HBASE-23055 Alter hbase:meta (#1043)
Make hbase:meta region schema dynamic. Patch has been under development a good while and its focus has changed a few times so its bloated with fixup from older versions. M hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java M hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java Shut down access to internals and removed unused methods. M hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java Cleanup/refactor section on replica-handling. M hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java Get hbase:meta schema from filesystem rather than from hard-coding.
This commit is contained in:
parent
00e64d83e8
commit
ba3463d9de
|
@ -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
|
||||
*/
|
||||
|
|
|
@ -666,22 +666,35 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
new DisableTableProcedureBiConsumer(tableName));
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility for completing passed TableState {@link CompletableFuture} <code>future</code>
|
||||
* using passed parameters. Sets error or boolean result ('true' if table matches
|
||||
* the passed-in targetState).
|
||||
*/
|
||||
private static CompletableFuture<Boolean> completeCheckTableState(
|
||||
CompletableFuture<Boolean> 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<Boolean> isTableEnabled(TableName tableName) {
|
||||
if (TableName.isMetaTableName(tableName)) {
|
||||
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));
|
||||
}
|
||||
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<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.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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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<Integer, String> 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<Integer, String> 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<String> 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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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<String> 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";
|
||||
|
|
|
@ -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<String, TableDescriptor> getByNamespace(String name)
|
||||
throws IOException;
|
||||
Map<String, TableDescriptor> 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<String, TableDescriptor> 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
|
||||
|
|
|
@ -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<InitMetaProcedure> optProc = procedureExecutor.getProcedures().stream()
|
||||
.filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny();
|
||||
initMetaProc = optProc.orElseGet(() -> {
|
||||
|
|
|
@ -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<TableName, TableState.State> 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<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.
|
||||
final Set<TableName> 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) {
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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<RegionInfo> 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<RegionInfo> copyOfRegions = new ArrayList<RegionInfo>(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<RegionInfo> 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<RegionInfo> 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<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;
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<byte []> 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<byte []> 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<byte[]> oldFamilies = unmodifiedTableDescriptor.getColumnFamilyNames();
|
||||
final Set<byte[]> 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<byte[]> originalFamilies = originalDescriptor.getColumnFamilyNames();
|
||||
final Set<byte[]> 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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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<TState> 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);
|
||||
|
|
|
@ -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<PeerModi
|
|||
return false;
|
||||
}
|
||||
Thread.sleep(SLEEP_INTERVAL_MS);
|
||||
} catch (TableStateNotFoundException e) {
|
||||
} catch (TableNotFoundException e) {
|
||||
return false;
|
||||
} catch (InterruptedException e) {
|
||||
throw (IOException) new InterruptedIOException(e.getMessage()).initCause(e);
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
/**
|
||||
/*
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
|
@ -41,6 +41,6 @@ public class MetaLocationSyncer extends ClientZKSyncer {
|
|||
|
||||
@Override
|
||||
Collection<String> getNodesToWatch() {
|
||||
return watcher.getZNodePaths().metaReplicaZNodes.values();
|
||||
return watcher.getZNodePaths().getMetaReplicaZNodes();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
|
|
|
@ -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<TableDescriptorBuilder, TableDescriptorBuilder> metaObserver) throws IOException {
|
||||
final Path rootdir, final boolean fsreadonly, final boolean usecache,
|
||||
Function<TableDescriptorBuilder, TableDescriptorBuilder> 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 <tt>true</tt>,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 <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);
|
||||
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;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
||||
|
|
|
@ -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"));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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++) {
|
||||
|
|
|
@ -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<byte[], Integer> 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(),
|
||||
|
|
|
@ -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<String, TableDescriptor> 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<String, TableDescriptor> 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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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):
|
||||
|
|
Loading…
Reference in New Issue