diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java index d1c3f78697f..22c71c11bb1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java @@ -220,11 +220,28 @@ public class TableDescriptorBuilder { RESERVED_KEYWORDS.add(IS_META_KEY); } + /** + * @deprecated namespace table has been folded into the ns family in meta table, do not use this + * any more. + */ @InterfaceAudience.Private + @Deprecated public final static String NAMESPACE_FAMILY_INFO = "info"; + + /** + * @deprecated namespace table has been folded into the ns family in meta table, do not use this + * any more. + */ @InterfaceAudience.Private + @Deprecated public final static byte[] NAMESPACE_FAMILY_INFO_BYTES = Bytes.toBytes(NAMESPACE_FAMILY_INFO); + + /** + * @deprecated namespace table has been folded into the ns family in meta table, do not use this + * any more. + */ @InterfaceAudience.Private + @Deprecated public final static byte[] NAMESPACE_COL_DESC_BYTES = Bytes.toBytes("d"); /** @@ -245,22 +262,21 @@ public class TableDescriptorBuilder { CP_HTD_ATTR_VALUE_PARAM_VALUE_PATTERN + "),?"); private static final Pattern CP_HTD_ATTR_KEY_PATTERN = Pattern.compile("^coprocessor\\$([0-9]+)$", Pattern.CASE_INSENSITIVE); + /** * Table descriptor for namespace table + * @deprecated We have folded the data in namespace table into meta table, so do not use it any + * more. */ - // TODO We used to set CacheDataInL1 for NS table. When we have BucketCache in file mode, now the - // NS data goes to File mode BC only. Test how that affect the system. If too much, we have to - // rethink about adding back the setCacheDataInL1 for NS table. - public static final TableDescriptor NAMESPACE_TABLEDESC - = TableDescriptorBuilder.newBuilder(TableName.NAMESPACE_TABLE_NAME) + @Deprecated + public static final TableDescriptor NAMESPACE_TABLEDESC = + TableDescriptorBuilder.newBuilder(TableName.NAMESPACE_TABLE_NAME) .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(NAMESPACE_FAMILY_INFO_BYTES) - // Ten is arbitrary number. Keep versions to help debugging. - .setMaxVersions(10) - .setInMemory(true) - .setBlocksize(8 * 1024) - .setScope(HConstants.REPLICATION_SCOPE_LOCAL) - .build()) + // Ten is arbitrary number. Keep versions to help debugging. + .setMaxVersions(10).setInMemory(true).setBlocksize(8 * 1024) + .setScope(HConstants.REPLICATION_SCOPE_LOCAL).build()) .build(); + private final ModifyableTableDescriptor desc; /** diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index fbfab4b60bc..fdc3d82677f 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -26,7 +26,6 @@ import java.util.Collections; import java.util.List; import java.util.UUID; import java.util.regex.Pattern; - import org.apache.commons.lang3.ArrayUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; @@ -508,6 +507,13 @@ public final class HConstants { public static final byte[] REPLICATION_BARRIER_FAMILY = Bytes.toBytes(REPLICATION_BARRIER_FAMILY_STR); + /** The namespace family as a string */ + public static final String NAMESPACE_FAMILY_STR = "ns"; + + /** The namespace family */ + public static final byte[] NAMESPACE_FAMILY = Bytes.toBytes(NAMESPACE_FAMILY_STR); + + public static final byte[] NAMESPACE_COL_DESC_QUALIFIER = Bytes.toBytes("d"); /** * The meta table version column qualifier. * We keep current version of the meta table in this column in -ROOT- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/NamespaceDescriptor.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/NamespaceDescriptor.java index d1b69ba16c5..07386b557c8 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/NamespaceDescriptor.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/NamespaceDescriptor.java @@ -166,7 +166,7 @@ public class NamespaceDescriptor { private Builder(NamespaceDescriptor ns) { this.bName = ns.name; - this.bConfiguration = ns.configuration; + this.bConfiguration.putAll(ns.configuration); } private Builder(String name) { diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java index e6cabbc3456..093bf20d348 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java @@ -80,9 +80,14 @@ public final class TableName implements Comparable { public static final TableName META_TABLE_NAME = valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); - /** The Namespace table's name. */ + /** + * The Namespace table's name. + * @deprecated We have folded the data in namespace table into meta table, so do not use it any + * more. + */ + @Deprecated public static final TableName NAMESPACE_TABLE_NAME = - valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "namespace"); + valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "namespace"); public static final String OLD_META_STR = ".META."; public static final String OLD_ROOT_STR = "-ROOT-"; diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto index 8bb69cf21cb..44ac952abc7 100644 --- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto +++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto @@ -117,7 +117,7 @@ enum CreateNamespaceState { CREATE_NAMESPACE_PREPARE = 1; CREATE_NAMESPACE_CREATE_DIRECTORY = 2; CREATE_NAMESPACE_INSERT_INTO_NS_TABLE = 3; - CREATE_NAMESPACE_UPDATE_ZK = 4; + CREATE_NAMESPACE_UPDATE_ZK = 4[deprecated=true]; CREATE_NAMESPACE_SET_NAMESPACE_QUOTA = 5; } @@ -128,7 +128,7 @@ message CreateNamespaceStateData { enum ModifyNamespaceState { MODIFY_NAMESPACE_PREPARE = 1; MODIFY_NAMESPACE_UPDATE_NS_TABLE = 2; - MODIFY_NAMESPACE_UPDATE_ZK = 3; + MODIFY_NAMESPACE_UPDATE_ZK = 3[deprecated=true]; } message ModifyNamespaceStateData { @@ -139,7 +139,7 @@ message ModifyNamespaceStateData { enum DeleteNamespaceState { DELETE_NAMESPACE_PREPARE = 1; DELETE_NAMESPACE_DELETE_FROM_NS_TABLE = 2; - DELETE_NAMESPACE_REMOVE_FROM_ZK = 3; + DELETE_NAMESPACE_REMOVE_FROM_ZK = 3[deprecated=true]; DELETE_NAMESPACE_DELETE_DIRECTORIES = 4; DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA = 5; } @@ -475,7 +475,8 @@ message ReopenTableRegionsStateData { } enum InitMetaState { - INIT_META_ASSIGN_META = 1; + INIT_META_ASSIGN_META = 1; + INIT_META_CREATE_NAMESPACES = 2; } message InitMetaStateData { diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java index 4a881d30d92..cdd68d15ab4 100644 --- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java +++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.rsgroup; import com.google.protobuf.ServiceException; - import java.io.ByteArrayInputStream; import java.io.IOException; import java.util.ArrayList; @@ -34,7 +33,6 @@ import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; import java.util.concurrent.atomic.AtomicBoolean; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -85,6 +83,7 @@ import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.common.collect.Maps; import org.apache.hbase.thirdparty.com.google.common.collect.Sets; + import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; @@ -757,12 +756,9 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager { assignedRegions.clear(); found.set(true); try { - conn.getTable(TableName.NAMESPACE_TABLE_NAME); - conn.getTable(RSGROUP_TABLE_NAME); boolean rootMetaFound = masterServices.getMetaTableLocator().verifyMetaRegionLocation( conn, masterServices.getZooKeeper(), 1); - final AtomicBoolean nsFound = new AtomicBoolean(false); if (rootMetaFound) { MetaTableAccessor.Visitor visitor = new DefaultVisitorBase() { @Override @@ -791,36 +787,13 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager { } foundRegions.add(info); } - if (TableName.NAMESPACE_TABLE_NAME.equals(info.getTable())) { - Cell cell = row.getColumnLatestCell(HConstants.CATALOG_FAMILY, - HConstants.SERVER_QUALIFIER); - ServerName sn = null; - if(cell != null) { - sn = ServerName.parseVersionedServerName(CellUtil.cloneValue(cell)); - } - if (sn == null) { - nsFound.set(false); - } else if (tsm.isTableState(TableName.NAMESPACE_TABLE_NAME, - TableState.State.ENABLED)) { - try { - ClientProtos.ClientService.BlockingInterface rs = conn.getClient(sn); - ClientProtos.GetRequest request = - RequestConverter.buildGetRequest(info.getRegionName(), - new Get(ROW_KEY)); - rs.get(null, request); - nsFound.set(true); - } catch(Exception ex) { - LOG.debug("Caught exception while verifying group region", ex); - } - } - } } return true; } }; MetaTableAccessor.fullScanRegions(conn, visitor); // if no regions in meta then we have to create the table - if (foundRegions.size() < 1 && rootMetaFound && !createSent && nsFound.get()) { + if (foundRegions.size() < 1 && rootMetaFound && !createSent) { createRSGroupTable(); createSent = true; } diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java index 7415ab57119..3860c59c4b3 100644 --- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java +++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java @@ -93,8 +93,8 @@ public class TestRSGroupsBasics extends TestRSGroupsBase { assertEquals(4, defaultInfo.getServers().size()); // Assignment of root and meta regions. int count = master.getAssignmentManager().getRegionStates().getRegionAssignments().size(); - //3 meta,namespace, group - assertEquals(3, count); + // 2 meta, group + assertEquals(2, count); } @Test diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon index 03056e14809..7bb6c40ac75 100644 --- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon +++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon @@ -447,8 +447,6 @@ AssignmentManager assignmentManager = master.getAssignmentManager(); description = "The hbase:acl table holds information about acl."; } else if (tableName.equals(VisibilityConstants.LABELS_TABLE_NAME)){ description = "The hbase:labels table holds information about visibility labels."; - } else if (tableName.equals(TableName.NAMESPACE_TABLE_NAME)){ - description = "The hbase:namespace table holds information about namespaces."; } else if (tableName.equals(QuotaUtil.QUOTA_TABLE_NAME)){ description = "The hbase:quota table holds quota information about number" + " or size of requests in a given time frame."; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java deleted file mode 100644 index 101c40ebd86..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java +++ /dev/null @@ -1,215 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hbase; - -import java.io.IOException; -import java.util.List; -import java.util.NavigableMap; -import java.util.NavigableSet; -import java.util.concurrent.ConcurrentSkipListMap; - -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.zookeeper.ZKListener; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.apache.hadoop.hbase.zookeeper.ZNodePaths; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.common.collect.Sets; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; - -/** - * Class servers two purposes: - * - * 1. Broadcast NamespaceDescriptor information via ZK - * (Done by the Master) - * 2. Consume broadcasted NamespaceDescriptor changes - * (Done by the RegionServers) - * - */ -@InterfaceAudience.Private -public class ZKNamespaceManager extends ZKListener { - private static final Logger LOG = LoggerFactory.getLogger(ZKNamespaceManager.class); - private final String nsZNode; - private final NavigableMap cache; - - public ZKNamespaceManager(ZKWatcher zkw) throws IOException { - super(zkw); - nsZNode = zkw.getZNodePaths().namespaceZNode; - cache = new ConcurrentSkipListMap<>(); - } - - public void start() throws IOException { - watcher.registerListener(this); - try { - if (ZKUtil.watchAndCheckExists(watcher, nsZNode)) { - List existing = - ZKUtil.getChildDataAndWatchForNewChildren(watcher, nsZNode); - if (existing != null) { - refreshNodes(existing); - } - } else { - ZKUtil.createWithParents(watcher, nsZNode); - } - } catch (KeeperException e) { - throw new IOException("Failed to initialize ZKNamespaceManager", e); - } - } - - public void stop() throws IOException { - this.watcher.unregisterListener(this); - } - - public NamespaceDescriptor get(String name) { - return cache.get(name); - } - - public void update(NamespaceDescriptor ns) throws IOException { - writeNamespace(ns); - cache.put(ns.getName(), ns); - } - - public void remove(String name) throws IOException { - deleteNamespace(name); - cache.remove(name); - } - - public NavigableSet list() throws IOException { - NavigableSet ret = - Sets.newTreeSet(NamespaceDescriptor.NAMESPACE_DESCRIPTOR_COMPARATOR); - for(NamespaceDescriptor ns: cache.values()) { - ret.add(ns); - } - return ret; - } - - @Override - public void nodeCreated(String path) { - if (nsZNode.equals(path)) { - try { - List nodes = - ZKUtil.getChildDataAndWatchForNewChildren(watcher, nsZNode); - refreshNodes(nodes); - } catch (KeeperException ke) { - String msg = "Error reading data from zookeeper"; - LOG.error(msg, ke); - watcher.abort(msg, ke); - } catch (IOException e) { - String msg = "Error parsing data from zookeeper"; - LOG.error(msg, e); - watcher.abort(msg, e); - } - } - } - - @Override - public void nodeDeleted(String path) { - if (nsZNode.equals(ZKUtil.getParent(path))) { - String nsName = ZKUtil.getNodeName(path); - cache.remove(nsName); - } - } - - @Override - public void nodeDataChanged(String path) { - if (nsZNode.equals(ZKUtil.getParent(path))) { - try { - byte[] data = ZKUtil.getDataAndWatch(watcher, path); - NamespaceDescriptor ns = - ProtobufUtil.toNamespaceDescriptor( - HBaseProtos.NamespaceDescriptor.parseFrom(data)); - cache.put(ns.getName(), ns); - } catch (KeeperException ke) { - String msg = "Error reading data from zookeeper for node "+path; - LOG.error(msg, ke); - // only option is to abort - watcher.abort(msg, ke); - } catch (IOException ioe) { - String msg = "Error deserializing namespace: "+path; - LOG.error(msg, ioe); - watcher.abort(msg, ioe); - } - } - } - - @Override - public void nodeChildrenChanged(String path) { - if (nsZNode.equals(path)) { - try { - List nodes = - ZKUtil.getChildDataAndWatchForNewChildren(watcher, nsZNode); - refreshNodes(nodes); - } catch (KeeperException ke) { - LOG.error("Error reading data from zookeeper for path "+path, ke); - watcher.abort("ZooKeeper error get node children for path "+path, ke); - } catch (IOException e) { - LOG.error("Error deserializing namespace child from: "+path, e); - watcher.abort("Error deserializing namespace child from: " + path, e); - } - } - } - - private void deleteNamespace(String name) throws IOException { - String zNode = ZNodePaths.joinZNode(nsZNode, name); - try { - ZKUtil.deleteNode(watcher, zNode); - } catch (KeeperException e) { - if (e instanceof KeeperException.NoNodeException) { - // If the node does not exist, it could be already deleted. Continue without fail. - LOG.warn("The ZNode " + zNode + " for namespace " + name + " does not exist."); - } else { - LOG.error("Failed updating permissions for namespace " + name, e); - throw new IOException("Failed updating permissions for namespace " + name, e); - } - } - } - - private void writeNamespace(NamespaceDescriptor ns) throws IOException { - String zNode = ZNodePaths.joinZNode(nsZNode, ns.getName()); - try { - ZKUtil.createWithParents(watcher, zNode); - ZKUtil.updateExistingNodeData(watcher, zNode, - ProtobufUtil.toProtoNamespaceDescriptor(ns).toByteArray(), -1); - } catch (KeeperException e) { - LOG.error("Failed updating permissions for namespace "+ns.getName(), e); - throw new IOException("Failed updating permissions for namespace "+ns.getName(), e); - } - } - - private void refreshNodes(List nodes) throws IOException { - for (ZKUtil.NodeAndData n : nodes) { - if (n.isEmpty()) continue; - String path = n.getNode(); - String namespace = ZKUtil.getNodeName(path); - byte[] nodeData = n.getData(); - if (LOG.isTraceEnabled()) { - LOG.trace("Updating namespace cache from node " + namespace + " with data: " + - Bytes.toStringBinary(nodeData)); - } - NamespaceDescriptor ns = - ProtobufUtil.toNamespaceDescriptor( - HBaseProtos.NamespaceDescriptor.parseFrom(nodeData)); - cache.put(ns.getName(), ns); - } - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java index 5af7614186e..1e631b0041d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java @@ -18,15 +18,10 @@ package org.apache.hadoop.hbase.master; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; import java.util.List; -import java.util.Set; - import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NamespaceNotFoundException; import org.apache.hadoop.hbase.ServiceNotRunningException; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure; import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; @@ -34,15 +29,17 @@ import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure; import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch; import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; -import org.apache.hbase.thirdparty.com.google.common.util.concurrent.AbstractService; import org.apache.hadoop.hbase.util.NonceKey; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; +import org.apache.hbase.thirdparty.com.google.common.util.concurrent.AbstractService; @InterfaceAudience.Private class ClusterSchemaServiceImpl extends AbstractService implements ClusterSchemaService { + private final TableNamespaceManager tableNamespaceManager; private final MasterServices masterServices; - private final static List EMPTY_NAMESPACE_LIST = - Collections.unmodifiableList(new ArrayList(0)); ClusterSchemaServiceImpl(final MasterServices masterServices) { this.masterServices = masterServices; @@ -51,9 +48,10 @@ class ClusterSchemaServiceImpl extends AbstractService implements ClusterSchemaS // All below are synchronized so consistent view on whether running or not. - private synchronized void checkIsRunning() throws ServiceNotRunningException { - if (!isRunning()) throw new ServiceNotRunningException(); + if (!isRunning()) { + throw new ServiceNotRunningException(); + } } @Override @@ -70,10 +68,6 @@ class ClusterSchemaServiceImpl extends AbstractService implements ClusterSchemaS protected void doStop() { // This is no stop for the table manager. notifyStopped(); - TableNamespaceManager tnsm = getTableNamespaceManager(); - if (tnsm != null) { - tnsm.stop("Stopping"); - } } @Override @@ -123,10 +117,8 @@ class ClusterSchemaServiceImpl extends AbstractService implements ClusterSchemaS @Override public List getNamespaces() throws IOException { checkIsRunning(); - Set set = getTableNamespaceManager().list(); - if (set == null || set.isEmpty()) return EMPTY_NAMESPACE_LIST; - List list = new ArrayList<>(set.size()); - list.addAll(set); - return Collections.unmodifiableList(list); + return getTableNamespaceManager().list().stream() + .sorted(NamespaceDescriptor.NAMESPACE_DESCRIPTOR_COMPARATOR) + .collect(ImmutableList.toImmutableList()); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index ae04283386f..428030d4f93 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -1112,10 +1112,7 @@ public class HMaster extends HRegionServer implements MasterServices { getChoreService().scheduleChore(catalogJanitorChore); this.serverManager.startChore(); - // NAMESPACE READ!!!! - // Here we expect hbase:namespace to be online. See inside initClusterSchemaService. - // TODO: Fix this. Namespace is a pain being a sort-of system table. Fold it in to hbase:meta. - // isNamespace does like isMeta and waits until namespace is onlined before allowing progress. + // Only for rolling upgrade, where we need to migrate the data in namespace table to meta table. if (!waitForNamespaceOnline()) { return; } @@ -1243,20 +1240,28 @@ public class HMaster extends HRegionServer implements MasterServices { /** * Check hbase:namespace table is assigned. If not, startup will hang looking for the ns table - * (TODO: Fix this! NS should not hold-up startup). + *

