HBASE-23055 Alter hbase:meta (#667)

Make it so hbase:meta can be altered. TableState for hbase:meta
    is kept in Master. State is in-memory transient so if  Master
    fails, hbase:meta is ENABLED again. hbase:meta schema will be
    bootstrapped from the filesystem. Changes to filesystem schema
    are atomic so we should be ok if Master fails mid-edit (TBD)
    Undoes a bunch of guards that prevented our being able to edit
    hbase:meta. At minimmum, need to add in a bunch of WARNING.

    TODO: Tests, more clarity around hbase:meta table state, and undoing
    references to hard-coded hbase:meta regioninfo.

    M hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
     Throw illegal access exception if you try to use MetaTableAccessor
     getting state of the hbase:meta table.

    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
     For table state, go to master rather than go to meta direct. Going
     to meta won't work for hbase;meta state. Puts load on Master.

    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
     Change isTableDisabled/Enabled implementation to ask the Master instead.
     This will give the Master's TableStateManager's opinion rather than
     client figuring it for themselves reading meta table direct.

    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
     TODO: Cleanup in here. Go to master for state, not to meta.

    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
     Logging cleanup.

    M hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
     Shutdown access.

    M hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java
     Just cleanup.

    M hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
     Add state holder for hbase:meta.
     Removed unused methods.

    M hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
     Shut down access.

    M hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
     Allow hbase:meta to be disabled.

    M hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java
     Allow hbase:meta to be enabled.

 Signed-off-by: Ramkrishna <ramkrishna.s.vasudevan@intel.com>
This commit is contained in:
Michael Stack 2019-09-30 09:29:51 -07:00 committed by GitHub
parent 944108c38a
commit 52176184df
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
32 changed files with 406 additions and 443 deletions

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -616,4 +616,5 @@ public class AsyncMetaTableAccessor {
} }
return -1; return -1;
} }
} }

View File

@ -303,11 +303,18 @@ public class MetaTableAccessor {
*/ */
public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo) public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)
throws IOException { throws IOException {
byte[] row = getMetaKeyForRegion(regionInfo); return getRegionLocation(getCatalogFamilyRow(connection, regionInfo),
Get get = new Get(row); 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); get.addFamily(HConstants.CATALOG_FAMILY);
Result r = get(getMetaHTable(connection), get); return get(getMetaHTable(connection), get);
return getRegionLocation(r, regionInfo, regionInfo.getReplicaId());
} }
/** Returns the row key to use for this regionInfo */ /** Returns the row key to use for this regionInfo */
@ -1077,7 +1084,7 @@ public class MetaTableAccessor {
public static TableState getTableState(Connection conn, TableName tableName) public static TableState getTableState(Connection conn, TableName tableName)
throws IOException { throws IOException {
if (tableName.equals(TableName.META_TABLE_NAME)) { if (tableName.equals(TableName.META_TABLE_NAME)) {
return new TableState(tableName, TableState.State.ENABLED); throw new IllegalAccessError("Go to the Master to find hbase:meta table state, not here");
} }
Table metaHTable = getMetaHTable(conn); Table metaHTable = getMetaHTable(conn);
Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getTableStateColumn()); Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getTableStateColumn());
@ -1105,7 +1112,8 @@ public class MetaTableAccessor {
} }
/** /**
* Updates state in META * Updates state in META.
* Do not use. For internal use only.
* @param conn connection to use * @param conn connection to use
* @param tableName table to look for * @param tableName table to look for
*/ */

View File

