diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index cba3f2dec1f..27e8875d861 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -304,11 +304,18 @@ public class MetaTableAccessor {
*/
public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)
throws IOException {
- byte[] row = getMetaKeyForRegion(regionInfo);
- Get get = new Get(row);
+ return getRegionLocation(getCatalogFamilyRow(connection, regionInfo),
+ regionInfo, regionInfo.getReplicaId());
+ }
+
+ /**
+ * @return Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta.
+ */
+ public static Result getCatalogFamilyRow(Connection connection, RegionInfo ri)
+ throws IOException {
+ Get get = new Get(getMetaKeyForRegion(ri));
get.addFamily(HConstants.CATALOG_FAMILY);
- Result r = get(getMetaHTable(connection), get);
- return getRegionLocation(r, regionInfo, regionInfo.getReplicaId());
+ return get(getMetaHTable(connection), get);
}
/** Returns the row key to use for this regionInfo */
@@ -1109,6 +1116,7 @@ public class MetaTableAccessor {
/**
* Updates state in META
+ * Do not use. For internal use only.
* @param conn connection to use
* @param tableName table to look for
*/
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 4a349d5591f..88022bb9cae 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -666,22 +666,35 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
new DisableTableProcedureBiConsumer(tableName));
}
+ /**
+ * Utility for completing passed TableState {@link CompletableFuture} future
+ * using passed parameters. Sets error or boolean result ('true' if table matches
+ * the passed-in targetState).
+ */
+ private static CompletableFuture completeCheckTableState(
+ CompletableFuture future, TableState tableState, Throwable error,
+ TableState.State targetState, TableName tableName) {
+ if (error != null) {
+ future.completeExceptionally(error);
+ } else {
+ if (tableState != null) {
+ future.complete(tableState.inStates(targetState));
+ } else {
+ future.completeExceptionally(new TableNotFoundException(tableName));
+ }
+ }
+ return future;
+ }
+
@Override
public CompletableFuture isTableEnabled(TableName tableName) {
if (TableName.isMetaTableName(tableName)) {
return CompletableFuture.completedFuture(true);
}
CompletableFuture future = new CompletableFuture<>();
- addListener(AsyncMetaTableAccessor.getTableState(metaTable, tableName), (state, error) -> {
- if (error != null) {
- future.completeExceptionally(error);
- return;
- }
- if (state.isPresent()) {
- future.complete(state.get().inStates(TableState.State.ENABLED));
- } else {
- future.completeExceptionally(new TableNotFoundException(tableName));
- }
+ addListener(AsyncMetaTableAccessor.getTableState(metaTable, tableName), (tableState, error) -> {
+ completeCheckTableState(future, tableState.isPresent()? tableState.get(): null, error,
+ TableState.State.ENABLED, tableName);
});
return future;
}
@@ -692,16 +705,9 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
return CompletableFuture.completedFuture(false);
}
CompletableFuture future = new CompletableFuture<>();
- addListener(AsyncMetaTableAccessor.getTableState(metaTable, tableName), (state, error) -> {
- if (error != null) {
- future.completeExceptionally(error);
- return;
- }
- if (state.isPresent()) {
- future.complete(state.get().inStates(TableState.State.DISABLED));
- } else {
- future.completeExceptionally(new TableNotFoundException(tableName));
- }
+ addListener(AsyncMetaTableAccessor.getTableState(metaTable, tableName), (tableState, error) -> {
+ completeCheckTableState(future, tableState.isPresent()? tableState.get(): null, error,
+ TableState.State.DISABLED, tableName);
});
return future;
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
index 08e3846e39d..b6bacc03e60 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
@@ -196,7 +196,7 @@ class ZKAsyncRegistry implements AsyncRegistry {
addListener(
zk.list(znodePaths.baseZNode)
.thenApply(children -> children.stream()
- .filter(c -> c.startsWith(znodePaths.metaZNodePrefix)).collect(Collectors.toList())),
+ .filter(c -> this.znodePaths.isMetaZNodePrefix(c)).collect(Collectors.toList())),
(metaReplicaZNodes, error) -> {
if (error != null) {
future.completeExceptionally(error);
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
index c8511d4dd61..98d7eb72eb0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.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
@@ -24,6 +24,7 @@ import static org.apache.hadoop.hbase.HConstants.SPLIT_LOGDIR_NAME;
import static org.apache.hadoop.hbase.HConstants.ZOOKEEPER_ZNODE_PARENT;
import static org.apache.hadoop.hbase.client.RegionInfo.DEFAULT_REPLICA_ID;
+import java.util.Collection;
import java.util.Optional;
import java.util.stream.IntStream;
import org.apache.hadoop.conf.Configuration;
@@ -40,15 +41,24 @@ public class ZNodePaths {
// TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved.
public static final char ZNODE_PATH_SEPARATOR = '/';
- public final static String META_ZNODE_PREFIX = "meta-region-server";
+ private static final String META_ZNODE_PREFIX = "meta-region-server";
private static final String DEFAULT_SNAPSHOT_CLEANUP_ZNODE = "snapshot-cleanup";
// base znode for this cluster
public final String baseZNode;
- // the prefix of meta znode, does not include baseZNode.
- public final String metaZNodePrefix;
- // znodes containing the locations of the servers hosting the meta replicas
- public final ImmutableMap metaReplicaZNodes;
+
+ /**
+ * The prefix of meta znode. Does not include baseZNode.
+ * Its a 'prefix' because meta replica id integer can be tagged on the end (if
+ * no number present, it is 'default' replica).
+ */
+ private final String metaZNodePrefix;
+
+ /**
+ * znodes containing the locations of the servers hosting the meta replicas
+ */
+ private final ImmutableMap metaReplicaZNodes;
+
// znode containing ephemeral nodes of the regionservers
public final String rsZNode;
// znode containing ephemeral nodes of the draining regionservers
@@ -154,21 +164,21 @@ public class ZNodePaths {
}
/**
- * Is the znode of any meta replica
- * @param node
- * @return true or false
+ * @return true if the znode is a meta region replica
*/
public boolean isAnyMetaReplicaZNode(String node) {
- if (metaReplicaZNodes.containsValue(node)) {
- return true;
- }
- return false;
+ return this.metaReplicaZNodes.containsValue(node);
}
/**
- * Get the znode string corresponding to a replicaId
- * @param replicaId
- * @return znode
+ * @return Meta Replica ZNodes
+ */
+ public Collection getMetaReplicaZNodes() {
+ return this.metaReplicaZNodes.values();
+ }
+
+ /**
+ * @return the znode string corresponding to a replicaId
*/
public String getZNodeForReplica(int replicaId) {
// return a newly created path but don't update the cache of paths
@@ -179,24 +189,21 @@ public class ZNodePaths {
}
/**
- * Parse the meta replicaId from the passed znode
+ * Parse the meta replicaId from the passed znode name.
* @param znode the name of the znode, does not include baseZNode
* @return replicaId
*/
public int getMetaReplicaIdFromZnode(String znode) {
- if (znode.equals(metaZNodePrefix)) {
- return RegionInfo.DEFAULT_REPLICA_ID;
- }
- return Integer.parseInt(znode.substring(metaZNodePrefix.length() + 1));
+ return znode.equals(metaZNodePrefix)?
+ RegionInfo.DEFAULT_REPLICA_ID:
+ Integer.parseInt(znode.substring(metaZNodePrefix.length() + 1));
}
/**
- * Is it the default meta replica's znode
- * @param znode the name of the znode, does not include baseZNode
- * @return true or false
+ * @return True if meta znode.
*/
- public boolean isDefaultMetaReplicaZnode(String znode) {
- return metaReplicaZNodes.get(DEFAULT_REPLICA_ID).equals(znode);
+ public boolean isMetaZNodePrefix(String znode) {
+ return znode != null && znode.startsWith(this.metaZNodePrefix);
}
/**
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 132d3e03be6..ce7abc8aef3 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
@@ -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
@@ -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;
@@ -1189,12 +1188,6 @@ public final class HConstants {
HBCK_SIDELINEDIR_NAME, HBASE_TEMP_DIRECTORY, MIGRATION_NAME
}));
- /** Directories that are not HBase user table directories */
- public static final List HBASE_NON_USER_TABLE_DIRS =
- Collections.unmodifiableList(Arrays.asList((String[])ArrayUtils.addAll(
- new String[] { TableName.META_TABLE_NAME.getNameAsString() },
- HBASE_NON_TABLE_DIRS.toArray())));
-
/** Health script related settings. */
public static final String HEALTH_SCRIPT_LOC = "hbase.node.health.script.location";
public static final String HEALTH_SCRIPT_TIMEOUT = "hbase.node.health.script.timeout";
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java
index 2537e7f83e7..85985814a2b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java
@@ -25,25 +25,19 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
/**
* Get, remove and modify table descriptors.
- * Used by servers to host descriptors.
*/
@InterfaceAudience.Private
public interface TableDescriptors {
/**
- * @param tableName
* @return TableDescriptor for tablename
- * @throws IOException
*/
- TableDescriptor get(final TableName tableName)
- throws IOException;
+ TableDescriptor get(final TableName tableName) throws IOException;
/**
* Get Map of all NamespaceDescriptors for a given namespace.
* @return Map of all descriptors.
- * @throws IOException
*/
- Map getByNamespace(String name)
- throws IOException;
+ Map getByNamespace(String name) throws IOException;
/**
* Get Map of all TableDescriptors. Populates the descriptor cache as a
@@ -51,25 +45,19 @@ public interface TableDescriptors {
* Notice: the key of map is the table name which contains namespace. It was generated by
* {@link TableName#getNameWithNamespaceInclAsString()}.
* @return Map of all descriptors.
- * @throws IOException
*/
Map getAll() throws IOException;
/**
* Add or update descriptor
* @param htd Descriptor to set into TableDescriptors
- * @throws IOException
*/
- void add(final TableDescriptor htd)
- throws IOException;
+ void update(final TableDescriptor htd) throws IOException;
/**
- * @param tablename
* @return Instance of table descriptor or null if none found.
- * @throws IOException
*/
- TableDescriptor remove(final TableName tablename)
- throws IOException;
+ TableDescriptor remove(final TableName tablename) throws IOException;
/**
* Enables the tabledescriptor cache
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 939e0ddcdff..b78a83571bc 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
@@ -999,7 +999,7 @@ public class HMaster extends HRegionServer implements MasterServices {
RegionState rs = this.assignmentManager.getRegionStates().
getRegionState(RegionInfoBuilder.FIRST_META_REGIONINFO);
LOG.info("hbase:meta {}", rs);
- if (rs.isOffline()) {
+ if (rs != null && rs.isOffline()) {
Optional optProc = procedureExecutor.getProcedures().stream()
.filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny();
initMetaProc = optProc.orElseGet(() -> {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
index 0ebd25beead..3c23610ff47 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.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
@@ -69,7 +69,7 @@ public class TableStateManager {
private final ConcurrentMap tableName2State =
new ConcurrentHashMap<>();
- public TableStateManager(MasterServices master) {
+ TableStateManager(MasterServices master) {
this.master = master;
}
@@ -88,61 +88,6 @@ public class TableStateManager {
}
}
- /**
- * Set table state to provided but only if table in specified states Caller should lock table on
- * write.
- * @param tableName table to change state for
- * @param newState new state
- * @param states states to check against
- * @return null if succeed or table state if failed
- */
- public TableState setTableStateIfInStates(TableName tableName, TableState.State newState,
- TableState.State... states) throws IOException {
- ReadWriteLock lock = tnLock.getLock(tableName);
- lock.writeLock().lock();
- try {
- TableState currentState = readMetaState(tableName);
- if (currentState == null) {
- throw new TableNotFoundException(tableName);
- }
- if (currentState.inStates(states)) {
- updateMetaState(tableName, newState);
- return null;
- } else {
- return currentState;
- }
- } finally {
- lock.writeLock().unlock();
- }
- }
-
- /**
- * Set table state to provided but only if table not in specified states Caller should lock table
- * on write.
- * @param tableName table to change state for
- * @param newState new state
- * @param states states to check against
- */
- public boolean setTableStateIfNotInStates(TableName tableName, TableState.State newState,
- TableState.State... states) throws IOException {
- ReadWriteLock lock = tnLock.getLock(tableName);
- lock.writeLock().lock();
- try {
- TableState currentState = readMetaState(tableName);
- if (currentState == null) {
- throw new TableNotFoundException(tableName);
- }
- if (!currentState.inStates(states)) {
- updateMetaState(tableName, newState);
- return true;
- } else {
- return false;
- }
- } finally {
- lock.writeLock().unlock();
- }
- }
-
public boolean isTableState(TableName tableName, TableState.State... states) {
try {
TableState tableState = getTableState(tableName);
@@ -156,6 +101,7 @@ public class TableStateManager {
public void setDeletedTable(TableName tableName) throws IOException {
if (tableName.equals(TableName.META_TABLE_NAME)) {
+ // Can't delete the hbase:meta table.
return;
}
ReadWriteLock lock = tnLock.getLock(tableName);
@@ -184,7 +130,7 @@ public class TableStateManager {
* @param states filter by states
* @return tables in given states
*/
- public Set getTablesInStates(TableState.State... states) throws IOException {
+ Set getTablesInStates(TableState.State... states) throws IOException {
// Only be called in region normalizer, will not use cache.
final Set rv = Sets.newHashSet();
MetaTableAccessor.fullScanTables(master.getConnection(), new MetaTableAccessor.Visitor() {
@@ -200,12 +146,6 @@ public class TableStateManager {
return rv;
}
- public static class TableStateNotFoundException extends TableNotFoundException {
- TableStateNotFoundException(TableName tableName) {
- super(tableName.getNameAsString());
- }
- }
-
@NonNull
public TableState getTableState(TableName tableName) throws IOException {
ReadWriteLock lock = tnLock.getLock(tableName);
@@ -213,7 +153,7 @@ public class TableStateManager {
try {
TableState currentState = readMetaState(tableName);
if (currentState == null) {
- throw new TableStateNotFoundException(tableName);
+ throw new TableNotFoundException("No state found for " + tableName);
}
return currentState;
} finally {
@@ -224,8 +164,8 @@ public class TableStateManager {
private void updateMetaState(TableName tableName, TableState.State newState) throws IOException {
if (tableName.equals(TableName.META_TABLE_NAME)) {
if (TableState.State.DISABLING.equals(newState) ||
- TableState.State.DISABLED.equals(newState)) {
- throw new IllegalArgumentIOException("Cannot disable the meta table; " + newState);
+ TableState.State.DISABLED.equals(newState)) {
+ throw new IllegalArgumentIOException("Cannot disable meta table; " + newState);
}
// Otherwise, just return; no need to set ENABLED on meta -- it is always ENABLED.
return;
@@ -237,7 +177,7 @@ public class TableStateManager {
succ = true;
} finally {
if (!succ) {
- tableName2State.remove(tableName);
+ this.tableName2State.remove(tableName);
}
}
metaStateUpdated(tableName, newState);
@@ -264,10 +204,8 @@ public class TableStateManager {
}
public void start() throws IOException {
- TableDescriptors tableDescriptors = master.getTableDescriptors();
migrateZooKeeper();
- Connection connection = master.getConnection();
- fixTableStates(tableDescriptors, connection);
+ fixTableStates(master.getTableDescriptors(), master.getConnection());
}
private void fixTableStates(TableDescriptors tableDescriptors, Connection connection)
@@ -336,7 +274,7 @@ public class TableStateManager {
TableState ts = null;
try {
ts = getTableState(entry.getKey());
- } catch (TableStateNotFoundException e) {
+ } catch (TableNotFoundException e) {
// This can happen; table exists but no TableState.
}
if (ts == null) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
index 60c407dbc7b..7d797320ad5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
@@ -146,8 +146,7 @@ public class RegionStateStore {
}
}
- public void updateRegionLocation(RegionStateNode regionStateNode)
- throws IOException {
+ void updateRegionLocation(RegionStateNode regionStateNode) throws IOException {
if (regionStateNode.getRegionInfo().isMetaRegion()) {
updateMetaLocation(regionStateNode.getRegionInfo(), regionStateNode.getRegionLocation(),
regionStateNode.getState());
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
index 34fde27d03c..a6c96a37e9e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
@@ -78,9 +78,7 @@ public class CreateTableProcedure
@Override
protected Flow executeFromState(final MasterProcedureEnv env, final CreateTableState state)
throws InterruptedException {
- if (LOG.isTraceEnabled()) {
- LOG.trace(this + " execute state=" + state);
- }
+ LOG.info("{} execute state={}", this, state);
try {
switch (state) {
case CREATE_TABLE_PRE_OPERATION:
@@ -320,8 +318,7 @@ public class CreateTableProcedure
// using a copy of descriptor, table will be created enabling first
final Path tempTableDir = FSUtils.getTableDir(tempdir, tableDescriptor.getTableName());
((FSTableDescriptors)(env.getMasterServices().getTableDescriptors()))
- .createTableDescriptorForTableDirectory(
- tempTableDir, tableDescriptor, false);
+ .createTableDescriptorForTableDirectory(tempTableDir, tableDescriptor, false);
// 2. Create Regions
newRegions = hdfsRegionHandler.createHdfsRegions(env, tempdir,
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
index 18c194f3287..a8a2a974bde 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
@@ -110,7 +110,7 @@ public class DisableTableProcedure
break;
case DISABLE_TABLE_ADD_REPLICATION_BARRIER:
if (env.getMasterServices().getTableDescriptors().get(tableName)
- .hasGlobalReplicationScope()) {
+ .hasGlobalReplicationScope()) {
MasterFileSystem fs = env.getMasterFileSystem();
try (BufferedMutator mutator = env.getMasterServices().getConnection()
.getBufferedMutator(TableName.META_TABLE_NAME)) {
@@ -243,7 +243,8 @@ public class DisableTableProcedure
private boolean prepareDisable(final MasterProcedureEnv env) throws IOException {
boolean canTableBeDisabled = true;
if (tableName.equals(TableName.META_TABLE_NAME)) {
- setFailure("master-disable-table", new ConstraintException("Cannot disable catalog table"));
+ setFailure("master-disable-table",
+ new ConstraintException("Cannot disable " + this.tableName));
canTableBeDisabled = false;
} else if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
setFailure("master-disable-table", new TableNotFoundException(tableName));
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java
index 06d6a2cfc52..614e4a203e1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.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
@@ -27,11 +27,9 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@@ -99,66 +97,55 @@ public class EnableTableProcedure
setNextState(EnableTableState.ENABLE_TABLE_MARK_REGIONS_ONLINE);
break;
case ENABLE_TABLE_MARK_REGIONS_ONLINE:
+ // Get the region replica count. If changed since disable, need to do
+ // more work assigning.
Connection connection = env.getMasterServices().getConnection();
- // we will need to get the tableDescriptor here to see if there is a change in the replica
- // count
- TableDescriptor hTableDescriptor =
+ TableDescriptor tableDescriptor =
env.getMasterServices().getTableDescriptors().get(tableName);
-
- // Get the replica count
- int regionReplicaCount = hTableDescriptor.getRegionReplication();
-
- // Get the regions for the table from memory; get both online and offline regions
- // ('true').
+ int configuredReplicaCount = tableDescriptor.getRegionReplication();
+ // Get regions for the table from memory; get both online and offline regions ('true').
List regionsOfTable =
env.getAssignmentManager().getRegionStates().getRegionsOfTable(tableName, true);
- int currentMaxReplica = 0;
- // Check if the regions in memory have replica regions as marked in META table
- for (RegionInfo regionInfo : regionsOfTable) {
- if (regionInfo.getReplicaId() > currentMaxReplica) {
- // Iterating through all the list to identify the highest replicaID region.
- // We can stop after checking with the first set of regions??
- currentMaxReplica = regionInfo.getReplicaId();
- }
- }
+ // How many replicas do we currently have? Check regions returned from
+ // in-memory state.
+ int currentMaxReplica = getMaxReplicaId(regionsOfTable);
- // read the META table to know the actual number of replicas for the table - if there
- // was a table modification on region replica then this will reflect the new entries also
- int replicasFound =
- getNumberOfReplicasFromMeta(connection, regionReplicaCount, regionsOfTable);
- assert regionReplicaCount - 1 == replicasFound;
- LOG.info(replicasFound + " META entries added for the given regionReplicaCount "
- + regionReplicaCount + " for the table " + tableName.getNameAsString());
- if (currentMaxReplica == (regionReplicaCount - 1)) {
+ // Read the META table to know the number of replicas the table currently has.
+ // If there was a table modification on region replica count then need to
+ // adjust replica counts here.
+ int replicasFound = TableName.isMetaTableName(this.tableName)?
+ 0: // TODO: Figure better what to do here for hbase:meta replica.
+ getReplicaCountInMeta(connection, configuredReplicaCount, regionsOfTable);
+ LOG.info("replicasFound={} (configuredReplicaCount={} for {}", replicasFound,
+ configuredReplicaCount, tableName.getNameAsString());
+ if (currentMaxReplica == (configuredReplicaCount - 1)) {
if (LOG.isDebugEnabled()) {
- LOG.debug("There is no change to the number of region replicas."
- + " Assigning the available regions." + " Current and previous"
- + "replica count is " + regionReplicaCount);
+ LOG.debug("No change in number of region replicas (configuredReplicaCount={});"
+ + " assigning.", configuredReplicaCount);
}
- } else if (currentMaxReplica > (regionReplicaCount - 1)) {
- // we have additional regions as the replica count has been decreased. Delete
+ } else if (currentMaxReplica > (configuredReplicaCount - 1)) {
+ // We have additional regions as the replica count has been decreased. Delete
// those regions because already the table is in the unassigned state
LOG.info("The number of replicas " + (currentMaxReplica + 1)
- + " is more than the region replica count " + regionReplicaCount);
+ + " is more than the region replica count " + configuredReplicaCount);
List copyOfRegions = new ArrayList(regionsOfTable);
for (RegionInfo regionInfo : copyOfRegions) {
- if (regionInfo.getReplicaId() > (regionReplicaCount - 1)) {
+ if (regionInfo.getReplicaId() > (configuredReplicaCount - 1)) {
// delete the region from the regionStates
env.getAssignmentManager().getRegionStates().deleteRegion(regionInfo);
// remove it from the list of regions of the table
- LOG.info("The regioninfo being removed is " + regionInfo + " "
- + regionInfo.getReplicaId());
+ LOG.info("Removed replica={} of {}", regionInfo.getRegionId(), regionInfo);
regionsOfTable.remove(regionInfo);
}
}
} else {
// the replicasFound is less than the regionReplication
- LOG.info("The number of replicas has been changed(increased)."
- + " Lets assign the new region replicas. The previous replica count was "
- + (currentMaxReplica + 1) + ". The current replica count is " + regionReplicaCount);
- regionsOfTable = RegionReplicaUtil.addReplicas(hTableDescriptor, regionsOfTable,
- currentMaxReplica + 1, regionReplicaCount);
+ LOG.info("Number of replicas has increased. Assigning new region replicas." +
+ "The previous replica count was {}. The current replica count is {}.",
+ (currentMaxReplica + 1), configuredReplicaCount);
+ regionsOfTable = RegionReplicaUtil.addReplicas(tableDescriptor, regionsOfTable,
+ currentMaxReplica + 1, configuredReplicaCount);
}
// Assign all the table regions. (including region replicas if added).
// createAssignProcedure will try to retain old assignments if possible.
@@ -186,9 +173,13 @@ public class EnableTableProcedure
return Flow.HAS_MORE_STATE;
}
- private int getNumberOfReplicasFromMeta(Connection connection, int regionReplicaCount,
+ /**
+ * @return Count of replicas found reading hbase:meta Region row or zk if
+ * asking about the hbase:meta table itself..
+ */
+ private int getReplicaCountInMeta(Connection connection, int regionReplicaCount,
List regionsOfTable) throws IOException {
- Result r = getRegionFromMeta(connection, regionsOfTable);
+ Result r = MetaTableAccessor.getCatalogFamilyRow(connection, regionsOfTable.get(0));
int replicasFound = 0;
for (int i = 1; i < regionReplicaCount; i++) {
// Since we have already added the entries to the META we will be getting only that here
@@ -201,16 +192,6 @@ public class EnableTableProcedure
return replicasFound;
}
- private Result getRegionFromMeta(Connection connection, List regionsOfTable)
- throws IOException {
- byte[] metaKeyForRegion = MetaTableAccessor.getMetaKeyForRegion(regionsOfTable.get(0));
- Get get = new Get(metaKeyForRegion);
- get.addFamily(HConstants.CATALOG_FAMILY);
- Table metaTable = MetaTableAccessor.getMetaHTable(connection);
- Result r = metaTable.get(get);
- return r;
- }
-
@Override
protected void rollbackState(final MasterProcedureEnv env, final EnableTableState state)
throws IOException {
@@ -408,4 +389,20 @@ public class EnableTableProcedure
}
}
}
+
+ /**
+ * @return Maximum region replica id found in passed list of regions.
+ */
+ private static int getMaxReplicaId(List regions) {
+ int max = 0;
+ for (RegionInfo regionInfo: regions) {
+ if (regionInfo.getReplicaId() > max) {
+ // Iterating through all the list to identify the highest replicaID region.
+ // We can stop after checking with the first set of regions??
+ max = regionInfo.getReplicaId();
+ }
+ }
+ return max;
+
+ }
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
index 8e435dd74fc..f90e06b2c20 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
@@ -19,6 +19,8 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@@ -41,6 +43,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
@@ -59,10 +62,19 @@ public class ModifyTableProcedure
private TableDescriptor modifiedTableDescriptor;
private boolean deleteColumnFamilyInModify;
private boolean shouldCheckDescriptor;
+ /**
+ * List of column families that cannot be deleted from the hbase:meta table.
+ * They are critical to cluster operation. This is a bit of an odd place to
+ * keep this list but then this is the tooling that does add/remove. Keeping
+ * it local!
+ */
+ private static final List UNDELETABLE_META_COLUMNFAMILIES =
+ Collections.unmodifiableList(Arrays.asList(
+ HConstants.CATALOG_FAMILY, HConstants.TABLE_FAMILY, HConstants.REPLICATION_BARRIER_FAMILY));
public ModifyTableProcedure() {
super();
- initilize(null, false);
+ initialize(null, false);
}
public ModifyTableProcedure(final MasterProcedureEnv env, final TableDescriptor htd)
@@ -81,12 +93,28 @@ public class ModifyTableProcedure
final TableDescriptor oldTableDescriptor, final boolean shouldCheckDescriptor)
throws HBaseIOException {
super(env, latch);
- initilize(oldTableDescriptor, shouldCheckDescriptor);
+ initialize(oldTableDescriptor, shouldCheckDescriptor);
this.modifiedTableDescriptor = newTableDescriptor;
preflightChecks(env, null/*No table checks; if changing peers, table can be online*/);
}
- private void initilize(final TableDescriptor unmodifiedTableDescriptor,
+ @Override
+ protected void preflightChecks(MasterProcedureEnv env, Boolean enabled) throws HBaseIOException {
+ super.preflightChecks(env, enabled);
+ if (this.modifiedTableDescriptor.isMetaTable()) {
+ // If we are modifying the hbase:meta table, make sure we are not deleting critical
+ // column families else we'll damage the cluster.
+ Set cfs = this.modifiedTableDescriptor.getColumnFamilyNames();
+ for (byte[] family : UNDELETABLE_META_COLUMNFAMILIES) {
+ if (!cfs.contains(family)) {
+ throw new HBaseIOException("Delete of hbase:meta column family " +
+ Bytes.toString(family));
+ }
+ }
+ }
+ }
+
+ private void initialize(final TableDescriptor unmodifiedTableDescriptor,
final boolean shouldCheckDescriptor) {
this.unmodifiedTableDescriptor = unmodifiedTableDescriptor;
this.shouldCheckDescriptor = shouldCheckDescriptor;
@@ -284,17 +312,27 @@ public class ModifyTableProcedure
"REGION_REPLICATION change is not supported for enabled tables");
}
}
+ this.deleteColumnFamilyInModify = isDeleteColumnFamily(unmodifiedTableDescriptor,
+ modifiedTableDescriptor);
+ }
- // Find out whether all column families in unmodifiedTableDescriptor also exists in
- // the modifiedTableDescriptor. This is to determine whether we are safe to rollback.
- final Set oldFamilies = unmodifiedTableDescriptor.getColumnFamilyNames();
- final Set newFamilies = modifiedTableDescriptor.getColumnFamilyNames();
- for (byte[] familyName : oldFamilies) {
+ /**
+ * Find out whether all column families in unmodifiedTableDescriptor also exists in
+ * the modifiedTableDescriptor.
+ * @return True if we are deleting a column family.
+ */
+ private static boolean isDeleteColumnFamily(TableDescriptor originalDescriptor,
+ TableDescriptor newDescriptor) {
+ boolean result = false;
+ final Set originalFamilies = originalDescriptor.getColumnFamilyNames();
+ final Set newFamilies = newDescriptor.getColumnFamilyNames();
+ for (byte[] familyName : originalFamilies) {
if (!newFamilies.contains(familyName)) {
- this.deleteColumnFamilyInModify = true;
+ result = true;
break;
}
}
+ return result;
}
/**
@@ -315,7 +353,7 @@ public class ModifyTableProcedure
* @throws IOException
**/
private void updateTableDescriptor(final MasterProcedureEnv env) throws IOException {
- env.getMasterServices().getTableDescriptors().add(modifiedTableDescriptor);
+ env.getMasterServices().getTableDescriptors().update(modifiedTableDescriptor);
}
/**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
index 244711de20a..4f6b1a1baed 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
@@ -368,7 +368,7 @@ public class RestoreSnapshotProcedure
* @throws IOException
**/
private void updateTableDescriptor(final MasterProcedureEnv env) throws IOException {
- env.getMasterServices().getTableDescriptors().add(modifiedTableDescriptor);
+ env.getMasterServices().getTableDescriptors().update(modifiedTableDescriptor);
}
/**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
index cdc0db28c4d..5ed7142c890 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
@@ -23,10 +23,10 @@ import java.util.HashMap;
import java.util.Map;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.master.TableStateManager;
-import org.apache.hadoop.hbase.master.TableStateManager.TableStateNotFoundException;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
@@ -139,7 +139,7 @@ public abstract class AbstractPeerProcedure extends AbstractPeerNoLockPr
return true;
}
Thread.sleep(SLEEP_INTERVAL_MS);
- } catch (TableStateNotFoundException e) {
+ } catch (TableNotFoundException e) {
return false;
} catch (InterruptedException e) {
throw (IOException) new InterruptedIOException(e.getMessage()).initCause(e);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index 007a2148362..8dd329f8569 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.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
@@ -20,10 +20,10 @@ package org.apache.hadoop.hbase.master.replication;
import java.io.IOException;
import java.io.InterruptedIOException;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.TableStateManager;
-import org.apache.hadoop.hbase.master.TableStateManager.TableStateNotFoundException;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
import org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure;
@@ -124,7 +124,7 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure getNodesToWatch() {
- return watcher.getZNodePaths().metaReplicaZNodes.values();
+ return watcher.getZNodePaths().getMetaReplicaZNodes();
}
-}
+}
\ No newline at end of file
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
index bf6699d5309..db0224ab4e8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
@@ -472,11 +472,10 @@ public final class SnapshotManifest {
public void consolidate() throws IOException {
if (getSnapshotFormat(desc) == SnapshotManifestV1.DESCRIPTOR_VERSION) {
- Path rootDir = FSUtils.getRootDir(conf);
LOG.info("Using old Snapshot Format");
// write a copy of descriptor to the snapshot directory
- new FSTableDescriptors(conf, workingDirFs, rootDir)
- .createTableDescriptorForTableDirectory(workingDir, htd, false);
+ FSTableDescriptors.createTableDescriptorForTableDirectory(workingDirFs, workingDir, htd,
+ false);
} else {
LOG.debug("Convert to Single Snapshot Manifest");
convertToV2SingleManifest();
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 8ae5b4b8ddd..7eb49cd2ad9 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
@@ -122,8 +122,9 @@ public class FSTableDescriptors implements TableDescriptors {
* @param fsreadonly True if we are read-only when it comes to filesystem
* operations; i.e. on remove, we do not do delete in fs.
*/
+ @VisibleForTesting
public FSTableDescriptors(final Configuration conf, final FileSystem fs,
- final Path rootdir, final boolean fsreadonly, final boolean usecache) throws IOException {
+ final Path rootdir, final boolean fsreadonly, final boolean usecache) throws IOException {
this(conf, fs, rootdir, fsreadonly, usecache, null);
}
@@ -135,18 +136,33 @@ public class FSTableDescriptors implements TableDescriptors {
* TODO: This is a workaround. Should remove this ugly code...
*/
public FSTableDescriptors(final Configuration conf, final FileSystem fs,
- final Path rootdir, final boolean fsreadonly, final boolean usecache,
- Function metaObserver) throws IOException {
+ final Path rootdir, final boolean fsreadonly, final boolean usecache,
+ Function metaObserver) throws IOException {
this.fs = fs;
this.rootdir = rootdir;
this.fsreadonly = fsreadonly;
this.usecache = usecache;
- this.metaTableDescriptor = metaObserver == null ? createMetaTableDescriptor(conf)
- : metaObserver.apply(createMetaTableDescriptorBuilder(conf)).build();
+ TableDescriptor td = null;
+ try {
+ td = getTableDescriptorFromFs(fs, rootdir, TableName.META_TABLE_NAME);
+ } catch (TableInfoMissingException e) {
+ td = metaObserver == null? createMetaTableDescriptor(conf):
+ metaObserver.apply(createMetaTableDescriptorBuilder(conf)).build();
+ if (!fsreadonly) {
+ LOG.info("Creating new hbase:meta table default descriptor/schema {}", td);
+ updateTableDescriptor(td);
+ }
+ }
+ this.metaTableDescriptor = td;
}
+ /**
+ *
+ * Make this private as soon as we've undone test dependency.
+ */
@VisibleForTesting
- public static TableDescriptorBuilder createMetaTableDescriptorBuilder(final Configuration conf) throws IOException {
+ public static TableDescriptorBuilder createMetaTableDescriptorBuilder(final Configuration conf)
+ throws IOException {
// TODO We used to set CacheDataInL1 for META table. When we have BucketCache in file mode, now
// the META table 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 META table CFs.
@@ -228,16 +244,6 @@ public class FSTableDescriptors implements TableDescriptors {
public TableDescriptor get(final TableName tablename)
throws IOException {
invocations++;
- if (TableName.META_TABLE_NAME.equals(tablename)) {
- cachehits++;
- return metaTableDescriptor;
- }
- // hbase:meta is already handled. If some one tries to get the descriptor for
- // .logs, .oldlogs or .corrupt throw an exception.
- if (HConstants.HBASE_NON_USER_TABLE_DIRS.contains(tablename.getNameAsString())) {
- throw new IOException("No descriptor found for non table = " + tablename);
- }
-
if (usecache) {
// Look in cache of descriptors.
TableDescriptor cachedtdm = this.cache.get(tablename);
@@ -332,19 +338,10 @@ public class FSTableDescriptors implements TableDescriptors {
* and updates the local cache with it.
*/
@Override
- public void add(TableDescriptor htd) throws IOException {
+ public void update(TableDescriptor htd) throws IOException {
if (fsreadonly) {
throw new NotImplementedException("Cannot add a table descriptor - in read only mode");
}
- TableName tableName = htd.getTableName();
- if (TableName.META_TABLE_NAME.equals(tableName)) {
- throw new NotImplementedException(HConstants.NOT_IMPLEMENTED);
- }
- if (HConstants.HBASE_NON_USER_TABLE_DIRS.contains(tableName.getNameAsString())) {
- throw new NotImplementedException(
- "Cannot add a table descriptor for a reserved subdirectory name: "
- + htd.getTableName().getNameAsString());
- }
updateTableDescriptor(htd);
}
@@ -369,26 +366,6 @@ public class FSTableDescriptors implements TableDescriptors {
return descriptor;
}
- /**
- * Checks if a current table info file exists for the given table
- *
- * @param tableName name of table
- * @return true if exists
- * @throws IOException
- */
- public boolean isTableInfoExists(TableName tableName) throws IOException {
- return getTableInfoPath(tableName) != null;
- }
-
- /**
- * Find the most current table info file for the given table in the hbase root directory.
- * @return The file status of the current table info file or null if it does not exist
- */
- private FileStatus getTableInfoPath(final TableName tableName) throws IOException {
- Path tableDir = getTableDir(tableName);
- return getTableInfoPath(tableDir);
- }
-
private FileStatus getTableInfoPath(Path tableDir)
throws IOException {
return getTableInfoPath(fs, tableDir, !fsreadonly);
@@ -403,7 +380,6 @@ public class FSTableDescriptors implements TableDescriptors {
* were sequence numbers).
*
* @return The file status of the current table info file or null if it does not exist
- * @throws IOException
*/
public static FileStatus getTableInfoPath(FileSystem fs, Path tableDir)
throws IOException {
@@ -421,7 +397,6 @@ public class FSTableDescriptors implements TableDescriptors {
* older files.
*
* @return The file status of the current table info file or null if none exist
- * @throws IOException
*/
private static FileStatus getTableInfoPath(FileSystem fs, Path tableDir, boolean removeOldFiles)
throws IOException {
@@ -609,21 +584,6 @@ public class FSTableDescriptors implements TableDescriptors {
return p;
}
- /**
- * Deletes all the table descriptor files from the file system.
- * Used in unit tests only.
- * @throws NotImplementedException if in read only mode
- */
- public void deleteTableDescriptorIfExists(TableName tableName) throws IOException {
- if (fsreadonly) {
- throw new NotImplementedException("Cannot delete a table descriptor - in read only mode");
- }
-
- Path tableDir = getTableDir(tableName);
- Path tableInfoDir = new Path(tableDir, TABLEINFO_DIR);
- deleteTableDescriptorFiles(fs, tableInfoDir, Integer.MAX_VALUE);
- }
-
/**
* Deletes files matching the table info file pattern within the given directory
* whose sequenceId is at most the given max sequenceId.
@@ -746,7 +706,8 @@ public class FSTableDescriptors implements TableDescriptors {
/**
* Create a new TableDescriptor in HDFS in the specified table directory. Happens when we create
- * a new table or snapshot a table.
+ * a new table during cluster start or in Clone and Create Table Procedures. Checks readOnly flag
+ * passed on construction.
* @param tableDir table directory under which we should write the file
* @param htd description of the table to write
* @param forceCreation if true,then even if previous table descriptor is present it will
@@ -755,11 +716,28 @@ public class FSTableDescriptors implements TableDescriptors {
* already exists and we weren't forcing the descriptor creation.
* @throws IOException if a filesystem error occurs
*/
- public boolean createTableDescriptorForTableDirectory(Path tableDir,
- TableDescriptor htd, boolean forceCreation) throws IOException {
- if (fsreadonly) {
+ public boolean createTableDescriptorForTableDirectory(Path tableDir, TableDescriptor htd,
+ boolean forceCreation) throws IOException {
+ if (this.fsreadonly) {
throw new NotImplementedException("Cannot create a table descriptor - in read only mode");
}
+ return createTableDescriptorForTableDirectory(this.fs, tableDir, htd, forceCreation);
+ }
+
+ /**
+ * Create a new TableDescriptor in HDFS in the specified table directory. Happens when we create
+ * a new table snapshoting. Does not enforce read-only. That is for caller to determine.
+ * @param fs Filesystem to use.
+ * @param tableDir table directory under which we should write the file
+ * @param htd description of the table to write
+ * @param forceCreation if true,then even if previous table descriptor is present it will
+ * be overwritten
+ * @return true if the we successfully created the file, false if the file
+ * already exists and we weren't forcing the descriptor creation.
+ * @throws IOException if a filesystem error occurs
+ */
+ public static boolean createTableDescriptorForTableDirectory(FileSystem fs,
+ Path tableDir, TableDescriptor htd, boolean forceCreation) throws IOException {
FileStatus status = getTableInfoPath(fs, tableDir);
if (status != null) {
LOG.debug("Current path=" + status.getPath());
@@ -772,9 +750,7 @@ public class FSTableDescriptors implements TableDescriptors {
}
}
}
- Path p = writeTableDescriptor(fs, htd, tableDir, status);
- return p != null;
+ return writeTableDescriptor(fs, htd, tableDir, status) != null;
}
-
}
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 79160cdd0ac..1f2e1b92d9d 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
@@ -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
@@ -492,17 +492,6 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
return new Path(fs.getWorkingDirectory(), "test-data");
}
- /**
- * @return META table descriptor
- */
- public TableDescriptorBuilder getMetaTableDescriptorBuilder() {
- try {
- return FSTableDescriptors.createMetaTableDescriptorBuilder(conf);
- } catch (IOException e) {
- throw new RuntimeException("Unable to create META table descriptor", e);
- }
- }
-
/**
* Returns a Path in the test filesystem, obtained from {@link #getTestFileSystem()}
* to write temporary test data. Call this method after setting up the mini dfs cluster
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
index c01fc45d5fd..57da486967a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
@@ -67,7 +67,7 @@ public class TestFSTableDescriptorForceCreation {
Path rootdir = new Path(UTIL.getDataTestDir(), name);
FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build();
- fstd.add(htd);
+ fstd.update(htd);
assertFalse("Should not create new table descriptor", fstd.createTableDescriptor(htd, false));
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java
new file mode 100644
index 00000000000..bbdb327ed04
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java
@@ -0,0 +1,119 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Test being able to edit hbase:meta.
+ */
+@Category({MiscTests.class, LargeTests.class})
+public class TestHBaseMetaEdit {
+ @ClassRule
+ public static final HBaseClassTestRule CLASS_RULE =
+ HBaseClassTestRule.forClass(TestHBaseMetaEdit.class);
+ @Rule
+ public TestName name = new TestName();
+ private final static HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+ @Before
+ public void before() throws Exception {
+ UTIL.startMiniCluster();
+ }
+
+ @After
+ public void after() throws Exception {
+ UTIL.shutdownMiniCluster();
+ }
+
+ /**
+ * Set versions, set HBASE-16213 indexed block encoding, and add a column family.
+ * Delete the column family. Then try to delete a core hbase:meta family (should fail).
+ * Verify they are all in place by looking at TableDescriptor AND by checking
+ * what the RegionServer sees after opening Region.
+ */
+ @Test
+ public void testEditMeta() throws IOException {
+ Admin admin = UTIL.getAdmin();
+ admin.tableExists(TableName.META_TABLE_NAME);
+ TableDescriptor originalDescriptor = admin.getDescriptor(TableName.META_TABLE_NAME);
+ ColumnFamilyDescriptor cfd = originalDescriptor.getColumnFamily(HConstants.CATALOG_FAMILY);
+ int oldVersions = cfd.getMaxVersions();
+ // Add '1' to current versions count. Set encoding too.
+ cfd = ColumnFamilyDescriptorBuilder.newBuilder(cfd).setMaxVersions(oldVersions + 1).
+ setConfiguration(ColumnFamilyDescriptorBuilder.DATA_BLOCK_ENCODING,
+ DataBlockEncoding.ROW_INDEX_V1.toString()).build();
+ admin.modifyColumnFamily(TableName.META_TABLE_NAME, cfd);
+ byte [] extraColumnFamilyName = Bytes.toBytes("xtra");
+ ColumnFamilyDescriptor newCfd =
+ ColumnFamilyDescriptorBuilder.newBuilder(extraColumnFamilyName).build();
+ admin.addColumnFamily(TableName.META_TABLE_NAME, newCfd);
+ TableDescriptor descriptor = admin.getDescriptor(TableName.META_TABLE_NAME);
+ // Assert new max versions is == old versions plus 1.
+ assertEquals(oldVersions + 1,
+ descriptor.getColumnFamily(HConstants.CATALOG_FAMILY).getMaxVersions());
+ descriptor = admin.getDescriptor(TableName.META_TABLE_NAME);
+ // Assert new max versions is == old versions plus 1.
+ assertEquals(oldVersions + 1,
+ descriptor.getColumnFamily(HConstants.CATALOG_FAMILY).getMaxVersions());
+ assertTrue(descriptor.getColumnFamily(newCfd.getName()) != null);
+ String encoding = descriptor.getColumnFamily(HConstants.CATALOG_FAMILY).getConfiguration().
+ get(ColumnFamilyDescriptorBuilder.DATA_BLOCK_ENCODING);
+ assertEquals(encoding, DataBlockEncoding.ROW_INDEX_V1.toString());
+ Region r = UTIL.getHBaseCluster().getRegionServer(0).
+ getRegion(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName());
+ assertEquals(oldVersions + 1,
+ r.getStore(HConstants.CATALOG_FAMILY).getColumnFamilyDescriptor().getMaxVersions());
+ encoding = r.getStore(HConstants.CATALOG_FAMILY).getColumnFamilyDescriptor().
+ getConfigurationValue(ColumnFamilyDescriptorBuilder.DATA_BLOCK_ENCODING);
+ assertEquals(encoding, DataBlockEncoding.ROW_INDEX_V1.toString());
+ assertTrue(r.getStore(extraColumnFamilyName) != null);
+ // Assert we can't drop critical hbase:meta column family but we can drop any other.
+ admin.deleteColumnFamily(TableName.META_TABLE_NAME, newCfd.getName());
+ descriptor = admin.getDescriptor(TableName.META_TABLE_NAME);
+ assertTrue(descriptor.getColumnFamily(newCfd.getName()) == null);
+ try {
+ admin.deleteColumnFamily(TableName.META_TABLE_NAME, HConstants.CATALOG_FAMILY);
+ fail("Should not reach here");
+ } catch (HBaseIOException hioe) {
+ assertTrue(hioe.getMessage().contains("Delete of hbase:meta"));
+ }
+ }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
index bc2a9fe84de..61dfda7448c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
@@ -327,7 +327,7 @@ public class MockMasterServices extends MockNoopMasterServices {
}
@Override
- public void add(TableDescriptor htd) throws IOException {
+ public void update(TableDescriptor htd) throws IOException {
// noop
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java
index 3da03416b44..1d31a4a37dd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java
@@ -36,17 +36,20 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
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.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.wal.WAL;
import org.junit.ClassRule;
import org.junit.Rule;
@@ -88,11 +91,11 @@ public class TestGetClosestAtOrBefore {
FileSystem filesystem = FileSystem.get(conf);
Path rootdir = UTIL.getDataTestDirOnTestFS();
// Up flush size else we bind up when we use default catalog flush of 16k.
- TableDescriptorBuilder metaBuilder = UTIL.getMetaTableDescriptorBuilder()
- .setMemStoreFlushSize(64 * 1024 * 1024);
-
+ TableDescriptors tds = new FSTableDescriptors(UTIL.getConfiguration());
+ TableDescriptor td = tds.get(TableName.META_TABLE_NAME);
+ td = TableDescriptorBuilder.newBuilder(td).setMemStoreFlushSize(64 * 1024 * 1024).build();
HRegion mr = HBaseTestingUtility.createRegionAndWAL(HRegionInfo.FIRST_META_REGIONINFO,
- rootdir, this.conf, metaBuilder.build());
+ rootdir, this.conf, td);
try {
// Write rows for three tables 'A', 'B', and 'C'.
for (char c = 'A'; c < 'D'; c++) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
index 63c3de18ef1..b852168b5ae 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
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.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.wal.WAL;
@@ -90,6 +91,9 @@ public class TestLogRollingNoCluster {
final Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
conf.set(WALFactory.WAL_PROVIDER, "filesystem");
FSUtils.setRootDir(conf, dir);
+ FSTableDescriptors fsTableDescriptors =
+ new FSTableDescriptors(TEST_UTIL.getConfiguration());
+ TableDescriptor metaTableDescriptor = fsTableDescriptors.get(TableName.META_TABLE_NAME);
conf.set("hbase.regionserver.hlog.writer.impl", HighLatencySyncWriter.class.getName());
final WALFactory wals = new WALFactory(conf, TestLogRollingNoCluster.class.getName());
final WAL wal = wals.getWAL(null);
@@ -101,7 +105,7 @@ public class TestLogRollingNoCluster {
try {
for (int i = 0; i < numThreads; i++) {
// Have each appending thread write 'count' entries
- appenders[i] = new Appender(wal, i, NUM_ENTRIES);
+ appenders[i] = new Appender(metaTableDescriptor, wal, i, NUM_ENTRIES);
}
for (int i = 0; i < numThreads; i++) {
appenders[i].start();
@@ -114,7 +118,7 @@ public class TestLogRollingNoCluster {
wals.close();
}
for (int i = 0; i < numThreads; i++) {
- assertFalse(appenders[i].isException());
+ assertFalse("Error: " + appenders[i].getException(), appenders[i].isException());
}
TEST_UTIL.shutdownMiniDFSCluster();
}
@@ -127,11 +131,13 @@ public class TestLogRollingNoCluster {
private final WAL wal;
private final int count;
private Exception e = null;
+ private final TableDescriptor metaTableDescriptor;
- Appender(final WAL wal, final int index, final int count) {
+ Appender(TableDescriptor metaTableDescriptor, final WAL wal, final int index, final int count) {
super("" + index);
this.wal = wal;
this.count = count;
+ this.metaTableDescriptor = metaTableDescriptor;
this.log = LoggerFactory.getLogger("Appender:" + getName());
}
@@ -161,9 +167,8 @@ public class TestLogRollingNoCluster {
byte[] bytes = Bytes.toBytes(i);
edit.add(new KeyValue(bytes, bytes, bytes, now, EMPTY_1K_ARRAY));
RegionInfo hri = RegionInfoBuilder.FIRST_META_REGIONINFO;
- TableDescriptor htd = TEST_UTIL.getMetaTableDescriptorBuilder().build();
NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
- for(byte[] fam : htd.getColumnFamilyNames()) {
+ for(byte[] fam: this.metaTableDescriptor.getColumnFamilyNames()) {
scopes.put(fam, 0);
}
final long txid = wal.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(),
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
index a0c37f27d4e..718cec39b84 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
@@ -169,7 +169,7 @@ public class TestFSTableDescriptors {
Path rootdir = new Path(UTIL.getDataTestDir(), name);
TableDescriptors htds = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build();
- htds.add(htd);
+ htds.update(htd);
assertNotNull(htds.remove(htd.getTableName()));
assertNull(htds.remove(htd.getTableName()));
}
@@ -322,8 +322,13 @@ public class TestFSTableDescriptors {
}
Map tables = tds.getAll();
+ // Remove hbase:meta from list. It shows up now since we made it dynamic. The schema
+ // is written into the fs by the FSTableDescriptors constructor now where before it
+ // didn't.
+ tables.remove(TableName.META_TABLE_NAME.getNameAsString());
assertEquals(4, tables.size());
+
String[] tableNamesOrdered =
new String[] { "bar:foo", "default:bar", "default:foo", "foo:bar" };
int i = 0;
@@ -359,12 +364,13 @@ public class TestFSTableDescriptors {
assertTrue(nonchtds.getAll().size() == chtds.getAll().size());
- // add a new entry for hbase:meta
- TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).build();
+ // add a new entry for random table name.
+ TableName random = TableName.valueOf("random");
+ TableDescriptor htd = TableDescriptorBuilder.newBuilder(random).build();
nonchtds.createTableDescriptor(htd);
- // hbase:meta will only increase the cachehit by 1
- assertTrue(nonchtds.getAll().size() == chtds.getAll().size());
+ // random will only increase the cachehit by 1
+ assertEquals(nonchtds.getAll().size(), chtds.getAll().size() + 1);
for (Map.Entry entry: nonchtds.getAll().entrySet()) {
String t = (String) entry.getKey();
@@ -394,9 +400,9 @@ public class TestFSTableDescriptors {
Path rootdir = new Path(UTIL.getDataTestDir(), name);
TableDescriptors htds = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build();
- htds.add(htd);
- htds.add(htd);
- htds.add(htd);
+ htds.update(htd);
+ htds.update(htd);
+ htds.update(htd);
}
@Test
diff --git a/hbase-shell/src/main/ruby/shell/commands/describe.rb b/hbase-shell/src/main/ruby/shell/commands/describe.rb
index 5ef02a03a84..0553755daba 100644
--- a/hbase-shell/src/main/ruby/shell/commands/describe.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/describe.rb
@@ -44,12 +44,15 @@ EOF
puts
end
formatter.footer
- puts
- formatter.header(%w[QUOTAS])
- count = quotas_admin.list_quotas(TABLE => table.to_s) do |_, quota|
- formatter.row([quota])
+ if table.to_s != 'hbase:meta'
+ # No QUOTAS if hbase:meta table
+ puts
+ formatter.header(%w[QUOTAS])
+ count = quotas_admin.list_quotas(TABLE => table.to_s) do |_, quota|
+ formatter.row([quota])
+ end
+ formatter.footer(count)
end
- formatter.footer(count)
end
# rubocop:enable Metrics/AbcSize, Metrics/MethodLength
end
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
index 10fd16582c7..9105d7e149f 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.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
@@ -61,14 +61,6 @@ public final class MetaTableLocator {
private MetaTableLocator() {
}
- /**
- * Checks if the meta region location is available.
- * @return true if meta region location is available, false if not
- */
- public static boolean isLocationAvailable(ZKWatcher zkw) {
- return getMetaRegionLocation(zkw) != null;
- }
-
/**
* @param zkw ZooKeeper watcher to be used
* @return meta table regions and their locations.
@@ -266,7 +258,7 @@ public final class MetaTableLocator {
}
/**
- * Load the meta region state from the meta server ZNode.
+ * Load the meta region state from the meta region server ZNode.
*
* @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @param replicaId the ID of the replica
@@ -306,10 +298,8 @@ public final class MetaTableLocator {
if (serverName == null) {
state = RegionState.State.OFFLINE;
}
- return new RegionState(
- RegionReplicaUtil.getRegionInfoForReplica(
- RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId),
- state, serverName);
+ return new RegionState(RegionReplicaUtil.getRegionInfoForReplica(
+ RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), state, serverName);
}
/**
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
index 5f2185e8287..5c9fcaba29d 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
@@ -2056,7 +2056,7 @@ public final class ZKUtil {
" byte(s) of data from znode " + znode +
(watcherSet? " and set watcher; ": "; data=") +
(data == null? "null": data.length == 0? "empty": (
- znode.startsWith(zkw.getZNodePaths().metaZNodePrefix)?
+ zkw.getZNodePaths().isMetaZNodePrefix(znode)?
getServerNameOrEmptyString(data):
znode.startsWith(zkw.getZNodePaths().backupMasterAddressesZNode)?
getServerNameOrEmptyString(data):