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