@ -90,6 +90,7 @@ import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
import org.apache.hadoop.hbase.security.access.Permission; import org.apache.hadoop.hbase.security.access.Permission;
import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil; import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
import org.apache.hadoop.hbase.security.access.UserPermission; import org.apache.hadoop.hbase.security.access.UserPermission;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
@ -192,6 +193,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDe
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
@ -666,42 +669,25 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
@Override @Override
public CompletableFuture<Boolean> isTableEnabled(TableName tableName) { public CompletableFuture<Boolean> isTableEnabled(TableName tableName) {
if (TableName.isMetaTableName(tableName)) { return isTableState(tableName, TableState.State.ENABLED);
return CompletableFuture.completedFuture(true);
}
CompletableFuture<Boolean> 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));
}
});
return future;
} }
@Override @Override
public CompletableFuture<Boolean> isTableDisabled(TableName tableName) { public CompletableFuture<Boolean> isTableDisabled(TableName tableName) {
if (TableName.isMetaTableName(tableName)) { return isTableState(tableName, TableState.State.DISABLED);
return CompletableFuture.completedFuture(false); }
}
CompletableFuture<Boolean> future = new CompletableFuture<>(); /**
addListener(AsyncMetaTableAccessor.getTableState(metaTable, tableName), (state, error) -> { * @return Future that calls Master getTableState and compares to <code>state</code>
if (error != null) { */
future.completeExceptionally(error); private CompletableFuture<Boolean> isTableState(TableName tableName, TableState.State state) {
return; return this.<Boolean> newMasterCaller().
} action((controller, stub) ->
if (state.isPresent()) { this.<GetTableStateRequest, GetTableStateResponse, Boolean> call(controller, stub,
future.complete(state.get().inStates(TableState.State.DISABLED)); GetTableStateRequest.newBuilder().
} else { setTableName(ProtobufUtil.toProtoTableName(tableName)).build(),
future.completeExceptionally(new TableNotFoundException(tableName)); (s, c, req, done) -> s.getTableState(c, req, done),
} resp -> resp.getTableState().getState().toString().equals(state.toString()))).call();
});
return future;
} }
@Override @Override

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -158,7 +158,8 @@ class ZKAsyncRegistry implements AsyncRegistry {
} }
Pair<RegionState.State, ServerName> stateAndServerName = getStateAndServerName(proto); Pair<RegionState.State, ServerName> stateAndServerName = getStateAndServerName(proto);
if (stateAndServerName.getFirst() != RegionState.State.OPEN) { if (stateAndServerName.getFirst() != RegionState.State.OPEN) {
LOG.warn("Meta region is in state " + stateAndServerName.getFirst()); LOG.warn("hbase:meta region (replicaId={}) is in state {}", replicaId,
stateAndServerName.getFirst());
} }
locs[DEFAULT_REPLICA_ID] = new HRegionLocation( locs[DEFAULT_REPLICA_ID] = new HRegionLocation(
getRegionInfoForDefaultReplica(FIRST_META_REGIONINFO), stateAndServerName.getSecond()); getRegionInfoForDefaultReplica(FIRST_META_REGIONINFO), stateAndServerName.getSecond());
@ -173,7 +174,7 @@ class ZKAsyncRegistry implements AsyncRegistry {
LOG.warn("Failed to fetch " + path, error); LOG.warn("Failed to fetch " + path, error);
locs[replicaId] = null; locs[replicaId] = null;
} else if (proto == null) { } else if (proto == null) {
LOG.warn("Meta znode for replica " + replicaId + " is null"); LOG.warn("hbase:meta znode for replica " + replicaId + " is null");
locs[replicaId] = null; locs[replicaId] = null;
} else { } else {
Pair<RegionState.State, ServerName> stateAndServerName = getStateAndServerName(proto); Pair<RegionState.State, ServerName> stateAndServerName = getStateAndServerName(proto);
@ -197,9 +198,8 @@ class ZKAsyncRegistry implements AsyncRegistry {
public CompletableFuture<RegionLocations> getMetaRegionLocation() { public CompletableFuture<RegionLocations> getMetaRegionLocation() {
CompletableFuture<RegionLocations> future = new CompletableFuture<>(); CompletableFuture<RegionLocations> future = new CompletableFuture<>();
addListener( addListener(
zk.list(znodePaths.baseZNode) zk.list(znodePaths.baseZNode).thenApply(children -> children.stream().
.thenApply(children -> children.stream() filter(c -> znodePaths.isMetaZNodePrefix(c)).collect(Collectors.toList())),
.filter(c -> c.startsWith(znodePaths.metaZNodePrefix)).collect(Collectors.toList())),
(metaReplicaZNodes, error) -> { (metaReplicaZNodes, error) -> {
if (error != null) { if (error != null) {
future.completeExceptionally(error); future.completeExceptionally(error);

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * 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.HConstants.ZOOKEEPER_ZNODE_PARENT;
import static org.apache.hadoop.hbase.client.RegionInfo.DEFAULT_REPLICA_ID; import static org.apache.hadoop.hbase.client.RegionInfo.DEFAULT_REPLICA_ID;
import java.util.Collection;
import java.util.Optional; import java.util.Optional;
import java.util.stream.IntStream; import java.util.stream.IntStream;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -40,15 +41,24 @@ public class ZNodePaths {
// TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved. // TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved.
public static final char ZNODE_PATH_SEPARATOR = '/'; 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"; private static final String DEFAULT_SNAPSHOT_CLEANUP_ZNODE = "snapshot-cleanup";
// base znode for this cluster // base znode for this cluster
public final String baseZNode; 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 * The prefix of meta znode. Does not include baseZNode.
public final ImmutableMap<Integer, String> metaReplicaZNodes; * 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<Integer, String> metaReplicaZNodes;
// znode containing ephemeral nodes of the regionservers // znode containing ephemeral nodes of the regionservers
public final String rsZNode; public final String rsZNode;
// znode containing ephemeral nodes of the draining regionservers // znode containing ephemeral nodes of the draining regionservers
@ -158,21 +168,21 @@ public class ZNodePaths {
} }
/** /**
* Is the znode of any meta replica * @return true if the znode is a meta region replica
* @param node
* @return true or false
*/ */
public boolean isAnyMetaReplicaZNode(String node) { public boolean isAnyMetaReplicaZNode(String node) {
if (metaReplicaZNodes.containsValue(node)) { return this.metaReplicaZNodes.containsValue(node);
return true;
}
return false;
} }
/** /**
* Get the znode string corresponding to a replicaId * @return Meta Replica ZNodes
* @param replicaId */
* @return znode public Collection<String> getMetaReplicaZNodes() {
return this.metaReplicaZNodes.values();
}
/**
* @return the znode string corresponding to a replicaId
*/ */
public String getZNodeForReplica(int replicaId) { public String getZNodeForReplica(int replicaId) {
// return a newly created path but don't update the cache of paths // return a newly created path but don't update the cache of paths
@ -183,24 +193,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 * @param znode the name of the znode, does not include baseZNode
* @return replicaId * @return replicaId
*/ */
public int getMetaReplicaIdFromZnode(String znode) { public int getMetaReplicaIdFromZnode(String znode) {
if (znode.equals(metaZNodePrefix)) { return znode.equals(metaZNodePrefix)?
return RegionInfo.DEFAULT_REPLICA_ID; RegionInfo.DEFAULT_REPLICA_ID:
} Integer.parseInt(znode.substring(metaZNodePrefix.length() + 1));
return Integer.parseInt(znode.substring(metaZNodePrefix.length() + 1));
} }
/** /**
* Is it the default meta replica's znode * @return True if meta znode.
* @param znode the name of the znode, does not include baseZNode
* @return true or false
*/ */
public boolean isDefaultMetaReplicaZnode(String znode) { public boolean isMetaZNodePrefix(String znode) {
return metaReplicaZNodes.get(DEFAULT_REPLICA_ID).equals(znode); return znode != null && znode.startsWith(this.metaZNodePrefix);
} }
/** /**

View File

@ -1175,8 +1175,7 @@ public final class HConstants {
/** Directories that are not HBase user table directories */ /** Directories that are not HBase user table directories */
public static final List<String> HBASE_NON_USER_TABLE_DIRS = public static final List<String> HBASE_NON_USER_TABLE_DIRS =
Collections.unmodifiableList(Arrays.asList((String[])ArrayUtils.addAll( Collections.unmodifiableList(Arrays.asList((String[])ArrayUtils.addAll(
new String[] { TableName.META_TABLE_NAME.getNameAsString() }, HBASE_NON_TABLE_DIRS.toArray())));
HBASE_NON_TABLE_DIRS.toArray())));
/** Health script related settings. */ /** Health script related settings. */
public static final String HEALTH_SCRIPT_LOC = "hbase.node.health.script.location"; public static final String HEALTH_SCRIPT_LOC = "hbase.node.health.script.location";

View File

@ -25,25 +25,19 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
/** /**
* Get, remove and modify table descriptors. * Get, remove and modify table descriptors.
* Used by servers to host descriptors.
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public interface TableDescriptors { public interface TableDescriptors {
/** /**
* @param tableName
* @return TableDescriptor for tablename * @return TableDescriptor for tablename
* @throws IOException
*/ */
TableDescriptor get(final TableName tableName) TableDescriptor get(final TableName tableName) throws IOException;
throws IOException;
/** /**
* Get Map of all NamespaceDescriptors for a given namespace. * Get Map of all NamespaceDescriptors for a given namespace.
* @return Map of all descriptors. * @return Map of all descriptors.
* @throws IOException
*/ */
Map<String, TableDescriptor> getByNamespace(String name) Map<String, TableDescriptor> getByNamespace(String name) throws IOException;
throws IOException;
/** /**
* Get Map of all TableDescriptors. Populates the descriptor cache as a * 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 * Notice: the key of map is the table name which contains namespace. It was generated by
* {@link TableName#getNameWithNamespaceInclAsString()}. * {@link TableName#getNameWithNamespaceInclAsString()}.
* @return Map of all descriptors. * @return Map of all descriptors.
* @throws IOException
*/ */
Map<String, TableDescriptor> getAll() throws IOException; Map<String, TableDescriptor> getAll() throws IOException;
/** /**
* Add or update descriptor * Add or update descriptor
* @param htd Descriptor to set into TableDescriptors * @param htd Descriptor to set into TableDescriptors
* @throws IOException
*/ */
void add(final TableDescriptor htd) void add(final TableDescriptor htd) throws IOException;
throws IOException;
/** /**
* @param tablename
* @return Instance of table descriptor or null if none found. * @return Instance of table descriptor or null if none found.
* @throws IOException
*/ */
TableDescriptor remove(final TableName tablename) TableDescriptor remove(final TableName tablename) throws IOException;
throws IOException;
/** /**
* Enables the tabledescriptor cache * Enables the tabledescriptor cache

View File

@ -49,7 +49,6 @@ import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Function;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import javax.servlet.ServletException; import javax.servlet.ServletException;
@ -670,10 +669,6 @@ public class HMaster extends HRegionServer implements MasterServices {
return connector.getLocalPort(); return connector.getLocalPort();
} }
@Override
protected Function<TableDescriptorBuilder, TableDescriptorBuilder> getMetaTableObserver() {
return builder -> builder.setRegionReplication(conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM));
}
/** /**
* For compatibility, if failed with regionserver credentials, try the master one * For compatibility, if failed with regionserver credentials, try the master one
*/ */
@ -1035,7 +1030,7 @@ public class HMaster extends HRegionServer implements MasterServices {
RegionState rs = this.assignmentManager.getRegionStates(). RegionState rs = this.assignmentManager.getRegionStates().
getRegionState(RegionInfoBuilder.FIRST_META_REGIONINFO); getRegionState(RegionInfoBuilder.FIRST_META_REGIONINFO);
LOG.info("hbase:meta {}", rs); LOG.info("hbase:meta {}", rs);
if (rs.isOffline()) { if (rs != null && rs.isOffline()) {
Optional<InitMetaProcedure> optProc = procedureExecutor.getProcedures().stream() Optional<InitMetaProcedure> optProc = procedureExecutor.getProcedures().stream()
.filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny(); .filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny();
initMetaProc = optProc.orElseGet(() -> { initMetaProc = optProc.orElseGet(() -> {

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
import org.apache.hadoop.hbase.util.IdReadWriteLock; import org.apache.hadoop.hbase.util.IdReadWriteLock;
import org.apache.hadoop.hbase.util.ZKDataMigrator; import org.apache.hadoop.hbase.util.ZKDataMigrator;
import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@ -53,8 +52,20 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
// TODO: Make this a guava Service // TODO: Make this a guava Service
@InterfaceAudience.Private @InterfaceAudience.Private
public class TableStateManager { public class TableStateManager {
private static final Logger LOG = LoggerFactory.getLogger(TableStateManager.class); private static final Logger LOG = LoggerFactory.getLogger(TableStateManager.class);
/**
* All table state is kept in hbase:meta except that of hbase:meta itself.
* hbase:meta state is kept here locally in this in-memory variable. State
* for hbase:meta is not persistent. If this process dies, the hbase:meta
* state reverts to enabled. State is used so we can edit hbase:meta as we
* would any other table by disabling, altering, and then re-enabling. If this
* process dies in the midst of an edit, the table reverts to enabled. Schema
* is read from the filesystem. It is changed atomically so if we die midway
* through an edit we should be good.
*/
private TableState.State metaTableState = TableState.State.ENABLED;
/** /**
* Set this key to false in Configuration to disable migrating table state from zookeeper so * Set this key to false in Configuration to disable migrating table state from zookeeper so
* hbase:meta table. * hbase:meta table.
@ -68,7 +79,7 @@ public class TableStateManager {
private final ConcurrentMap<TableName, TableState.State> tableName2State = private final ConcurrentMap<TableName, TableState.State> tableName2State =
new ConcurrentHashMap<>(); new ConcurrentHashMap<>();
public TableStateManager(MasterServices master) { TableStateManager(MasterServices master) {
this.master = master; this.master = master;
} }
@ -87,61 +98,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) { public boolean isTableState(TableName tableName, TableState.State... states) {
try { try {
TableState tableState = getTableState(tableName); TableState tableState = getTableState(tableName);
@ -155,6 +111,7 @@ public class TableStateManager {
public void setDeletedTable(TableName tableName) throws IOException { public void setDeletedTable(TableName tableName) throws IOException {
if (tableName.equals(TableName.META_TABLE_NAME)) { if (tableName.equals(TableName.META_TABLE_NAME)) {
// Can't delete the hbase:meta table.
return; return;
} }
ReadWriteLock lock = tnLock.getLock(tableName); ReadWriteLock lock = tnLock.getLock(tableName);
@ -183,7 +140,7 @@ public class TableStateManager {
* @param states filter by states * @param states filter by states
* @return tables in given states * @return tables in given states
*/ */
public Set<TableName> getTablesInStates(TableState.State... states) throws IOException { Set<TableName> getTablesInStates(TableState.State... states) throws IOException {
// Only be called in region normalizer, will not use cache. // Only be called in region normalizer, will not use cache.
final Set<TableName> rv = Sets.newHashSet(); final Set<TableName> rv = Sets.newHashSet();
MetaTableAccessor.fullScanTables(master.getConnection(), new MetaTableAccessor.Visitor() { MetaTableAccessor.fullScanTables(master.getConnection(), new MetaTableAccessor.Visitor() {
@ -199,12 +156,6 @@ public class TableStateManager {
return rv; return rv;
} }
public static class TableStateNotFoundException extends TableNotFoundException {
TableStateNotFoundException(TableName tableName) {
super(tableName.getNameAsString());
}
}
@NonNull @NonNull
public TableState getTableState(TableName tableName) throws IOException { public TableState getTableState(TableName tableName) throws IOException {
ReadWriteLock lock = tnLock.getLock(tableName); ReadWriteLock lock = tnLock.getLock(tableName);
@ -212,7 +163,7 @@ public class TableStateManager {
try { try {
TableState currentState = readMetaState(tableName); TableState currentState = readMetaState(tableName);
if (currentState == null) { if (currentState == null) {
throw new TableStateNotFoundException(tableName); throw new TableNotFoundException("No state found for " + tableName);
} }
return currentState; return currentState;
} finally { } finally {
@ -221,22 +172,18 @@ public class TableStateManager {
} }
private void updateMetaState(TableName tableName, TableState.State newState) throws IOException { 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);
}
// Otherwise, just return; no need to set ENABLED on meta -- it is always ENABLED.
return;
}
boolean succ = false; boolean succ = false;
try { try {
MetaTableAccessor.updateTableState(master.getConnection(), tableName, newState); if (tableName.equals(TableName.META_TABLE_NAME)) {
tableName2State.put(tableName, newState); this.metaTableState = newState;
} else {
MetaTableAccessor.updateTableState(master.getConnection(), tableName, newState);
}
this.tableName2State.put(tableName, newState);
succ = true; succ = true;
} finally { } finally {
if (!succ) { if (!succ) {
tableName2State.remove(tableName); this.tableName2State.remove(tableName);
} }
} }
metaStateUpdated(tableName, newState); metaStateUpdated(tableName, newState);
@ -255,7 +202,9 @@ public class TableStateManager {
if (state != null) { if (state != null) {
return new TableState(tableName, state); return new TableState(tableName, state);
} }
TableState tableState = MetaTableAccessor.getTableState(master.getConnection(), tableName); TableState tableState = tableName.equals(TableName.META_TABLE_NAME)?
new TableState(TableName.META_TABLE_NAME, this.metaTableState):
MetaTableAccessor.getTableState(master.getConnection(), tableName);
if (tableState != null) { if (tableState != null) {
tableName2State.putIfAbsent(tableName, tableState.getState()); tableName2State.putIfAbsent(tableName, tableState.getState());
} }
@ -263,10 +212,8 @@ public class TableStateManager {
} }
public void start() throws IOException { public void start() throws IOException {
TableDescriptors tableDescriptors = master.getTableDescriptors();
migrateZooKeeper(); migrateZooKeeper();
Connection connection = master.getConnection(); fixTableStates(master.getTableDescriptors(), master.getConnection());
fixTableStates(tableDescriptors, connection);
} }
private void fixTableStates(TableDescriptors tableDescriptors, Connection connection) private void fixTableStates(TableDescriptors tableDescriptors, Connection connection)
@ -335,7 +282,7 @@ public class TableStateManager {
TableState ts = null; TableState ts = null;
try { try {
ts = getTableState(entry.getKey()); ts = getTableState(entry.getKey());
} catch (TableStateNotFoundException e) { } catch (TableNotFoundException e) {
// This can happen; table exists but no TableState. // This can happen; table exists but no TableState.
} }
if (ts == null) { if (ts == null) {
@ -377,4 +324,4 @@ public class TableStateManager {
LOG.warn("Failed deleting table state from zookeeper", e); LOG.warn("Failed deleting table state from zookeeper", e);
} }
} }
} }

View File

@ -147,8 +147,7 @@ public class RegionStateStore {
} }
} }
public void updateRegionLocation(RegionStateNode regionStateNode) void updateRegionLocation(RegionStateNode regionStateNode) throws IOException {
throws IOException {
if (regionStateNode.getRegionInfo().isMetaRegion()) { if (regionStateNode.getRegionInfo().isMetaRegion()) {
updateMetaLocation(regionStateNode.getRegionInfo(), regionStateNode.getRegionLocation(), updateMetaLocation(regionStateNode.getRegionInfo(), regionStateNode.getRegionLocation(),
regionStateNode.getState()); regionStateNode.getState());

View File

@ -78,9 +78,7 @@ public class CreateTableProcedure
@Override @Override
protected Flow executeFromState(final MasterProcedureEnv env, final CreateTableState state) protected Flow executeFromState(final MasterProcedureEnv env, final CreateTableState state)
throws InterruptedException { throws InterruptedException {
if (LOG.isTraceEnabled()) { LOG.info("{} execute state={}", this, state);
LOG.trace(this + " execute state=" + state);
}
try { try {
switch (state) { switch (state) {
case CREATE_TABLE_PRE_OPERATION: case CREATE_TABLE_PRE_OPERATION:

View File

@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.BufferedMutator; import org.apache.hadoop.hbase.client.BufferedMutator;
import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.TableStateManager; import org.apache.hadoop.hbase.master.TableStateManager;
@ -109,8 +108,8 @@ public class DisableTableProcedure
setNextState(DisableTableState.DISABLE_TABLE_ADD_REPLICATION_BARRIER); setNextState(DisableTableState.DISABLE_TABLE_ADD_REPLICATION_BARRIER);
break; break;
case DISABLE_TABLE_ADD_REPLICATION_BARRIER: case DISABLE_TABLE_ADD_REPLICATION_BARRIER:
if (env.getMasterServices().getTableDescriptors().get(tableName) if (env.getMasterServices().getTableDescriptors().get(tableName).
.hasGlobalReplicationScope()) { hasGlobalReplicationScope()) {
MasterFileSystem fs = env.getMasterFileSystem(); MasterFileSystem fs = env.getMasterFileSystem();
try (BufferedMutator mutator = env.getMasterServices().getConnection() try (BufferedMutator mutator = env.getMasterServices().getConnection()
.getBufferedMutator(TableName.META_TABLE_NAME)) { .getBufferedMutator(TableName.META_TABLE_NAME)) {
@ -242,10 +241,7 @@ public class DisableTableProcedure
*/ */
private boolean prepareDisable(final MasterProcedureEnv env) throws IOException { private boolean prepareDisable(final MasterProcedureEnv env) throws IOException {
boolean canTableBeDisabled = true; boolean canTableBeDisabled = true;
if (tableName.equals(TableName.META_TABLE_NAME)) { if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
setFailure("master-disable-table", new ConstraintException("Cannot disable catalog table"));
canTableBeDisabled = false;
} else if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
setFailure("master-disable-table", new TableNotFoundException(tableName)); setFailure("master-disable-table", new TableNotFoundException(tableName));
canTableBeDisabled = false; canTableBeDisabled = false;
} else if (!skipTableStateCheck) { } else if (!skipTableStateCheck) {

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -20,18 +20,19 @@ package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import org.apache.hadoop.hbase.Cell; import java.util.concurrent.ExecutionException;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.AsyncConnection;
import org.apache.hadoop.hbase.client.AsyncTable;
import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; 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.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@ -45,6 +46,7 @@ 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;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableState; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableState;
@InterfaceAudience.Private @InterfaceAudience.Private
public class EnableTableProcedure public class EnableTableProcedure
extends AbstractStateMachineTableProcedure<EnableTableState> { extends AbstractStateMachineTableProcedure<EnableTableState> {
@ -57,7 +59,8 @@ public class EnableTableProcedure
/** /**
* Constructor * Constructor
* @param env MasterProcedureEnv *
* @param env MasterProcedureEnv
* @param tableName the table to operate on * @param tableName the table to operate on
*/ */
public EnableTableProcedure(MasterProcedureEnv env, TableName tableName) { public EnableTableProcedure(MasterProcedureEnv env, TableName tableName) {
@ -66,11 +69,12 @@ public class EnableTableProcedure
/** /**
* Constructor * Constructor
* @param env MasterProcedureEnv *
* @param env MasterProcedureEnv
* @param tableName the table to operate on * @param tableName the table to operate on
*/ */
public EnableTableProcedure(MasterProcedureEnv env, TableName tableName, public EnableTableProcedure(MasterProcedureEnv env, TableName tableName,
ProcedurePrepareLatch syncLatch) { ProcedurePrepareLatch syncLatch) {
super(env, syncLatch); super(env, syncLatch);
this.tableName = tableName; this.tableName = tableName;
} }
@ -99,66 +103,53 @@ public class EnableTableProcedure
setNextState(EnableTableState.ENABLE_TABLE_MARK_REGIONS_ONLINE); setNextState(EnableTableState.ENABLE_TABLE_MARK_REGIONS_ONLINE);
break; break;
case ENABLE_TABLE_MARK_REGIONS_ONLINE: case ENABLE_TABLE_MARK_REGIONS_ONLINE:
Connection connection = env.getMasterServices().getConnection(); // Get the region replica count. If changed since disable, need to do
// we will need to get the tableDescriptor here to see if there is a change in the replica // more work assigning.
// count AsyncClusterConnection connection = env.getMasterServices().getAsyncClusterConnection();
TableDescriptor hTableDescriptor = TableDescriptor tableDescriptor =
env.getMasterServices().getTableDescriptors().get(tableName); env.getMasterServices().getTableDescriptors().get(tableName);
int configuredReplicaCount = tableDescriptor.getRegionReplication();
// Get the replica count // Get regions for the table from memory; get both online and offline regions ('true').
int regionReplicaCount = hTableDescriptor.getRegionReplication();
// Get the regions for the table from memory; get both online and offline regions
// ('true').
List<RegionInfo> regionsOfTable = List<RegionInfo> regionsOfTable =
env.getAssignmentManager().getRegionStates().getRegionsOfTable(tableName, true); env.getAssignmentManager().getRegionStates().getRegionsOfTable(tableName, true);
int currentMaxReplica = 0; // How many replicas do we currently have? Check regions returned from
// Check if the regions in memory have replica regions as marked in META table // in-memory state.
for (RegionInfo regionInfo : regionsOfTable) { int currentMaxReplica = getMaxReplicaId(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();
}
}
// read the META table to know the actual number of replicas for the table - if there // Read the META table to know the number of replicas the table currently has.
// was a table modification on region replica then this will reflect the new entries also // If there was a table modification on region replica count then need to
int replicasFound = // adjust replica counts here.
getNumberOfReplicasFromMeta(connection, regionReplicaCount, regionsOfTable); int replicasFound = getReplicaCount(connection, this.tableName);
assert regionReplicaCount - 1 == replicasFound; LOG.info("replicasFound={} (configuredReplicaCount={} for {}", replicasFound,
LOG.info(replicasFound + " META entries added for the given regionReplicaCount " configuredReplicaCount, tableName.getNameAsString());
+ regionReplicaCount + " for the table " + tableName.getNameAsString()); if (currentMaxReplica == (configuredReplicaCount - 1)) {
if (currentMaxReplica == (regionReplicaCount - 1)) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("There is no change to the number of region replicas." LOG.debug("No change in number of region replicas (configuredReplicaCount={});"
+ " Assigning the available regions." + " Current and previous" + " assigning.", configuredReplicaCount);
+ "replica count is " + regionReplicaCount);
} }
} else if (currentMaxReplica > (regionReplicaCount - 1)) { } else if (currentMaxReplica > (configuredReplicaCount - 1)) {
// we have additional regions as the replica count has been decreased. Delete // We have additional regions as the replica count has been decreased. Delete
// those regions because already the table is in the unassigned state // those regions because already the table is in the unassigned state
LOG.info("The number of replicas " + (currentMaxReplica + 1) 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<RegionInfo> copyOfRegions = new ArrayList<RegionInfo>(regionsOfTable); List<RegionInfo> copyOfRegions = new ArrayList<>(regionsOfTable);
for (RegionInfo regionInfo : copyOfRegions) { for (RegionInfo regionInfo : copyOfRegions) {
if (regionInfo.getReplicaId() > (regionReplicaCount - 1)) { if (regionInfo.getReplicaId() > (configuredReplicaCount - 1)) {
// delete the region from the regionStates // delete the region from the regionStates
env.getAssignmentManager().getRegionStates().deleteRegion(regionInfo); env.getAssignmentManager().getRegionStates().deleteRegion(regionInfo);
// remove it from the list of regions of the table // remove it from the list of regions of the table
LOG.info("The regioninfo being removed is " + regionInfo + " " LOG.info("Removed replica={} of {}", regionInfo.getRegionId(), regionInfo);
+ regionInfo.getReplicaId());
regionsOfTable.remove(regionInfo); regionsOfTable.remove(regionInfo);
} }
} }
} else { } else {
// the replicasFound is less than the regionReplication // the replicasFound is less than the regionReplication
LOG.info("The number of replicas has been changed(increased)." LOG.info("Number of replicas has increased. Assigning new region replicas." +
+ " Lets assign the new region replicas. The previous replica count was " "The previous replica count was {}. The current replica count is {}.",
+ (currentMaxReplica + 1) + ". The current replica count is " + regionReplicaCount); (currentMaxReplica + 1), configuredReplicaCount);
regionsOfTable = RegionReplicaUtil.addReplicas(hTableDescriptor, regionsOfTable, regionsOfTable = RegionReplicaUtil.addReplicas(tableDescriptor, regionsOfTable,
currentMaxReplica + 1, regionReplicaCount); currentMaxReplica + 1, configuredReplicaCount);
} }
// Assign all the table regions. (including region replicas if added). // Assign all the table regions. (including region replicas if added).
// createAssignProcedure will try to retain old assignments if possible. // createAssignProcedure will try to retain old assignments if possible.
@ -175,40 +166,32 @@ public class EnableTableProcedure
default: default:
throw new UnsupportedOperationException("unhandled state=" + state); throw new UnsupportedOperationException("unhandled state=" + state);
} }
} catch (IOException e) { } catch (IOException | ExecutionException e) {
if (isRollbackSupported(state)) { if (isRollbackSupported(state)) {
setFailure("master-enable-table", e); setFailure("master-enable-table", getCause(e));
} else { } else {
LOG.warn( LOG.warn("Retryable error enabling {}, state={}", tableName, state, getCause(e));
"Retriable error trying to enable table=" + tableName + " (in state=" + state + ")", e);
} }
} }
return Flow.HAS_MORE_STATE; return Flow.HAS_MORE_STATE;
} }
private int getNumberOfReplicasFromMeta(Connection connection, int regionReplicaCount, /**
List<RegionInfo> regionsOfTable) throws IOException { * @return If ExecutionException, pull out the cause.
Result r = getRegionFromMeta(connection, regionsOfTable); */
int replicasFound = 0; private Throwable getCause(Exception e) {
for (int i = 1; i < regionReplicaCount; i++) { return e instanceof ExecutionException? ((ExecutionException)e).getCause(): e;
// Since we have already added the entries to the META we will be getting only that here
List<Cell> columnCells =
r.getColumnCells(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(i));
if (!columnCells.isEmpty()) {
replicasFound++;
}
}
return replicasFound;
} }
private Result getRegionFromMeta(Connection connection, List<RegionInfo> regionsOfTable) /**
throws IOException { * @return If hbase;meta table, it goes to the registry implementation which is what we want.
byte[] metaKeyForRegion = MetaTableAccessor.getMetaKeyForRegion(regionsOfTable.get(0)); */
Get get = new Get(metaKeyForRegion); private int getReplicaCount(AsyncConnection connection, TableName tableName)
get.addFamily(HConstants.CATALOG_FAMILY); throws ExecutionException, InterruptedException {
Table metaTable = MetaTableAccessor.getMetaHTable(connection); AsyncTable t = connection.getTable(TableName.META_TABLE_NAME);
Result r = metaTable.get(get); List<HRegionLocation> rls =
return r; t.getRegionLocator().getRegionLocations(HConstants.EMPTY_START_ROW, true).get();
return rls.size();
} }
@Override @Override
@ -408,4 +391,19 @@ public class EnableTableProcedure
} }
} }
} }
/**
* @return Maximum region replica id found in passed list of regions.
*/
private static int getMaxReplicaId(List<RegionInfo> 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;
}
} }

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -23,10 +23,10 @@ import java.util.HashMap;
import java.util.Map; import java.util.Map;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName; 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.Connection;
import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.master.TableStateManager; 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.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface; import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch; import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
@ -140,7 +140,7 @@ public abstract class AbstractPeerProcedure<TState> extends AbstractPeerNoLockPr
return true; return true;
} }
Thread.sleep(SLEEP_INTERVAL_MS); Thread.sleep(SLEEP_INTERVAL_MS);
} catch (TableStateNotFoundException e) { } catch (TableNotFoundException e) {
return false; return false;
} catch (InterruptedException e) { } catch (InterruptedException e) {
throw (IOException) new InterruptedIOException(e.getMessage()).initCause(e); throw (IOException) new InterruptedIOException(e.getMessage()).initCause(e);

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * 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.IOException;
import java.io.InterruptedIOException; import java.io.InterruptedIOException;
import org.apache.hadoop.hbase.TableName; 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.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.TableStateManager; 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.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch; import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
import org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure; import org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure;
@ -125,7 +125,7 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure<PeerModi
return false; return false;
} }
Thread.sleep(SLEEP_INTERVAL_MS); Thread.sleep(SLEEP_INTERVAL_MS);
} catch (TableStateNotFoundException e) { } catch (TableNotFoundException e) {
return false; return false;
} catch (InterruptedException e) { } catch (InterruptedException e) {
throw (IOException) new InterruptedIOException(e.getMessage()).initCause(e); throw (IOException) new InterruptedIOException(e.getMessage()).initCause(e);

View File

@ -1,4 +1,4 @@
/** /*
* *
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
@ -41,6 +41,6 @@ public class MetaLocationSyncer extends ClientZKSyncer {
@Override @Override
Collection<String> getNodesToWatch() { Collection<String> getNodesToWatch() {
return watcher.getZNodePaths().metaReplicaZNodes.values(); return watcher.getZNodePaths().getMetaReplicaZNodes();
} }
} }

View File

@ -744,9 +744,14 @@ public class HRegionServer extends HasThread implements
protected TableDescriptors getFsTableDescriptors() throws IOException { protected TableDescriptors getFsTableDescriptors() throws IOException {
return new FSTableDescriptors(this.conf, return new FSTableDescriptors(this.conf,
this.fs, this.rootDir, !canUpdateTableDescriptor(), false, getMetaTableObserver()); this.fs, this.rootDir, !canUpdateTableDescriptor(), false);
} }
/**
* @deprecated Since 2.3.0. Not needed anymore. Was used by Master to pass in replication
* setting on hbase:meta construction. To be removed in hbase4.
*/
@Deprecated
protected Function<TableDescriptorBuilder, TableDescriptorBuilder> getMetaTableObserver() { protected Function<TableDescriptorBuilder, TableDescriptorBuilder> getMetaTableObserver() {
return null; return null;
} }

View File

@ -1,4 +1,6 @@
/** /*
* Copyright The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -22,6 +24,7 @@ import java.util.function.BiPredicate;
import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.replication.SyncReplicationState;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
/** /**
* Check whether we need to reject the replication request from source cluster. * Check whether we need to reject the replication request from source cluster.
*/ */

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -24,7 +24,6 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.TreeMap; import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Function;
import java.util.regex.Matcher; import java.util.regex.Matcher;
import java.util.regex.Pattern; import java.util.regex.Pattern;
@ -99,10 +98,7 @@ public class FSTableDescriptors implements TableDescriptors {
// TODO. // TODO.
private final Map<TableName, TableDescriptor> cache = new ConcurrentHashMap<>(); private final Map<TableName, TableDescriptor> cache = new ConcurrentHashMap<>();
/** private final Configuration configuration;
* Table descriptor for <code>hbase:meta</code> catalog table
*/
private final TableDescriptor metaTableDescriptor;
/** /**
* Construct a FSTableDescriptors instance using the hbase root dir of the given * Construct a FSTableDescriptors instance using the hbase root dir of the given
@ -123,30 +119,22 @@ public class FSTableDescriptors implements TableDescriptors {
* operations; i.e. on remove, we do not do delete in fs. * operations; i.e. on remove, we do not do delete in fs.
*/ */
public FSTableDescriptors(final Configuration conf, final FileSystem fs, 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);
}
/**
* @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.
* @param metaObserver Used by HMaster. It need to modify the META_REPLICAS_NUM for meta table descriptor.
* see HMaster#finishActiveMasterInitialization
* 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<TableDescriptorBuilder, TableDescriptorBuilder> metaObserver) throws IOException {
this.fs = fs; this.fs = fs;
this.rootdir = rootdir; this.rootdir = rootdir;
this.fsreadonly = fsreadonly; this.fsreadonly = fsreadonly;
this.usecache = usecache; this.usecache = usecache;
this.metaTableDescriptor = metaObserver == null ? createMetaTableDescriptor(conf) this.configuration = conf;
: metaObserver.apply(createMetaTableDescriptorBuilder(conf)).build();
} }
/**
* Should be private
* @deprecated Since 2.3.0. Should be for internal use only. Used by testing.
*/
@Deprecated
@VisibleForTesting @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 // 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, // 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. // we have to rethink about adding back the setCacheDataInL1 for META table CFs.
@ -191,7 +179,10 @@ public class FSTableDescriptors implements TableDescriptors {
.build()) .build())
.setCoprocessor(CoprocessorDescriptorBuilder.newBuilder( .setCoprocessor(CoprocessorDescriptorBuilder.newBuilder(
MultiRowMutationEndpoint.class.getName()) MultiRowMutationEndpoint.class.getName())
.setPriority(Coprocessor.PRIORITY_SYSTEM).build()); .setPriority(Coprocessor.PRIORITY_SYSTEM)
.build())
.setRegionReplication(conf.getInt(HConstants.META_REPLICAS_NUM,
HConstants.DEFAULT_META_REPLICA_NUM));
} }
@VisibleForTesting @VisibleForTesting
@ -228,16 +219,11 @@ public class FSTableDescriptors implements TableDescriptors {
public TableDescriptor get(final TableName tablename) public TableDescriptor get(final TableName tablename)
throws IOException { throws IOException {
invocations++; invocations++;
if (TableName.META_TABLE_NAME.equals(tablename)) { // If some one tries to get the descriptor for
cachehits++;
return metaTableDescriptor;
}
// hbase:meta is already handled. If some one tries to get the descriptor for
// .logs, .oldlogs or .corrupt throw an exception. // .logs, .oldlogs or .corrupt throw an exception.
if (HConstants.HBASE_NON_USER_TABLE_DIRS.contains(tablename.getNameAsString())) { if (HConstants.HBASE_NON_USER_TABLE_DIRS.contains(tablename.getNameAsString())) {
throw new IOException("No descriptor found for non table = " + tablename); throw new IOException("No descriptor found for non table = " + tablename);
} }
if (usecache) { if (usecache) {
// Look in cache of descriptors. // Look in cache of descriptors.
TableDescriptor cachedtdm = this.cache.get(tablename); TableDescriptor cachedtdm = this.cache.get(tablename);
@ -246,24 +232,27 @@ public class FSTableDescriptors implements TableDescriptors {
return cachedtdm; return cachedtdm;
} }
} }
TableDescriptor tdmt = null; TableDescriptor td = null;
try { try {
tdmt = getTableDescriptorFromFs(fs, rootdir, tablename); td = getTableDescriptorFromFs(fs, rootdir, tablename);
} catch (NullPointerException e) { } catch (NullPointerException e) {
LOG.debug("Exception during readTableDecriptor. Current table name = " LOG.debug("Exception during readTableDecriptor; tableName={}", tablename, e);
+ tablename, e);
} catch (TableInfoMissingException e) { } catch (TableInfoMissingException e) {
// ignore. This is regular operation if (TableName.isMetaTableName(tablename)) {
// If we tried to access hbase:meta and it not there, create it.
td = createMetaTableDescriptor(this.configuration);
LOG.info("Creating new hbase:meta table default descriptor/schema {}", td);
}
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.debug("Exception during readTableDecriptor. Current table name = " LOG.debug("Exception during readTableDecriptor. Current table name = "
+ tablename, ioe); + tablename, ioe);
} }
// last HTD written wins // last HTD written wins
if (usecache && tdmt != null) { if (usecache && td != null) {
this.cache.put(tablename, tdmt); this.cache.put(tablename, td);
} }
return tdmt; return td;
} }
/** /**
@ -273,16 +262,21 @@ public class FSTableDescriptors implements TableDescriptors {
public Map<String, TableDescriptor> getAll() public Map<String, TableDescriptor> getAll()
throws IOException { throws IOException {
Map<String, TableDescriptor> tds = new TreeMap<>(); Map<String, TableDescriptor> tds = new TreeMap<>();
if (fsvisited && usecache) { if (fsvisited && usecache) {
if (this.cache.get(TableName.META_TABLE_NAME) == null) {
// This get will create hbase:meta if it does not exist. Will also populate cache.
get(TableName.META_TABLE_NAME);
}
for (Map.Entry<TableName, TableDescriptor> entry: this.cache.entrySet()) { for (Map.Entry<TableName, TableDescriptor> entry: this.cache.entrySet()) {
tds.put(entry.getKey().getNameWithNamespaceInclAsString(), entry.getValue()); tds.put(entry.getKey().getNameWithNamespaceInclAsString(), entry.getValue());
} }
// add hbase:meta to the response
tds.put(this.metaTableDescriptor.getTableName().getNameAsString(), metaTableDescriptor);
} else { } else {
LOG.trace("Fetching table descriptors from the filesystem."); LOG.trace("Fetching table descriptors from the filesystem.");
boolean allvisited = true; boolean allvisited = true;
// Add hbase:meta descriptor. The get will create hbase:meta in fs if doesn't
// exist. FSUtils listing table names in fs skip meta dirs. TODO: Fill out
// FSUtils with methods to get userspace tables and system tables.
tds.put(TableName.META_TABLE_NAME.toString(), get(TableName.META_TABLE_NAME));
for (Path d : FSUtils.getTableDirs(fs, rootdir)) { for (Path d : FSUtils.getTableDirs(fs, rootdir)) {
TableDescriptor htd = null; TableDescriptor htd = null;
try { try {
@ -336,14 +330,9 @@ public class FSTableDescriptors implements TableDescriptors {
if (fsreadonly) { if (fsreadonly) {
throw new NotImplementedException("Cannot add a table descriptor - in read only mode"); throw new NotImplementedException("Cannot add a table descriptor - in read only mode");
} }
TableName tableName = htd.getTableName(); if (HConstants.HBASE_NON_USER_TABLE_DIRS.contains(htd.getTableName().getNameAsString())) {
if (TableName.META_TABLE_NAME.equals(tableName)) { throw new NotImplementedException("Cannot add Descriptor for reserved subdirectory name: " +
throw new NotImplementedException(HConstants.NOT_IMPLEMENTED); htd.getTableName().getNameAsString());
}
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); updateTableDescriptor(htd);
} }
@ -369,26 +358,6 @@ public class FSTableDescriptors implements TableDescriptors {
return descriptor; 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) private FileStatus getTableInfoPath(Path tableDir)
throws IOException { throws IOException {
return getTableInfoPath(fs, tableDir, !fsreadonly); return getTableInfoPath(fs, tableDir, !fsreadonly);
@ -403,7 +372,6 @@ public class FSTableDescriptors implements TableDescriptors {
* were sequence numbers). * were sequence numbers).
* *
* @return The file status of the current table info file or null if it does not exist * @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) public static FileStatus getTableInfoPath(FileSystem fs, Path tableDir)
throws IOException { throws IOException {
@ -421,7 +389,6 @@ public class FSTableDescriptors implements TableDescriptors {
* older files. * older files.
* *
* @return The file status of the current table info file or null if none exist * @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) private static FileStatus getTableInfoPath(FileSystem fs, Path tableDir, boolean removeOldFiles)
throws IOException { throws IOException {
@ -609,21 +576,6 @@ public class FSTableDescriptors implements TableDescriptors {
return p; 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 * Deletes files matching the table info file pattern within the given directory
* whose sequenceId is at most the given max sequenceId. * whose sequenceId is at most the given max sequenceId.
@ -760,6 +712,27 @@ public class FSTableDescriptors implements TableDescriptors {
if (fsreadonly) { if (fsreadonly) {
throw new NotImplementedException("Cannot create a table descriptor - in read only mode"); throw new NotImplementedException("Cannot create a table descriptor - in read only mode");
} }
return createTableDescriptorForTableDirectory(this.fs, tableDir, htd, forceCreation);
}
/**
* Create a new TableDescriptor in the specified table directory and filesystem. Happens when we
* create a new table or snapshot a table. This method doesn't require creationg of an
* {@link FSTableDescriptors} instance so it takes a bunch of arguments. Users of the method
* above used to create an FSTableDescriptors instance just to run the method. That was fine
* until construction started expecting to be able to read the hbase:meta schema. Snapshotting
* to some random dir would fail construction if no hbase:meta schema available.
* @param fs Filesystem to write to. Snapshot can set it to other than that of running system.
* @param tableDir table directory under which we should write the file
* @param htd description of the table to write
* @param forceCreation if <tt>true</tt>,then even if previous table descriptor is present it will
* be overwritten
* @return <tt>true</tt> if the we successfully created the file, <tt>false</tt> 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); FileStatus status = getTableInfoPath(fs, tableDir);
if (status != null) { if (status != null) {
LOG.debug("Current path=" + status.getPath()); LOG.debug("Current path=" + status.getPath());
@ -775,6 +748,5 @@ public class FSTableDescriptors implements TableDescriptors {
Path p = writeTableDescriptor(fs, htd, tableDir, status); Path p = writeTableDescriptor(fs, htd, tableDir, status);
return p != null; return p != null;
} }
} }

View File

@ -121,7 +121,6 @@ import org.apache.hadoop.hbase.security.visibility.VisibilityLabelsCache;
import org.apache.hadoop.hbase.trace.TraceUtil; import org.apache.hadoop.hbase.trace.TraceUtil;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
@ -157,6 +156,7 @@ import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
/** /**
* Facility for testing HBase. Replacement for * Facility for testing HBase. Replacement for
* old HBaseTestCase and HBaseClusterTestCase functionality. * old HBaseTestCase and HBaseClusterTestCase functionality.
@ -494,17 +494,6 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
return new Path(fs.getWorkingDirectory(), "test-data"); 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()} * 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 * to write temporary test data. Call this method after setting up the mini dfs cluster

View File

@ -0,0 +1,109 @@
/*
* 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 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.
* 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.disableTable(TableName.META_TABLE_NAME);
TableDescriptor descriptor = admin.getDescriptor(TableName.META_TABLE_NAME);
ColumnFamilyDescriptor cfd = descriptor.getColumnFamily(HConstants.CATALOG_FAMILY);
byte [] extraColumnFamilyName = Bytes.toBytes("xtra");
ColumnFamilyDescriptor newCfd =
ColumnFamilyDescriptorBuilder.newBuilder(extraColumnFamilyName).build();
int oldVersions = cfd.getMaxVersions();
// Add '1' to current versions count.
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);
admin.addColumnFamily(TableName.META_TABLE_NAME, newCfd);
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());
admin.enableTable(TableName.META_TABLE_NAME);
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);
}
}

View File

@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.Waiter.Predicate; import org.apache.hadoop.hbase.Waiter.Predicate;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager; import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
@ -504,22 +503,6 @@ public class TestAdmin2 extends TestAdminBase {
return regionServer; return regionServer;
} }
@Test
public void testDisableCatalogTable() throws Exception {
try {
ADMIN.disableTable(TableName.META_TABLE_NAME);
fail("Expected to throw ConstraintException");
} catch (ConstraintException e) {
}
// Before the fix for HBASE-6146, the below table creation was failing as the hbase:meta table
// actually getting disabled by the disableTable() call.
HTableDescriptor htd =
new HTableDescriptor(TableName.valueOf(Bytes.toBytes(name.getMethodName())));
HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes("cf1"));
htd.addFamily(hcd);
TEST_UTIL.getAdmin().createTable(htd);
}
@Test @Test
public void testIsEnabledOrDisabledOnUnknownTable() throws Exception { public void testIsEnabledOrDisabledOnUnknownTable() throws Exception {
try { try {

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license * 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 * 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 * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the

View File

@ -39,7 +39,6 @@ import java.util.Set;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
/** /**
* Class to test asynchronous table admin operations * Class to test asynchronous table admin operations
@ -54,18 +53,6 @@ public class TestAsyncTableAdminApi2 extends TestAsyncAdminBase {
public static final HBaseClassTestRule CLASS_RULE = public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestAsyncTableAdminApi2.class); HBaseClassTestRule.forClass(TestAsyncTableAdminApi2.class);
@Test
public void testDisableCatalogTable() throws Exception {
try {
this.admin.disableTable(TableName.META_TABLE_NAME).join();
fail("Expected to throw ConstraintException");
} catch (Exception e) {
}
// Before the fix for HBASE-6146, the below table creation was failing as the hbase:meta table
// actually getting disabled by the disableTable() call.
createTableWithDefaultConf(tableName);
}
@Test @Test
public void testAddColumnFamily() throws Exception { public void testAddColumnFamily() throws Exception {
// Create a table with two families // Create a table with two families

View File

@ -18,12 +18,9 @@
package org.apache.hadoop.hbase.client; package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME; import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
@ -32,7 +29,6 @@ import java.util.Optional;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import org.apache.hadoop.hbase.AsyncMetaTableAccessor; import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
@ -201,14 +197,6 @@ public class TestAsyncTableAdminApi3 extends TestAsyncAdminBase {
ok = false; ok = false;
} }
assertTrue(ok); assertTrue(ok);
// meta table can not be disabled.
try {
admin.disableTable(TableName.META_TABLE_NAME).get();
fail("meta table can not be disabled");
} catch (ExecutionException e) {
Throwable cause = e.getCause();
assertThat(cause, instanceOf(DoNotRetryIOException.class));
}
} }
@Test @Test

View File

@ -147,6 +147,13 @@ public class TestMetaWithReplicas {
public void testMetaHTDReplicaCount() throws Exception { public void testMetaHTDReplicaCount() throws Exception {
assertEquals(3, assertEquals(3,
TEST_UTIL.getAdmin().getDescriptor(TableName.META_TABLE_NAME).getRegionReplication()); TEST_UTIL.getAdmin().getDescriptor(TableName.META_TABLE_NAME).getRegionReplication());
try (AsyncConnection connection =
ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get()) {
AsyncTable t = connection.getTable(TableName.META_TABLE_NAME);
List<HRegionLocation> rls =
t.getRegionLocator().getRegionLocations(HConstants.EMPTY_START_ROW, true).get();
assertEquals(3, rls.size());
}
} }
@Test @Test

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information

View File

@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WAL;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule; import org.junit.Rule;
@ -87,8 +88,10 @@ public class TestGetClosestAtOrBefore {
public void testUsingMetaAndBinary() throws IOException { public void testUsingMetaAndBinary() throws IOException {
FileSystem filesystem = FileSystem.get(conf); FileSystem filesystem = FileSystem.get(conf);
Path rootdir = UTIL.getDataTestDirOnTestFS(); Path rootdir = UTIL.getDataTestDirOnTestFS();
// Up flush size else we bind up when we use default catalog flush of 16k. // Up flush size else we bind up when we use default catalog flush of 16k.
TableDescriptorBuilder metaBuilder = UTIL.getMetaTableDescriptorBuilder() TableDescriptorBuilder metaBuilder =
FSTableDescriptors.createMetaTableDescriptorBuilder(UTIL.getConfiguration())
.setMemStoreFlushSize(64 * 1024 * 1024); .setMemStoreFlushSize(64 * 1024 * 1024);
HRegion mr = HBaseTestingUtility.createRegionAndWAL(HRegionInfo.FIRST_META_REGIONINFO, HRegion mr = HBaseTestingUtility.createRegionAndWAL(HRegionInfo.FIRST_META_REGIONINFO,

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes; 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.FSUtils;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WAL;
@ -77,8 +78,6 @@ public class TestLogRollingNoCluster {
/** /**
* Spin up a bunch of threads and have them all append to a WAL. Roll the * Spin up a bunch of threads and have them all append to a WAL. Roll the
* WAL frequently to try and trigger NPE. * WAL frequently to try and trigger NPE.
* @throws IOException
* @throws InterruptedException
*/ */
@Test @Test
public void testContendedLogRolling() throws Exception { public void testContendedLogRolling() throws Exception {
@ -161,7 +160,8 @@ public class TestLogRollingNoCluster {
byte[] bytes = Bytes.toBytes(i); byte[] bytes = Bytes.toBytes(i);
edit.add(new KeyValue(bytes, bytes, bytes, now, EMPTY_1K_ARRAY)); edit.add(new KeyValue(bytes, bytes, bytes, now, EMPTY_1K_ARRAY));
RegionInfo hri = RegionInfoBuilder.FIRST_META_REGIONINFO; RegionInfo hri = RegionInfoBuilder.FIRST_META_REGIONINFO;
TableDescriptor htd = TEST_UTIL.getMetaTableDescriptorBuilder().build(); TableDescriptor htd =
FSTableDescriptors.createMetaTableDescriptor(TEST_UTIL.getConfiguration());
NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
for(byte[] fam : htd.getColumnFamilyNames()) { for(byte[] fam : htd.getColumnFamilyNames()) {
scopes.put(fam, 0); scopes.put(fam, 0);

View File

@ -322,7 +322,9 @@ public class TestFSTableDescriptors {
} }
Map<String, TableDescriptor> tables = tds.getAll(); Map<String, TableDescriptor> tables = tds.getAll();
assertEquals(4, tables.size()); assertEquals(5, tables.size());
// Remove because it messes up below order test.
tables.remove(TableName.META_TABLE_NAME.toString());
String[] tableNamesOrdered = String[] tableNamesOrdered =
new String[] { "bar:foo", "default:bar", "default:foo", "foo:bar" }; new String[] { "bar:foo", "default:bar", "default:foo", "foo:bar" };
@ -368,6 +370,9 @@ public class TestFSTableDescriptors {
for (Map.Entry<String, TableDescriptor> entry: nonchtds.getAll().entrySet()) { for (Map.Entry<String, TableDescriptor> entry: nonchtds.getAll().entrySet()) {
String t = (String) entry.getKey(); String t = (String) entry.getKey();
if (t.equals(TableName.META_TABLE_NAME.toString())) {
continue;
}
TableDescriptor nchtd = entry.getValue(); TableDescriptor nchtd = entry.getValue();
assertTrue("expected " + htd.toString() + assertTrue("expected " + htd.toString() +
" got: " + chtds.get(TableName.valueOf(t)).toString(), " got: " + chtds.get(TableName.valueOf(t)).toString(),

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -61,14 +61,6 @@ public final class MetaTableLocator {
private 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 * @param zkw ZooKeeper watcher to be used
* @return meta table regions and their locations. * @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 zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @param replicaId the ID of the replica * @param replicaId the ID of the replica
@ -306,10 +298,8 @@ public final class MetaTableLocator {
if (serverName == null) { if (serverName == null) {
state = RegionState.State.OFFLINE; state = RegionState.State.OFFLINE;
} }
return new RegionState( return new RegionState(RegionReplicaUtil.getRegionInfoForReplica(
RegionReplicaUtil.getRegionInfoForReplica( RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), state, serverName);
RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId),
state, serverName);
} }
/** /**

View File

@ -2057,7 +2057,7 @@ public final class ZKUtil {
" byte(s) of data from znode " + znode + " byte(s) of data from znode " + znode +
(watcherSet? " and set watcher; ": "; data=") + (watcherSet? " and set watcher; ": "; data=") +
(data == null? "null": data.length == 0? "empty": ( (data == null? "null": data.length == 0? "empty": (
znode.startsWith(zkw.getZNodePaths().metaZNodePrefix)? zkw.getZNodePaths().isMetaZNodePrefix(znode)?
getServerNameOrEmptyString(data): getServerNameOrEmptyString(data):
znode.startsWith(zkw.getZNodePaths().backupMasterAddressesZNode)? znode.startsWith(zkw.getZNodePaths().backupMasterAddressesZNode)?
getServerNameOrEmptyString(data): getServerNameOrEmptyString(data):