+ * This is for rolling upgrading, later we will migrate the data in ns table to the ns family of + * meta table. And if this is a new clsuter, this method will return immediately as there will be + * no namespace table/region. * @return True if namespace table is up/online. */ - @VisibleForTesting - public boolean waitForNamespaceOnline() throws InterruptedException { - List ris = this.assignmentManager.getRegionStates(). - getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME); + private boolean waitForNamespaceOnline() throws InterruptedException, IOException { + TableState nsTableState = + MetaTableAccessor.getTableState(getClusterConnection(), TableName.NAMESPACE_TABLE_NAME); + if (nsTableState == null || nsTableState.isDisabled()) { + // this means we have already migrated the data and disabled or deleted the namespace table, + // or this is a new depliy which does not have a namespace table from the beginning. + return true; + } + List ris = + this.assignmentManager.getRegionStates().getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME); if (ris.isEmpty()) { - // If empty, means we've not assigned the namespace table yet... Just return true so startup - // continues and the namespace table gets created. + // maybe this will not happen any more, but anyway, no harm to add a check here... return true; } // Else there are namespace regions up in meta. Ensure they are assigned before we go on. - for (RegionInfo ri: ris) { + for (RegionInfo ri : ris) { isRegionOnline(ri); } return true; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java index aefeebe0989..21178e56c5d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java @@ -19,324 +19,159 @@ package org.apache.hadoop.hbase.master; import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.NavigableSet; - +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellBuilderFactory; -import org.apache.hadoop.hbase.CellBuilderType; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.NamespaceDescriptor; -import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ZKNamespaceManager; +import org.apache.hadoop.hbase.client.BufferedMutator; +import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.constraint.ConstraintException; -import org.apache.hadoop.hbase.exceptions.TimeoutIOException; -import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; -import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch; -import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; -import org.apache.hbase.thirdparty.com.google.common.collect.Sets; +import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hbase.thirdparty.com.google.protobuf.CodedInputStream; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.Threads; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** - * This is a helper class used internally to manage the namespace metadata that is stored in - * TableName.NAMESPACE_TABLE_NAME. It also mirrors updates to the ZK store by forwarding updates to - * {@link org.apache.hadoop.hbase.ZKNamespaceManager}. - * - * WARNING: Do not use. Go via the higher-level {@link ClusterSchema} API instead. This manager - * is likely to go aways anyways. + * This is a helper class used internally to manage the namespace metadata that is stored in the ns + * family in meta table. */ @InterfaceAudience.Private -@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC", - justification="TODO: synchronize access on nsTable but it is done in tiers above and this " + - "class is going away/shrinking") -public class TableNamespaceManager implements Stoppable { - private static final Logger LOG = LoggerFactory.getLogger(TableNamespaceManager.class); - private volatile boolean stopped = false; - - private Configuration conf; - private MasterServices masterServices; - private Table nsTable = null; // FindBugs: IS2_INCONSISTENT_SYNC TODO: Access is not synchronized - private ZKNamespaceManager zkNamespaceManager; - private boolean initialized; +public class TableNamespaceManager { public static final String KEY_MAX_REGIONS = "hbase.namespace.quota.maxregions"; public static final String KEY_MAX_TABLES = "hbase.namespace.quota.maxtables"; static final String NS_INIT_TIMEOUT = "hbase.master.namespace.init.timeout"; static final int DEFAULT_NS_INIT_TIMEOUT = 300000; + private final ConcurrentMap cache = new ConcurrentHashMap<>(); + + private final MasterServices masterServices; + TableNamespaceManager(MasterServices masterServices) { this.masterServices = masterServices; - this.conf = masterServices.getConfiguration(); + } + + private void migrateNamespaceTable() throws IOException { + try (Table nsTable = masterServices.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME); + ResultScanner scanner = nsTable.getScanner( + new Scan().addFamily(TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES).readAllVersions()); + BufferedMutator mutator = + masterServices.getConnection().getBufferedMutator(TableName.META_TABLE_NAME)) { + for (Result result;;) { + result = scanner.next(); + if (result == null) { + break; + } + Put put = new Put(result.getRow()); + result + .getColumnCells(TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES, + TableDescriptorBuilder.NAMESPACE_COL_DESC_BYTES) + .forEach(c -> put.addColumn(HConstants.NAMESPACE_FAMILY, + HConstants.NAMESPACE_COL_DESC_QUALIFIER, c.getTimestamp(), CellUtil.cloneValue(c))); + mutator.mutate(put); + } + } + // schedule a disable procedure instead of block waiting here, as when disabling a table we will + // wait until master is initialized, but we are part of the initialization... + masterServices.getMasterProcedureExecutor().submitProcedure( + new DisableTableProcedure(masterServices.getMasterProcedureExecutor().getEnvironment(), + TableName.NAMESPACE_TABLE_NAME, false)); + } + + private void loadNamespaceIntoCache() throws IOException { + try (Table table = masterServices.getConnection().getTable(TableName.META_TABLE_NAME); + ResultScanner scanner = table.getScanner(HConstants.NAMESPACE_FAMILY)) { + for (Result result;;) { + result = scanner.next(); + if (result == null) { + break; + } + Cell cell = result.getColumnLatestCell(HConstants.NAMESPACE_FAMILY, + HConstants.NAMESPACE_COL_DESC_QUALIFIER); + NamespaceDescriptor ns = ProtobufUtil + .toNamespaceDescriptor(HBaseProtos.NamespaceDescriptor.parseFrom(CodedInputStream + .newInstance(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()))); + cache.put(ns.getName(), ns); + } + } } public void start() throws IOException { - if (!MetaTableAccessor.tableExists(masterServices.getConnection(), - TableName.NAMESPACE_TABLE_NAME)) { - LOG.info("Namespace table not found. Creating..."); - createNamespaceTable(masterServices); - } - - try { - // Wait for the namespace table to be initialized. - long startTime = EnvironmentEdgeManager.currentTime(); - int timeout = conf.getInt(NS_INIT_TIMEOUT, DEFAULT_NS_INIT_TIMEOUT); - while (!isTableAvailableAndInitialized()) { - if (EnvironmentEdgeManager.currentTime() - startTime + 100 > timeout) { - // We can't do anything if ns is not online. - throw new IOException("Timedout " + timeout + "ms waiting for namespace table to " - + "be assigned and enabled: " + getTableState()); - } - Thread.sleep(100); - } - } catch (InterruptedException e) { - throw (InterruptedIOException) new InterruptedIOException().initCause(e); + TableState nsTableState = MetaTableAccessor.getTableState(masterServices.getConnection(), + TableName.NAMESPACE_TABLE_NAME); + if (nsTableState != null && nsTableState.isEnabled()) { + migrateNamespaceTable(); } + loadNamespaceIntoCache(); } - private synchronized Table getNamespaceTable() throws IOException { - if (!isTableNamespaceManagerInitialized()) { - throw new IOException(this.getClass().getName() + " isn't ready to serve"); - } - return nsTable; - } - - /* + /** * check whether a namespace has already existed. */ - public boolean doesNamespaceExist(final String namespaceName) throws IOException { - if (nsTable == null) { - throw new IOException(this.getClass().getName() + " isn't ready to serve"); - } - return (get(nsTable, namespaceName) != null); + public boolean doesNamespaceExist(String namespaceName) throws IOException { + return cache.containsKey(namespaceName); } - public synchronized NamespaceDescriptor get(String name) throws IOException { - if (!isTableNamespaceManagerInitialized()) { - return null; - } - return zkNamespaceManager.get(name); + public NamespaceDescriptor get(String name) throws IOException { + return cache.get(name); } - private NamespaceDescriptor get(Table table, String name) throws IOException { - Result res = table.get(new Get(Bytes.toBytes(name))); - if (res.isEmpty()) { - return null; - } - byte[] val = CellUtil.cloneValue(res.getColumnLatestCell( - HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES, HTableDescriptor.NAMESPACE_COL_DESC_BYTES)); - return - ProtobufUtil.toNamespaceDescriptor( - HBaseProtos.NamespaceDescriptor.parseFrom(val)); + public void addOrUpdateNamespace(NamespaceDescriptor ns) throws IOException { + insertNamespaceToMeta(masterServices.getConnection(), ns); + cache.put(ns.getName(), ns); } - public void insertIntoNSTable(final NamespaceDescriptor ns) throws IOException { - if (nsTable == null) { - throw new IOException(this.getClass().getName() + " isn't ready to serve"); - } - byte[] row = Bytes.toBytes(ns.getName()); - Put p = new Put(row, true); - p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY) - .setRow(row) - .setFamily(TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES) - .setQualifier(TableDescriptorBuilder.NAMESPACE_COL_DESC_BYTES) - .setTimestamp(p.getTimestamp()) - .setType(Cell.Type.Put) - .setValue(ProtobufUtil.toProtoNamespaceDescriptor(ns).toByteArray()) - .build()); - nsTable.put(p); - } - - public void updateZKNamespaceManager(final NamespaceDescriptor ns) throws IOException { - try { - zkNamespaceManager.update(ns); - } catch (IOException ex) { - String msg = "Failed to update namespace information in ZK."; - LOG.error(msg, ex); - throw new IOException(msg, ex); - } - } - - public void removeFromNSTable(final String namespaceName) throws IOException { - if (nsTable == null) { - throw new IOException(this.getClass().getName() + " isn't ready to serve"); - } - Delete d = new Delete(Bytes.toBytes(namespaceName)); - nsTable.delete(d); - } - - public void removeFromZKNamespaceManager(final String namespaceName) throws IOException { - zkNamespaceManager.remove(namespaceName); - } - - public synchronized NavigableSet list() throws IOException { - NavigableSet ret = - Sets.newTreeSet(NamespaceDescriptor.NAMESPACE_DESCRIPTOR_COMPARATOR); - ResultScanner scanner = - getNamespaceTable().getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES); - try { - for(Result r : scanner) { - byte[] val = CellUtil.cloneValue(r.getColumnLatestCell( - HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES, - HTableDescriptor.NAMESPACE_COL_DESC_BYTES)); - ret.add(ProtobufUtil.toNamespaceDescriptor( - HBaseProtos.NamespaceDescriptor.parseFrom(val))); - } - } finally { - scanner.close(); - } - return ret; - } - - private void createNamespaceTable(MasterServices masterServices) throws IOException { - masterServices.createSystemTable(HTableDescriptor.NAMESPACE_TABLEDESC); - } - - @SuppressWarnings("deprecation") - private boolean isTableNamespaceManagerInitialized() throws IOException { - if (initialized) { - this.nsTable = this.masterServices.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME); - return true; - } - return false; - } - - /** - * Create Namespace in a blocking manner. Keeps trying until - * {@link ClusterSchema#HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY} expires. - * Note, by-passes notifying coprocessors and name checks. Use for system namespaces only. - */ - private void blockingCreateNamespace(final NamespaceDescriptor namespaceDescriptor) + public static void insertNamespaceToMeta(Connection conn, NamespaceDescriptor ns) throws IOException { - ClusterSchema clusterSchema = this.masterServices.getClusterSchema(); - long procId = clusterSchema.createNamespace(namespaceDescriptor, null, ProcedurePrepareLatch.getNoopLatch()); - block(this.masterServices, procId); - } - - - /** - * An ugly utility to be removed when refactor TableNamespaceManager. - * @throws TimeoutIOException - */ - private static void block(final MasterServices services, final long procId) - throws TimeoutIOException { - int timeoutInMillis = services.getConfiguration(). - getInt(ClusterSchema.HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY, - ClusterSchema.DEFAULT_HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT); - long deadlineTs = EnvironmentEdgeManager.currentTime() + timeoutInMillis; - ProcedureExecutor procedureExecutor = - services.getMasterProcedureExecutor(); - while(EnvironmentEdgeManager.currentTime() < deadlineTs) { - if (procedureExecutor.isFinished(procId)) return; - // Sleep some - Threads.sleep(10); + byte[] row = Bytes.toBytes(ns.getName()); + Put put = new Put(row, true).addColumn(HConstants.NAMESPACE_FAMILY, + HConstants.NAMESPACE_COL_DESC_QUALIFIER, + ProtobufUtil.toProtoNamespaceDescriptor(ns).toByteArray()); + try (Table table = conn.getTable(TableName.META_TABLE_NAME)) { + table.put(put); } - throw new TimeoutIOException("Procedure pid=" + procId + " is still running"); } - /** - * This method checks if the namespace table is assigned and then - * tries to create its Table reference. If it was already created before, it also makes - * sure that the connection isn't closed. - * @return true if the namespace table manager is ready to serve, false otherwise - */ - @SuppressWarnings("deprecation") - public synchronized boolean isTableAvailableAndInitialized() - throws IOException { - // Did we already get a table? If so, still make sure it's available - if (isTableNamespaceManagerInitialized()) { - return true; + public void deleteNamespace(String namespaceName) throws IOException { + Delete d = new Delete(Bytes.toBytes(namespaceName)); + try (Table table = masterServices.getConnection().getTable(TableName.META_TABLE_NAME)) { + table.delete(d); } - - // Now check if the table is assigned, if not then fail fast - if (isTableAssigned() && isTableEnabled()) { - try { - boolean initGoodSofar = true; - nsTable = this.masterServices.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME); - zkNamespaceManager = new ZKNamespaceManager(masterServices.getZooKeeper()); - zkNamespaceManager.start(); - - if (get(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE.getName()) == null) { - blockingCreateNamespace(NamespaceDescriptor.DEFAULT_NAMESPACE); - } - if (get(nsTable, NamespaceDescriptor.SYSTEM_NAMESPACE.getName()) == null) { - blockingCreateNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE); - } - - if (!initGoodSofar) { - // some required namespace is created asynchronized. We should complete init later. - return false; - } - - ResultScanner scanner = nsTable.getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES); - try { - for (Result result : scanner) { - byte[] val = CellUtil.cloneValue(result.getColumnLatestCell( - 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; - return true; - } catch (IOException ie) { - LOG.warn("Caught exception in initializing namespace table manager", ie); - if (nsTable != null) { - nsTable.close(); - } - throw ie; - } - } - return false; + cache.remove(namespaceName); } - private TableState getTableState() throws IOException { - return masterServices.getTableStateManager().getTableState(TableName.NAMESPACE_TABLE_NAME); - } - - private boolean isTableEnabled() throws IOException { - return getTableState().isEnabled(); - } - - private boolean isTableAssigned() { - // TODO: we have a better way now (wait on event) - return masterServices.getAssignmentManager() - .getRegionStates().hasTableRegionStates(TableName.NAMESPACE_TABLE_NAME); + public List list() throws IOException { + return cache.values().stream().collect(Collectors.toList()); } public void validateTableAndRegionCount(NamespaceDescriptor desc) throws IOException { if (getMaxRegions(desc) <= 0) { - throw new ConstraintException("The max region quota for " + desc.getName() - + " is less than or equal to zero."); + throw new ConstraintException( + "The max region quota for " + desc.getName() + " is less than or equal to zero."); } if (getMaxTables(desc) <= 0) { - throw new ConstraintException("The max tables quota for " + desc.getName() - + " is less than or equal to zero."); + throw new ConstraintException( + "The max tables quota for " + desc.getName() + " is less than or equal to zero."); } } @@ -371,31 +206,4 @@ public class TableNamespaceManager implements Stoppable { } return maxRegions; } - - @Override - public boolean isStopped() { - return this.stopped; - } - - @Override - public void stop(String why) { - if (this.stopped) { - return; - } - try { - if (this.zkNamespaceManager != null) { - this.zkNamespaceManager.stop(); - } - } catch (IOException ioe) { - LOG.warn("Failed NamespaceManager close", ioe); - } - try { - if (this.nsTable != null) { - this.nsTable.close(); - } - } catch (IOException ioe) { - LOG.warn("Failed Namespace Table close", ioe); - } - this.stopped = true; - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java index 3a87bbc4609..4fba2ddc211 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java @@ -366,7 +366,7 @@ public final class LockProcedure extends Procedure } private LockInterface setupNamespaceLock() throws IllegalArgumentException { - this.tableName = TableName.NAMESPACE_TABLE_NAME; + this.tableName = TableProcedureInterface.DUMMY_NAMESPACE_TABLE_NAME; switch (type) { case EXCLUSIVE: this.opType = TableOperationType.EDIT; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java index 341d1168b0e..e751034e2e0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java @@ -18,18 +18,25 @@ package org.apache.hadoop.hbase.master.procedure; +import java.io.IOException; +import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.master.MasterFileSystem; +import org.apache.hadoop.hbase.master.TableNamespaceManager; import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hadoop.hbase.procedure2.StateMachineProcedure; +import org.apache.hadoop.hbase.util.FSUtils; /** - * Base class for all the Namespace procedures that want to use a StateMachineProcedure. - * It provide some basic helpers like basic locking and basic toStringClassDetails(). + * Base class for all the Namespace procedures that want to use a StateMachineProcedure. It provide + * some basic helpers like basic locking and basic toStringClassDetails(). */ @InterfaceAudience.Private public abstract class AbstractStateMachineNamespaceProcedure - extends StateMachineProcedure - implements TableProcedureInterface { + extends StateMachineProcedure implements TableProcedureInterface { private final ProcedurePrepareLatch syncLatch; @@ -52,7 +59,7 @@ public abstract class AbstractStateMachineNamespaceProcedure @Override public TableName getTableName() { - return TableName.NAMESPACE_TABLE_NAME; + return DUMMY_NAMESPACE_TABLE_NAME; } @Override @@ -83,6 +90,35 @@ public abstract class AbstractStateMachineNamespaceProcedure env.getProcedureScheduler().wakeNamespaceExclusiveLock(this, getNamespaceName()); } + /** + * Insert/update the row into the ns family of meta table. + * @param env MasterProcedureEnv + */ + protected static void addOrUpdateNamespace(MasterProcedureEnv env, NamespaceDescriptor ns) + throws IOException { + getTableNamespaceManager(env).addOrUpdateNamespace(ns); + } + + protected static TableNamespaceManager getTableNamespaceManager(MasterProcedureEnv env) { + return env.getMasterServices().getClusterSchema().getTableNamespaceManager(); + } + + /** + * Create the namespace directory + * @param env MasterProcedureEnv + * @param nsDescriptor NamespaceDescriptor + */ + protected static void createDirectory(MasterProcedureEnv env, NamespaceDescriptor nsDescriptor) + throws IOException { + createDirectory(env.getMasterServices().getMasterFileSystem(), nsDescriptor); + } + + @VisibleForTesting + public static void createDirectory(MasterFileSystem mfs, NamespaceDescriptor nsDescriptor) + throws IOException { + mfs.getFileSystem().mkdirs(FSUtils.getNamespaceDir(mfs.getRootDir(), nsDescriptor.getName())); + } + protected void releaseSyncLatch() { ProcedurePrepareLatch.releaseLatch(syncLatch, this); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java index 2f56e833b87..28f75859c8c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java @@ -15,23 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import java.io.IOException; - import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NamespaceExistException; +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.master.MasterFileSystem; -import org.apache.hadoop.hbase.master.TableNamespaceManager; -import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceState; -import org.apache.hadoop.hbase.util.FSUtils; /** * The procedure to create a new namespace. @@ -42,10 +38,8 @@ public class CreateNamespaceProcedure private static final Logger LOG = LoggerFactory.getLogger(CreateNamespaceProcedure.class); private NamespaceDescriptor nsDescriptor; - private Boolean traceEnabled; public CreateNamespaceProcedure() { - this.traceEnabled = null; } public CreateNamespaceProcedure(final MasterProcedureEnv env, @@ -57,43 +51,40 @@ public class CreateNamespaceProcedure final NamespaceDescriptor nsDescriptor, ProcedurePrepareLatch latch) { super(env, latch); this.nsDescriptor = nsDescriptor; - this.traceEnabled = null; } @Override protected Flow executeFromState(final MasterProcedureEnv env, final CreateNamespaceState state) throws InterruptedException { - if (isTraceEnabled()) { - LOG.trace(this + " execute state=" + state); - } + LOG.trace("{} execute state={}", this, state); try { switch (state) { - case CREATE_NAMESPACE_PREPARE: - boolean success = prepareCreate(env); - releaseSyncLatch(); - if (!success) { - assert isFailed() : "createNamespace should have an exception here"; + case CREATE_NAMESPACE_PREPARE: + boolean success = prepareCreate(env); + releaseSyncLatch(); + if (!success) { + assert isFailed() : "createNamespace should have an exception here"; + return Flow.NO_MORE_STATE; + } + setNextState(CreateNamespaceState.CREATE_NAMESPACE_CREATE_DIRECTORY); + break; + case CREATE_NAMESPACE_CREATE_DIRECTORY: + createDirectory(env, nsDescriptor); + setNextState(CreateNamespaceState.CREATE_NAMESPACE_INSERT_INTO_NS_TABLE); + break; + case CREATE_NAMESPACE_INSERT_INTO_NS_TABLE: + addOrUpdateNamespace(env, nsDescriptor); + setNextState(CreateNamespaceState.CREATE_NAMESPACE_SET_NAMESPACE_QUOTA); + break; + case CREATE_NAMESPACE_UPDATE_ZK: + // not used any more + setNextState(CreateNamespaceState.CREATE_NAMESPACE_SET_NAMESPACE_QUOTA); + break; + case CREATE_NAMESPACE_SET_NAMESPACE_QUOTA: + setNamespaceQuota(env, nsDescriptor); return Flow.NO_MORE_STATE; - } - setNextState(CreateNamespaceState.CREATE_NAMESPACE_CREATE_DIRECTORY); - break; - case CREATE_NAMESPACE_CREATE_DIRECTORY: - createDirectory(env, nsDescriptor); - setNextState(CreateNamespaceState.CREATE_NAMESPACE_INSERT_INTO_NS_TABLE); - break; - case CREATE_NAMESPACE_INSERT_INTO_NS_TABLE: - insertIntoNSTable(env, nsDescriptor); - setNextState(CreateNamespaceState.CREATE_NAMESPACE_UPDATE_ZK); - break; - case CREATE_NAMESPACE_UPDATE_ZK: - updateZKNamespaceManager(env, nsDescriptor); - setNextState(CreateNamespaceState.CREATE_NAMESPACE_SET_NAMESPACE_QUOTA); - break; - case CREATE_NAMESPACE_SET_NAMESPACE_QUOTA: - setNamespaceQuota(env, nsDescriptor); - return Flow.NO_MORE_STATE; - default: - throw new UnsupportedOperationException(this + " unhandled state=" + state); + default: + throw new UnsupportedOperationException(this + " unhandled state=" + state); } } catch (IOException e) { if (isRollbackSupported(state)) { @@ -145,39 +136,26 @@ public class CreateNamespaceProcedure } @Override - protected void serializeStateData(ProcedureStateSerializer serializer) - throws IOException { + protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { super.serializeStateData(serializer); MasterProcedureProtos.CreateNamespaceStateData.Builder createNamespaceMsg = - MasterProcedureProtos.CreateNamespaceStateData.newBuilder().setNamespaceDescriptor( - ProtobufUtil.toProtoNamespaceDescriptor(this.nsDescriptor)); + MasterProcedureProtos.CreateNamespaceStateData.newBuilder() + .setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(this.nsDescriptor)); serializer.serialize(createNamespaceMsg.build()); } @Override - protected void deserializeStateData(ProcedureStateSerializer serializer) - throws IOException { + protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { super.deserializeStateData(serializer); MasterProcedureProtos.CreateNamespaceStateData createNamespaceMsg = - serializer.deserialize(MasterProcedureProtos.CreateNamespaceStateData.class); + serializer.deserialize(MasterProcedureProtos.CreateNamespaceStateData.class); nsDescriptor = ProtobufUtil.toNamespaceDescriptor(createNamespaceMsg.getNamespaceDescriptor()); } - private boolean isBootstrapNamespace() { - return nsDescriptor.equals(NamespaceDescriptor.DEFAULT_NAMESPACE) || - nsDescriptor.equals(NamespaceDescriptor.SYSTEM_NAMESPACE); - } - @Override protected boolean waitInitialized(MasterProcedureEnv env) { - // Namespace manager might not be ready if master is not fully initialized, - // return false to reject user namespace creation; return true for default - // and system namespace creation (this is part of master initialization). - if (isBootstrapNamespace()) { - return false; - } return env.waitInitialized(this); } @@ -202,90 +180,26 @@ public class CreateNamespaceProcedure /** * Action before any real action of creating namespace. * @param env MasterProcedureEnv - * @throws IOException */ private boolean prepareCreate(final MasterProcedureEnv env) throws IOException { if (getTableNamespaceManager(env).doesNamespaceExist(nsDescriptor.getName())) { setFailure("master-create-namespace", - new NamespaceExistException("Namespace " + nsDescriptor.getName() + " already exists")); + new NamespaceExistException("Namespace " + nsDescriptor.getName() + " already exists")); return false; } getTableNamespaceManager(env).validateTableAndRegionCount(nsDescriptor); return true; } - /** - * Create the namespace directory - * @param env MasterProcedureEnv - * @param nsDescriptor NamespaceDescriptor - * @throws IOException - */ - protected static void createDirectory( - final MasterProcedureEnv env, - final NamespaceDescriptor nsDescriptor) throws IOException { - MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem(); - mfs.getFileSystem().mkdirs( - FSUtils.getNamespaceDir(mfs.getRootDir(), nsDescriptor.getName())); - } - - /** - * Insert the row into ns table - * @param env MasterProcedureEnv - * @param nsDescriptor NamespaceDescriptor - * @throws IOException - */ - protected static void insertIntoNSTable( - final MasterProcedureEnv env, - final NamespaceDescriptor nsDescriptor) throws IOException { - getTableNamespaceManager(env).insertIntoNSTable(nsDescriptor); - } - - /** - * Update ZooKeeper. - * @param env MasterProcedureEnv - * @param nsDescriptor NamespaceDescriptor - * @throws IOException - */ - protected static void updateZKNamespaceManager( - final MasterProcedureEnv env, - final NamespaceDescriptor nsDescriptor) throws IOException { - getTableNamespaceManager(env).updateZKNamespaceManager(nsDescriptor); - } - /** * Set quota for the namespace * @param env MasterProcedureEnv * @param nsDescriptor NamespaceDescriptor - * @throws IOException **/ - protected static void setNamespaceQuota( - final MasterProcedureEnv env, + private static void setNamespaceQuota(final MasterProcedureEnv env, final NamespaceDescriptor nsDescriptor) throws IOException { if (env.getMasterServices().isInitialized()) { env.getMasterServices().getMasterQuotaManager().setNamespaceQuota(nsDescriptor); } } - - private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) { - return env.getMasterServices().getClusterSchema().getTableNamespaceManager(); - } - - /** - * The procedure could be restarted from a different machine. If the variable is null, we need to - * retrieve it. - * @return traceEnabled - */ - private Boolean isTraceEnabled() { - if (traceEnabled == null) { - traceEnabled = LOG.isTraceEnabled(); - } - return traceEnabled; - } - - @Override - protected boolean shouldWaitClientAck(MasterProcedureEnv env) { - // hbase and default namespaces are created on bootstrap internally by the system - // the client does not know about this procedures. - return !isBootstrapNamespace(); - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java index 8369a19564c..d3749a2300f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java @@ -15,29 +15,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import java.io.FileNotFoundException; import java.io.IOException; - import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NamespaceNotFoundException; +import org.apache.hadoop.hbase.constraint.ConstraintException; +import org.apache.hadoop.hbase.master.MasterFileSystem; +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.util.FSUtils; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.constraint.ConstraintException; -import org.apache.hadoop.hbase.master.MasterFileSystem; -import org.apache.hadoop.hbase.master.TableNamespaceManager; -import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceState; -import org.apache.hadoop.hbase.util.FSUtils; /** * The procedure to remove a namespace. @@ -49,64 +47,59 @@ public class DeleteNamespaceProcedure private NamespaceDescriptor nsDescriptor; private String namespaceName; - private Boolean traceEnabled; public DeleteNamespaceProcedure() { - this.nsDescriptor = null; - this.traceEnabled = null; } - public DeleteNamespaceProcedure(final MasterProcedureEnv env, final String namespaceName) { + public DeleteNamespaceProcedure(MasterProcedureEnv env, String namespaceName) { this(env, namespaceName, null); } - public DeleteNamespaceProcedure(final MasterProcedureEnv env, final String namespaceName, + public DeleteNamespaceProcedure(MasterProcedureEnv env, String namespaceName, final ProcedurePrepareLatch latch) { super(env, latch); this.namespaceName = namespaceName; - this.nsDescriptor = null; - this.traceEnabled = null; } @Override - protected Flow executeFromState(final MasterProcedureEnv env, final DeleteNamespaceState state) + protected Flow executeFromState(MasterProcedureEnv env, DeleteNamespaceState state) throws InterruptedException { LOG.info(this.toString()); try { switch (state) { - case DELETE_NAMESPACE_PREPARE: - boolean present = prepareDelete(env); - releaseSyncLatch(); - if (!present) { - assert isFailed() : "Delete namespace should have an exception here"; + case DELETE_NAMESPACE_PREPARE: + boolean present = prepareDelete(env); + releaseSyncLatch(); + if (!present) { + assert isFailed() : "Delete namespace should have an exception here"; + return Flow.NO_MORE_STATE; + } + setNextState(DeleteNamespaceState.DELETE_NAMESPACE_DELETE_FROM_NS_TABLE); + break; + case DELETE_NAMESPACE_DELETE_FROM_NS_TABLE: + deleteNamespace(env, namespaceName); + setNextState(DeleteNamespaceState.DELETE_NAMESPACE_DELETE_DIRECTORIES); + break; + case DELETE_NAMESPACE_REMOVE_FROM_ZK: + // not used any more + setNextState(DeleteNamespaceState.DELETE_NAMESPACE_DELETE_DIRECTORIES); + break; + case DELETE_NAMESPACE_DELETE_DIRECTORIES: + deleteDirectory(env, namespaceName); + setNextState(DeleteNamespaceState.DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA); + break; + case DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA: + removeNamespaceQuota(env, namespaceName); return Flow.NO_MORE_STATE; - } - setNextState(DeleteNamespaceState.DELETE_NAMESPACE_DELETE_FROM_NS_TABLE); - break; - case DELETE_NAMESPACE_DELETE_FROM_NS_TABLE: - deleteFromNSTable(env, namespaceName); - setNextState(DeleteNamespaceState.DELETE_NAMESPACE_REMOVE_FROM_ZK); - break; - case DELETE_NAMESPACE_REMOVE_FROM_ZK: - removeFromZKNamespaceManager(env, namespaceName); - setNextState(DeleteNamespaceState.DELETE_NAMESPACE_DELETE_DIRECTORIES); - break; - case DELETE_NAMESPACE_DELETE_DIRECTORIES: - deleteDirectory(env, namespaceName); - setNextState(DeleteNamespaceState.DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA); - break; - case DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA: - removeNamespaceQuota(env, namespaceName); - return Flow.NO_MORE_STATE; - default: - throw new UnsupportedOperationException(this + " unhandled state=" + state); + default: + throw new UnsupportedOperationException(this + " unhandled state=" + state); } } catch (IOException e) { if (isRollbackSupported(state)) { setFailure("master-delete-namespace", e); } else { - LOG.warn("Retriable error trying to delete namespace " + namespaceName + - " (in state=" + state + ")", e); + LOG.warn("Retriable error trying to delete namespace " + namespaceName + " (in state=" + + state + ")", e); } } return Flow.HAS_MORE_STATE; @@ -139,7 +132,7 @@ public class DeleteNamespaceProcedure @Override protected DeleteNamespaceState getState(final int stateId) { - return DeleteNamespaceState.valueOf(stateId); + return DeleteNamespaceState.forNumber(stateId); } @Override @@ -153,30 +146,28 @@ public class DeleteNamespaceProcedure } @Override - protected void serializeStateData(ProcedureStateSerializer serializer) - throws IOException { + protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { super.serializeStateData(serializer); MasterProcedureProtos.DeleteNamespaceStateData.Builder deleteNamespaceMsg = - MasterProcedureProtos.DeleteNamespaceStateData.newBuilder().setNamespaceName(namespaceName); + MasterProcedureProtos.DeleteNamespaceStateData.newBuilder().setNamespaceName(namespaceName); if (this.nsDescriptor != null) { - deleteNamespaceMsg.setNamespaceDescriptor( - ProtobufUtil.toProtoNamespaceDescriptor(this.nsDescriptor)); + deleteNamespaceMsg + .setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(this.nsDescriptor)); } serializer.serialize(deleteNamespaceMsg.build()); } @Override - protected void deserializeStateData(ProcedureStateSerializer serializer) - throws IOException { + protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { super.deserializeStateData(serializer); MasterProcedureProtos.DeleteNamespaceStateData deleteNamespaceMsg = - serializer.deserialize(MasterProcedureProtos.DeleteNamespaceStateData.class); + serializer.deserialize(MasterProcedureProtos.DeleteNamespaceStateData.class); namespaceName = deleteNamespaceMsg.getNamespaceName(); if (deleteNamespaceMsg.hasNamespaceDescriptor()) { nsDescriptor = - ProtobufUtil.toNamespaceDescriptor(deleteNamespaceMsg.getNamespaceDescriptor()); + ProtobufUtil.toNamespaceDescriptor(deleteNamespaceMsg.getNamespaceDescriptor()); } } @@ -193,7 +184,6 @@ public class DeleteNamespaceProcedure /** * Action before any real action of deleting namespace. * @param env MasterProcedureEnv - * @throws IOException */ private boolean prepareDelete(final MasterProcedureEnv env) throws IOException { if (getTableNamespaceManager(env).doesNamespaceExist(namespaceName) == false) { @@ -201,8 +191,8 @@ public class DeleteNamespaceProcedure return false; } if (NamespaceDescriptor.RESERVED_NAMESPACES.contains(namespaceName)) { - setFailure("master-delete-namespace", new ConstraintException( - "Reserved namespace "+ namespaceName +" cannot be removed.")); + setFailure("master-delete-namespace", + new ConstraintException("Reserved namespace " + namespaceName + " cannot be removed.")); return false; } @@ -214,9 +204,9 @@ public class DeleteNamespaceProcedure return false; } if (tableCount > 0) { - setFailure("master-delete-namespace", new ConstraintException( - "Only empty namespaces can be removed. Namespace "+ namespaceName + " has " - + tableCount +" tables")); + setFailure("master-delete-namespace", + new ConstraintException("Only empty namespaces can be removed. Namespace " + namespaceName + + " has " + tableCount + " tables")); return false; } @@ -226,76 +216,28 @@ public class DeleteNamespaceProcedure } /** - * delete the row from namespace table + * delete the row from the ns family in meta table. * @param env MasterProcedureEnv * @param namespaceName name of the namespace in string format - * @throws IOException */ - protected static void deleteFromNSTable( - final MasterProcedureEnv env, - final String namespaceName) throws IOException { - getTableNamespaceManager(env).removeFromNSTable(namespaceName); - } - - /** - * undo the delete - * @param env MasterProcedureEnv - * @throws IOException - */ - private void undoDeleteFromNSTable(final MasterProcedureEnv env) { - try { - if (nsDescriptor != null) { - CreateNamespaceProcedure.insertIntoNSTable(env, nsDescriptor); - } - } catch (Exception e) { - // Ignore - LOG.debug("Rollback of deleteFromNSTable throws exception: " + e); - } - } - - /** - * remove from ZooKeeper. - * @param env MasterProcedureEnv - * @param namespaceName name of the namespace in string format - * @throws IOException - */ - protected static void removeFromZKNamespaceManager( - final MasterProcedureEnv env, - final String namespaceName) throws IOException { - getTableNamespaceManager(env).removeFromZKNamespaceManager(namespaceName); - } - - /** - * undo the remove from ZooKeeper - * @param env MasterProcedureEnv - * @throws IOException - */ - private void undoRemoveFromZKNamespaceManager(final MasterProcedureEnv env) { - try { - if (nsDescriptor != null) { - CreateNamespaceProcedure.updateZKNamespaceManager(env, nsDescriptor); - } - } catch (Exception e) { - // Ignore - LOG.debug("Rollback of removeFromZKNamespaceManager throws exception: " + e); - } + private static void deleteNamespace(MasterProcedureEnv env, String namespaceName) + throws IOException { + getTableNamespaceManager(env).deleteNamespace(namespaceName); } /** * Delete the namespace directories from the file system * @param env MasterProcedureEnv * @param namespaceName name of the namespace in string format - * @throws IOException */ - protected static void deleteDirectory( - final MasterProcedureEnv env, - final String namespaceName) throws IOException { + private static void deleteDirectory(MasterProcedureEnv env, String namespaceName) + throws IOException { MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem(); FileSystem fs = mfs.getFileSystem(); Path p = FSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName); try { - for(FileStatus status : fs.listStatus(p)) { + for (FileStatus status : fs.listStatus(p)) { if (!HConstants.HBASE_NON_TABLE_DIRS.contains(status.getPath().getName())) { throw new IOException("Namespace directory contains table dir: " + status.getPath()); } @@ -309,58 +251,13 @@ public class DeleteNamespaceProcedure } } - /** - * undo delete directory - * @param env MasterProcedureEnv - * @throws IOException - */ - private void rollbackDeleteDirectory(final MasterProcedureEnv env) throws IOException { - try { - CreateNamespaceProcedure.createDirectory(env, nsDescriptor); - } catch (Exception e) { - // Ignore exception - LOG.debug("Rollback of deleteDirectory throws exception: " + e); - } - } - /** * remove quota for the namespace * @param env MasterProcedureEnv * @param namespaceName name of the namespace in string format - * @throws IOException **/ - protected static void removeNamespaceQuota( - final MasterProcedureEnv env, - final String namespaceName) throws IOException { + private static void removeNamespaceQuota(final MasterProcedureEnv env, final String namespaceName) + throws IOException { env.getMasterServices().getMasterQuotaManager().removeNamespaceQuota(namespaceName); } - - /** - * undo remove quota for the namespace - * @param env MasterProcedureEnv - * @throws IOException - **/ - private void rollbacRemoveNamespaceQuota(final MasterProcedureEnv env) throws IOException { - try { - CreateNamespaceProcedure.setNamespaceQuota(env, nsDescriptor); - } catch (Exception e) { - // Ignore exception - LOG.debug("Rollback of removeNamespaceQuota throws exception: " + e); - } - } - - private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) { - return env.getMasterServices().getClusterSchema().getTableNamespaceManager(); - } - /** - * The procedure could be restarted from a different machine. If the variable is null, we need to - * retrieve it. - * @return traceEnabled - */ - private Boolean isTraceEnabled() { - if (traceEnabled == null) { - traceEnabled = LOG.isTraceEnabled(); - } - return traceEnabled; - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java index 024f3ea4045..5ca7972fc5a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java @@ -17,6 +17,11 @@ */ package org.apache.hadoop.hbase.master.procedure; +import static org.apache.hadoop.hbase.NamespaceDescriptor.DEFAULT_NAMESPACE; +import static org.apache.hadoop.hbase.NamespaceDescriptor.SYSTEM_NAMESPACE; +import static org.apache.hadoop.hbase.master.TableNamespaceManager.insertNamespaceToMeta; +import static org.apache.hadoop.hbase.master.procedure.AbstractStateMachineNamespaceProcedure.createDirectory; + import java.io.IOException; import java.util.Arrays; import java.util.concurrent.CountDownLatch; @@ -25,11 +30,15 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; +import org.apache.hadoop.hbase.procedure2.ProcedureUtil; import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.InitMetaState; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.InitMetaStateData; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos; /** * This procedure is used to initialize meta table for a new hbase deploy. It will just schedule an @@ -38,8 +47,12 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.I @InterfaceAudience.Private public class InitMetaProcedure extends AbstractStateMachineTableProcedure { + private static final Logger LOG = LoggerFactory.getLogger(InitMetaProcedure.class); + private CountDownLatch latch = new CountDownLatch(1); + private int attempts; + @Override public TableName getTableName() { return TableName.META_TABLE_NAME; @@ -53,10 +66,32 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure { case ENABLE: return true; case EDIT: - // we allow concurrent edit on the NS table - return !proc.getTableName().equals(TableName.NAMESPACE_TABLE_NAME); + // we allow concurrent edit on the ns family in meta table + return !proc.getTableName().equals(TableProcedureInterface.DUMMY_NAMESPACE_TABLE_NAME); case READ: return false; // region operations are using the shared-lock on the table diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index c10a824b63e..4b6da53edd2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -1,4 +1,4 @@ -/* +/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -95,7 +95,6 @@ import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.RegionTooBusyException; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.UnknownScannerException; @@ -8298,8 +8297,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi */ public byte[] checkSplit() { // Can't split META - if (this.getRegionInfo().isMetaRegion() || - TableName.NAMESPACE_TABLE_NAME.equals(this.getRegionInfo().getTable())) { + if (this.getRegionInfo().isMetaRegion()) { if (shouldForceSplit()) { LOG.warn("Cannot split meta region in HBase 0.20 and above"); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 24743b91261..c6e3eee9904 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -779,13 +779,10 @@ public class HRegionServer extends HasThread implements } /** - * Create a 'smarter' Connection, one that is capable of by-passing RPC if the request is to - * the local server; i.e. a short-circuit Connection. Safe to use going to local or remote - * server. Create this instance in a method can be intercepted and mocked in tests. - * @throws IOException + * Create a 'smarter' Connection, one that is capable of by-passing RPC if the request is to the + * local server; i.e. a short-circuit Connection. Safe to use going to local or remote server. */ - @VisibleForTesting - protected ClusterConnection createClusterConnection() throws IOException { + private ClusterConnection createClusterConnection() throws IOException { Configuration conf = this.conf; if (conf.get(HConstants.CLIENT_ZOOKEEPER_QUORUM) != null) { // Use server ZK cluster for server-issued connections, so we clone @@ -796,8 +793,15 @@ public class HRegionServer extends HasThread implements // Create a cluster connection that when appropriate, can short-circuit and go directly to the // local server if the request is to the local server bypassing RPC. Can be used for both local // and remote invocations. - return ConnectionUtils.createShortCircuitConnection(conf, null, userProvider.getCurrent(), - serverName, rpcServices, rpcServices); + ClusterConnection conn = ConnectionUtils.createShortCircuitConnection(conf, null, + userProvider.getCurrent(), serverName, rpcServices, rpcServices); + // This is used to initialize the batch thread pool inside the connection implementation. + // When deploy a fresh cluster, we may first use the cluster connection in InitMetaProcedure, + // which will be executed inside the PEWorker, and then the batch thread pool will inherit the + // thread group of PEWorker, which will be destroy when shutting down the ProcedureExecutor. It + // will cause lots of procedure related UTs to fail, so here let's initialize it first, no harm. + conn.getTable(TableName.META_TABLE_NAME).close(); + return conn; } /** @@ -823,7 +827,6 @@ public class HRegionServer extends HasThread implements /** * Setup our cluster connection if not already initialized. - * @throws IOException */ protected synchronized void setupClusterConnection() throws IOException { if (clusterConnection == null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java index 1a84bfdea29..835fc0d0149 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java @@ -875,7 +875,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor, // wait till the acl table is created. if (AccessControlLists.isAclTable(desc)) { this.aclTabAvailable = true; - } else if (!(TableName.NAMESPACE_TABLE_NAME.equals(desc.getTableName()))) { + } else { if (!aclTabAvailable) { LOG.warn("Not adding owner permission for table " + desc.getTableName() + ". " + AccessControlLists.ACL_TABLE_NAME + " is not yet created. " diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java index 28c7ec3c2f3..3dce0de4afc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java @@ -178,6 +178,17 @@ public class FSTableDescriptors implements TableDescriptors { // Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore. .setBloomFilterType(BloomType.NONE) .build()) + .setColumnFamily(ColumnFamilyDescriptorBuilder + .newBuilder(HConstants.NAMESPACE_FAMILY) + .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS, + HConstants.DEFAULT_HBASE_META_VERSIONS)) + .setInMemory(true) + .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE, + HConstants.DEFAULT_HBASE_META_BLOCK_SIZE)) + .setScope(HConstants.REPLICATION_SCOPE_LOCAL) + // Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore. + .setBloomFilterType(BloomType.NONE) + .build()) .setCoprocessor(CoprocessorDescriptorBuilder.newBuilder( MultiRowMutationEndpoint.class.getName()) .setPriority(Coprocessor.PRIORITY_SYSTEM).build()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 3f383d6e9dd..655bbdbe015 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -3436,7 +3436,6 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { */ public void waitUntilAllSystemRegionsAssigned() throws IOException { waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME); - waitUntilAllRegionsAssigned(TableName.NAMESPACE_TABLE_NAME); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java index 9aecf06626a..44b06de18d5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java @@ -58,8 +58,8 @@ public class TestGlobalMemStoreSize { private static final Logger LOG = LoggerFactory.getLogger(TestGlobalMemStoreSize.class); private static int regionServerNum = 4; private static int regionNum = 16; - // total region num = region num + root and meta regions - private static int totalRegionNum = regionNum+2; + // total region num = region num + meta regions + private static int totalRegionNum = regionNum + 1; private HBaseTestingUtility TEST_UTIL; private MiniHBaseCluster cluster; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java index c9db891023a..5d5bc50ca33 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -63,7 +62,6 @@ public class TestNamespace { private static HBaseTestingUtility TEST_UTIL; protected static Admin admin; protected static HBaseCluster cluster; - private static ZKNamespaceManager zkNamespaceManager; private String prefix = "TestNamespace"; @Rule @@ -76,9 +74,6 @@ public class TestNamespace { admin = TEST_UTIL.getAdmin(); cluster = TEST_UTIL.getHBaseCluster(); master = ((MiniHBaseCluster)cluster).getMaster(); - zkNamespaceManager = - new ZKNamespaceManager(master.getZooKeeper()); - zkNamespaceManager.start(); LOG.info("Done initializing cluster"); } @@ -107,19 +102,16 @@ public class TestNamespace { admin.getNamespaceDescriptor(NamespaceDescriptor.DEFAULT_NAMESPACE.getName()); assertNotNull(ns); assertEquals(ns.getName(), NamespaceDescriptor.DEFAULT_NAMESPACE.getName()); - assertNotNull(zkNamespaceManager.get(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR)); ns = admin.getNamespaceDescriptor(NamespaceDescriptor.SYSTEM_NAMESPACE.getName()); assertNotNull(ns); assertEquals(ns.getName(), NamespaceDescriptor.SYSTEM_NAMESPACE.getName()); - assertNotNull(zkNamespaceManager.get(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)); assertEquals(2, admin.listNamespaceDescriptors().length); //verify existence of system tables Set systemTables = Sets.newHashSet( - TableName.META_TABLE_NAME, - TableName.NAMESPACE_TABLE_NAME); + TableName.META_TABLE_NAME); HTableDescriptor[] descs = admin.listTableDescriptorsByNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE.getName()); assertEquals(systemTables.size(), descs.length); @@ -181,18 +173,9 @@ public class TestNamespace { //create namespace and verify admin.createNamespace(NamespaceDescriptor.create(nsName).build()); assertEquals(3, admin.listNamespaceDescriptors().length); - TEST_UTIL.waitFor(60000, new Waiter.Predicate() { - @Override - public boolean evaluate() throws Exception { - return zkNamespaceManager.list().size() == 3; - } - }); - assertNotNull(zkNamespaceManager.get(nsName)); //remove namespace and verify admin.deleteNamespace(nsName); assertEquals(2, admin.listNamespaceDescriptors().length); - assertEquals(2, zkNamespaceManager.list().size()); - assertNull(zkNamespaceManager.get(nsName)); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNamespaceAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNamespaceAdminApi.java index f74b09551b0..8de32b86a34 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNamespaceAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNamespaceAdminApi.java @@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -29,14 +27,10 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NamespaceExistException; import org.apache.hadoop.hbase.NamespaceNotFoundException; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.Waiter; -import org.apache.hadoop.hbase.ZKNamespaceManager; -import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.BeforeClass; @@ -58,8 +52,6 @@ public class TestAsyncNamespaceAdminApi extends TestAsyncAdminBase { HBaseClassTestRule.forClass(TestAsyncNamespaceAdminApi.class); private String prefix = "TestNamespace"; - private static HMaster master; - private static ZKNamespaceManager zkNamespaceManager; @BeforeClass public static void setUpBeforeClass() throws Exception { @@ -69,9 +61,6 @@ public class TestAsyncNamespaceAdminApi extends TestAsyncAdminBase { TEST_UTIL.getConfiguration().setInt(START_LOG_ERRORS_AFTER_COUNT_KEY, 0); TEST_UTIL.startMiniCluster(1); ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get(); - master = TEST_UTIL.getHBaseCluster().getMaster(); - zkNamespaceManager = new ZKNamespaceManager(master.getZooKeeper()); - zkNamespaceManager.start(); LOG.info("Done initializing cluster"); } @@ -83,18 +72,9 @@ public class TestAsyncNamespaceAdminApi extends TestAsyncAdminBase { // create namespace and verify admin.createNamespace(NamespaceDescriptor.create(nsName).build()).join(); assertEquals(3, admin.listNamespaceDescriptors().get().size()); - TEST_UTIL.waitFor(60000, new Waiter.Predicate() { - @Override - public boolean evaluate() throws Exception { - return zkNamespaceManager.list().size() == 3; - } - }); - assertNotNull(zkNamespaceManager.get(nsName)); // delete namespace and verify admin.deleteNamespace(nsName).join(); assertEquals(2, admin.listNamespaceDescriptors().get().size()); - assertEquals(2, zkNamespaceManager.list().size()); - assertNull(zkNamespaceManager.get(nsName)); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java index c4285b44687..65bc3f619b0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java @@ -6396,7 +6396,7 @@ public class TestFromClientSide { scan.setCaching(1); // Filter out any records scan.setFilter(new FilterList(new FirstKeyOnlyFilter(), new InclusiveStopFilter(new byte[0]))); - try (Table table = TEST_UTIL.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME)) { + try (Table table = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) { try (ResultScanner s = table.getScanner(scan)) { assertNull(s.next()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java index a11064db9c3..388c53dc050 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java @@ -505,19 +505,19 @@ public abstract class AbstractTestDLS { LOG.debug("Waiting for no more RIT\n"); blockUntilNoRIT(zkw, master); NavigableSet regions = HBaseTestingUtility.getAllOnlineRegions(cluster); - LOG.debug("Verifying only catalog and namespace regions are assigned\n"); - if (regions.size() != 2) { + LOG.debug("Verifying only catalog region is assigned\n"); + if (regions.size() != 1) { for (String oregion : regions) LOG.debug("Region still online: " + oregion); } - assertEquals(2 + existingRegions, regions.size()); + assertEquals(1 + existingRegions, regions.size()); LOG.debug("Enabling table\n"); TEST_UTIL.getAdmin().enableTable(tableName); LOG.debug("Waiting for no more RIT\n"); blockUntilNoRIT(zkw, master); LOG.debug("Verifying there are " + numRegions + " assigned on cluster\n"); regions = HBaseTestingUtility.getAllOnlineRegions(cluster); - assertEquals(numRegions + 2 + existingRegions, regions.size()); + assertEquals(numRegions + 1 + existingRegions, regions.size()); return table; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java index b07950190db..75d9ee199b9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java @@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.master; import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CompatibilityFactory; -import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -42,40 +41,51 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos; -@Category({MasterTests.class, MediumTests.class}) +@Category({ MasterTests.class, MediumTests.class }) public class TestMasterMetrics { @ClassRule public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestMasterMetrics.class); + HBaseClassTestRule.forClass(TestMasterMetrics.class); private static final Logger LOG = LoggerFactory.getLogger(TestMasterMetrics.class); - private static final MetricsAssertHelper metricsHelper = CompatibilityFactory - .getInstance(MetricsAssertHelper.class); + private static final MetricsAssertHelper metricsHelper = + CompatibilityFactory.getInstance(MetricsAssertHelper.class); private static MiniHBaseCluster cluster; private static HMaster master; - private static HBaseTestingUtility TEST_UTIL; + private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); public static class MyMaster extends HMaster { + public MyMaster(Configuration conf) throws IOException, KeeperException, InterruptedException { super(conf); } + @Override - protected void tryRegionServerReport( - long reportStartTime, long reportEndTime) { + protected void tryRegionServerReport(long reportStartTime, long reportEndTime) { // do nothing } + } + public static class MyRegionServer extends MiniHBaseCluster.MiniHBaseClusterRegionServer { + + public MyRegionServer(Configuration conf) throws IOException, InterruptedException { + super(conf); + } + + @Override + protected void tryRegionServerReport(long reportStartTime, long reportEndTime) { + // do nothing + } } @BeforeClass public static void startCluster() throws Exception { LOG.info("Starting cluster"); - TEST_UTIL = new HBaseTestingUtility(); // Set master class and use default values for other options. - StartMiniClusterOption option = StartMiniClusterOption.builder() - .masterClass(MyMaster.class).build(); + StartMiniClusterOption option = StartMiniClusterOption.builder().masterClass(MyMaster.class) + .rsClass(MyRegionServer.class).build(); TEST_UTIL.startMiniCluster(option); cluster = TEST_UTIL.getHBaseCluster(); LOG.info("Waiting for active/ready master"); @@ -85,61 +95,44 @@ public class TestMasterMetrics { @AfterClass public static void after() throws Exception { - if (TEST_UTIL != null) { - TEST_UTIL.shutdownMiniCluster(); - } + master.stopMaster(); + TEST_UTIL.shutdownMiniCluster(); } @Test public void testClusterRequests() throws Exception { - // sending fake request to master to see how metric value has changed - RegionServerStatusProtos.RegionServerReportRequest.Builder request = - RegionServerStatusProtos.RegionServerReportRequest.newBuilder(); + RegionServerStatusProtos.RegionServerReportRequest.newBuilder(); ServerName serverName = cluster.getMaster(0).getServerName(); request.setServer(ProtobufUtil.toServerName(serverName)); long expectedRequestNumber = 10000; MetricsMasterSource masterSource = master.getMasterMetrics().getMetricsSource(); ClusterStatusProtos.ServerLoad sl = ClusterStatusProtos.ServerLoad.newBuilder() - .setTotalNumberOfRequests(expectedRequestNumber) - .build(); + .setTotalNumberOfRequests(expectedRequestNumber).build(); request.setLoad(sl); master.getMasterRpcServices().regionServerReport(null, request.build()); - boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration()); - if (tablesOnMaster) { - metricsHelper.assertCounter("cluster_requests", expectedRequestNumber, masterSource); - } else { - metricsHelper.assertCounterGt("cluster_requests", expectedRequestNumber, masterSource); - - } + metricsHelper.assertCounter("cluster_requests", expectedRequestNumber, masterSource); expectedRequestNumber = 15000; - sl = ClusterStatusProtos.ServerLoad.newBuilder() - .setTotalNumberOfRequests(expectedRequestNumber) - .build(); + sl = ClusterStatusProtos.ServerLoad.newBuilder().setTotalNumberOfRequests(expectedRequestNumber) + .build(); request.setLoad(sl); master.getMasterRpcServices().regionServerReport(null, request.build()); - if (tablesOnMaster) { - metricsHelper.assertCounter("cluster_requests", expectedRequestNumber, masterSource); - } else { - metricsHelper.assertCounterGt("cluster_requests", expectedRequestNumber, masterSource); - } - - master.stopMaster(); + metricsHelper.assertCounter("cluster_requests", expectedRequestNumber, masterSource); } @Test public void testDefaultMasterMetrics() throws Exception { MetricsMasterSource masterSource = master.getMasterMetrics().getMetricsSource(); boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration()); - metricsHelper.assertGauge( "numRegionServers",1 + (tablesOnMaster? 1: 0), masterSource); - metricsHelper.assertGauge( "averageLoad", 1 + (tablesOnMaster? 0: 1), masterSource); - metricsHelper.assertGauge( "numDeadRegionServers", 0, masterSource); + metricsHelper.assertGauge("numRegionServers", 1 + (tablesOnMaster ? 1 : 0), masterSource); + metricsHelper.assertGauge("averageLoad", 1, masterSource); + metricsHelper.assertGauge("numDeadRegionServers", 0, masterSource); metricsHelper.assertGauge("masterStartTime", master.getMasterStartTime(), masterSource); metricsHelper.assertGauge("masterActiveTime", master.getMasterActiveTime(), masterSource); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java index 3f4590c0424..4d5c946f715 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java @@ -200,7 +200,7 @@ public class TestMasterOperationsForRegionReplicas { StartMiniClusterOption option = StartMiniClusterOption.builder() .numRegionServers(numSlaves).rsPorts(rsports).build(); TEST_UTIL.startMiniHBaseCluster(option); - TEST_UTIL.waitTableEnabled(tableName); + TEST_UTIL.waitTableAvailable(tableName); validateFromSnapshotFromMeta(TEST_UTIL, tableName, numRegions, numReplica, ADMIN.getConnection()); @@ -208,7 +208,7 @@ public class TestMasterOperationsForRegionReplicas { // one server running TEST_UTIL.shutdownMiniHBaseCluster(); TEST_UTIL.startMiniHBaseCluster(); - TEST_UTIL.waitTableEnabled(tableName); + TEST_UTIL.waitTableAvailable(tableName); validateSingleRegionServerAssignment(ADMIN.getConnection(), numRegions, numReplica); for (int i = 1; i < numSlaves; i++) { //restore the cluster TEST_UTIL.getMiniHBaseCluster().startRegionServer(); @@ -334,7 +334,7 @@ public class TestMasterOperationsForRegionReplicas { connection); snapshot.initialize(); Map regionToServerMap = snapshot.getRegionToRegionServerMap(); - assert(regionToServerMap.size() == numRegions * numReplica + 1); //'1' for the namespace + assert(regionToServerMap.size() == numRegions * numReplica); Map> serverToRegionMap = snapshot.getRegionServerToRegionMap(); for (Map.Entry> entry : serverToRegionMap.entrySet()) { if (entry.getKey().equals(util.getHBaseCluster().getMaster().getServerName())) { @@ -361,14 +361,14 @@ public class TestMasterOperationsForRegionReplicas { connection); snapshot.initialize(); Map regionToServerMap = snapshot.getRegionToRegionServerMap(); - assertEquals(regionToServerMap.size(), numRegions * numReplica + 1); + assertEquals(regionToServerMap.size(), numRegions * numReplica); Map> serverToRegionMap = snapshot.getRegionServerToRegionMap(); assertEquals("One Region Only", 1, serverToRegionMap.keySet().size()); for (Map.Entry> entry : serverToRegionMap.entrySet()) { if (entry.getKey().equals(TEST_UTIL.getHBaseCluster().getMaster().getServerName())) { continue; } - assertEquals(entry.getValue().size(), numRegions * numReplica + 1); + assertEquals(entry.getValue().size(), numRegions * numReplica); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java index 665e5c6df66..fca2866e380 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java @@ -91,9 +91,8 @@ public class TestMasterRestartAfterDisablingTable { TEST_UTIL.getAdmin().disableTable(tableName); NavigableSet regions = HBaseTestingUtility.getAllOnlineRegions(cluster); - assertEquals( - "The number of regions for the table tableRestart should be 0 and only" - + "the catalog and namespace tables should be present.", 2, regions.size()); + assertEquals("The number of regions for the table tableRestart should be 0 and only" + + "the catalog table should be present.", 1, regions.size()); List masterThreads = cluster.getMasterThreads(); MasterThread activeMaster = null; @@ -120,12 +119,10 @@ public class TestMasterRestartAfterDisablingTable { TEST_UTIL.waitUntilNoRegionsInTransition(60000); log("Verifying there are " + numRegions + " assigned on cluster\n"); regions = HBaseTestingUtility.getAllOnlineRegions(cluster); - assertEquals("The assigned regions were not onlined after master" - + " switch except for the catalog and namespace tables.", - 6, regions.size()); - assertTrue("The table should be in enabled state", - cluster.getMaster().getTableStateManager() - .isTableState(TableName.valueOf(name.getMethodName()), TableState.State.ENABLED)); + assertEquals("The assigned regions were not onlined after master" + + " switch except for the catalog table.", 5, regions.size()); + assertTrue("The table should be in enabled state", cluster.getMaster().getTableStateManager() + .isTableState(TableName.valueOf(name.getMethodName()), TableState.State.ENABLED)); ht.close(); TEST_UTIL.shutdownMiniCluster(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java new file mode 100644 index 00000000000..d114317329e --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java @@ -0,0 +1,108 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.master; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.NamespaceDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineNamespaceProcedure; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; + +/** + * Testcase for HBASE-21154. + */ +@Category({ MasterTests.class, LargeTests.class }) +public class TestMigrateNamespaceTable { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMigrateNamespaceTable.class); + + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + + @BeforeClass + public static void setUp() throws Exception { + UTIL.startMiniCluster(1); + } + + @AfterClass + public static void tearDow() throws Exception { + UTIL.shutdownMiniCluster(); + } + + @Test + public void testMigrate() throws IOException, InterruptedException { + UTIL.getAdmin().createTable(TableDescriptorBuilder.NAMESPACE_TABLEDESC); + try (Table table = UTIL.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME)) { + for (int i = 0; i < 5; i++) { + NamespaceDescriptor nd = NamespaceDescriptor.create("Test-NS-" + i) + .addConfiguration("key-" + i, "value-" + i).build(); + table.put(new Put(Bytes.toBytes(nd.getName())).addColumn( + TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES, + TableDescriptorBuilder.NAMESPACE_COL_DESC_BYTES, + ProtobufUtil.toProtoNamespaceDescriptor(nd).toByteArray())); + AbstractStateMachineNamespaceProcedure + .createDirectory(UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem(), nd); + } + } + MasterThread masterThread = UTIL.getMiniHBaseCluster().getMasterThread(); + masterThread.getMaster().stop("For testing"); + masterThread.join(); + UTIL.getMiniHBaseCluster().startMaster(); + // 5 + default and system('hbase') + assertEquals(7, UTIL.getAdmin().listNamespaceDescriptors().length); + for (int i = 0; i < 5; i++) { + NamespaceDescriptor nd = UTIL.getAdmin().getNamespaceDescriptor("Test-NS-" + i); + assertEquals("Test-NS-" + i, nd.getName()); + assertEquals(1, nd.getConfiguration().size()); + assertEquals("value-" + i, nd.getConfigurationValue("key-" + i)); + } + UTIL.waitFor(30000, () -> UTIL.getAdmin().isTableDisabled(TableName.NAMESPACE_TABLE_NAME)); + + masterThread = UTIL.getMiniHBaseCluster().getMasterThread(); + masterThread.getMaster().stop("For testing"); + masterThread.join(); + + UTIL.getMiniHBaseCluster().startMaster(); + + // make sure that we could still restart the cluster after disabling the namespace table. + assertEquals(7, UTIL.getAdmin().listNamespaceDescriptors().length); + + // let's delete the namespace table + UTIL.getAdmin().deleteTable(TableName.NAMESPACE_TABLE_NAME); + assertFalse(UTIL.getAdmin().tableExists(TableName.NAMESPACE_TABLE_NAME)); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java index 04720cdaa02..4ba18764daf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java @@ -81,7 +81,7 @@ public class TestRestartCluster { } List allRegions = MetaTableAccessor.getAllRegions(UTIL.getConnection(), false); - assertEquals(4, allRegions.size()); + assertEquals(3, allRegions.size()); LOG.info("\n\nShutting down cluster"); UTIL.shutdownMiniHBaseCluster(); @@ -96,7 +96,7 @@ public class TestRestartCluster { // Otherwise we're reusing an Connection that has gone stale because // the shutdown of the cluster also called shut of the connection. allRegions = MetaTableAccessor.getAllRegions(UTIL.getConnection(), false); - assertEquals(4, allRegions.size()); + assertEquals(3, allRegions.size()); LOG.info("\n\nWaiting for tables to be available"); for(TableName TABLE: TABLES) { try { @@ -201,9 +201,6 @@ public class TestRestartCluster { snapshot.getRegionToRegionServerMap(); assertEquals(regionToRegionServerMap.size(), newRegionToRegionServerMap.size()); for (Map.Entry entry : newRegionToRegionServerMap.entrySet()) { - if (TableName.NAMESPACE_TABLE_NAME.equals(entry.getKey().getTable())) { - continue; - } ServerName oldServer = regionToRegionServerMap.get(entry.getKey()); ServerName currentServer = entry.getValue(); LOG.info( diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java index ff8ad0b339c..8a6f708f104 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java @@ -103,11 +103,13 @@ public class TestRollingRestart { log("Waiting for no more RIT\n"); TEST_UTIL.waitUntilNoRegionsInTransition(60000); NavigableSet regions = HBaseTestingUtility.getAllOnlineRegions(cluster); - log("Verifying only catalog and namespace regions are assigned\n"); - if (regions.size() != 2) { - for (String oregion : regions) log("Region still online: " + oregion); + log("Verifying only catalog region is assigned\n"); + if (regions.size() != 1) { + for (String oregion : regions) { + log("Region still online: " + oregion); + } } - assertEquals(2, regions.size()); + assertEquals(1, regions.size()); log("Enabling table\n"); TEST_UTIL.getAdmin().enableTable(tableName); log("Waiting for no more RIT\n"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionMoveAndAbandon.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionMoveAndAbandon.java index 05047c6dfca..bb95a6f417d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionMoveAndAbandon.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionMoveAndAbandon.java @@ -64,6 +64,7 @@ public class TestRegionMoveAndAbandon { private MiniZooKeeperCluster zkCluster; private HRegionServer rs1; private HRegionServer rs2; + private TableName tableName; private RegionInfo regionInfo; @Before @@ -75,10 +76,10 @@ public class TestRegionMoveAndAbandon { rs1 = cluster.getRegionServer(0); rs2 = cluster.getRegionServer(1); assertEquals(2, cluster.getRegionServerThreads().size()); - // We'll use hbase:namespace for our testing - UTIL.waitTableAvailable(TableName.NAMESPACE_TABLE_NAME, 30_000); - regionInfo = - Iterables.getOnlyElement(cluster.getRegions(TableName.NAMESPACE_TABLE_NAME)).getRegionInfo(); + tableName = TableName.valueOf(name.getMethodName()); + UTIL.createTable(tableName, Bytes.toBytes("cf")); + UTIL.waitTableAvailable(tableName, 30_000); + regionInfo = Iterables.getOnlyElement(cluster.getRegions(tableName)).getRegionInfo(); } @After @@ -105,7 +106,7 @@ public class TestRegionMoveAndAbandon { // Stop RS1 cluster.killRegionServer(rs1.getServerName()); // Region should get moved to RS2 - UTIL.waitTableAvailable(TableName.NAMESPACE_TABLE_NAME, 30_000); + UTIL.waitTableAvailable(tableName, 30_000); // Restart the master LOG.info("Killing master {}", cluster.getMaster().getServerName()); cluster.killMaster(cluster.getMaster().getServerName()); @@ -120,7 +121,7 @@ public class TestRegionMoveAndAbandon { UTIL.waitFor(30_000, new Waiter.Predicate() { @Override public boolean evaluate() throws Exception { - try (Table nsTable = UTIL.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME)) { + try (Table nsTable = UTIL.getConnection().getTable(tableName)) { // Doesn't matter what we're getting. We just want to make sure we can access the region nsTable.get(new Get(Bytes.toBytes("a"))); return true; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java index a74a3e5949b..4bc566cde21 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java @@ -160,7 +160,6 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase { LoadBalancer balancer = master.getLoadBalancer(); List regions = admin.getRegions(tableName); regions.addAll(admin.getTableRegions(TableName.META_TABLE_NAME)); - regions.addAll(admin.getTableRegions(TableName.NAMESPACE_TABLE_NAME)); List servers = Lists.newArrayList( admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet()); Map> map = balancer.roundRobinAssignment(regions, servers); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java index 66e72d0b038..484ebaae45b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java @@ -902,7 +902,7 @@ public class TestMasterProcedureScheduler { @Override public TableName getTableName() { - return TableName.NAMESPACE_TABLE_NAME; + return TableProcedureInterface.DUMMY_NAMESPACE_TABLE_NAME; } @Override @@ -1009,7 +1009,7 @@ public class TestMasterProcedureScheduler { LockedResource tableResource = locks.get(1); assertLockResource(tableResource, LockedResourceType.TABLE, - TableName.NAMESPACE_TABLE_NAME.getNameAsString()); + TableProcedureInterface.DUMMY_NAMESPACE_TABLE_NAME.getNameAsString()); assertSharedLock(tableResource, 1); assertTrue(tableResource.getWaitingProcedures().isEmpty()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyNamespaceProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyNamespaceProcedure.java index e76aacd247c..4622f79c43d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyNamespaceProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyNamespaceProcedure.java @@ -240,11 +240,9 @@ public class TestModifyNamespaceProcedure { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true); // Modify - nsd.setConfiguration(nsKey, nsValue); - // Start the Modify procedure && kill the executor - long procId = procExec.submitProcedure( - new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd)); + long procId = procExec.submitProcedure(new ModifyNamespaceProcedure(procExec.getEnvironment(), + NamespaceDescriptor.create(nsd).addConfiguration(nsKey, nsValue).build())); int lastStep = 2; // failing before MODIFY_NAMESPACE_UPDATE_NS_TABLE MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, lastStep); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java index 32fb173c2ea..9ebad27213f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java @@ -136,8 +136,6 @@ public class TestProcedurePriority { .stream().filter(t -> !t.getRegionServer().getRegions(TableName.META_TABLE_NAME).isEmpty()) .findAny().get(); HRegionServer rsNoMeta = UTIL.getOtherRegionServer(rsWithMetaThread.getRegionServer()); - // wait for NS table initialization to avoid our error inject affecting master initialization - UTIL.waitTableAvailable(TableName.NAMESPACE_TABLE_NAME); FAIL = true; UTIL.getMiniHBaseCluster().killRegionServer(rsNoMeta.getServerName()); // wait until all the worker thread are stuck, which means that the stuck checker will start to diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestLogRoller.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestLogRoller.java index e8c94236da2..7892d4478f2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestLogRoller.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestLogRoller.java @@ -24,9 +24,11 @@ import static org.junit.Assert.assertTrue; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.wal.FSHLog; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.junit.After; import org.junit.Before; @@ -49,6 +51,9 @@ public class TestLogRoller { public void setup() throws Exception { TEST_UTIL.getConfiguration().setInt("hbase.regionserver.logroll.period", logRollPeriod); TEST_UTIL.startMiniCluster(1); + TableName name = TableName.valueOf("Test"); + TEST_UTIL.createTable(name, Bytes.toBytes("cf")); + TEST_UTIL.waitTableAvailable(name); } @After diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java index 9bbce091440..0e7c019de66 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java @@ -443,7 +443,6 @@ public class TestPerColumnFamilyFlush { * When a log roll is about to happen, we do a flush of the regions who will be affected by the * log roll. These flushes cannot be a selective flushes, otherwise we cannot roll the logs. This * test ensures that we do a full-flush in that scenario. - * @throws IOException */ @Test public void testFlushingWhenLogRolling() throws Exception { @@ -467,12 +466,6 @@ public class TestPerColumnFamilyFlush { TEST_UTIL.startMiniCluster(numRegionServers); try { Table table = TEST_UTIL.createTable(tableName, FAMILIES); - // Force flush the namespace table so edits to it are not hanging around as oldest - // edits. Otherwise, below, when we make maximum number of WAL files, then it will be - // the namespace region that is flushed and not the below 'desiredRegion'. - try (Admin admin = TEST_UTIL.getConnection().getAdmin()) { - admin.flush(TableName.NAMESPACE_TABLE_NAME); - } Pair desiredRegionAndServer = getRegionWithName(tableName); final HRegion desiredRegion = desiredRegionAndServer.getFirst(); assertTrue("Could not find a region which hosts the new region.", desiredRegion != null); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithRestartScenarios.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithRestartScenarios.java index d8a907467d8..a4c09188ee2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithRestartScenarios.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithRestartScenarios.java @@ -135,7 +135,7 @@ public class TestRegionReplicasWithRestartScenarios { checkDuplicates(onlineRegions3); assertFalse(res); int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size(); - assertEquals(62, totalRegions); + assertEquals(61, totalRegions); } private boolean checkDuplicates(Collection onlineRegions3) throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java index 79ee15dba6d..40c3d290e30 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java @@ -239,7 +239,7 @@ public class TestRegionServerMetrics { @Test public void testRegionCount() throws Exception { - metricsHelper.assertGauge("regionCount", TABLES_ON_MASTER? 1: 3, serverSource); + metricsHelper.assertGauge("regionCount", TABLES_ON_MASTER ? 1 : 2, serverSource); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java index 83bd9abecd0..4c19aa0a824 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java @@ -254,9 +254,6 @@ public abstract class AbstractTestLogRolling { final WAL log = server.getWAL(region.getRegionInfo()); Store s = region.getStore(HConstants.CATALOG_FAMILY); - //have to flush namespace to ensure it doesn't affect wall tests - admin.flush(TableName.NAMESPACE_TABLE_NAME); - // Put some stuff into table, to make sure we have some files to compact. for (int i = 1; i <= 2; ++i) { doPut(table, i); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java index f2c5e50493a..7faaefb8f3b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java @@ -73,14 +73,7 @@ public class TestReplicationWALEntryFilters { assertNull(filter.filter(metaEntry)); - // ns table - WALKeyImpl key2 = - new WALKeyImpl(new byte[0], TableName.NAMESPACE_TABLE_NAME, System.currentTimeMillis()); - Entry nsEntry = new Entry(key2, null); - assertNull(filter.filter(nsEntry)); - // user table - WALKeyImpl key3 = new WALKeyImpl(new byte[0], TableName.valueOf("foo"), System.currentTimeMillis()); Entry userEntry = new Entry(key3, null); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java index 85ab16d6851..0855559d573 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java @@ -80,7 +80,6 @@ public class TestHBaseFsckMOB extends BaseTestHBaseFsck { admin.setBalancerRunning(false, true); TEST_UTIL.waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME); - TEST_UTIL.waitUntilAllRegionsAssigned(TableName.NAMESPACE_TABLE_NAME); } @AfterClass