diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java index eb050fcc0be..c37a63fb3e6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java @@ -1414,27 +1414,11 @@ public class HTableDescriptor implements WritableComparable { * HBaseAdmin#getTableDescriptor(TableName.META_TABLE_NAME) instead. */ @Deprecated - public static final HTableDescriptor META_TABLEDESC = new HTableDescriptor( - TableName.META_TABLE_NAME, - new HColumnDescriptor[] { - new HColumnDescriptor(HConstants.CATALOG_FAMILY) - // Ten is arbitrary number. Keep versions to help debugging. - .setMaxVersions(HConstants.DEFAULT_HBASE_META_VERSIONS) - .setInMemory(true) - .setBlocksize(HConstants.DEFAULT_HBASE_META_BLOCK_SIZE) - .setScope(HConstants.REPLICATION_SCOPE_LOCAL) - // Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore. - .setBloomFilterType(BloomType.NONE) - // Enable cache of data blocks in L1 if more than one caching tier deployed: - // e.g. if using CombinedBlockCache (BucketCache). - .setCacheDataInL1(true) - }); + public static final HTableDescriptor META_TABLEDESC; static { try { - META_TABLEDESC.addCoprocessor( - "org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint", - null, Coprocessor.PRIORITY_SYSTEM, null); + META_TABLEDESC = metaTableDescriptor(new Configuration()); } catch (IOException ex) { //LOG.warn("exception in loading coprocessor for the hbase:meta table"); throw new RuntimeException(ex); @@ -1604,21 +1588,32 @@ public class HTableDescriptor implements WritableComparable { public static HTableDescriptor metaTableDescriptor(final Configuration conf) throws IOException { HTableDescriptor metaDescriptor = new HTableDescriptor( - TableName.META_TABLE_NAME, - new HColumnDescriptor[] { - new HColumnDescriptor(HConstants.CATALOG_FAMILY) - .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS, - HConstants.DEFAULT_HBASE_META_VERSIONS)) - .setInMemory(true) - .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE, - HConstants.DEFAULT_HBASE_META_BLOCK_SIZE)) - .setScope(HConstants.REPLICATION_SCOPE_LOCAL) - // Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore. - .setBloomFilterType(BloomType.NONE) - }); + TableName.META_TABLE_NAME, + new HColumnDescriptor[] { + new HColumnDescriptor(HConstants.CATALOG_FAMILY) + .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS, + HConstants.DEFAULT_HBASE_META_VERSIONS)) + .setInMemory(true) + .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE, + HConstants.DEFAULT_HBASE_META_BLOCK_SIZE)) + .setScope(HConstants.REPLICATION_SCOPE_LOCAL) + // Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore. + .setBloomFilterType(BloomType.NONE), + new HColumnDescriptor(HConstants.TABLE_FAMILY) + // Ten is arbitrary number. Keep versions to help debugging. + .setMaxVersions(10) + .setInMemory(true) + .setBlocksize(8 * 1024) + .setScope(HConstants.REPLICATION_SCOPE_LOCAL) + // Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore. + .setBloomFilterType(BloomType.NONE) + // Enable cache of data blocks in L1 if more than one caching tier deployed: + // e.g. if using CombinedBlockCache (BucketCache). + .setCacheDataInL1(true) + }); metaDescriptor.addCoprocessor( - "org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint", - null, Coprocessor.PRIORITY_SYSTEM, null); + "org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint", + null, Coprocessor.PRIORITY_SYSTEM, null); return metaDescriptor; } 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 37ef788de10..8d486bf5ab0 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 @@ -17,9 +17,23 @@ */ package org.apache.hadoop.hbase; +import javax.annotation.Nullable; +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.ServiceException; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -38,6 +52,8 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableState; +import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; @@ -50,17 +66,6 @@ import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; -import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.Set; -import java.util.SortedMap; -import java.util.TreeMap; -import java.util.regex.Matcher; -import java.util.regex.Pattern; /** * Read/write operations on region and assignment information store in @@ -80,6 +85,11 @@ public class MetaTableAccessor { * HRI defined which is called default replica. * * Meta layout (as of 0.98 + HBASE-10070) is like: + * + * For each table there is single row in column family 'table' formatted: + * including namespace and columns are: + * table: state => contains table state + * * For each table range, there is a single row, formatted like: * ,,,. This row corresponds to the regionName * of the default region replica. @@ -122,6 +132,24 @@ public class MetaTableAccessor { META_REGION_PREFIX, 0, len); } + + @InterfaceAudience.Private + public enum QueryType { + ALL(HConstants.TABLE_FAMILY, HConstants.CATALOG_FAMILY), + REGION(HConstants.CATALOG_FAMILY), + TABLE(HConstants.TABLE_FAMILY); + + private final byte[][] families; + + QueryType(byte[]... families) { + this.families = families; + } + + byte[][] getFamilies() { + return this.families; + } + } + /** The delimiter for meta columns for replicaIds > 0 */ protected static final char META_REPLICA_ID_DELIMITER = '_'; @@ -133,40 +161,64 @@ public class MetaTableAccessor { // Reading operations // //////////////////////// - /** - * Performs a full scan of a hbase:meta table. - * @return List of {@link org.apache.hadoop.hbase.client.Result} + /** + * Performs a full scan of hbase:meta for regions. + * @param connection connection we're using + * @param visitor Visitor invoked against each row in regions family. * @throws IOException */ - public static List fullScanOfMeta(Connection connection) - throws IOException { - CollectAllVisitor v = new CollectAllVisitor(); - fullScan(connection, v, null); - return v.getResults(); + public static void fullScanRegions(Connection connection, + final Visitor visitor) + throws IOException { + fullScan(connection, visitor, null, QueryType.REGION); + } + + /** + * Performs a full scan of hbase:meta for regions. + * @param connection connection we're using + * @throws IOException + */ + public static List fullScanRegions(Connection connection) + throws IOException { + return fullScan(connection, QueryType.REGION); + } + + /** + * Performs a full scan of hbase:meta for tables. + * @param connection connection we're using + * @param visitor Visitor invoked against each row in tables family. + * @throws IOException + */ + public static void fullScanTables(Connection connection, + final Visitor visitor) + throws IOException { + fullScan(connection, visitor, null, QueryType.TABLE); } /** * Performs a full scan of hbase:meta. * @param connection connection we're using * @param visitor Visitor invoked against each row. + * @param type scanned part of meta * @throws IOException */ public static void fullScan(Connection connection, - final Visitor visitor) + final Visitor visitor, QueryType type) throws IOException { - fullScan(connection, visitor, null); + fullScan(connection, visitor, null, type); } /** * Performs a full scan of hbase:meta. * @param connection connection we're using + * @param type scanned part of meta * @return List of {@link Result} * @throws IOException */ - public static List fullScan(Connection connection) + public static List fullScan(Connection connection, QueryType type) throws IOException { CollectAllVisitor v = new CollectAllVisitor(); - fullScan(connection, v, null); + fullScan(connection, v, null, type); return v.getResults(); } @@ -307,6 +359,7 @@ public class MetaTableAccessor { * @return null if it doesn't contain merge qualifier, else two merge regions * @throws IOException */ + @Nullable public static Pair getRegionsFromMergeQualifier( Connection connection, byte[] regionName) throws IOException { Result result = getRegionResult(connection, regionName); @@ -329,42 +382,9 @@ public class MetaTableAccessor { public static boolean tableExists(Connection connection, final TableName tableName) throws IOException { - if (tableName.equals(TableName.META_TABLE_NAME)) { - // Catalog tables always exist. - return true; - } - // Make a version of ResultCollectingVisitor that only collects the first - CollectingVisitor visitor = new CollectingVisitor() { - private HRegionInfo current = null; - - @Override - public boolean visit(Result r) throws IOException { - RegionLocations locations = getRegionLocations(r); - if (locations == null || locations.getRegionLocation().getRegionInfo() == null) { - LOG.warn("No serialized HRegionInfo in " + r); - return true; - } - this.current = locations.getRegionLocation().getRegionInfo(); - if (this.current == null) { - LOG.warn("No serialized HRegionInfo in " + r); - return true; - } - if (!isInsideTable(this.current, tableName)) return false; - // Else call super and add this Result to the collection. - super.visit(r); - // Stop collecting regions from table after we get one. - return false; - } - - @Override - void add(Result r) { - // Add the current HRI. - this.results.add(this.current); - } - }; - fullScan(connection, visitor, getTableStartRowForMeta(tableName)); - // If visitor has results >= 1 then table exists. - return visitor.getResults().size() >= 1; + // Catalog tables always exist. + return tableName.equals(TableName.META_TABLE_NAME) + || getTableState(connection, tableName) != null; } /** @@ -400,6 +420,7 @@ public class MetaTableAccessor { return getListOfHRegionInfos(result); } + @Nullable static List getListOfHRegionInfos(final List> pairs) { if (pairs == null || pairs.isEmpty()) return null; List result = new ArrayList(pairs.size()); @@ -519,7 +540,7 @@ public class MetaTableAccessor { } } }; - fullScan(connection, visitor, getTableStartRowForMeta(tableName)); + fullScan(connection, visitor, getTableStartRowForMeta(tableName), QueryType.REGION); return visitor.getResults(); } @@ -551,7 +572,7 @@ public class MetaTableAccessor { } } }; - fullScan(connection, v); + fullScan(connection, v, QueryType.REGION); return hris; } @@ -562,17 +583,22 @@ public class MetaTableAccessor { public boolean visit(Result r) throws IOException { if (r == null || r.isEmpty()) return true; LOG.info("fullScanMetaAndPrint.Current Meta Row: " + r); - RegionLocations locations = getRegionLocations(r); - if (locations == null) return true; - for (HRegionLocation loc : locations.getRegionLocations()) { - if (loc != null) { - LOG.info("fullScanMetaAndPrint.HRI Print= " + loc.getRegionInfo()); + TableState state = getTableState(r); + if (state != null) { + LOG.info("Table State: " + state); + } else { + RegionLocations locations = getRegionLocations(r); + if (locations == null) return true; + for (HRegionLocation loc : locations.getRegionLocations()) { + if (loc != null) { + LOG.info("fullScanMetaAndPrint.HRI Print= " + loc.getRegionInfo()); + } } } return true; } }; - fullScan(connection, v); + fullScan(connection, v, QueryType.ALL); } /** @@ -581,20 +607,40 @@ public class MetaTableAccessor { * @param visitor Visitor invoked against each row. * @param startrow Where to start the scan. Pass null if want to begin scan * at first row. + * @param type scanned part of meta * hbase:meta, the default (pass false to scan hbase:meta) * @throws IOException */ public static void fullScan(Connection connection, - final Visitor visitor, final byte [] startrow) + final Visitor visitor, @Nullable final byte[] startrow, QueryType type) throws IOException { + fullScan(connection, visitor, startrow, type, false); + } + + /** + * Performs a full scan of a catalog table. + * @param connection connection we're using + * @param visitor Visitor invoked against each row. + * @param startrow Where to start the scan. Pass null if want to begin scan + * at first row. + * @param type scanned part of meta + * @param raw read raw data including Delete tumbstones + * hbase:meta, the default (pass false to scan hbase:meta) + * @throws IOException + */ + public static void fullScan(Connection connection, + final Visitor visitor, @Nullable final byte[] startrow, QueryType type, boolean raw) throws IOException { Scan scan = new Scan(); + scan.setRaw(raw); if (startrow != null) scan.setStartRow(startrow); if (startrow == null) { int caching = connection.getConfiguration() .getInt(HConstants.HBASE_META_SCANNER_CACHING, 100); scan.setCaching(caching); } - scan.addFamily(HConstants.CATALOG_FAMILY); + for (byte[] family : type.getFamilies()) { + scan.addFamily(family); + } Table metaTable = getMetaHTable(connection); ResultScanner scanner = null; try { @@ -615,10 +661,18 @@ public class MetaTableAccessor { * Returns the column family used for meta columns. * @return HConstants.CATALOG_FAMILY. */ - protected static byte[] getFamily() { + protected static byte[] getCatalogFamily() { return HConstants.CATALOG_FAMILY; } + /** + * Returns the column family used for table columns. + * @return HConstants.TABLE_FAMILY. + */ + protected static byte[] getTableFamily() { + return HConstants.TABLE_FAMILY; + } + /** * Returns the column qualifier for serialized region info * @return HConstants.REGIONINFO_QUALIFIER @@ -627,6 +681,15 @@ public class MetaTableAccessor { return HConstants.REGIONINFO_QUALIFIER; } + /** + * Returns the column qualifier for serialized table state + * + * @return HConstants.TABLE_STATE_QUALIFIER + */ + protected static byte[] getStateColumn() { + return HConstants.TABLE_STATE_QUALIFIER; + } + /** * Returns the column qualifier for server column for replicaId * @param replicaId the replicaId of the region @@ -693,14 +756,15 @@ public class MetaTableAccessor { * @param r Result to pull from * @return A ServerName instance or null if necessary fields not found or empty. */ + @Nullable private static ServerName getServerName(final Result r, final int replicaId) { byte[] serverColumn = getServerColumn(replicaId); - Cell cell = r.getColumnLatestCell(getFamily(), serverColumn); + Cell cell = r.getColumnLatestCell(getCatalogFamily(), serverColumn); if (cell == null || cell.getValueLength() == 0) return null; String hostAndPort = Bytes.toString( cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); byte[] startcodeColumn = getStartCodeColumn(replicaId); - cell = r.getColumnLatestCell(getFamily(), startcodeColumn); + cell = r.getColumnLatestCell(getCatalogFamily(), startcodeColumn); if (cell == null || cell.getValueLength() == 0) return null; return ServerName.valueOf(hostAndPort, Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength())); @@ -713,7 +777,7 @@ public class MetaTableAccessor { * @return SeqNum, or HConstants.NO_SEQNUM if there's no value written. */ private static long getSeqNumDuringOpen(final Result r, final int replicaId) { - Cell cell = r.getColumnLatestCell(getFamily(), getSeqNumColumn(replicaId)); + Cell cell = r.getColumnLatestCell(getCatalogFamily(), getSeqNumColumn(replicaId)); if (cell == null || cell.getValueLength() == 0) return HConstants.NO_SEQNUM; return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); } @@ -723,6 +787,7 @@ public class MetaTableAccessor { * @return an HRegionLocationList containing all locations for the region range or null if * we can't deserialize the result. */ + @Nullable public static RegionLocations getRegionLocations(final Result r) { if (r == null) return null; HRegionInfo regionInfo = getHRegionInfo(r, getRegionInfoColumn()); @@ -733,7 +798,7 @@ public class MetaTableAccessor { locations.add(getRegionLocation(r, regionInfo, 0)); - NavigableMap infoMap = familyMap.get(getFamily()); + NavigableMap infoMap = familyMap.get(getCatalogFamily()); if (infoMap == null) return new RegionLocations(locations); // iterate until all serverName columns are seen @@ -795,8 +860,9 @@ public class MetaTableAccessor { * @param qualifier Column family qualifier * @return An HRegionInfo instance or null. */ + @Nullable private static HRegionInfo getHRegionInfo(final Result r, byte [] qualifier) { - Cell cell = r.getColumnLatestCell(getFamily(), qualifier); + Cell cell = r.getColumnLatestCell(getCatalogFamily(), qualifier); if (cell == null) return null; return HRegionInfo.parseFromOrNull(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); @@ -830,6 +896,80 @@ public class MetaTableAccessor { return new PairOfSameType(mergeA, mergeB); } + /** + * Fetch table state for given table from META table + * @param conn connection to use + * @param tableName table to fetch state for + * @return state + * @throws IOException + */ + @Nullable + public static TableState getTableState(Connection conn, TableName tableName) + throws IOException { + Table metaHTable = getMetaHTable(conn); + Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getStateColumn()); + long time = EnvironmentEdgeManager.currentTime(); + get.setTimeRange(0, time); + Result result = + metaHTable.get(get); + return getTableState(result); + } + + /** + * Fetch table states from META table + * @param conn connection to use + * @return map {tableName -> state} + * @throws IOException + */ + public static Map getTableStates(Connection conn) + throws IOException { + final Map states = new LinkedHashMap<>(); + Visitor collector = new Visitor() { + @Override + public boolean visit(Result r) throws IOException { + TableState state = getTableState(r); + if (state != null) + states.put(state.getTableName(), state); + return true; + } + }; + fullScanTables(conn, collector); + return states; + } + + /** + * Updates state in META + * @param conn connection to use + * @param tableName table to look for + * @throws IOException + */ + public static void updateTableState(Connection conn, TableName tableName, + TableState.State actual) throws IOException { + updateTableState(conn, new TableState(tableName, actual)); + } + + /** + * Decode table state from META Result. + * Should contain cell from HConstants.TABLE_FAMILY + * @param r result + * @return null if not found + * @throws IOException + */ + @Nullable + public static TableState getTableState(Result r) + throws IOException { + Cell cell = r.getColumnLatestCell(getTableFamily(), getStateColumn()); + if (cell == null) return null; + try { + return TableState.parseFrom(TableName.valueOf(r.getRow()), + Arrays.copyOfRange(cell.getValueArray(), + cell.getValueOffset(), cell.getValueOffset() + cell.getValueLength())); + } catch (DeserializationException e) { + throw new IOException(e); + } + + } + /** * Implementations 'visit' a catalog table row. */ @@ -927,7 +1067,8 @@ public class MetaTableAccessor { */ public static Put makePutFromRegionInfo(HRegionInfo regionInfo) throws IOException { - Put put = new Put(regionInfo.getRegionName()); + long now = EnvironmentEdgeManager.currentTime(); + Put put = new Put(regionInfo.getRegionName(), now); addRegionInfo(put, regionInfo); return put; } @@ -940,7 +1081,9 @@ public class MetaTableAccessor { if (regionInfo == null) { throw new IllegalArgumentException("Can't make a delete for null region"); } + long now = EnvironmentEdgeManager.currentTime(); Delete delete = new Delete(regionInfo.getRegionName()); + delete.addFamily(getCatalogFamily(), now); return delete; } @@ -1041,14 +1184,15 @@ public class MetaTableAccessor { throws IOException { int absoluteIndex = replicaIndexToDeleteFrom + numReplicasToRemove; for (byte[] row : metaRows) { + long now = EnvironmentEdgeManager.currentTime(); Delete deleteReplicaLocations = new Delete(row); for (int i = replicaIndexToDeleteFrom; i < absoluteIndex; i++) { - deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY, - getServerColumn(i)); - deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY, - getSeqNumColumn(i)); - deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY, - getStartCodeColumn(i)); + deleteReplicaLocations.addColumns(getCatalogFamily(), + getServerColumn(i), now); + deleteReplicaLocations.addColumns(getCatalogFamily(), + getSeqNumColumn(i), now); + deleteReplicaLocations.addColumns(getCatalogFamily(), + getStartCodeColumn(i), now); } deleteFromMetaTable(connection, deleteReplicaLocations); } @@ -1178,7 +1322,8 @@ public class MetaTableAccessor { public static void addDaughter(final Connection connection, final HRegionInfo regionInfo, final ServerName sn, final long openSeqNum) throws NotAllMetaRegionsOnlineException, IOException { - Put put = new Put(regionInfo.getRegionName()); + long now = EnvironmentEdgeManager.currentTime(); + Put put = new Put(regionInfo.getRegionName(), now); addRegionInfo(put, regionInfo); if (sn != null) { addLocation(put, sn, openSeqNum, regionInfo.getReplicaId()); @@ -1279,6 +1424,45 @@ public class MetaTableAccessor { } } + /** + * Update state of the table in meta. + * @param connection what we use for update + * @param state new state + * @throws IOException + */ + public static void updateTableState(Connection connection, TableState state) + throws IOException { + Put put = makePutFromTableState(state); + putToMetaTable(connection, put); + LOG.info( + "Updated table " + state.getTableName() + " state to " + state.getState() + " in META"); + } + + /** + * Construct PUT for given state + * @param state new state + */ + public static Put makePutFromTableState(TableState state) { + long time = EnvironmentEdgeManager.currentTime(); + Put put = new Put(state.getTableName().getName(), time); + put.add(getTableFamily(), getStateColumn(), state.convert().toByteArray()); + return put; + } + + /** + * Remove state for table from meta + * @param connection to use for deletion + * @param table to delete state for + */ + public static void deleteTableState(Connection connection, TableName table) + throws IOException { + long time = EnvironmentEdgeManager.currentTime(); + Delete delete = new Delete(table.getName()); + delete.addColumns(getTableFamily(), getStateColumn(), time); + deleteFromMetaTable(connection, delete); + LOG.info("Deleted table " + table + " state from META"); + } + /** * Performs an atomic multi-Mutate operation against the given table. */ @@ -1344,7 +1528,8 @@ public class MetaTableAccessor { HRegionInfo regionInfo, ServerName sn, long openSeqNum) throws IOException { // region replicas are kept in the primary region's row - Put put = new Put(getMetaKeyForRegion(regionInfo)); + long time = EnvironmentEdgeManager.currentTime(); + Put put = new Put(getMetaKeyForRegion(regionInfo), time); addLocation(put, sn, openSeqNum, regionInfo.getReplicaId()); putToMetaTable(connection, put); LOG.info("Updated row " + regionInfo.getRegionNameAsString() + @@ -1360,7 +1545,9 @@ public class MetaTableAccessor { public static void deleteRegion(Connection connection, HRegionInfo regionInfo) throws IOException { + long time = EnvironmentEdgeManager.currentTime(); Delete delete = new Delete(regionInfo.getRegionName()); + delete.addFamily(getCatalogFamily(), time); deleteFromMetaTable(connection, delete); LOG.info("Deleted " + regionInfo.getRegionNameAsString()); } @@ -1374,8 +1561,11 @@ public class MetaTableAccessor { public static void deleteRegions(Connection connection, List regionsInfo) throws IOException { List deletes = new ArrayList(regionsInfo.size()); + long time = EnvironmentEdgeManager.currentTime(); for (HRegionInfo hri: regionsInfo) { - deletes.add(new Delete(hri.getRegionName())); + Delete e = new Delete(hri.getRegionName()); + e.addFamily(getCatalogFamily(), time); + deletes.add(e); } deleteFromMetaTable(connection, deletes); LOG.info("Deleted " + regionsInfo); @@ -1395,7 +1585,7 @@ public class MetaTableAccessor { List mutation = new ArrayList(); if (regionsToRemove != null) { for (HRegionInfo hri: regionsToRemove) { - mutation.add(new Delete(hri.getRegionName())); + mutation.add(makeDeleteFromRegionInfo(hri)); } } if (regionsToAdd != null) { @@ -1438,9 +1628,10 @@ public class MetaTableAccessor { */ public static void deleteMergeQualifiers(Connection connection, final HRegionInfo mergedRegion) throws IOException { + long time = EnvironmentEdgeManager.currentTime(); Delete delete = new Delete(mergedRegion.getRegionName()); - delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER); - delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER); + delete.addColumns(getCatalogFamily(), HConstants.MERGEA_QUALIFIER, time); + delete.addColumns(getCatalogFamily(), HConstants.MERGEB_QUALIFIER, time); deleteFromMetaTable(connection, delete); LOG.info("Deleted references in merged region " + mergedRegion.getRegionNameAsString() + ", qualifier=" @@ -1450,7 +1641,7 @@ public class MetaTableAccessor { private static Put addRegionInfo(final Put p, final HRegionInfo hri) throws IOException { - p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, + p.addImmutable(getCatalogFamily(), HConstants.REGIONINFO_QUALIFIER, hri.toByteArray()); return p; } @@ -1459,20 +1650,20 @@ public class MetaTableAccessor { // using regionserver's local time as the timestamp of Put. // See: HBASE-11536 long now = EnvironmentEdgeManager.currentTime(); - p.addImmutable(HConstants.CATALOG_FAMILY, getServerColumn(replicaId), now, + p.addImmutable(getCatalogFamily(), getServerColumn(replicaId), now, Bytes.toBytes(sn.getHostAndPort())); - p.addImmutable(HConstants.CATALOG_FAMILY, getStartCodeColumn(replicaId), now, + p.addImmutable(getCatalogFamily(), getStartCodeColumn(replicaId), now, Bytes.toBytes(sn.getStartcode())); - p.addImmutable(HConstants.CATALOG_FAMILY, getSeqNumColumn(replicaId), now, + p.addImmutable(getCatalogFamily(), getSeqNumColumn(replicaId), now, Bytes.toBytes(openSeqNum)); return p; } public static Put addEmptyLocation(final Put p, int replicaId) { long now = EnvironmentEdgeManager.currentTime(); - p.addImmutable(HConstants.CATALOG_FAMILY, getServerColumn(replicaId), now, null); - p.addImmutable(HConstants.CATALOG_FAMILY, getStartCodeColumn(replicaId), now, null); - p.addImmutable(HConstants.CATALOG_FAMILY, getSeqNumColumn(replicaId), now, null); + p.addImmutable(getCatalogFamily(), getServerColumn(replicaId), now, null); + p.addImmutable(getCatalogFamily(), getStartCodeColumn(replicaId), now, null); + p.addImmutable(getCatalogFamily(), getSeqNumColumn(replicaId), now, null); return p; } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java new file mode 100644 index 00000000000..3f449274637 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java @@ -0,0 +1,56 @@ +/** + * 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.client; + +import java.io.Closeable; +import java.io.IOException; + +/** + * A RetryingCallable for generic connection operations. + * @param return type + */ +abstract class ConnectionCallable implements RetryingCallable, Closeable { + protected Connection connection; + + public ConnectionCallable(final Connection connection) { + this.connection = connection; + } + + @Override + public void prepare(boolean reload) throws IOException { + } + + @Override + public void close() throws IOException { + } + + @Override + public void throwable(Throwable t, boolean retrying) { + } + + @Override + public String getExceptionMessageAdditionalDetail() { + return ""; + } + + @Override + public long sleep(long pause, int tries) { + return ConnectionUtils.getPauseTime(pause, tries); + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java index 9b8f324c00f..4b267c0a9f3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java @@ -18,6 +18,8 @@ */ package org.apache.hadoop.hbase.client; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; import java.io.InterruptedIOException; @@ -37,9 +39,12 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import com.google.common.annotations.VisibleForTesting; +import com.google.protobuf.BlockingRpcChannel; +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -60,8 +65,6 @@ import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture; -import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; -import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase; import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy; import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicyFactory; import org.apache.hadoop.hbase.client.coprocessor.Batch; @@ -168,6 +171,7 @@ import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ExceptionUtil; +import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; import org.apache.hadoop.hbase.zookeeper.ZKUtil; @@ -175,11 +179,6 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.ipc.RemoteException; import org.apache.zookeeper.KeeperException; -import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.BlockingRpcChannel; -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; - /** * An internal, non-instantiable class that manages creation of {@link HConnection}s. */ @@ -892,7 +891,7 @@ class ConnectionManager { @Override public boolean isTableEnabled(TableName tableName) throws IOException { - return this.registry.isTableOnlineState(tableName, true); + return getTableState(tableName).inStates(TableState.State.ENABLED); } @Override @@ -902,7 +901,7 @@ class ConnectionManager { @Override public boolean isTableDisabled(TableName tableName) throws IOException { - return this.registry.isTableOnlineState(tableName, false); + return getTableState(tableName).inStates(TableState.State.DISABLED); } @Override @@ -912,30 +911,7 @@ class ConnectionManager { @Override public boolean isTableAvailable(final TableName tableName) throws IOException { - final AtomicBoolean available = new AtomicBoolean(true); - final AtomicInteger regionCount = new AtomicInteger(0); - MetaScannerVisitor visitor = new MetaScannerVisitorBase() { - @Override - public boolean processRow(Result row) throws IOException { - HRegionInfo info = MetaScanner.getHRegionInfo(row); - if (info != null && !info.isSplitParent()) { - if (tableName.equals(info.getTable())) { - ServerName server = HRegionInfo.getServerName(row); - if (server == null) { - available.set(false); - return false; - } - regionCount.incrementAndGet(); - } else if (tableName.compareTo(info.getTable()) < 0) { - // Return if we are done with the current table - return false; - } - } - return true; - } - }; - MetaScanner.metaScan(this, visitor, tableName); - return available.get() && (regionCount.get() > 0); + return isTableAvailable(tableName, null); } @Override @@ -944,44 +920,62 @@ class ConnectionManager { } @Override - public boolean isTableAvailable(final TableName tableName, final byte[][] splitKeys) + public boolean isTableAvailable(final TableName tableName, @Nullable final byte[][] splitKeys) throws IOException { - final AtomicBoolean available = new AtomicBoolean(true); - final AtomicInteger regionCount = new AtomicInteger(0); - MetaScannerVisitor visitor = new MetaScannerVisitorBase() { - @Override - public boolean processRow(Result row) throws IOException { - HRegionInfo info = MetaScanner.getHRegionInfo(row); - if (info != null && !info.isSplitParent()) { - if (tableName.equals(info.getTable())) { - ServerName server = HRegionInfo.getServerName(row); - if (server == null) { - available.set(false); - return false; - } - if (!Bytes.equals(info.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) { - for (byte[] splitKey : splitKeys) { - // Just check if the splitkey is available - if (Bytes.equals(info.getStartKey(), splitKey)) { - regionCount.incrementAndGet(); - break; - } - } - } else { - // Always empty start row should be counted - regionCount.incrementAndGet(); - } - } else if (tableName.compareTo(info.getTable()) < 0) { - // Return if we are done with the current table - return false; + try { + if (!isTableEnabled(tableName)) { + LOG.debug("Table " + tableName + " not enabled"); + return false; + } + ClusterConnection connection = getConnectionInternal(getConfiguration()); + List> locations = MetaTableAccessor + .getTableRegionsAndLocations(getKeepAliveZooKeeperWatcher(), + connection, tableName, true); + int notDeployed = 0; + int regionCount = 0; + for (Pair pair : locations) { + HRegionInfo info = pair.getFirst(); + if (pair.getSecond() == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Table " + tableName + " has not deployed region " + pair.getFirst() + .getEncodedName()); } + notDeployed++; + } else if (splitKeys != null + && !Bytes.equals(info.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) { + for (byte[] splitKey : splitKeys) { + // Just check if the splitkey is available + if (Bytes.equals(info.getStartKey(), splitKey)) { + regionCount++; + break; + } + } + } else { + // Always empty start row should be counted + regionCount++; + } + } + if (notDeployed > 0) { + if (LOG.isDebugEnabled()) { + LOG.debug("Table " + tableName + " has " + notDeployed + " regions"); + } + return false; + } else if (splitKeys != null && regionCount != splitKeys.length + 1) { + if (LOG.isDebugEnabled()) { + LOG.debug("Table " + tableName + " expected to have " + (splitKeys.length + 1) + + " regions, but only " + regionCount + " available"); + } + return false; + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Table " + tableName + " should be available"); } return true; } - }; - MetaScanner.metaScan(this, visitor, tableName); - // +1 needs to be added so that the empty start row is also taken into account - return available.get() && (regionCount.get() == splitKeys.length + 1); + } catch (TableNotFoundException tnfe) { + LOG.warn("Table " + tableName + " not enabled, it is not exists"); + return false; + } } @Override @@ -2412,6 +2406,15 @@ class ConnectionManager { return getHTableDescriptorsByTableName(tableNames); } + @Nonnull + public TableState getTableState(TableName tableName) throws IOException { + ClusterConnection conn = getConnectionInternal(getConfiguration()); + TableState tableState = MetaTableAccessor.getTableState(conn, tableName); + if (tableState == null) + throw new TableNotFoundException(tableName); + return tableState; + } + @Override public NonceGenerator getNonceGenerator() { return this.nonceGenerator; @@ -2433,7 +2436,7 @@ class ConnectionManager { GetTableDescriptorsResponse htds; try { GetTableDescriptorsRequest req = - RequestConverter.buildGetTableDescriptorsRequest(tableName); + RequestConverter.buildGetTableDescriptorsRequest(tableName); htds = master.getTableDescriptors(null, req); } catch (ServiceException se) { throw ProtobufUtil.getRemoteException(se); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index 520b9538e80..cae61329186 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -18,6 +18,7 @@ */ package org.apache.hadoop.hbase.client; +import java.io.Closeable; import java.io.IOException; import java.io.InterruptedIOException; import java.net.SocketTimeoutException; @@ -32,6 +33,9 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Pattern; +import com.google.common.annotations.VisibleForTesting; +import com.google.protobuf.ByteString; +import com.google.protobuf.ServiceException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -147,10 +151,6 @@ import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.util.StringUtils; import org.apache.zookeeper.KeeperException; -import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.ByteString; -import com.google.protobuf.ServiceException; - /** * HBaseAdmin is no longer a client API. It is marked InterfaceAudience.Private indicating that * this is an HBase-internal class as defined in @@ -282,7 +282,12 @@ public class HBaseAdmin implements Admin { */ @Override public boolean tableExists(final TableName tableName) throws IOException { - return MetaTableAccessor.tableExists(connection, tableName); + return executeCallable(new ConnectionCallable(getConnection()) { + @Override + public Boolean call(int callTimeout) throws ServiceException, IOException { + return MetaTableAccessor.tableExists(connection, tableName); + } + }); } public boolean tableExists(final byte[] tableName) @@ -543,11 +548,11 @@ public class HBaseAdmin implements Admin { } int numRegs = (splitKeys == null ? 1 : splitKeys.length + 1) * desc.getRegionReplication(); int prevRegCount = 0; - boolean doneWithMetaScan = false; + boolean tableWasEnabled = false; for (int tries = 0; tries < this.numRetries * this.retryLongerMultiplier; ++tries) { - if (!doneWithMetaScan) { - // Wait for new table to come on-line + if (tableWasEnabled) { + // Wait all table regions comes online final AtomicInteger actualRegCount = new AtomicInteger(0); MetaScannerVisitor visitor = new MetaScannerVisitorBase() { @Override @@ -595,17 +600,26 @@ public class HBaseAdmin implements Admin { tries = -1; } } else { - doneWithMetaScan = true; - tries = -1; + return; } - } else if (isTableEnabled(desc.getTableName())) { - return; } else { - try { // Sleep - Thread.sleep(getPauseTime(tries)); - } catch (InterruptedException e) { - throw new InterruptedIOException("Interrupted when waiting" + - " for table to be enabled; meta scan was done"); + try { + tableWasEnabled = isTableAvailable(desc.getTableName()); + } catch (TableNotFoundException tnfe) { + LOG.debug( + "Table " + desc.getTableName() + " was not enabled, sleeping, still " + numRetries + + " retries left"); + } + if (tableWasEnabled) { + // no we will scan meta to ensure all regions are online + tries = -1; + } else { + try { // Sleep + Thread.sleep(getPauseTime(tries)); + } catch (InterruptedException e) { + throw new InterruptedIOException("Interrupted when waiting" + + " for table to be enabled; meta scan was done"); + } } } } @@ -694,24 +708,11 @@ public class HBaseAdmin implements Admin { }); int failures = 0; - // Wait until all regions deleted for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) { try { - // Find whether all regions are deleted. - List regionLations = - MetaScanner.listTableRegionLocations(conf, connection, tableName); - - // let us wait until hbase:meta table is updated and - // HMaster removes the table from its HTableDescriptors - if (regionLations == null || regionLations.size() == 0) { - HTableDescriptor htd = getTableDescriptorByTableName(tableName); - - if (htd == null) { - // table could not be found in master - we are done. - tableExists = false; - break; - } - } + tableExists = tableExists(tableName); + if (!tableExists) + break; } catch (IOException ex) { failures++; if(failures >= numRetries - 1) { // no more tries left @@ -1105,9 +1106,17 @@ public class HBaseAdmin implements Admin { * @throws IOException if a remote or network exception occurs */ @Override - public boolean isTableEnabled(TableName tableName) throws IOException { + public boolean isTableEnabled(final TableName tableName) throws IOException { checkTableExistence(tableName); - return connection.isTableEnabled(tableName); + return executeCallable(new ConnectionCallable(getConnection()) { + @Override + public Boolean call(int callTimeout) throws ServiceException, IOException { + TableState tableState = MetaTableAccessor.getTableState(connection, tableName); + if (tableState == null) + throw new TableNotFoundException(tableName); + return tableState.inStates(TableState.State.ENABLED); + } + }); } public boolean isTableEnabled(byte[] tableName) throws IOException { @@ -2284,10 +2293,15 @@ public class HBaseAdmin implements Admin { */ private TableName checkTableExists(final TableName tableName) throws IOException { - if (!MetaTableAccessor.tableExists(connection, tableName)) { - throw new TableNotFoundException(tableName); - } - return tableName; + return executeCallable(new ConnectionCallable(getConnection()) { + @Override + public TableName call(int callTimeout) throws ServiceException, IOException { + if (!MetaTableAccessor.tableExists(connection, tableName)) { + throw new TableNotFoundException(tableName); + } + return tableName; + } + }); } /** @@ -3617,7 +3631,8 @@ public class HBaseAdmin implements Admin { }); } - private V executeCallable(MasterCallable callable) throws IOException { + private & Closeable, V> V executeCallable(C callable) + throws IOException { RpcRetryingCaller caller = rpcCallerFactory.newCaller(); try { return caller.callWithRetries(callable, operationTimeout); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index d6f066dde81..4afaf227dce 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -128,6 +128,7 @@ public class HTable implements HTableInterface { protected AsyncProcess multiAp; private RpcRetryingCallerFactory rpcCallerFactory; private RpcControllerFactory rpcControllerFactory; + private final HTableDescriptor metaTableDescriptor; /** * Creates an object to access a HBase table. @@ -172,12 +173,14 @@ public class HTable implements HTableInterface { this.cleanupPoolOnClose = this.cleanupConnectionOnClose = true; if (conf == null) { this.connection = null; + this.metaTableDescriptor = HTableDescriptor.metaTableDescriptor(new Configuration()); return; } this.connection = ConnectionManager.getConnectionInternal(conf); this.configuration = conf; this.pool = getDefaultExecutor(conf); + this.metaTableDescriptor = HTableDescriptor.metaTableDescriptor(conf); this.finishSetup(); } @@ -197,6 +200,7 @@ public class HTable implements HTableInterface { this.configuration = connection.getConfiguration(); this.pool = getDefaultExecutor(this.configuration); + this.metaTableDescriptor = HTableDescriptor.metaTableDescriptor(configuration); this.finishSetup(); } @@ -257,6 +261,7 @@ public class HTable implements HTableInterface { } this.tableName = tableName; this.cleanupConnectionOnClose = true; + this.metaTableDescriptor = HTableDescriptor.metaTableDescriptor(conf); this.finishSetup(); } @@ -303,6 +308,7 @@ public class HTable implements HTableInterface { this.cleanupConnectionOnClose = false; this.connection = connection; this.configuration = connection.getConfiguration(); + this.metaTableDescriptor = HTableDescriptor.metaTableDescriptor(configuration); this.tableConfiguration = tableConfig; this.pool = pool; if (pool == null) { @@ -329,6 +335,7 @@ public class HTable implements HTableInterface { tableConfiguration = new TableConfiguration(connection.getConfiguration()); cleanupPoolOnClose = false; cleanupConnectionOnClose = false; + this.metaTableDescriptor = HTableDescriptor.metaTableDescriptor(new Configuration()); // used from tests, don't trust the connection is real this.mutator = new BufferedMutatorImpl(conn, null, null, params); } @@ -565,7 +572,7 @@ public class HTable implements HTableInterface { // TODO: This is the same as HBaseAdmin.getTableDescriptor(). Only keep one. if (tableName == null) return null; if (tableName.equals(TableName.META_TABLE_NAME)) { - return HTableDescriptor.META_TABLEDESC; + return metaTableDescriptor; } HTableDescriptor htd = executeMasterCallable( new MasterCallable(getConnection()) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Registry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Registry.java index 58ec3c45399..9debd63b80f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Registry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Registry.java @@ -21,7 +21,6 @@ import java.io.IOException; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.TableName; /** * Cluster registry. @@ -46,12 +45,6 @@ interface Registry { */ String getClusterId(); - /** - * @param enabled Return true if table is enabled - * @throws IOException - */ - boolean isTableOnlineState(TableName tableName, boolean enabled) throws IOException; - /** * @return Count of 'running' regionservers * @throws IOException diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java new file mode 100644 index 00000000000..964d401e63e --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java @@ -0,0 +1,219 @@ +/** + * 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.client; + +import com.google.protobuf.InvalidProtocolBufferException; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; + +/** + * Represents table state. + */ +@InterfaceAudience.Private +public class TableState { + + @InterfaceAudience.Public + @InterfaceStability.Evolving + public static enum State { + ENABLED, + DISABLED, + DISABLING, + ENABLING; + + /** + * Covert from PB version of State + * + * @param state convert from + * @return POJO + */ + public static State convert(HBaseProtos.TableState.State state) { + State ret; + switch (state) { + case ENABLED: + ret = State.ENABLED; + break; + case DISABLED: + ret = State.DISABLED; + break; + case DISABLING: + ret = State.DISABLING; + break; + case ENABLING: + ret = State.ENABLING; + break; + default: + throw new IllegalStateException(state.toString()); + } + return ret; + } + + /** + * Covert to PB version of State + * + * @return PB + */ + public HBaseProtos.TableState.State convert() { + HBaseProtos.TableState.State state; + switch (this) { + case ENABLED: + state = HBaseProtos.TableState.State.ENABLED; + break; + case DISABLED: + state = HBaseProtos.TableState.State.DISABLED; + break; + case DISABLING: + state = HBaseProtos.TableState.State.DISABLING; + break; + case ENABLING: + state = HBaseProtos.TableState.State.ENABLING; + break; + default: + throw new IllegalStateException(this.toString()); + } + return state; + } + + } + + private final TableName tableName; + private final State state; + + /** + * Create instance of TableState. + * @param state table state + */ + public TableState(TableName tableName, State state) { + this.tableName = tableName; + this.state = state; + } + + /** + * @return table state + */ + public State getState() { + return state; + } + + /** + * Table name for state + * + * @return milliseconds + */ + public TableName getTableName() { + return tableName; + } + + /** + * Check that table in given states + * @param state state + * @return true if satisfies + */ + public boolean inStates(State state) { + return this.state.equals(state); + } + + /** + * Check that table in given states + * @param states state list + * @return true if satisfies + */ + public boolean inStates(State... states) { + for (State s : states) { + if (s.equals(this.state)) + return true; + } + return false; + } + + + /** + * Covert to PB version of TableState + * @return PB + */ + public HBaseProtos.TableState convert() { + return HBaseProtos.TableState.newBuilder() + .setState(this.state.convert()).build(); + } + + /** + * Covert from PB version of TableState + * + * @param tableName table this state of + * @param tableState convert from + * @return POJO + */ + public static TableState convert(TableName tableName, HBaseProtos.TableState tableState) { + TableState.State state = State.convert(tableState.getState()); + return new TableState(tableName, state); + } + + public static TableState parseFrom(TableName tableName, byte[] bytes) + throws DeserializationException { + try { + return convert(tableName, HBaseProtos.TableState.parseFrom(bytes)); + } catch (InvalidProtocolBufferException e) { + throw new DeserializationException(e); + } + } + + /** + * Static version of state checker + * @param state desired + * @param target equals to any of + * @return true if satisfies + */ + public static boolean isInStates(State state, State... target) { + for (State tableState : target) { + if (state.equals(tableState)) + return true; + } + return false; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + TableState that = (TableState) o; + + if (state != that.state) return false; + if (tableName != null ? !tableName.equals(that.tableName) : that.tableName != null) + return false; + + return true; + } + + @Override + public int hashCode() { + int result = (tableName != null ? tableName.hashCode() : 0); + result = 31 * result + (state != null ? state.hashCode() : 0); + return result; + } + + @Override + public String toString() { + return "TableState{" + + ", tableName=" + tableName + + ", state=" + state + + '}'; + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java index 3fb3a05c222..26aca1858fa 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; -import java.io.InterruptedIOException; import java.util.List; import org.apache.commons.logging.Log; @@ -27,10 +26,8 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKClusterId; -import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.zookeeper.KeeperException; @@ -116,24 +113,6 @@ class ZooKeeperRegistry implements Registry { return this.clusterId; } - @Override - public boolean isTableOnlineState(TableName tableName, boolean enabled) - throws IOException { - ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher(); - try { - if (enabled) { - return ZKTableStateClientSideReader.isEnabledTable(zkw, tableName); - } - return ZKTableStateClientSideReader.isDisabledTable(zkw, tableName); - } catch (KeeperException e) { - throw new IOException("Enable/Disable failed", e); - } catch (InterruptedException e) { - throw new InterruptedIOException(); - } finally { - zkw.close(); - } - } - @Override public int getCurrentNrHRS() throws IOException { ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableStateClientSideReader.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableStateClientSideReader.java deleted file mode 100644 index d12231be2cb..00000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableStateClientSideReader.java +++ /dev/null @@ -1,202 +0,0 @@ -/** - * Copyright The Apache Software Foundation - * - * 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.zookeeper; - -import com.google.protobuf.InvalidProtocolBufferException; - -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; -import org.apache.zookeeper.KeeperException; - -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -/** - * Non-instantiable class that provides helper functions to learn - * about HBase table state for code running on client side (hence, not having - * access to consensus context). - * - * Doesn't cache any table state, just goes directly to ZooKeeper. - * TODO: decouple this class from ZooKeeper. - */ -@InterfaceAudience.Private -public class ZKTableStateClientSideReader { - - private ZKTableStateClientSideReader() {} - - /** - * Go to zookeeper and see if state of table is {@code ZooKeeperProtos.Table.State#DISABLED}. - * This method does not use cache. - * This method is for clients other than AssignmentManager - * @param zkw ZooKeeperWatcher instance to use - * @param tableName table we're checking - * @return True if table is enabled. - * @throws KeeperException - */ - public static boolean isDisabledTable(final ZooKeeperWatcher zkw, - final TableName tableName) - throws KeeperException, InterruptedException { - ZooKeeperProtos.Table.State state = getTableState(zkw, tableName); - return isTableState(ZooKeeperProtos.Table.State.DISABLED, state); - } - - /** - * Go to zookeeper and see if state of table is {@code ZooKeeperProtos.Table.State#ENABLED}. - * This method does not use cache. - * This method is for clients other than AssignmentManager - * @param zkw ZooKeeperWatcher instance to use - * @param tableName table we're checking - * @return True if table is enabled. - * @throws KeeperException - */ - public static boolean isEnabledTable(final ZooKeeperWatcher zkw, - final TableName tableName) - throws KeeperException, InterruptedException { - return getTableState(zkw, tableName) == ZooKeeperProtos.Table.State.ENABLED; - } - - /** - * Go to zookeeper and see if state of table is {@code ZooKeeperProtos.Table.State#DISABLING} - * of {@code ZooKeeperProtos.Table.State#DISABLED}. - * This method does not use cache. - * This method is for clients other than AssignmentManager. - * @param zkw ZooKeeperWatcher instance to use - * @param tableName table we're checking - * @return True if table is enabled. - * @throws KeeperException - */ - public static boolean isDisablingOrDisabledTable(final ZooKeeperWatcher zkw, - final TableName tableName) - throws KeeperException, InterruptedException { - ZooKeeperProtos.Table.State state = getTableState(zkw, tableName); - return isTableState(ZooKeeperProtos.Table.State.DISABLING, state) || - isTableState(ZooKeeperProtos.Table.State.DISABLED, state); - } - - /** - * Gets a list of all the tables set as disabled in zookeeper. - * @return Set of disabled tables, empty Set if none - * @throws KeeperException - */ - public static Set getDisabledTables(ZooKeeperWatcher zkw) - throws KeeperException, InterruptedException { - Set disabledTables = new HashSet(); - List children = - ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode); - for (String child: children) { - TableName tableName = - TableName.valueOf(child); - ZooKeeperProtos.Table.State state = getTableState(zkw, tableName); - if (state == ZooKeeperProtos.Table.State.DISABLED) disabledTables.add(tableName); - } - return disabledTables; - } - - /** - * Gets a list of all the tables set as disabled in zookeeper. - * @return Set of disabled tables, empty Set if none - * @throws KeeperException - */ - public static Set getDisabledOrDisablingTables(ZooKeeperWatcher zkw) - throws KeeperException, InterruptedException { - return - getTablesInStates( - zkw, - ZooKeeperProtos.Table.State.DISABLED, - ZooKeeperProtos.Table.State.DISABLING); - } - - /** - * Gets a list of all the tables set as enabling in zookeeper. - * @param zkw ZooKeeperWatcher instance to use - * @return Set of enabling tables, empty Set if none - * @throws KeeperException - * @throws InterruptedException - */ - public static Set getEnablingTables(ZooKeeperWatcher zkw) - throws KeeperException, InterruptedException { - return getTablesInStates(zkw, ZooKeeperProtos.Table.State.ENABLING); - } - - /** - * Gets a list of tables that are set as one of the passing in states in zookeeper. - * @param zkw ZooKeeperWatcher instance to use - * @param states the list of states that a table could be in - * @return Set of tables in one of the states, empty Set if none - * @throws KeeperException - * @throws InterruptedException - */ - private static Set getTablesInStates( - ZooKeeperWatcher zkw, - ZooKeeperProtos.Table.State... states) - throws KeeperException, InterruptedException { - Set tableNameSet = new HashSet(); - List children = ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode); - TableName tableName; - ZooKeeperProtos.Table.State tableState; - for (String child: children) { - tableName = TableName.valueOf(child); - tableState = getTableState(zkw, tableName); - for (ZooKeeperProtos.Table.State state : states) { - if (tableState == state) { - tableNameSet.add(tableName); - break; - } - } - } - return tableNameSet; - } - - static boolean isTableState(final ZooKeeperProtos.Table.State expectedState, - final ZooKeeperProtos.Table.State currentState) { - return currentState != null && currentState.equals(expectedState); - } - - /** - * @param zkw ZooKeeperWatcher instance to use - * @param tableName table we're checking - * @return Null or {@link ZooKeeperProtos.Table.State} found in znode. - * @throws KeeperException - */ - static ZooKeeperProtos.Table.State getTableState(final ZooKeeperWatcher zkw, - final TableName tableName) - throws KeeperException, InterruptedException { - String znode = ZKUtil.joinZNode(zkw.tableZNode, tableName.getNameAsString()); - byte [] data = ZKUtil.getData(zkw, znode); - if (data == null || data.length <= 0) return null; - try { - ProtobufUtil.expectPBMagicPrefix(data); - ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder(); - int magicLen = ProtobufUtil.lengthOfPBMagic(); - ZooKeeperProtos.Table t = builder.mergeFrom(data, magicLen, data.length - magicLen).build(); - return t.getState(); - } catch (InvalidProtocolBufferException e) { - KeeperException ke = new KeeperException.DataInconsistencyException(); - ke.initCause(e); - throw ke; - } catch (DeserializationException e) { - throw ZKUtil.convert(e); - } - } -} diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java index 2d50c1b0540..dbe62045775 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java @@ -128,12 +128,6 @@ public class TestClientNoCluster extends Configured implements Tool { return HConstants.CLUSTER_ID_DEFAULT; } - @Override - public boolean isTableOnlineState(TableName tableName, boolean enabled) - throws IOException { - return enabled; - } - @Override public int getCurrentNrHRS() throws IOException { return 1; 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 2704516f889..42254c6fa62 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 @@ -19,7 +19,6 @@ package org.apache.hadoop.hbase; import static org.apache.hadoop.hbase.io.hfile.BlockType.MAGIC_LENGTH; -import java.nio.ByteBuffer; import java.nio.charset.Charset; import java.util.Arrays; import java.util.Collections; @@ -443,6 +442,16 @@ public final class HConstants { /** The upper-half merge region column qualifier */ public static final byte[] MERGEB_QUALIFIER = Bytes.toBytes("mergeB"); + /** The catalog family as a string*/ + public static final String TABLE_FAMILY_STR = "table"; + + /** The catalog family */ + public static final byte [] TABLE_FAMILY = Bytes.toBytes(TABLE_FAMILY_STR); + + /** The serialized table state qualifier */ + public static final byte[] TABLE_STATE_QUALIFIER = Bytes.toBytes("state"); + + /** * The meta table version column qualifier. * We keep current version of the meta table in this column in -ROOT- @@ -730,7 +739,8 @@ public final class HConstants { /** * The client scanner timeout period in milliseconds. */ - public static final String HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD = "hbase.client.scanner.timeout.period"; + public static final String HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD = + "hbase.client.scanner.timeout.period"; /** * Use {@link #HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD} instead. diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java index 1dbce4d3771..16ccf1bd590 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java @@ -2404,6 +2404,601 @@ public final class HBaseProtos { // @@protoc_insertion_point(class_scope:TableSchema) } + public interface TableStateOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .TableState.State state = 1; + /** + * required .TableState.State state = 1; + * + *
+     * This is the table's state.
+     * 
+ */ + boolean hasState(); + /** + * required .TableState.State state = 1; + * + *
+     * This is the table's state.
+     * 
+ */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState(); + } + /** + * Protobuf type {@code TableState} + * + *
+   ** Denotes state of the table 
+   * 
+ */ + public static final class TableState extends + com.google.protobuf.GeneratedMessage + implements TableStateOrBuilder { + // Use TableState.newBuilder() to construct. + private TableState(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private TableState(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final TableState defaultInstance; + public static TableState getDefaultInstance() { + return defaultInstance; + } + + public TableState getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private TableState( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + int rawValue = input.readEnum(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State value = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(1, rawValue); + } else { + bitField0_ |= 0x00000001; + state_ = value; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_TableState_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_TableState_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public TableState parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new TableState(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + /** + * Protobuf enum {@code TableState.State} + * + *
+     * Table's current state
+     * 
+ */ + public enum State + implements com.google.protobuf.ProtocolMessageEnum { + /** + * ENABLED = 0; + */ + ENABLED(0, 0), + /** + * DISABLED = 1; + */ + DISABLED(1, 1), + /** + * DISABLING = 2; + */ + DISABLING(2, 2), + /** + * ENABLING = 3; + */ + ENABLING(3, 3), + ; + + /** + * ENABLED = 0; + */ + public static final int ENABLED_VALUE = 0; + /** + * DISABLED = 1; + */ + public static final int DISABLED_VALUE = 1; + /** + * DISABLING = 2; + */ + public static final int DISABLING_VALUE = 2; + /** + * ENABLING = 3; + */ + public static final int ENABLING_VALUE = 3; + + + public final int getNumber() { return value; } + + public static State valueOf(int value) { + switch (value) { + case 0: return ENABLED; + case 1: return DISABLED; + case 2: return DISABLING; + case 3: return ENABLING; + default: return null; + } + } + + public static com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static com.google.protobuf.Internal.EnumLiteMap + internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() { + public State findValueByNumber(int number) { + return State.valueOf(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + return getDescriptor().getValues().get(index); + } + public final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.getDescriptor().getEnumTypes().get(0); + } + + private static final State[] VALUES = values(); + + public static State valueOf( + com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + return VALUES[desc.getIndex()]; + } + + private final int index; + private final int value; + + private State(int index, int value) { + this.index = index; + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:TableState.State) + } + + private int bitField0_; + // required .TableState.State state = 1; + public static final int STATE_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State state_; + /** + * required .TableState.State state = 1; + * + *
+     * This is the table's state.
+     * 
+ */ + public boolean hasState() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .TableState.State state = 1; + * + *
+     * This is the table's state.
+     * 
+ */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState() { + return state_; + } + + private void initFields() { + state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasState()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeEnum(1, state_.getNumber()); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeEnumSize(1, state_.getNumber()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState other = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState) obj; + + boolean result = true; + result = result && (hasState() == other.hasState()); + if (hasState()) { + result = result && + (getState() == other.getState()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasState()) { + hash = (37 * hash) + STATE_FIELD_NUMBER; + hash = (53 * hash) + hashEnum(getState()); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code TableState} + * + *
+     ** Denotes state of the table 
+     * 
+ */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableStateOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_TableState_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_TableState_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_TableState_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState build() { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState result = new org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.state_ = state_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.getDefaultInstance()) return this; + if (other.hasState()) { + setState(other.getState()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasState()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required .TableState.State state = 1; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED; + /** + * required .TableState.State state = 1; + * + *
+       * This is the table's state.
+       * 
+ */ + public boolean hasState() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .TableState.State state = 1; + * + *
+       * This is the table's state.
+       * 
+ */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState() { + return state_; + } + /** + * required .TableState.State state = 1; + * + *
+       * This is the table's state.
+       * 
+ */ + public Builder setState(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + state_ = value; + onChanged(); + return this; + } + /** + * required .TableState.State state = 1; + * + *
+       * This is the table's state.
+       * 
+ */ + public Builder clearState() { + bitField0_ = (bitField0_ & ~0x00000001); + state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:TableState) + } + + static { + defaultInstance = new TableState(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:TableState) + } + public interface ColumnFamilySchemaOrBuilder extends com.google.protobuf.MessageOrBuilder { @@ -16206,6 +16801,11 @@ public final class HBaseProtos { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_TableSchema_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_TableState_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_TableState_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor internal_static_ColumnFamilySchema_descriptor; private static @@ -16321,46 +16921,49 @@ public final class HBaseProtos { "Name\022#\n\nattributes\030\002 \003(\0132\017.BytesBytesPai" + "r\022,\n\017column_families\030\003 \003(\0132\023.ColumnFamil" + "ySchema\022&\n\rconfiguration\030\004 \003(\0132\017.NameStr" + - "ingPair\"o\n\022ColumnFamilySchema\022\014\n\004name\030\001 " + - "\002(\014\022#\n\nattributes\030\002 \003(\0132\017.BytesBytesPair" + - "\022&\n\rconfiguration\030\003 \003(\0132\017.NameStringPair" + - "\"\232\001\n\nRegionInfo\022\021\n\tregion_id\030\001 \002(\004\022\036\n\nta", - "ble_name\030\002 \002(\0132\n.TableName\022\021\n\tstart_key\030" + - "\003 \001(\014\022\017\n\007end_key\030\004 \001(\014\022\017\n\007offline\030\005 \001(\010\022" + - "\r\n\005split\030\006 \001(\010\022\025\n\nreplica_id\030\007 \001(\005:\0010\"1\n" + - "\014FavoredNodes\022!\n\014favored_node\030\001 \003(\0132\013.Se" + - "rverName\"\225\001\n\017RegionSpecifier\0222\n\004type\030\001 \002" + - "(\0162$.RegionSpecifier.RegionSpecifierType" + - "\022\r\n\005value\030\002 \002(\014\"?\n\023RegionSpecifierType\022\017" + - "\n\013REGION_NAME\020\001\022\027\n\023ENCODED_REGION_NAME\020\002" + - "\"%\n\tTimeRange\022\014\n\004from\030\001 \001(\004\022\n\n\002to\030\002 \001(\004\"" + - "A\n\nServerName\022\021\n\thost_name\030\001 \002(\t\022\014\n\004port", - "\030\002 \001(\r\022\022\n\nstart_code\030\003 \001(\004\"\033\n\013Coprocesso" + - "r\022\014\n\004name\030\001 \002(\t\"-\n\016NameStringPair\022\014\n\004nam" + - "e\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\",\n\rNameBytesPair\022" + - "\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \001(\014\"/\n\016BytesByt" + - "esPair\022\r\n\005first\030\001 \002(\014\022\016\n\006second\030\002 \002(\014\",\n" + - "\rNameInt64Pair\022\014\n\004name\030\001 \001(\t\022\r\n\005value\030\002 " + - "\001(\003\"\275\001\n\023SnapshotDescription\022\014\n\004name\030\001 \002(" + - "\t\022\r\n\005table\030\002 \001(\t\022\030\n\rcreation_time\030\003 \001(\003:" + - "\0010\022.\n\004type\030\004 \001(\0162\031.SnapshotDescription.T" + - "ype:\005FLUSH\022\017\n\007version\030\005 \001(\005\".\n\004Type\022\014\n\010D", - "ISABLED\020\000\022\t\n\005FLUSH\020\001\022\r\n\tSKIPFLUSH\020\002\"}\n\024P" + - "rocedureDescription\022\021\n\tsignature\030\001 \002(\t\022\020" + - "\n\010instance\030\002 \001(\t\022\030\n\rcreation_time\030\003 \001(\003:" + - "\0010\022&\n\rconfiguration\030\004 \003(\0132\017.NameStringPa" + - "ir\"\n\n\010EmptyMsg\"\033\n\007LongMsg\022\020\n\010long_msg\030\001 " + - "\002(\003\"\037\n\tDoubleMsg\022\022\n\ndouble_msg\030\001 \002(\001\"\'\n\r" + - "BigDecimalMsg\022\026\n\016bigdecimal_msg\030\001 \002(\014\"5\n" + - "\004UUID\022\026\n\016least_sig_bits\030\001 \002(\004\022\025\n\rmost_si" + - "g_bits\030\002 \002(\004\"K\n\023NamespaceDescriptor\022\014\n\004n" + - "ame\030\001 \002(\014\022&\n\rconfiguration\030\002 \003(\0132\017.NameS", - "tringPair\"$\n\020RegionServerInfo\022\020\n\010infoPor" + - "t\030\001 \001(\005*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS" + - "_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n" + - "\020GREATER_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_O" + - "P\020\006B>\n*org.apache.hadoop.hbase.protobuf." + - "generatedB\013HBaseProtosH\001\240\001\001" + "ingPair\"o\n\nTableState\022 \n\005state\030\001 \002(\0162\021.T" + + "ableState.State\"?\n\005State\022\013\n\007ENABLED\020\000\022\014\n" + + "\010DISABLED\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENABLING\020\003" + + "\"o\n\022ColumnFamilySchema\022\014\n\004name\030\001 \002(\014\022#\n\n", + "attributes\030\002 \003(\0132\017.BytesBytesPair\022&\n\rcon" + + "figuration\030\003 \003(\0132\017.NameStringPair\"\232\001\n\nRe" + + "gionInfo\022\021\n\tregion_id\030\001 \002(\004\022\036\n\ntable_nam" + + "e\030\002 \002(\0132\n.TableName\022\021\n\tstart_key\030\003 \001(\014\022\017" + + "\n\007end_key\030\004 \001(\014\022\017\n\007offline\030\005 \001(\010\022\r\n\005spli" + + "t\030\006 \001(\010\022\025\n\nreplica_id\030\007 \001(\005:\0010\"1\n\014Favore" + + "dNodes\022!\n\014favored_node\030\001 \003(\0132\013.ServerNam" + + "e\"\225\001\n\017RegionSpecifier\0222\n\004type\030\001 \002(\0162$.Re" + + "gionSpecifier.RegionSpecifierType\022\r\n\005val" + + "ue\030\002 \002(\014\"?\n\023RegionSpecifierType\022\017\n\013REGIO", + "N_NAME\020\001\022\027\n\023ENCODED_REGION_NAME\020\002\"%\n\tTim" + + "eRange\022\014\n\004from\030\001 \001(\004\022\n\n\002to\030\002 \001(\004\"A\n\nServ" + + "erName\022\021\n\thost_name\030\001 \002(\t\022\014\n\004port\030\002 \001(\r\022" + + "\022\n\nstart_code\030\003 \001(\004\"\033\n\013Coprocessor\022\014\n\004na" + + "me\030\001 \002(\t\"-\n\016NameStringPair\022\014\n\004name\030\001 \002(\t" + + "\022\r\n\005value\030\002 \002(\t\",\n\rNameBytesPair\022\014\n\004name" + + "\030\001 \002(\t\022\r\n\005value\030\002 \001(\014\"/\n\016BytesBytesPair\022" + + "\r\n\005first\030\001 \002(\014\022\016\n\006second\030\002 \002(\014\",\n\rNameIn" + + "t64Pair\022\014\n\004name\030\001 \001(\t\022\r\n\005value\030\002 \001(\003\"\275\001\n" + + "\023SnapshotDescription\022\014\n\004name\030\001 \002(\t\022\r\n\005ta", + "ble\030\002 \001(\t\022\030\n\rcreation_time\030\003 \001(\003:\0010\022.\n\004t" + + "ype\030\004 \001(\0162\031.SnapshotDescription.Type:\005FL" + + "USH\022\017\n\007version\030\005 \001(\005\".\n\004Type\022\014\n\010DISABLED" + + "\020\000\022\t\n\005FLUSH\020\001\022\r\n\tSKIPFLUSH\020\002\"}\n\024Procedur" + + "eDescription\022\021\n\tsignature\030\001 \002(\t\022\020\n\010insta" + + "nce\030\002 \001(\t\022\030\n\rcreation_time\030\003 \001(\003:\0010\022&\n\rc" + + "onfiguration\030\004 \003(\0132\017.NameStringPair\"\n\n\010E" + + "mptyMsg\"\033\n\007LongMsg\022\020\n\010long_msg\030\001 \002(\003\"\037\n\t" + + "DoubleMsg\022\022\n\ndouble_msg\030\001 \002(\001\"\'\n\rBigDeci" + + "malMsg\022\026\n\016bigdecimal_msg\030\001 \002(\014\"5\n\004UUID\022\026", + "\n\016least_sig_bits\030\001 \002(\004\022\025\n\rmost_sig_bits\030" + + "\002 \002(\004\"K\n\023NamespaceDescriptor\022\014\n\004name\030\001 \002" + + "(\014\022&\n\rconfiguration\030\002 \003(\0132\017.NameStringPa" + + "ir\"$\n\020RegionServerInfo\022\020\n\010infoPort\030\001 \001(\005" + + "*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQU" + + "AL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATE" + + "R_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006B>\n*" + + "org.apache.hadoop.hbase.protobuf.generat" + + "edB\013HBaseProtosH\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -16379,122 +16982,128 @@ public final class HBaseProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_TableSchema_descriptor, new java.lang.String[] { "TableName", "Attributes", "ColumnFamilies", "Configuration", }); - internal_static_ColumnFamilySchema_descriptor = + internal_static_TableState_descriptor = getDescriptor().getMessageTypes().get(2); + internal_static_TableState_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_TableState_descriptor, + new java.lang.String[] { "State", }); + internal_static_ColumnFamilySchema_descriptor = + getDescriptor().getMessageTypes().get(3); internal_static_ColumnFamilySchema_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ColumnFamilySchema_descriptor, new java.lang.String[] { "Name", "Attributes", "Configuration", }); internal_static_RegionInfo_descriptor = - getDescriptor().getMessageTypes().get(3); + getDescriptor().getMessageTypes().get(4); internal_static_RegionInfo_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RegionInfo_descriptor, new java.lang.String[] { "RegionId", "TableName", "StartKey", "EndKey", "Offline", "Split", "ReplicaId", }); internal_static_FavoredNodes_descriptor = - getDescriptor().getMessageTypes().get(4); + getDescriptor().getMessageTypes().get(5); internal_static_FavoredNodes_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_FavoredNodes_descriptor, new java.lang.String[] { "FavoredNode", }); internal_static_RegionSpecifier_descriptor = - getDescriptor().getMessageTypes().get(5); + getDescriptor().getMessageTypes().get(6); internal_static_RegionSpecifier_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RegionSpecifier_descriptor, new java.lang.String[] { "Type", "Value", }); internal_static_TimeRange_descriptor = - getDescriptor().getMessageTypes().get(6); + getDescriptor().getMessageTypes().get(7); internal_static_TimeRange_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_TimeRange_descriptor, new java.lang.String[] { "From", "To", }); internal_static_ServerName_descriptor = - getDescriptor().getMessageTypes().get(7); + getDescriptor().getMessageTypes().get(8); internal_static_ServerName_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ServerName_descriptor, new java.lang.String[] { "HostName", "Port", "StartCode", }); internal_static_Coprocessor_descriptor = - getDescriptor().getMessageTypes().get(8); + getDescriptor().getMessageTypes().get(9); internal_static_Coprocessor_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Coprocessor_descriptor, new java.lang.String[] { "Name", }); internal_static_NameStringPair_descriptor = - getDescriptor().getMessageTypes().get(9); + getDescriptor().getMessageTypes().get(10); internal_static_NameStringPair_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_NameStringPair_descriptor, new java.lang.String[] { "Name", "Value", }); internal_static_NameBytesPair_descriptor = - getDescriptor().getMessageTypes().get(10); + getDescriptor().getMessageTypes().get(11); internal_static_NameBytesPair_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_NameBytesPair_descriptor, new java.lang.String[] { "Name", "Value", }); internal_static_BytesBytesPair_descriptor = - getDescriptor().getMessageTypes().get(11); + getDescriptor().getMessageTypes().get(12); internal_static_BytesBytesPair_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_BytesBytesPair_descriptor, new java.lang.String[] { "First", "Second", }); internal_static_NameInt64Pair_descriptor = - getDescriptor().getMessageTypes().get(12); + getDescriptor().getMessageTypes().get(13); internal_static_NameInt64Pair_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_NameInt64Pair_descriptor, new java.lang.String[] { "Name", "Value", }); internal_static_SnapshotDescription_descriptor = - getDescriptor().getMessageTypes().get(13); + getDescriptor().getMessageTypes().get(14); internal_static_SnapshotDescription_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SnapshotDescription_descriptor, new java.lang.String[] { "Name", "Table", "CreationTime", "Type", "Version", }); internal_static_ProcedureDescription_descriptor = - getDescriptor().getMessageTypes().get(14); + getDescriptor().getMessageTypes().get(15); internal_static_ProcedureDescription_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ProcedureDescription_descriptor, new java.lang.String[] { "Signature", "Instance", "CreationTime", "Configuration", }); internal_static_EmptyMsg_descriptor = - getDescriptor().getMessageTypes().get(15); + getDescriptor().getMessageTypes().get(16); internal_static_EmptyMsg_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_EmptyMsg_descriptor, new java.lang.String[] { }); internal_static_LongMsg_descriptor = - getDescriptor().getMessageTypes().get(16); + getDescriptor().getMessageTypes().get(17); internal_static_LongMsg_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_LongMsg_descriptor, new java.lang.String[] { "LongMsg", }); internal_static_DoubleMsg_descriptor = - getDescriptor().getMessageTypes().get(17); + getDescriptor().getMessageTypes().get(18); internal_static_DoubleMsg_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_DoubleMsg_descriptor, new java.lang.String[] { "DoubleMsg", }); internal_static_BigDecimalMsg_descriptor = - getDescriptor().getMessageTypes().get(18); + getDescriptor().getMessageTypes().get(19); internal_static_BigDecimalMsg_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_BigDecimalMsg_descriptor, new java.lang.String[] { "BigdecimalMsg", }); internal_static_UUID_descriptor = - getDescriptor().getMessageTypes().get(19); + getDescriptor().getMessageTypes().get(20); internal_static_UUID_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_UUID_descriptor, new java.lang.String[] { "LeastSigBits", "MostSigBits", }); internal_static_NamespaceDescriptor_descriptor = - getDescriptor().getMessageTypes().get(20); + getDescriptor().getMessageTypes().get(21); internal_static_NamespaceDescriptor_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_NamespaceDescriptor_descriptor, new java.lang.String[] { "Name", "Configuration", }); internal_static_RegionServerInfo_descriptor = - getDescriptor().getMessageTypes().get(21); + getDescriptor().getMessageTypes().get(22); internal_static_RegionServerInfo_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RegionServerInfo_descriptor, diff --git a/hbase-protocol/src/main/protobuf/HBase.proto b/hbase-protocol/src/main/protobuf/HBase.proto index 24941ffdff0..b08c4f61719 100644 --- a/hbase-protocol/src/main/protobuf/HBase.proto +++ b/hbase-protocol/src/main/protobuf/HBase.proto @@ -44,6 +44,19 @@ message TableSchema { repeated NameStringPair configuration = 4; } +/** Denotes state of the table */ +message TableState { + // Table's current state + enum State { + ENABLED = 0; + DISABLED = 1; + DISABLING = 2; + ENABLING = 3; + } + // This is the table's state. + required State state = 1; +} + /** * Column Family Schema * Inspired by the rest ColumSchemaMessage diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/CoordinatedStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/CoordinatedStateManager.java index b7bfa751c24..bd0268af702 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/CoordinatedStateManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/CoordinatedStateManager.java @@ -55,12 +55,4 @@ public interface CoordinatedStateManager { * @return instance of Server coordinated state manager runs within */ Server getServer(); - - /** - * Returns implementation of TableStateManager. - * @throws InterruptedException if operation is interrupted - * @throws CoordinatedStateException if error happens in underlying coordination mechanism - */ - TableStateManager getTableStateManager() throws InterruptedException, - CoordinatedStateException; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/MetaMigrationConvertingToPB.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/MetaMigrationConvertingToPB.java index 13bebd3d978..ba9397fa2ab 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/MetaMigrationConvertingToPB.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/MetaMigrationConvertingToPB.java @@ -149,7 +149,7 @@ public class MetaMigrationConvertingToPB { static long updateMeta(final MasterServices masterServices) throws IOException { LOG.info("Starting update of META"); ConvertToPBMetaVisitor v = new ConvertToPBMetaVisitor(masterServices); - MetaTableAccessor.fullScan(masterServices.getConnection(), v); + MetaTableAccessor.fullScanRegions(masterServices.getConnection(), v); LOG.info("Finished update of META. Total rows updated:" + v.numMigratedRows); return v.numMigratedRows; } @@ -160,7 +160,7 @@ public class MetaMigrationConvertingToPB { * @throws IOException */ static boolean isMetaTableUpdated(final HConnection hConnection) throws IOException { - List results = MetaTableAccessor.fullScanOfMeta(hConnection); + List results = MetaTableAccessor.fullScanRegions(hConnection); if (results == null || results.isEmpty()) { LOG.info("hbase:meta doesn't have any entries to update."); return true; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/TableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/TableStateManager.java deleted file mode 100644 index 70e1af28b95..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/TableStateManager.java +++ /dev/null @@ -1,115 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase; - -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; - -import java.io.InterruptedIOException; -import java.util.Set; - -/** - * Helper class for table state management for operations running inside - * RegionServer or HMaster. - * Depending on implementation, fetches information from HBase system table, - * local data store, ZooKeeper ensemble or somewhere else. - * Code running on client side (with no coordinated state context) shall instead use - * {@link org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader} - */ -@InterfaceAudience.Private -public interface TableStateManager { - - /** - * Sets the table into desired state. Fails silently if the table is already in this state. - * @param tableName table to process - * @param state new state of this table - * @throws CoordinatedStateException if error happened when trying to set table state - */ - void setTableState(TableName tableName, ZooKeeperProtos.Table.State state) - throws CoordinatedStateException; - - /** - * Sets the specified table into the newState, but only if the table is already in - * one of the possibleCurrentStates (otherwise no operation is performed). - * @param tableName table to process - * @param newState new state for the table - * @param states table should be in one of these states for the operation - * to be performed - * @throws CoordinatedStateException if error happened while performing operation - * @return true if operation succeeded, false otherwise - */ - boolean setTableStateIfInStates(TableName tableName, ZooKeeperProtos.Table.State newState, - ZooKeeperProtos.Table.State... states) - throws CoordinatedStateException; - - /** - * Sets the specified table into the newState, but only if the table is NOT in - * one of the possibleCurrentStates (otherwise no operation is performed). - * @param tableName table to process - * @param newState new state for the table - * @param states table should NOT be in one of these states for the operation - * to be performed - * @throws CoordinatedStateException if error happened while performing operation - * @return true if operation succeeded, false otherwise - */ - boolean setTableStateIfNotInStates(TableName tableName, ZooKeeperProtos.Table.State newState, - ZooKeeperProtos.Table.State... states) - throws CoordinatedStateException; - - /** - * @return true if the table is in any one of the listed states, false otherwise. - */ - boolean isTableState(TableName tableName, ZooKeeperProtos.Table.State... states); - - /** - * Mark table as deleted. Fails silently if the table is not currently marked as disabled. - * @param tableName table to be deleted - * @throws CoordinatedStateException if error happened while performing operation - */ - void setDeletedTable(TableName tableName) throws CoordinatedStateException; - - /** - * Checks if table is present. - * - * @param tableName table we're checking - * @return true if the table is present, false otherwise - */ - boolean isTablePresent(TableName tableName); - - /** - * @return set of tables which are in any one of the listed states, empty Set if none - */ - Set getTablesInStates(ZooKeeperProtos.Table.State... states) - throws InterruptedIOException, CoordinatedStateException; - - /** - * If the table is found in the given state the in-memory state is removed. This - * helps in cases where CreateTable is to be retried by the client in case of - * failures. If deletePermanentState is true - the flag kept permanently is - * also reset. - * - * @param tableName table we're working on - * @param states if table isn't in any one of these states, operation aborts - * @param deletePermanentState if true, reset the permanent flag - * @throws CoordinatedStateException if error happened in underlying coordination engine - */ - void checkAndRemoveTableState(TableName tableName, ZooKeeperProtos.Table.State states, - boolean deletePermanentState) - throws CoordinatedStateException; -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java index f79e5d8dfbb..674b8b01bd0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java @@ -17,11 +17,9 @@ */ package org.apache.hadoop.hbase.coordination; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.CoordinatedStateException; import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.Server; -import org.apache.hadoop.hbase.TableStateManager; +import org.apache.hadoop.hbase.classification.InterfaceAudience; /** * Base class for {@link org.apache.hadoop.hbase.CoordinatedStateManager} implementations. @@ -49,9 +47,6 @@ public abstract class BaseCoordinatedStateManager implements CoordinatedStateMan return null; } - @Override - public abstract TableStateManager getTableStateManager() throws InterruptedException, - CoordinatedStateException; /** * Method to retrieve coordination for split log worker */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java index 2f739befb4f..a8b6f2aa43f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java @@ -19,14 +19,10 @@ package org.apache.hadoop.hbase.coordination; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.CoordinatedStateException; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.Server; -import org.apache.hadoop.hbase.TableStateManager; -import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; -import org.apache.zookeeper.KeeperException; /** * ZooKeeper-based implementation of {@link org.apache.hadoop.hbase.CoordinatedStateManager}. @@ -60,16 +56,6 @@ public class ZkCoordinatedStateManager extends BaseCoordinatedStateManager { return server; } - @Override - public TableStateManager getTableStateManager() throws InterruptedException, - CoordinatedStateException { - try { - return new ZKTableStateManager(server.getZooKeeper()); - } catch (KeeperException e) { - throw new CoordinatedStateException(e); - } - } - @Override public SplitLogWorkerCoordination getSplitLogWorkerCoordination() { return splitLogWorkerCoordination; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkOpenRegionCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkOpenRegionCoordination.java index 812bbe25ccb..b523e2edf99 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkOpenRegionCoordination.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkOpenRegionCoordination.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.RegionState; @@ -309,7 +310,7 @@ public class ZkOpenRegionCoordination implements OpenRegionCoordination { } if (!openedNodeDeleted) { if (assignmentManager.getTableStateManager().isTableState(regionInfo.getTable(), - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { + TableState.State.DISABLED, TableState.State.DISABLING)) { debugLog(regionInfo, "Opened region " + regionInfo.getShortNameToLog() + " but " + "this table is disabled, triggering close of region"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java index 3e895b4d066..0f75f2e2428 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.master; import java.io.IOException; -import java.io.InterruptedIOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -43,9 +42,10 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.LinkedHashMultimap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -55,18 +55,17 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.RegionTransition; -import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.TableStateManager; -import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.RegionReplicaUtil; -import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager; import org.apache.hadoop.hbase.coordination.OpenRegionCoordination; import org.apache.hadoop.hbase.coordination.RegionMergeCoordination; @@ -89,12 +88,10 @@ import org.apache.hadoop.hbase.master.handler.EnableTableHandler; import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.regionserver.RegionAlreadyInTransitionException; import org.apache.hadoop.hbase.regionserver.RegionOpeningState; import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; import org.apache.hadoop.hbase.util.ConfigUtil; -import org.apache.hadoop.hbase.wal.DefaultWALProvider; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.KeyLocker; @@ -102,6 +99,7 @@ import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.PairOfSameType; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Triple; +import org.apache.hadoop.hbase.wal.DefaultWALProvider; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKAssign; import org.apache.hadoop.hbase.zookeeper.ZKUtil; @@ -113,9 +111,6 @@ import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.KeeperException.NodeExistsException; import org.apache.zookeeper.data.Stat; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.LinkedHashMultimap; - /** * Manages and performs region assignment. *

@@ -134,7 +129,7 @@ public class AssignmentManager extends ZooKeeperListener { = "hbase.assignment.already.intransition.waittime"; static final int DEFAULT_ALREADY_IN_TRANSITION_WAITTIME = 60000; // 1 minute - protected final Server server; + protected final MasterServices server; private ServerManager serverManager; @@ -267,10 +262,11 @@ public class AssignmentManager extends ZooKeeperListener { * @throws KeeperException * @throws IOException */ - public AssignmentManager(Server server, ServerManager serverManager, + public AssignmentManager(MasterServices server, ServerManager serverManager, final LoadBalancer balancer, final ExecutorService service, MetricsMaster metricsMaster, - final TableLockManager tableLockManager) throws KeeperException, + final TableLockManager tableLockManager, + final TableStateManager tableStateManager) throws KeeperException, IOException, CoordinatedStateException { super(server.getZooKeeper()); this.server = server; @@ -284,15 +280,6 @@ public class AssignmentManager extends ZooKeeperListener { this.shouldAssignRegionsWithFavoredNodes = conf.getClass( HConstants.HBASE_MASTER_LOADBALANCER_CLASS, Object.class).equals( FavoredNodeLoadBalancer.class); - try { - if (server.getCoordinatedStateManager() != null) { - this.tableStateManager = server.getCoordinatedStateManager().getTableStateManager(); - } else { - this.tableStateManager = null; - } - } catch (InterruptedException e) { - throw new InterruptedIOException(); - } // This is the max attempts, not retries, so it should be at least 1. this.maximumAttempts = Math.max(1, this.server.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10)); @@ -302,6 +289,7 @@ public class AssignmentManager extends ZooKeeperListener { int maxThreads = conf.getInt("hbase.assignment.threads.max", 30); this.threadPoolExecutorService = Threads.getBoundedCachedThreadPool( maxThreads, 60L, TimeUnit.SECONDS, Threads.newDaemonThreadFactory("AM.")); + this.tableStateManager = tableStateManager; this.regionStates = new RegionStates( server, tableStateManager, serverManager, regionStateStore); @@ -601,8 +589,8 @@ public class AssignmentManager extends ZooKeeperListener { if (!failover) { disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates( - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING, - ZooKeeperProtos.Table.State.ENABLING); + TableState.State.DISABLED, TableState.State.DISABLING, + TableState.State.ENABLING); // Clean re/start, mark all user regions closed before reassignment allRegions = regionStates.closeAllUserRegions( @@ -1337,7 +1325,7 @@ public class AssignmentManager extends ZooKeeperListener { LOG.debug("Znode " + regionNameStr + " deleted, state: " + rs); boolean disabled = getTableStateManager().isTableState(regionInfo.getTable(), - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING); + TableState.State.DISABLED, TableState.State.DISABLING); ServerName serverName = rs.getServerName(); if (serverManager.isServerOnline(serverName)) { @@ -2085,7 +2073,7 @@ public class AssignmentManager extends ZooKeeperListener { // will not be in ENABLING or ENABLED state. TableName tableName = region.getTable(); if (!tableStateManager.isTableState(tableName, - ZooKeeperProtos.Table.State.ENABLED, ZooKeeperProtos.Table.State.ENABLING)) { + TableState.State.ENABLED, TableState.State.ENABLING)) { LOG.debug("Setting table " + tableName + " to ENABLED state."); setEnabledTable(tableName); } @@ -2279,8 +2267,8 @@ public class AssignmentManager extends ZooKeeperListener { private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) { if (this.tableStateManager.isTableState(region.getTable(), - ZooKeeperProtos.Table.State.DISABLED, - ZooKeeperProtos.Table.State.DISABLING) || replicasToClose.contains(region)) { + TableState.State.DISABLED, + TableState.State.DISABLING) || replicasToClose.contains(region)) { LOG.info("Table " + region.getTable() + " is disabled or disabling;" + " skipping assign of " + region.getRegionNameAsString()); offlineDisabledRegion(region); @@ -2771,7 +2759,7 @@ public class AssignmentManager extends ZooKeeperListener { for (HRegionInfo hri : regionsFromMetaScan) { TableName tableName = hri.getTable(); if (!tableStateManager.isTableState(tableName, - ZooKeeperProtos.Table.State.ENABLED)) { + TableState.State.ENABLED)) { setEnabledTable(tableName); } } @@ -2842,15 +2830,15 @@ public class AssignmentManager extends ZooKeeperListener { Set rebuildUserRegions() throws IOException, KeeperException, CoordinatedStateException { Set disabledOrEnablingTables = tableStateManager.getTablesInStates( - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.ENABLING); + TableState.State.DISABLED, TableState.State.ENABLING); Set disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates( - ZooKeeperProtos.Table.State.DISABLED, - ZooKeeperProtos.Table.State.DISABLING, - ZooKeeperProtos.Table.State.ENABLING); + TableState.State.DISABLED, + TableState.State.DISABLING, + TableState.State.ENABLING); // Region assignment from META - List results = MetaTableAccessor.fullScanOfMeta(server.getConnection()); + List results = MetaTableAccessor.fullScanRegions(server.getConnection()); // Get any new but slow to checkin region server that joined the cluster Set onlineServers = serverManager.getOnlineServers().keySet(); // Set of offline servers to be returned @@ -2917,7 +2905,7 @@ public class AssignmentManager extends ZooKeeperListener { // this will be used in rolling restarts if (!disabledOrDisablingOrEnabling.contains(tableName) && !getTableStateManager().isTableState(tableName, - ZooKeeperProtos.Table.State.ENABLED)) { + TableState.State.ENABLED)) { setEnabledTable(tableName); } } @@ -2936,7 +2924,7 @@ public class AssignmentManager extends ZooKeeperListener { private void recoverTableInDisablingState() throws KeeperException, IOException, CoordinatedStateException { Set disablingTables = - tableStateManager.getTablesInStates(ZooKeeperProtos.Table.State.DISABLING); + tableStateManager.getTablesInStates(TableState.State.DISABLING); if (disablingTables.size() != 0) { for (TableName tableName : disablingTables) { // Recover by calling DisableTableHandler @@ -2960,7 +2948,7 @@ public class AssignmentManager extends ZooKeeperListener { private void recoverTableInEnablingState() throws KeeperException, IOException, CoordinatedStateException { Set enablingTables = tableStateManager. - getTablesInStates(ZooKeeperProtos.Table.State.ENABLING); + getTablesInStates(TableState.State.ENABLING); if (enablingTables.size() != 0) { for (TableName tableName : enablingTables) { // Recover by calling EnableTableHandler @@ -3368,7 +3356,7 @@ public class AssignmentManager extends ZooKeeperListener { server.abort("Unexpected ZK exception deleting node " + hri, ke); } if (tableStateManager.isTableState(hri.getTable(), - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { + TableState.State.DISABLED, TableState.State.DISABLING)) { regionStates.regionOffline(hri); it.remove(); continue; @@ -3390,7 +3378,7 @@ public class AssignmentManager extends ZooKeeperListener { HRegionInfo hri = plan.getRegionInfo(); TableName tableName = hri.getTable(); if (tableStateManager.isTableState(tableName, - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { + TableState.State.DISABLED, TableState.State.DISABLING)) { LOG.info("Ignored moving region of disabling/disabled table " + tableName); return; @@ -3437,11 +3425,11 @@ public class AssignmentManager extends ZooKeeperListener { protected void setEnabledTable(TableName tableName) { try { this.tableStateManager.setTableState(tableName, - ZooKeeperProtos.Table.State.ENABLED); - } catch (CoordinatedStateException e) { + TableState.State.ENABLED); + } catch (IOException e) { // here we can abort as it is the start up flow String errorMsg = "Unable to ensure that the table " + tableName - + " will be" + " enabled because of a ZooKeeper issue"; + + " will be" + " enabled because of a IOException issue"; LOG.error(errorMsg); this.server.abort(errorMsg, e); } @@ -3450,6 +3438,8 @@ public class AssignmentManager extends ZooKeeperListener { /** * Set region as OFFLINED up in zookeeper asynchronously. * @param state + * @param cb + * @param destination * @return True if we succeeded, false otherwise (State was incorrect or failed * updating zk). */ @@ -3537,7 +3527,7 @@ public class AssignmentManager extends ZooKeeperListener { // When there are more than one region server a new RS is selected as the // destination and the same is updated in the region plan. (HBASE-5546) if (getTableStateManager().isTableState(hri.getTable(), - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) || + TableState.State.DISABLED, TableState.State.DISABLING) || replicasToClose.contains(hri)) { offlineDisabledRegion(hri); return; @@ -3571,14 +3561,14 @@ public class AssignmentManager extends ZooKeeperListener { // reset the count, if any failedOpenTracker.remove(hri.getEncodedName()); if (getTableStateManager().isTableState(hri.getTable(), - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { + TableState.State.DISABLED, TableState.State.DISABLING)) { invokeUnAssign(hri); } } private void onRegionClosed(final HRegionInfo hri) { if (getTableStateManager().isTableState(hri.getTable(), - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) || + TableState.State.DISABLED, TableState.State.DISABLING) || replicasToClose.contains(hri)) { offlineDisabledRegion(hri); return; @@ -3615,7 +3605,7 @@ public class AssignmentManager extends ZooKeeperListener { // User could disable the table before master knows the new region. if (getTableStateManager().isTableState(p.getTable(), - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { + TableState.State.DISABLED, TableState.State.DISABLING)) { invokeUnAssign(a); invokeUnAssign(b); } else { @@ -3641,7 +3631,7 @@ public class AssignmentManager extends ZooKeeperListener { regionOffline(b); if (getTableStateManager().isTableState(p.getTable(), - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { + TableState.State.DISABLED, TableState.State.DISABLING)) { invokeUnAssign(p); } } @@ -3674,7 +3664,7 @@ public class AssignmentManager extends ZooKeeperListener { // User could disable the table before master knows the new region. if (getTableStateManager().isTableState(p.getTable(), - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { + TableState.State.DISABLED, TableState.State.DISABLING)) { invokeUnAssign(p); } else { Callable mergeReplicasCallable = new Callable() { @@ -3700,7 +3690,7 @@ public class AssignmentManager extends ZooKeeperListener { regionOffline(p); if (getTableStateManager().isTableState(p.getTable(), - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { + TableState.State.DISABLED, TableState.State.DISABLING)) { invokeUnAssign(a); invokeUnAssign(b); } @@ -3816,7 +3806,7 @@ public class AssignmentManager extends ZooKeeperListener { // User could disable the table before master knows the new region. if (tableStateManager.isTableState(p.getTable(), - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { + TableState.State.DISABLED, TableState.State.DISABLING)) { unassign(p); } } @@ -3946,7 +3936,7 @@ public class AssignmentManager extends ZooKeeperListener { // User could disable the table before master knows the new region. if (tableStateManager.isTableState(p.getTable(), - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { + TableState.State.DISABLED, TableState.State.DISABLING)) { unassign(hri_a); unassign(hri_b); } 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 0e654b336aa..5da7c6495a3 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 @@ -70,6 +70,7 @@ import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.MetaScanner; @@ -108,7 +109,6 @@ import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost; import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.RSRpcServices; @@ -127,6 +127,7 @@ import org.apache.hadoop.hbase.util.HasThread; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.VersionInfo; +import org.apache.hadoop.hbase.util.ZKDataMigrator; import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker; import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker; import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; @@ -244,6 +245,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server { // manager of assignment nodes in zookeeper AssignmentManager assignmentManager; + // handle table states + private TableStateManager tableStateManager; + // buffer for "fatal error" notices from region servers // in the cluster. This is only used for assisting // operations/debugging. @@ -421,6 +425,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server { return connector.getLocalPort(); } + @Override + protected TableDescriptors getFsTableDescriptors() throws IOException { + return super.getFsTableDescriptors(); + } + /** * For compatibility, if failed with regionserver credentials, try the master one */ @@ -524,7 +533,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { this.loadBalancerTracker.start(); this.assignmentManager = new AssignmentManager(this, serverManager, this.balancer, this.service, this.metricsMaster, - this.tableLockManager); + this.tableLockManager, this.tableStateManager); zooKeeper.registerListenerFirst(assignmentManager); this.regionServerTracker = new RegionServerTracker(zooKeeper, this, @@ -613,6 +622,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server { // Invalidate all write locks held previously this.tableLockManager.reapWriteLocks(); + this.tableStateManager = new TableStateManager(this); + status.setStatus("Initializing ZK system trackers"); initializeZKBasedSystemTrackers(); @@ -689,6 +700,14 @@ public class HMaster extends HRegionServer implements MasterServices, Server { // assigned when master is shutting down if(isStopped()) return; + // migrating existent table state from zk + for (Map.Entry entry : ZKDataMigrator + .queryForTableStates(getZooKeeper()).entrySet()) { + LOG.info("Converting state from zk to new states:" + entry); + tableStateManager.setTableState(entry.getKey(), entry.getValue()); + } + ZKUtil.deleteChildrenRecursively(getZooKeeper(), getZooKeeper().tableZNode); + status.setStatus("Submitting log splitting work for previously failed region servers"); // Master has recovered hbase:meta region server and we put // other failed region servers in a queue to be handled later by SSH @@ -880,6 +899,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server { this.fileSystemManager.splitMetaLog(previouslyFailedMetaRSs); } + this.assignmentManager.setEnabledTable(TableName.META_TABLE_NAME); + tableStateManager.start(); + // Make sure a hbase:meta location is set. We need to enable SSH here since // if the meta region server is died at this time, we need it to be re-assigned // by SSH so that system tables can be assigned. @@ -955,7 +977,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { private void enableMeta(TableName metaTableName) { if (!this.assignmentManager.getTableStateManager().isTableState(metaTableName, - ZooKeeperProtos.Table.State.ENABLED)) { + TableState.State.ENABLED)) { this.assignmentManager.setEnabledTable(metaTableName); } } @@ -1186,7 +1208,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server { if (rpCount < plans.size() && // if performing next balance exceeds cutoff time, exit the loop (System.currentTimeMillis() + (totalRegPlanExecTime / rpCount)) > cutoffTime) { - //TODO: After balance, there should not be a cutoff time (keeping it as a security net for now) + //TODO: After balance, there should not be a cutoff time (keeping it as + // a security net for now) LOG.debug("No more balancing till next balance run; maximumBalanceTime=" + maximumBalanceTime); break; @@ -1474,7 +1497,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server { LOG.fatal("Failed to become active master", t); // HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility if (t instanceof NoClassDefFoundError && - t.getMessage().contains("org/apache/hadoop/hdfs/protocol/FSConstants$SafeModeAction")) { + t.getMessage() + .contains("org/apache/hadoop/hdfs/protocol/FSConstants$SafeModeAction")) { // improved error message for this special case abort("HBase is having a problem with its Hadoop jars. You may need to " + "recompile HBase against Hadoop version " @@ -1718,11 +1742,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server { if (isCatalogTable(tableName)) { throw new IOException("Can't modify catalog tables"); } - if (!MetaTableAccessor.tableExists(getConnection(), tableName)) { + if (!tableStateManager.isTableExists(tableName)) { throw new TableNotFoundException(tableName); } - if (!getAssignmentManager().getTableStateManager(). - isTableState(tableName, ZooKeeperProtos.Table.State.DISABLED)) { + if (!tableStateManager.isTableState(tableName, TableState.State.DISABLED)) { throw new TableNotDisabledException(tableName); } } @@ -2200,15 +2223,18 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } for (HTableDescriptor desc: htds) { - if (includeSysTables || !desc.getTableName().isSystemTable()) { + if (tableStateManager.isTablePresent(desc.getTableName()) + && (includeSysTables || !desc.getTableName().isSystemTable())) { descriptors.add(desc); } } } else { for (TableName s: tableNameList) { - HTableDescriptor desc = tableDescriptors.get(s); - if (desc != null) { - descriptors.add(desc); + if (tableStateManager.isTablePresent(s)) { + HTableDescriptor desc = tableDescriptors.get(s); + if (desc != null) { + descriptors.add(desc); + } } } } @@ -2307,4 +2333,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server { public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException { return getClusterStatus().getLastMajorCompactionTsForRegion(regionName); } + + @Override + public TableStateManager getTableStateManager() { + return tableStateManager; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java index 458e53c8e3f..09aa52860be 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java @@ -266,4 +266,9 @@ public interface MasterServices extends Server { * @throws IOException */ public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException; + + /** + * @return table state manager + */ + public TableStateManager getTableStateManager(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java index 18a781fdd63..f7ef70fac5b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java @@ -241,7 +241,8 @@ public class RegionStateStore { } } // Called when meta is not on master - multiHConnection.processBatchCallback(Arrays.asList(put), TableName.META_TABLE_NAME, null, null); + multiHConnection.processBatchCallback(Arrays.asList(put), + TableName.META_TABLE_NAME, null, null); } catch (IOException ioe) { LOG.error("Failed to persist region state " + newState, ioe); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java index c27c0eeb606..a770c460984 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java @@ -38,15 +38,12 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.RegionTransition; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.TableStateManager; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.client.RegionReplicaUtil; -import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.master.RegionState.State; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Pair; @@ -153,13 +150,13 @@ public class RegionStates { private final TableStateManager tableStateManager; private final RegionStateStore regionStateStore; private final ServerManager serverManager; - private final Server server; + private final MasterServices server; // The maximum time to keep a log split info in region states map static final String LOG_SPLIT_TIME = "hbase.master.maximum.logsplit.keeptime"; static final long DEFAULT_LOG_SPLIT_TIME = 7200000L; // 2 hours - RegionStates(final Server master, final TableStateManager tableStateManager, + RegionStates(final MasterServices master, final TableStateManager tableStateManager, final ServerManager serverManager, final RegionStateStore regionStateStore) { this.tableStateManager = tableStateManager; this.regionStateStore = regionStateStore; @@ -594,7 +591,7 @@ public class RegionStates { if (oldServerName != null && serverHoldings.containsKey(oldServerName)) { if (newState == State.MERGED || newState == State.SPLIT || hri.isMetaRegion() || tableStateManager.isTableState(hri.getTable(), - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) { + TableState.State.DISABLED, TableState.State.DISABLING)) { // Offline the region only if it's merged/split, or the table is disabled/disabling. // Otherwise, offline it from this server only when it is online on a different server. LOG.info("Offlined " + hri.getShortNameToLog() + " from " + oldServerName); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java index b03611c5301..39beba8f264 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java @@ -141,7 +141,7 @@ public class SnapshotOfRegionAssignmentFromMeta { } }; // Scan hbase:meta to pick up user regions - MetaTableAccessor.fullScan(connection, v); + MetaTableAccessor.fullScanRegions(connection, v); //regionToRegionServerMap = regions; LOG.info("Finished to scan the hbase:meta for the current region assignment" + "snapshot"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java index c22294b60ae..e33baa67338 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java @@ -80,8 +80,7 @@ public class TableNamespaceManager { } public void start() throws IOException { - if (!MetaTableAccessor.tableExists(masterServices.getConnection(), - TableName.NAMESPACE_TABLE_NAME)) { + if (!masterServices.getTableStateManager().isTableExists(TableName.NAMESPACE_TABLE_NAME)) { LOG.info("Namespace table not found. Creating..."); createNamespaceTable(masterServices); } 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 new file mode 100644 index 00000000000..d72bdaa8382 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java @@ -0,0 +1,285 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.master; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import com.google.common.collect.Sets; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +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.TableNotFoundException; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.TableState; + +/** + * This is a helper class used to manage table states. + * States persisted in tableinfo and cached internally. + */ +@InterfaceAudience.Private +public class TableStateManager { + private static final Log LOG = LogFactory.getLog(TableStateManager.class); + + private final ReadWriteLock lock = new ReentrantReadWriteLock(); + private final MasterServices master; + + public TableStateManager(MasterServices master) { + this.master = master; + } + + /** + * Set table state to provided. + * Caller should lock table on write. + * + * @param tableName table to change state for + * @param newState new state + * @throws java.io.IOException + */ + public void setTableState(TableName tableName, TableState.State newState) throws IOException { + lock.writeLock().lock(); + try { + writeMetaState(tableName, newState); + } finally { + lock.writeLock().unlock(); + } + + } + + /** + * 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 + * @throws java.io.IOException + */ + public boolean setTableStateIfInStates(TableName tableName, + TableState.State newState, + TableState.State... states) + throws IOException { + lock.writeLock().lock(); + try { + TableState currentState = readMetaState(tableName); + if (currentState == null) { + throw new TableNotFoundException(tableName); + } + if (currentState.inStates(states)) { + writeMetaState(tableName, newState); + return true; + } else { + return false; + } + } 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 + * @throws java.io.IOException + */ + public boolean setTableStateIfNotInStates(TableName tableName, + TableState.State newState, + TableState.State... states) + throws IOException { + TableState currentState = readMetaState(tableName); + if (currentState == null) { + throw new TableNotFoundException(tableName); + } + if (!currentState.inStates(states)) { + writeMetaState(tableName, newState); + return true; + } else { + return false; + } + } + + public boolean isTableExists(TableName tableName) throws IOException { + return readMetaState(tableName) != null; + } + + public boolean isTableState(TableName tableName, TableState.State... states) { + try { + TableState.State tableState = getTableState(tableName); + return TableState.isInStates(tableState, states); + } catch (IOException e) { + LOG.error("Unable to get table " + tableName + " state, probably table not exists"); + return false; + } + } + + public void setDeletedTable(TableName tableName) throws IOException { + if (tableName.equals(TableName.META_TABLE_NAME)) + return; + deleteFromMeta(tableName); + } + + public boolean isTablePresent(TableName tableName) throws IOException { + return readMetaState(tableName) != null; + } + + /** + * Return all tables in given states. + * + * @param states filter by states + * @return tables in given states + * @throws java.io.IOException + */ + public Set getTablesInStates(final TableState.State... states) throws IOException { + final Set rv = Sets.newHashSet(); + MetaTableAccessor.fullScanTables(master.getConnection(), new MetaTableAccessor.Visitor() { + @Override + public boolean visit(Result r) throws IOException { + TableState tableState = MetaTableAccessor.getTableState(r); + if (tableState != null && tableState.inStates(states)) + rv.add(tableState.getTableName()); + return true; + } + }); + return rv; + } + + @Nonnull + public TableState.State getTableState(TableName tableName) throws IOException { + TableState currentState = readMetaState(tableName); + if (currentState == null) { + throw new TableNotFoundException(tableName); + } + return currentState.getState(); + } + + @Nullable + public TableState readMetaState(TableName tableName) throws IOException { + if (tableName.equals(TableName.META_TABLE_NAME)) + return new TableState(tableName, TableState.State.ENABLED); + return fetchMetaState(tableName); + } + + protected void writeMetaState(TableName tableName, TableState.State newState) + throws IOException { + MetaTableAccessor.updateTableState(master.getConnection(), tableName, newState); + } + + protected TableState fetchMetaState(TableName tableName) throws IOException { + return MetaTableAccessor.getTableState(master.getConnection(), tableName); + } + + protected void deleteFromMeta(TableName tableName) throws IOException { + MetaTableAccessor.deleteTableState(master.getConnection(), tableName); + } + + @SuppressWarnings("deprecation") + public void start() throws IOException { + TableDescriptors tableDescriptors = master.getTableDescriptors(); + Connection connection = master.getConnection(); + fixTableStates(tableDescriptors, connection); + } + + protected static void fixTableStates(TableDescriptors tableDescriptors, Connection connection) + throws IOException { + final Map allDescriptors = + tableDescriptors.getAll(); + final Map states = new HashMap<>(); + MetaTableAccessor.fullScanTables(connection, new MetaTableAccessor.Visitor() { + @Override + public boolean visit(Result r) throws IOException { + TableState state = MetaTableAccessor.getTableState(r); + if (state != null) + states.put(state.getTableName().getNameAsString(), state); + return true; + } + }); + for (Map.Entry entry : allDescriptors.entrySet()) { + String table = entry.getKey(); + if (table.equals(TableName.META_TABLE_NAME.getNameAsString())) + continue; + if (!states.containsKey(table)) { + LOG.warn("Found table with no state, assuming ENABLED"); + MetaTableAccessor.updateTableState(connection, TableName.valueOf(table), + TableState.State.ENABLED); + } + } + } + + /** + * An inmemory implementation + */ + @InterfaceAudience.Private + public static class InMemoryTableStateManager extends TableStateManager { + + private HashMap map = new HashMap<>(); + + public InMemoryTableStateManager() { + super(null); + } + + @Override + public void start() throws IOException { + // do nothing + } + + @Override + protected void writeMetaState(TableName tableName, TableState.State newState) + throws IOException { + map.put(tableName, newState); + } + + @Override + protected TableState fetchMetaState(TableName tableName) throws IOException { + TableState.State state = map.get(tableName); + if (state == null) + return null; + return new TableState(tableName, state); + } + + @Override + protected void deleteFromMeta(TableName tableName) throws IOException { + map.remove(tableName); + } + + @Override + public Set getTablesInStates(TableState.State... states) throws IOException { + Set tns = new HashSet<>(); + for (Map.Entry entry : map.entrySet()) { + if (TableState.isInStates(entry.getValue(), states)) + tns.add(entry.getKey()); + } + return tns; + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java index e0e4ee4d6c6..a8ba14f7cff 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java @@ -23,11 +23,11 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.Server; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.RegionState; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; /** * Handles CLOSED region event on Master. @@ -93,7 +93,7 @@ public class ClosedRegionHandler extends EventHandler implements TotesHRegionInf LOG.debug("Handling CLOSED event for " + regionInfo.getEncodedName()); // Check if this table is being disabled or not if (this.assignmentManager.getTableStateManager().isTableState(this.regionInfo.getTable(), - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) || + TableState.State.DISABLED, TableState.State.DISABLING) || assignmentManager.getReplicasToClose().contains(regionInfo)) { assignmentManager.offlineDisabledRegion(regionInfo); return; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java index f9d0d24f86f..e6f66507af5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java @@ -31,14 +31,15 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CoordinatedStateException; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.TableExistsException; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionReplicaUtil; -import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.ipc.RequestContext; @@ -49,7 +50,7 @@ import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.TableLockManager; import org.apache.hadoop.hbase.master.TableLockManager.TableLock; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; +import org.apache.hadoop.hbase.master.TableStateManager; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.util.FSTableDescriptors; @@ -68,7 +69,9 @@ public class CreateTableHandler extends EventHandler { protected final Configuration conf; private final AssignmentManager assignmentManager; private final TableLockManager tableLockManager; + private final TableStateManager tableStateManager; private final HRegionInfo [] newRegions; + private final MasterServices masterServices; private final TableLock tableLock; private User activeUser; @@ -81,8 +84,10 @@ public class CreateTableHandler extends EventHandler { this.hTableDescriptor = hTableDescriptor; this.conf = conf; this.newRegions = newRegions; + this.masterServices = masterServices; this.assignmentManager = masterServices.getAssignmentManager(); this.tableLockManager = masterServices.getTableLockManager(); + this.tableStateManager = masterServices.getTableStateManager(); this.tableLock = this.tableLockManager.writeLock(this.hTableDescriptor.getTableName() , EventType.C_M_CREATE_TABLE.toString()); @@ -118,11 +123,9 @@ public class CreateTableHandler extends EventHandler { boolean success = false; try { TableName tableName = this.hTableDescriptor.getTableName(); - if (MetaTableAccessor.tableExists(this.server.getConnection(), tableName)) { + if (tableStateManager.isTableExists(tableName)) { throw new TableExistsException(tableName); } - - checkAndSetEnablingTable(assignmentManager, tableName); success = true; } finally { if (!success) { @@ -132,47 +135,6 @@ public class CreateTableHandler extends EventHandler { return this; } - static void checkAndSetEnablingTable(final AssignmentManager assignmentManager, - final TableName tableName) throws IOException { - // If we have multiple client threads trying to create the table at the - // same time, given the async nature of the operation, the table - // could be in a state where hbase:meta table hasn't been updated yet in - // the process() function. - // Use enabling state to tell if there is already a request for the same - // table in progress. This will introduce a new zookeeper call. Given - // createTable isn't a frequent operation, that should be ok. - // TODO: now that we have table locks, re-evaluate above -- table locks are not enough. - // We could have cleared the hbase.rootdir and not zk. How can we detect this case? - // Having to clean zk AND hdfs is awkward. - try { - if (!assignmentManager.getTableStateManager().setTableStateIfNotInStates(tableName, - ZooKeeperProtos.Table.State.ENABLING, - ZooKeeperProtos.Table.State.ENABLING, - ZooKeeperProtos.Table.State.ENABLED)) { - throw new TableExistsException(tableName); - } - } catch (CoordinatedStateException e) { - throw new IOException("Unable to ensure that the table will be" + - " enabling because of a ZooKeeper issue", e); - } - } - - static void removeEnablingTable(final AssignmentManager assignmentManager, - final TableName tableName) { - // Try deleting the enabling node in case of error - // If this does not happen then if the client tries to create the table - // again with the same Active master - // It will block the creation saying TableAlreadyExists. - try { - assignmentManager.getTableStateManager().checkAndRemoveTableState(tableName, - ZooKeeperProtos.Table.State.ENABLING, false); - } catch (CoordinatedStateException e) { - // Keeper exception should not happen here - LOG.error("Got a keeper exception while removing the ENABLING table znode " - + tableName, e); - } - } - @Override public String toString() { String name = "UnknownServerName"; @@ -180,16 +142,16 @@ public class CreateTableHandler extends EventHandler { name = server.getServerName().toString(); } return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" + - this.hTableDescriptor.getTableName(); + this.hTableDescriptor.getTableName(); } @Override public void process() { TableName tableName = this.hTableDescriptor.getTableName(); LOG.info("Create table " + tableName); - + HMaster master = ((HMaster) this.server); try { - final MasterCoprocessorHost cpHost = ((HMaster) this.server).getMasterCoprocessorHost(); + final MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost(); if (cpHost != null) { cpHost.preCreateTableHandler(this.hTableDescriptor, this.newRegions); } @@ -218,9 +180,6 @@ public class CreateTableHandler extends EventHandler { releaseTableLock(); LOG.info("Table, " + this.hTableDescriptor.getTableName() + ", creation " + (exception == null ? "successful" : "failed. " + exception)); - if (exception != null) { - removeEnablingTable(this.assignmentManager, this.hTableDescriptor.getTableName()); - } } /** @@ -243,9 +202,13 @@ public class CreateTableHandler extends EventHandler { FileSystem fs = fileSystemManager.getFileSystem(); // 1. Create Table Descriptor + // using a copy of descriptor, table will be created enabling first + HTableDescriptor underConstruction = new HTableDescriptor( + this.hTableDescriptor); Path tempTableDir = FSUtils.getTableDir(tempdir, tableName); - new FSTableDescriptors(this.conf).createTableDescriptorForTableDirectory( - tempTableDir, this.hTableDescriptor, false); + ((FSTableDescriptors)(masterServices.getTableDescriptors())) + .createTableDescriptorForTableDirectory( + tempTableDir, underConstruction, false); Path tableDir = FSUtils.getTableDir(fileSystemManager.getRootDir(), tableName); // 2. Create Regions @@ -253,9 +216,15 @@ public class CreateTableHandler extends EventHandler { // 3. Move Table temp directory to the hbase root location if (!fs.rename(tempTableDir, tableDir)) { throw new IOException("Unable to move table from temp=" + tempTableDir + - " to hbase root=" + tableDir); + " to hbase root=" + tableDir); } + // populate descriptors cache to be visible in getAll + masterServices.getTableDescriptors().get(tableName); + + MetaTableAccessor.updateTableState(this.server.getConnection(), hTableDescriptor.getTableName(), + TableState.State.ENABLING); + if (regionInfos != null && regionInfos.size() > 0) { // 4. Add regions to META addRegionsToMeta(regionInfos, hTableDescriptor.getRegionReplication()); @@ -271,23 +240,18 @@ public class CreateTableHandler extends EventHandler { ModifyRegionUtils.assignRegions(assignmentManager, regionInfos); } - // 8. Set table enabled flag up in zk. - try { - assignmentManager.getTableStateManager().setTableState(tableName, - ZooKeeperProtos.Table.State.ENABLED); - } catch (CoordinatedStateException e) { - throw new IOException("Unable to ensure that " + tableName + " will be" + - " enabled because of a ZooKeeper issue", e); - } + // 8. Enable table + assignmentManager.getTableStateManager().setTableState(tableName, + TableState.State.ENABLED); - // 8. Update the tabledescriptor cache. + // 9. Update the tabledescriptor cache. ((HMaster) this.server).getTableDescriptors().get(tableName); } /** * Create any replicas for the regions (the default replicas that was * already created is passed to the method) - * @param hTableDescriptor + * @param hTableDescriptor descriptor to use * @param regions default replicas * @return the combined list of default and non-default replicas */ @@ -325,7 +289,7 @@ public class CreateTableHandler extends EventHandler { * @return the list of regions created */ protected List handleCreateHdfsRegions(final Path tableRootDir, - final TableName tableName) + final TableName tableName) throws IOException { return ModifyRegionUtils.createRegions(conf, tableRootDir, hTableDescriptor, newRegions, null); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java index 93dcc84b48b..2049945c965 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java @@ -72,7 +72,7 @@ public class DeleteTableHandler extends TableEventHandler { AssignmentManager am = this.masterServices.getAssignmentManager(); RegionStates states = am.getRegionStates(); long waitTime = server.getConfiguration(). - getLong("hbase.master.wait.on.region", 5 * 60 * 1000); + getLong("hbase.master.wait.on.region", 5 * 60 * 1000); for (HRegionInfo region : regions) { long done = System.currentTimeMillis() + waitTime; while (System.currentTimeMillis() < done) { @@ -87,12 +87,12 @@ public class DeleteTableHandler extends TableEventHandler { throw (InterruptedIOException)new InterruptedIOException().initCause(e); } LOG.debug("Waiting on region to clear regions in transition; " - + am.getRegionStates().getRegionTransitionState(region)); + + am.getRegionStates().getRegionTransitionState(region)); } if (states.isRegionInTransition(region)) { throw new IOException("Waited hbase.master.wait.on.region (" + - waitTime + "ms) for region to leave region " + - region.getRegionNameAsString() + " in transitions"); + waitTime + "ms) for region to leave region " + + region.getRegionNameAsString() + " in transitions"); } } } @@ -108,33 +108,33 @@ public class DeleteTableHandler extends TableEventHandler { // 1. Wait because of region in transition waitRegionInTransition(regions); - try { - // 2. Remove table from hbase:meta and HDFS - removeTableData(regions); - } finally { - // 3. Update table descriptor cache - LOG.debug("Removing '" + tableName + "' descriptor."); - this.masterServices.getTableDescriptors().remove(tableName); - - AssignmentManager am = this.masterServices.getAssignmentManager(); - - // 4. Clean up regions of the table in RegionStates. - LOG.debug("Removing '" + tableName + "' from region states."); - am.getRegionStates().tableDeleted(tableName); - - // 5. If entry for this table in zk, and up in AssignmentManager, remove it. - LOG.debug("Marking '" + tableName + "' as deleted."); - am.getTableStateManager().setDeletedTable(tableName); - - // 6.Clean any remaining rows for this table. - cleanAnyRemainingRows(); - } + // 2. Remove table from hbase:meta and HDFS + removeTableData(regions); if (cpHost != null) { cpHost.postDeleteTableHandler(this.tableName); } } + private void cleanupTableState() throws IOException { + // 3. Update table descriptor cache + LOG.debug("Removing '" + tableName + "' descriptor."); + this.masterServices.getTableDescriptors().remove(tableName); + + AssignmentManager am = this.masterServices.getAssignmentManager(); + + // 4. Clean up regions of the table in RegionStates. + LOG.debug("Removing '" + tableName + "' from region states."); + am.getRegionStates().tableDeleted(tableName); + + // 5.Clean any remaining rows for this table. + cleanAnyRemainingRows(); + + // 6. If entry for this table states, remove it. + LOG.debug("Marking '" + tableName + "' as deleted."); + am.getTableStateManager().setDeletedTable(tableName); + } + /** * There may be items for this table still up in hbase:meta in the case where the * info:regioninfo column was empty because of some write error. Remove ALL rows from hbase:meta @@ -153,7 +153,7 @@ public class DeleteTableHandler extends TableEventHandler { } if (!deletes.isEmpty()) { LOG.warn("Deleting some vestigal " + deletes.size() + " rows of " + this.tableName + - " from " + TableName.META_TABLE_NAME); + " from " + TableName.META_TABLE_NAME); metaTable.delete(deletes); } } @@ -163,34 +163,38 @@ public class DeleteTableHandler extends TableEventHandler { * Removes the table from hbase:meta and archives the HDFS files. */ protected void removeTableData(final List regions) - throws IOException, CoordinatedStateException { - // 1. Remove regions from META - LOG.debug("Deleting regions from META"); - MetaTableAccessor.deleteRegions(this.server.getConnection(), regions); + throws IOException, CoordinatedStateException { + try { + // 1. Remove regions from META + LOG.debug("Deleting regions from META"); + MetaTableAccessor.deleteRegions(this.server.getConnection(), regions); - // ----------------------------------------------------------------------- - // NOTE: At this point we still have data on disk, but nothing in hbase:meta - // if the rename below fails, hbck will report an inconsistency. - // ----------------------------------------------------------------------- + // ----------------------------------------------------------------------- + // NOTE: At this point we still have data on disk, but nothing in hbase:meta + // if the rename below fails, hbck will report an inconsistency. + // ----------------------------------------------------------------------- - // 2. Move the table in /hbase/.tmp - MasterFileSystem mfs = this.masterServices.getMasterFileSystem(); - Path tempTableDir = mfs.moveTableToTemp(tableName); + // 2. Move the table in /hbase/.tmp + MasterFileSystem mfs = this.masterServices.getMasterFileSystem(); + Path tempTableDir = mfs.moveTableToTemp(tableName); - // 3. Archive regions from FS (temp directory) - FileSystem fs = mfs.getFileSystem(); - for (HRegionInfo hri : regions) { - LOG.debug("Archiving region " + hri.getRegionNameAsString() + " from FS"); - HFileArchiver.archiveRegion(fs, mfs.getRootDir(), - tempTableDir, HRegion.getRegionDir(tempTableDir, hri.getEncodedName())); + // 3. Archive regions from FS (temp directory) + FileSystem fs = mfs.getFileSystem(); + for (HRegionInfo hri : regions) { + LOG.debug("Archiving region " + hri.getRegionNameAsString() + " from FS"); + HFileArchiver.archiveRegion(fs, mfs.getRootDir(), + tempTableDir, HRegion.getRegionDir(tempTableDir, hri.getEncodedName())); + } + + // 4. Delete table directory from FS (temp directory) + if (!fs.delete(tempTableDir, true)) { + LOG.error("Couldn't delete " + tempTableDir); + } + + LOG.debug("Table '" + tableName + "' archived!"); + } finally { + cleanupTableState(); } - - // 4. Delete table directory from FS (temp directory) - if (!fs.delete(tempTableDir, true)) { - LOG.error("Couldn't delete " + tempTableDir); - } - - LOG.debug("Table '" + tableName + "' archived!"); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java index aaacfaab2af..f6ddf60490e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.constraint.ConstraintException; import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventType; @@ -39,11 +40,12 @@ import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.BulkAssigner; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; +import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.RegionStates; import org.apache.hadoop.hbase.master.TableLockManager; import org.apache.hadoop.hbase.master.RegionState.State; import org.apache.hadoop.hbase.master.TableLockManager.TableLock; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; +import org.apache.hadoop.hbase.master.TableStateManager; import org.apache.htrace.Trace; /** @@ -55,16 +57,18 @@ public class DisableTableHandler extends EventHandler { private final TableName tableName; private final AssignmentManager assignmentManager; private final TableLockManager tableLockManager; + private final TableStateManager tableStateManager; private final boolean skipTableStateCheck; private TableLock tableLock; - public DisableTableHandler(Server server, TableName tableName, + public DisableTableHandler(MasterServices server, TableName tableName, AssignmentManager assignmentManager, TableLockManager tableLockManager, boolean skipTableStateCheck) { super(server, EventType.C_M_DISABLE_TABLE); this.tableName = tableName; this.assignmentManager = assignmentManager; this.tableLockManager = tableLockManager; + this.tableStateManager = server.getTableStateManager(); this.skipTableStateCheck = skipTableStateCheck; } @@ -81,7 +85,7 @@ public class DisableTableHandler extends EventHandler { boolean success = false; try { // Check if table exists - if (!MetaTableAccessor.tableExists(this.server.getConnection(), tableName)) { + if (!tableStateManager.isTableExists(tableName)) { throw new TableNotFoundException(tableName); } @@ -91,16 +95,11 @@ public class DisableTableHandler extends EventHandler { // DISABLED or ENABLED. //TODO: reevaluate this since we have table locks now if (!skipTableStateCheck) { - try { - if (!this.assignmentManager.getTableStateManager().setTableStateIfInStates( - this.tableName, ZooKeeperProtos.Table.State.DISABLING, - ZooKeeperProtos.Table.State.ENABLED)) { - LOG.info("Table " + tableName + " isn't enabled; skipping disable"); - throw new TableNotEnabledException(this.tableName); - } - } catch (CoordinatedStateException e) { - throw new IOException("Unable to ensure that the table will be" + - " disabling because of a coordination engine issue", e); + if (!tableStateManager.setTableStateIfInStates( + this.tableName, TableState.State.DISABLING, + TableState.State.ENABLED)) { + LOG.info("Table " + tableName + " isn't enabled; skipping disable"); + throw new TableNotEnabledException(this.tableName); } } success = true; @@ -157,8 +156,8 @@ public class DisableTableHandler extends EventHandler { private void handleDisableTable() throws IOException, CoordinatedStateException { // Set table disabling flag up in zk. - this.assignmentManager.getTableStateManager().setTableState(this.tableName, - ZooKeeperProtos.Table.State.DISABLING); + tableStateManager.setTableState(this.tableName, + TableState.State.DISABLING); boolean done = false; while (true) { // Get list of online regions that are of this table. Regions that are @@ -186,8 +185,8 @@ public class DisableTableHandler extends EventHandler { } } // Flip the table to disabled if success. - if (done) this.assignmentManager.getTableStateManager().setTableState(this.tableName, - ZooKeeperProtos.Table.State.DISABLED); + if (done) this.tableStateManager.setTableState(this.tableName, + TableState.State.DISABLED); LOG.info("Disabled table, " + this.tableName + ", is done=" + done); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java index 243ec2d8143..b237582cc3f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.master.AssignmentManager; @@ -47,7 +48,7 @@ import org.apache.hadoop.hbase.master.RegionStates; import org.apache.hadoop.hbase.master.ServerManager; import org.apache.hadoop.hbase.master.TableLockManager; import org.apache.hadoop.hbase.master.TableLockManager.TableLock; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; +import org.apache.hadoop.hbase.master.TableStateManager; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; @@ -60,25 +61,20 @@ public class EnableTableHandler extends EventHandler { private final TableName tableName; private final AssignmentManager assignmentManager; private final TableLockManager tableLockManager; + private final TableStateManager tableStateManager; private boolean skipTableStateCheck = false; private TableLock tableLock; private MasterServices services; - public EnableTableHandler(Server server, TableName tableName, + public EnableTableHandler(MasterServices services, TableName tableName, AssignmentManager assignmentManager, TableLockManager tableLockManager, boolean skipTableStateCheck) { - super(server, EventType.C_M_ENABLE_TABLE); + super(services, EventType.C_M_ENABLE_TABLE); this.tableName = tableName; this.assignmentManager = assignmentManager; this.tableLockManager = tableLockManager; + this.tableStateManager = services.getTableStateManager(); this.skipTableStateCheck = skipTableStateCheck; - } - - public EnableTableHandler(MasterServices services, TableName tableName, - AssignmentManager assignmentManager, - TableLockManager tableLockManager, boolean skipTableStateCheck) { - this((Server)services, tableName, assignmentManager, tableLockManager, - skipTableStateCheck); this.services = services; } @@ -92,20 +88,11 @@ public class EnableTableHandler extends EventHandler { boolean success = false; try { // Check if table exists - if (!MetaTableAccessor.tableExists(this.server.getConnection(), tableName)) { + if (!this.tableStateManager.isTableExists(tableName)) { // retainAssignment is true only during recovery. In normal case it is false if (!this.skipTableStateCheck) { throw new TableNotFoundException(tableName); } - try { - this.assignmentManager.getTableStateManager().checkAndRemoveTableState(tableName, - ZooKeeperProtos.Table.State.ENABLING, true); - throw new TableNotFoundException(tableName); - } catch (CoordinatedStateException e) { - // TODO : Use HBCK to clear such nodes - LOG.warn("Failed to delete the ENABLING node for the table " + tableName - + ". The table will remain unusable. Run HBCK to manually fix the problem."); - } } // There could be multiple client requests trying to disable or enable @@ -113,16 +100,11 @@ public class EnableTableHandler extends EventHandler { // After that, no other requests can be accepted until the table reaches // DISABLED or ENABLED. if (!skipTableStateCheck) { - try { - if (!this.assignmentManager.getTableStateManager().setTableStateIfInStates( - this.tableName, ZooKeeperProtos.Table.State.ENABLING, - ZooKeeperProtos.Table.State.DISABLED)) { - LOG.info("Table " + tableName + " isn't disabled; skipping enable"); - throw new TableNotDisabledException(this.tableName); - } - } catch (CoordinatedStateException e) { - throw new IOException("Unable to ensure that the table will be" + - " enabling because of a coordination engine issue", e); + if (!this.assignmentManager.getTableStateManager().setTableStateIfInStates( + this.tableName, TableState.State.ENABLING, + TableState.State.DISABLED)) { + LOG.info("Table " + tableName + " isn't disabled; skipping enable"); + throw new TableNotDisabledException(this.tableName); } } success = true; @@ -185,7 +167,7 @@ public class EnableTableHandler extends EventHandler { // Set table enabling flag up in zk. this.assignmentManager.getTableStateManager().setTableState(this.tableName, - ZooKeeperProtos.Table.State.ENABLING); + TableState.State.ENABLING); boolean done = false; ServerManager serverManager = ((HMaster)this.server).getServerManager(); // Get the regions of this table. We're done when all listed @@ -250,7 +232,7 @@ public class EnableTableHandler extends EventHandler { if (done) { // Flip the table to enabled. this.assignmentManager.getTableStateManager().setTableState( - this.tableName, ZooKeeperProtos.Table.State.ENABLED); + this.tableName, TableState.State.ENABLED); LOG.info("Table '" + this.tableName + "' was successfully enabled. Status: done=" + done); } else { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java index baa8513ce9b..8b909ad7550 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java @@ -37,12 +37,12 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterServices; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.util.Bytes; @InterfaceAudience.Private @@ -65,7 +65,7 @@ public class ModifyTableHandler extends TableEventHandler { // Check operation is possible on the table in its current state // Also checks whether the table exists if (masterServices.getAssignmentManager().getTableStateManager() - .isTableState(this.htd.getTableName(), ZooKeeperProtos.Table.State.ENABLED) + .isTableState(this.htd.getTableName(), TableState.State.ENABLED) && this.htd.getRegionReplication() != getTableDescriptor().getRegionReplication()) { throw new IOException("REGION_REPLICATION change is not supported for enabled tables"); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java index 7789ee13ee1..d6b0f0c2498 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.RegionReplicaUtil; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.master.AssignmentManager; @@ -45,7 +46,6 @@ import org.apache.hadoop.hbase.master.RegionState.State; import org.apache.hadoop.hbase.master.RegionStates; import org.apache.hadoop.hbase.master.ServerManager; import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; import org.apache.hadoop.hbase.util.ConfigUtil; import org.apache.hadoop.hbase.zookeeper.ZKAssign; @@ -285,7 +285,7 @@ public class ServerShutdownHandler extends EventHandler { } else if (rit != null) { if ((rit.isPendingCloseOrClosing() || rit.isOffline()) && am.getTableStateManager().isTableState(hri.getTable(), - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) || + TableState.State.DISABLED, TableState.State.DISABLING) || am.getReplicasToClose().contains(hri)) { // If the table was partially disabled and the RS went down, we should clear the RIT // and remove the node for the region. @@ -375,7 +375,7 @@ public class ServerShutdownHandler extends EventHandler { } // If table is not disabled but the region is offlined, boolean disabled = assignmentManager.getTableStateManager().isTableState(hri.getTable(), - ZooKeeperProtos.Table.State.DISABLED); + TableState.State.DISABLED); if (disabled){ LOG.info("The table " + hri.getTable() + " was disabled. Hence not proceeding."); @@ -388,7 +388,7 @@ public class ServerShutdownHandler extends EventHandler { return false; } boolean disabling = assignmentManager.getTableStateManager().isTableState(hri.getTable(), - ZooKeeperProtos.Table.State.DISABLING); + TableState.State.DISABLING); if (disabling) { LOG.info("The table " + hri.getTable() + " is disabled. Hence not assigning region" + hri.getEncodedName()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java index 1b141fc4349..e7546d75c4b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java @@ -40,12 +40,12 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.RegionLocator; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.master.BulkReOpen; import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.TableLockManager.TableLock; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -136,7 +136,7 @@ public abstract class TableEventHandler extends EventHandler { handleTableOperation(hris); if (eventType.isOnlineSchemaChangeSupported() && this.masterServices. getAssignmentManager().getTableStateManager().isTableState( - tableName, ZooKeeperProtos.Table.State.ENABLED)) { + tableName, TableState.State.ENABLED)) { if (reOpenAllRegions(hris)) { LOG.info("Completed table operation " + eventType + " on table " + tableName); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java index 4d4e3314924..3ab3e319429 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java @@ -23,6 +23,7 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -31,12 +32,12 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterServices; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.ModifyRegionUtils; @@ -93,54 +94,49 @@ public class TruncateTableHandler extends DeleteTableHandler { AssignmentManager assignmentManager = this.masterServices.getAssignmentManager(); - // 1. Set table znode - CreateTableHandler.checkAndSetEnablingTable(assignmentManager, tableName); - try { - // 1. Create Table Descriptor - Path tempTableDir = FSUtils.getTableDir(tempdir, this.tableName); - new FSTableDescriptors(server.getConfiguration()) - .createTableDescriptorForTableDirectory(tempTableDir, this.hTableDescriptor, false); - Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), this.tableName); + // 1. Create Table Descriptor + HTableDescriptor underConstruction = new HTableDescriptor(this.hTableDescriptor); + Path tempTableDir = FSUtils.getTableDir(tempdir, this.tableName); - HRegionInfo[] newRegions; - if (this.preserveSplits) { - newRegions = regions.toArray(new HRegionInfo[regions.size()]); - LOG.info("Truncate will preserve " + newRegions.length + " regions"); - } else { - newRegions = new HRegionInfo[1]; - newRegions[0] = new HRegionInfo(this.tableName, null, null); - LOG.info("Truncate will not preserve the regions"); - } + ((FSTableDescriptors)(masterServices.getTableDescriptors())) + .createTableDescriptorForTableDirectory(tempTableDir, underConstruction, false); + Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), this.tableName); - // 2. Create Regions - List regionInfos = ModifyRegionUtils.createRegions( + HRegionInfo[] newRegions; + if (this.preserveSplits) { + newRegions = regions.toArray(new HRegionInfo[regions.size()]); + LOG.info("Truncate will preserve " + newRegions.length + " regions"); + } else { + newRegions = new HRegionInfo[1]; + newRegions[0] = new HRegionInfo(this.tableName, null, null); + LOG.info("Truncate will not preserve the regions"); + } + + // 2. Create Regions + List regionInfos = ModifyRegionUtils.createRegions( masterServices.getConfiguration(), tempdir, this.hTableDescriptor, newRegions, null); - // 3. Move Table temp directory to the hbase root location - if (!fs.rename(tempTableDir, tableDir)) { - throw new IOException("Unable to move table from temp=" + tempTableDir + + // 3. Move Table temp directory to the hbase root location + if (!fs.rename(tempTableDir, tableDir)) { + throw new IOException("Unable to move table from temp=" + tempTableDir + " to hbase root=" + tableDir); - } + } - // 4. Add regions to META - MetaTableAccessor.addRegionsToMeta(masterServices.getConnection(), + // populate descriptors cache to be visible in getAll + masterServices.getTableDescriptors().get(tableName); + + assignmentManager.getTableStateManager().setTableState(tableName, + TableState.State.ENABLING); + // 4. Add regions to META + MetaTableAccessor.addRegionsToMeta(masterServices.getConnection(), regionInfos, hTableDescriptor.getRegionReplication()); - // 5. Trigger immediate assignment of the regions in round-robin fashion - ModifyRegionUtils.assignRegions(assignmentManager, regionInfos); + // 5. Trigger immediate assignment of the regions in round-robin fashion + ModifyRegionUtils.assignRegions(assignmentManager, regionInfos); - // 6. Set table enabled flag up in zk. - try { - assignmentManager.getTableStateManager().setTableState(tableName, - ZooKeeperProtos.Table.State.ENABLED); - } catch (CoordinatedStateException e) { - throw new IOException("Unable to ensure that " + tableName + " will be" + - " enabled because of a ZooKeeper issue", e); - } - } catch (IOException e) { - CreateTableHandler.removeEnablingTable(assignmentManager, tableName); - throw e; - } + // 6. Set table enabled flag up in zk. + assignmentManager.getTableStateManager().setTableState(tableName, + TableState.State.ENABLED); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java index 44435a2c076..627f9c71a4a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.errorhandling.ForeignException; import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.master.AssignmentManager; @@ -63,7 +64,6 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; @@ -559,14 +559,14 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable TableName snapshotTable = TableName.valueOf(snapshot.getTable()); AssignmentManager assignmentMgr = master.getAssignmentManager(); if (assignmentMgr.getTableStateManager().isTableState(snapshotTable, - ZooKeeperProtos.Table.State.ENABLED)) { + TableState.State.ENABLED)) { LOG.debug("Table enabled, starting distributed snapshot."); snapshotEnabledTable(snapshot); LOG.debug("Started snapshot: " + ClientSnapshotDescriptionUtils.toString(snapshot)); } // For disabled table, snapshot is created by the master else if (assignmentMgr.getTableStateManager().isTableState(snapshotTable, - ZooKeeperProtos.Table.State.DISABLED)) { + TableState.State.DISABLED)) { LOG.debug("Table is disabled, running snapshot entirely on master."); snapshotDisabledTable(snapshot); LOG.debug("Started snapshot: " + ClientSnapshotDescriptionUtils.toString(snapshot)); @@ -697,7 +697,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable // Execute the restore/clone operation if (MetaTableAccessor.tableExists(master.getConnection(), tableName)) { if (master.getAssignmentManager().getTableStateManager().isTableState( - TableName.valueOf(fsSnapshot.getTable()), ZooKeeperProtos.Table.State.ENABLED)) { + TableName.valueOf(fsSnapshot.getTable()), TableState.State.ENABLED)) { throw new UnsupportedOperationException("Table '" + TableName.valueOf(fsSnapshot.getTable()) + "' must be disabled in order to " + "perform a restore operation" + diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index ed5ff41671e..1b046dcceab 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -560,6 +560,11 @@ public class HRegionServer extends HasThread implements this.choreService = new ChoreService(getServerName().toString()); } + protected TableDescriptors getFsTableDescriptors() throws IOException { + return new FSTableDescriptors(this.conf, + this.fs, this.rootDir, !canUpdateTableDescriptor(), false); + } + protected void login(UserProvider user, String host) throws IOException { user.login("hbase.regionserver.keytab.file", "hbase.regionserver.kerberos.principal", host); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java index d219ed2ddfa..4dfd61b2603 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java @@ -18,11 +18,6 @@ package org.apache.hadoop.hbase.security.visibility; -import static org.apache.hadoop.hbase.HConstants.OperationStatusCode.SANITY_CHECK_FAILURE; -import static org.apache.hadoop.hbase.HConstants.OperationStatusCode.SUCCESS; -import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_FAMILY; -import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME; - import java.io.IOException; import java.net.InetAddress; import java.util.ArrayList; @@ -31,6 +26,12 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import com.google.common.collect.Lists; +import com.google.common.collect.MapMaker; +import com.google.protobuf.ByteString; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -43,7 +44,6 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagRewriteCell; @@ -104,12 +104,10 @@ import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; -import com.google.common.collect.Lists; -import com.google.common.collect.MapMaker; -import com.google.protobuf.ByteString; -import com.google.protobuf.RpcCallback; -import com.google.protobuf.RpcController; -import com.google.protobuf.Service; +import static org.apache.hadoop.hbase.HConstants.OperationStatusCode.SANITY_CHECK_FAILURE; +import static org.apache.hadoop.hbase.HConstants.OperationStatusCode.SUCCESS; +import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_FAMILY; +import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME; /** * Coprocessor that has both the MasterObserver and RegionObserver implemented that supports in @@ -180,7 +178,8 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements public void postStartMaster(ObserverContext ctx) throws IOException { // Need to create the new system table for labels here MasterServices master = ctx.getEnvironment().getMasterServices(); - if (!MetaTableAccessor.tableExists(master.getConnection(), LABELS_TABLE_NAME)) { + if (!ctx.getEnvironment().getMasterServices() + .getTableStateManager().isTableExists(LABELS_TABLE_NAME)) { HTableDescriptor labelsTable = new HTableDescriptor(LABELS_TABLE_NAME); HColumnDescriptor labelsColumn = new HColumnDescriptor(LABELS_TABLE_FAMILY); labelsColumn.setBloomFilterType(BloomType.NONE); 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 7cd26739204..bd94fc54af5 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 @@ -30,7 +30,7 @@ import java.util.regex.Pattern; import org.apache.commons.lang.NotImplementedException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -198,8 +198,9 @@ public class FSTableDescriptors implements TableDescriptors { htds.put(entry.getKey().toString(), entry.getValue()); } // add hbase:meta to the response - htds.put(HTableDescriptor.META_TABLEDESC.getTableName().getNameAsString(), - HTableDescriptor.META_TABLEDESC); + + htds.put(metaTableDescriptor.getTableName().getNameAsString(), + metaTableDescriptor); } else { LOG.debug("Fetching table descriptors from the filesystem."); boolean allvisited = true; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index 601aff59226..10e58db8680 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -17,10 +17,10 @@ */ package org.apache.hadoop.hbase.util; +import javax.annotation.Nonnull; import java.io.Closeable; import java.io.FileNotFoundException; import java.io.IOException; -import java.io.InterruptedIOException; import java.io.PrintWriter; import java.io.StringWriter; import java.net.InetAddress; @@ -85,17 +85,12 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.HConnectable; -import org.apache.hadoop.hbase.client.HConnection; -import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.MetaScanner; import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase; @@ -104,6 +99,7 @@ import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.master.MasterFileSystem; @@ -124,8 +120,6 @@ import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandlerImpl; import org.apache.hadoop.hbase.util.hbck.TableLockChecker; import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; -import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader; -import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException; @@ -137,7 +131,6 @@ import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; import org.apache.zookeeper.KeeperException; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; @@ -247,7 +240,8 @@ public class HBaseFsck extends Configured implements Closeable { // hbase:meta are always checked private Set tablesIncluded = new HashSet(); private int maxMerge = DEFAULT_MAX_MERGE; // maximum number of overlapping regions to merge - private int maxOverlapsToSideline = DEFAULT_OVERLAPS_TO_SIDELINE; // maximum number of overlapping regions to sideline + // maximum number of overlapping regions to sideline + private int maxOverlapsToSideline = DEFAULT_OVERLAPS_TO_SIDELINE; private boolean sidelineBigOverlaps = false; // sideline overlaps with >maxMerge regions private Path sidelineDir = null; @@ -269,8 +263,6 @@ public class HBaseFsck extends Configured implements Closeable { * to detect and correct consistency (hdfs/meta/deployment) problems. */ private TreeMap regionInfoMap = new TreeMap(); - private TreeSet disabledTables = - new TreeSet(); // Empty regioninfo qualifiers in hbase:meta private Set emptyRegionInfoQualifiers = new HashSet(); @@ -294,6 +286,8 @@ public class HBaseFsck extends Configured implements Closeable { private Map> orphanTableDirs = new HashMap>(); + private Map tableStates = + new HashMap(); /** * List of orphaned table ZNodes @@ -500,7 +494,7 @@ public class HBaseFsck extends Configured implements Closeable { fixes = 0; regionInfoMap.clear(); emptyRegionInfoQualifiers.clear(); - disabledTables.clear(); + tableStates.clear(); errors.clear(); tablesInfo.clear(); orphanHdfsDirs.clear(); @@ -584,15 +578,15 @@ public class HBaseFsck extends Configured implements Closeable { reportTablesInFlux(); } + // Get disabled tables states + loadTableStates(); + // load regiondirs and regioninfos from HDFS if (shouldCheckHdfs()) { loadHdfsRegionDirs(); loadHdfsRegionInfos(); } - // Get disabled tables from ZooKeeper - loadDisabledTables(); - // fix the orphan tables fixOrphanTables(); @@ -630,9 +624,6 @@ public class HBaseFsck extends Configured implements Closeable { checkAndFixTableLocks(); - // Check (and fix if requested) orphaned table ZNodes - checkAndFixOrphanedTableZNodes(); - // Remove the hbck lock unlockHbck(); @@ -763,6 +754,15 @@ public class HBaseFsck extends Configured implements Closeable { } } + /** + * Load the list of disabled tables in ZK into local set. + * @throws IOException + */ + private void loadTableStates() throws IOException { + tableStates = MetaTableAccessor.getTableStates(connection); + } + + /** * Orphaned regions are regions without a .regioninfo file in them. We "adopt" * these orphans by creating a new region, and moving the column families, @@ -1266,6 +1266,8 @@ public class HBaseFsck extends Configured implements Closeable { } TableInfo ti = e.getValue(); + puts.add(MetaTableAccessor + .makePutFromTableState(new TableState(ti.tableName, TableState.State.ENABLED))); for (Entry> spl : ti.sc.getStarts().asMap() .entrySet()) { Collection his = spl.getValue(); @@ -1517,39 +1519,13 @@ public class HBaseFsck extends Configured implements Closeable { return backupDir; } - /** - * Load the list of disabled tables in ZK into local set. - * @throws ZooKeeperConnectionException - * @throws IOException - */ - private void loadDisabledTables() - throws ZooKeeperConnectionException, IOException { - HConnectionManager.execute(new HConnectable(getConf()) { - @Override - public Void connect(HConnection connection) throws IOException { - ZooKeeperWatcher zkw = createZooKeeperWatcher(); - try { - for (TableName tableName : - ZKTableStateClientSideReader.getDisabledOrDisablingTables(zkw)) { - disabledTables.add(tableName); - } - } catch (KeeperException ke) { - throw new IOException(ke); - } catch (InterruptedException e) { - throw new InterruptedIOException(); - } finally { - zkw.close(); - } - return null; - } - }); - } /** * Check if the specified region's table is disabled. */ - private boolean isTableDisabled(HRegionInfo regionInfo) { - return disabledTables.contains(regionInfo.getTable()); + private boolean isTableDisabled(@Nonnull TableName tableName) { + TableState state = tableStates.get(tableName); + return state != null && state.inStates(TableState.State.DISABLED, TableState.State.DISABLING); } /** @@ -1619,15 +1595,24 @@ public class HBaseFsck extends Configured implements Closeable { HConstants.EMPTY_START_ROW, false, false); if (rl == null) { errors.reportError(ERROR_CODE.NULL_META_REGION, - "META region or some of its attributes are null."); + "META region was not found in Zookeeper"); return false; } for (HRegionLocation metaLocation : rl.getRegionLocations()) { // Check if Meta region is valid and existing - if (metaLocation == null || metaLocation.getRegionInfo() == null || - metaLocation.getHostname() == null) { + if (metaLocation == null ) { errors.reportError(ERROR_CODE.NULL_META_REGION, - "META region or some of its attributes are null."); + "META region location is null"); + return false; + } + if (metaLocation.getRegionInfo() == null) { + errors.reportError(ERROR_CODE.NULL_META_REGION, + "META location regionInfo is null"); + return false; + } + if (metaLocation.getHostname() == null) { + errors.reportError(ERROR_CODE.NULL_META_REGION, + "META location hostName is null"); return false; } ServerName sn = metaLocation.getServerName(); @@ -1722,6 +1707,55 @@ public class HBaseFsck extends Configured implements Closeable { } } setCheckHdfs(prevHdfsCheck); + + if (shouldCheckHdfs()) { + checkAndFixTableStates(); + } + } + + /** + * Check and fix table states, assumes full info available: + * - tableInfos + * - empty tables loaded + */ + private void checkAndFixTableStates() throws IOException { + // first check dangling states + for (Entry entry : tableStates.entrySet()) { + TableName tableName = entry.getKey(); + TableState tableState = entry.getValue(); + TableInfo tableInfo = tablesInfo.get(tableName); + if (isTableIncluded(tableName) + && !tableName.isSystemTable() + && tableInfo == null) { + if (fixMeta) { + MetaTableAccessor.deleteTableState(connection, tableName); + TableState state = MetaTableAccessor.getTableState(connection, tableName); + if (state != null) { + errors.reportError(ERROR_CODE.ORPHAN_TABLE_STATE, + tableName + " unable to delete dangling table state " + tableState); + } + } else { + errors.reportError(ERROR_CODE.ORPHAN_TABLE_STATE, + tableName + " has dangling table state " + tableState); + } + } + } + // check that all tables have states + for (TableName tableName : tablesInfo.keySet()) { + if (isTableIncluded(tableName) && !tableStates.containsKey(tableName)) { + if (fixMeta) { + MetaTableAccessor.updateTableState(connection, tableName, TableState.State.ENABLED); + TableState newState = MetaTableAccessor.getTableState(connection, tableName); + if (newState == null) { + errors.reportError(ERROR_CODE.NO_TABLE_STATE, + "Unable to change state for table " + tableName + " in meta "); + } + } else { + errors.reportError(ERROR_CODE.NO_TABLE_STATE, + tableName + " has no state in meta "); + } + } + } } private void preCheckPermission() throws IOException, AccessDeniedException { @@ -1965,8 +1999,8 @@ public class HBaseFsck extends Configured implements Closeable { hasMetaAssignment && isDeployed && !isMultiplyDeployed && hbi.metaEntry.regionServer.equals(hbi.deployedOn.get(0)); boolean splitParent = - (hbi.metaEntry == null)? false: hbi.metaEntry.isSplit() && hbi.metaEntry.isOffline(); - boolean shouldBeDeployed = inMeta && !isTableDisabled(hbi.metaEntry); + inMeta && hbi.metaEntry.isSplit() && hbi.metaEntry.isOffline(); + boolean shouldBeDeployed = inMeta && !isTableDisabled(hbi.metaEntry.getTable()); boolean recentlyModified = inHdfs && hbi.getModTime() + timelag > System.currentTimeMillis(); @@ -2749,7 +2783,7 @@ public class HBaseFsck extends Configured implements Closeable { // When table is disabled no need to check for the region chain. Some of the regions // accidently if deployed, this below code might report some issues like missing start // or end regions or region hole in chain and may try to fix which is unwanted. - if (disabledTables.contains(this.tableName)) { + if (isTableDisabled(this.tableName)) { return true; } int originalErrorsCount = errors.getErrorList().size(); @@ -3036,58 +3070,6 @@ public class HBaseFsck extends Configured implements Closeable { } } - /** - * Check whether a orphaned table ZNode exists and fix it if requested. - * @throws IOException - * @throws KeeperException - * @throws InterruptedException - */ - private void checkAndFixOrphanedTableZNodes() - throws IOException, KeeperException, InterruptedException { - ZooKeeperWatcher zkw = createZooKeeperWatcher(); - - try { - Set enablingTables = ZKTableStateClientSideReader.getEnablingTables(zkw); - String msg; - TableInfo tableInfo; - - for (TableName tableName : enablingTables) { - // Check whether the table exists in hbase - tableInfo = tablesInfo.get(tableName); - if (tableInfo != null) { - // Table exists. This table state is in transit. No problem for this table. - continue; - } - - msg = "Table " + tableName + " not found in hbase:meta. Orphaned table ZNode found."; - LOG.warn(msg); - orphanedTableZNodes.add(tableName); - errors.reportError(ERROR_CODE.ORPHANED_ZK_TABLE_ENTRY, msg); - } - - if (orphanedTableZNodes.size() > 0 && this.fixTableZNodes) { - ZKTableStateManager zkTableStateMgr = new ZKTableStateManager(zkw); - - for (TableName tableName : orphanedTableZNodes) { - try { - // Set the table state to be disabled so that if we made mistake, we can trace - // the history and figure it out. - // Another choice is to call checkAndRemoveTableState() to delete the orphaned ZNode. - // Both approaches works. - zkTableStateMgr.setTableState(tableName, ZooKeeperProtos.Table.State.DISABLED); - } catch (CoordinatedStateException e) { - // This exception should not happen here - LOG.error( - "Got a CoordinatedStateException while fixing the ENABLING table znode " + tableName, - e); - } - } - } - } finally { - zkw.close(); - } - } - /** * Check values in regionInfo for hbase:meta * Check if zero or more than one regions with hbase:meta are found. @@ -3597,12 +3579,15 @@ public class HBaseFsck extends Configured implements Closeable { public interface ErrorReporter { enum ERROR_CODE { UNKNOWN, NO_META_REGION, NULL_META_REGION, NO_VERSION_FILE, NOT_IN_META_HDFS, NOT_IN_META, - NOT_IN_META_OR_DEPLOYED, NOT_IN_HDFS_OR_DEPLOYED, NOT_IN_HDFS, SERVER_DOES_NOT_MATCH_META, NOT_DEPLOYED, + NOT_IN_META_OR_DEPLOYED, NOT_IN_HDFS_OR_DEPLOYED, NOT_IN_HDFS, SERVER_DOES_NOT_MATCH_META, + NOT_DEPLOYED, MULTI_DEPLOYED, SHOULD_NOT_BE_DEPLOYED, MULTI_META_REGION, RS_CONNECT_FAILURE, FIRST_REGION_STARTKEY_NOT_EMPTY, LAST_REGION_ENDKEY_NOT_EMPTY, DUPE_STARTKEYS, HOLE_IN_REGION_CHAIN, OVERLAP_IN_REGION_CHAIN, REGION_CYCLE, DEGENERATE_REGION, ORPHAN_HDFS_REGION, LINGERING_SPLIT_PARENT, NO_TABLEINFO_FILE, LINGERING_REFERENCE_HFILE, - WRONG_USAGE, EMPTY_META_CELL, EXPIRED_TABLE_LOCK, ORPHANED_ZK_TABLE_ENTRY, BOUNDARIES_ERROR + WRONG_USAGE, EMPTY_META_CELL, EXPIRED_TABLE_LOCK, + ORPHANED_ZK_TABLE_ENTRY, BOUNDARIES_ERROR, + ORPHAN_TABLE_STATE, NO_TABLE_STATE } void clear(); void report(String message); @@ -3759,10 +3744,10 @@ public class HBaseFsck extends Configured implements Closeable { private HBaseFsck hbck; private ServerName rsinfo; private ErrorReporter errors; - private HConnection connection; + private ClusterConnection connection; WorkItemRegion(HBaseFsck hbck, ServerName info, - ErrorReporter errors, HConnection connection) { + ErrorReporter errors, ClusterConnection connection) { this.hbck = hbck; this.rsinfo = info; this.errors = errors; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java index f773b06f503..e3676df314d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java @@ -18,8 +18,11 @@ package org.apache.hadoop.hbase.util; import java.io.IOException; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import com.google.protobuf.InvalidProtocolBufferException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -27,6 +30,9 @@ import org.apache.hadoop.conf.Configured; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.TableState; +import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; @@ -134,6 +140,79 @@ public class ZKDataMigrator extends Configured implements Tool { return 0; } + /** + * Method for table states migration. + * Used when upgrading from pre-2.0 to 2.0 + * Reading state from zk, applying them to internal state + * and delete. + * Used by master to clean migration from zk based states to + * table descriptor based states. + */ + @SuppressWarnings("deprecation") + @Deprecated + public static Map queryForTableStates(ZooKeeperWatcher zkw) + throws KeeperException, InterruptedException { + Map rv = new HashMap<>(); + List children = ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode); + if (children == null) + return rv; + for (String child: children) { + TableName tableName = TableName.valueOf(child); + ZooKeeperProtos.Table.State state = getTableState(zkw, tableName); + TableState.State newState = TableState.State.ENABLED; + if (state != null) { + switch (state) { + case ENABLED: + newState = TableState.State.ENABLED; + break; + case DISABLED: + newState = TableState.State.DISABLED; + break; + case DISABLING: + newState = TableState.State.DISABLING; + break; + case ENABLING: + newState = TableState.State.ENABLING; + break; + default: + } + } + rv.put(tableName, newState); + } + return rv; + } + + /** + * Gets table state from ZK. + * @param zkw ZooKeeperWatcher instance to use + * @param tableName table we're checking + * @return Null or {@link ZooKeeperProtos.DeprecatedTableState.State} found in znode. + * @throws KeeperException + */ + @Deprecated + private static ZooKeeperProtos.Table.State getTableState( + final ZooKeeperWatcher zkw, final TableName tableName) + throws KeeperException, InterruptedException { + String znode = ZKUtil.joinZNode(zkw.tableZNode, tableName.getNameAsString()); + byte [] data = ZKUtil.getData(zkw, znode); + if (data == null || data.length <= 0) return null; + try { + ProtobufUtil.expectPBMagicPrefix(data); + ZooKeeperProtos.Table.Builder builder = + ZooKeeperProtos.Table.newBuilder(); + int magicLen = ProtobufUtil.lengthOfPBMagic(); + ZooKeeperProtos.Table t = builder.mergeFrom(data, + magicLen, data.length - magicLen).build(); + return t.getState(); + } catch (InvalidProtocolBufferException e) { + KeeperException ke = new KeeperException.DataInconsistencyException(); + ke.initCause(e); + throw ke; + } catch (DeserializationException e) { + throw ZKUtil.convert(e); + } + } + private void checkAndMigrateTableStatesToPB(ZooKeeperWatcher zkw) throws KeeperException, InterruptedException { List tables = ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java index ff205c61927..2c03b418a1b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java @@ -25,6 +25,7 @@ import java.io.InterruptedIOException; import java.text.ParseException; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; import java.util.List; @@ -49,6 +50,10 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.protobuf.ServiceException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -60,20 +65,16 @@ import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.CoordinatedStateException; import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.TableStateManager; -import org.apache.hadoop.hbase.Tag; -import org.apache.hadoop.hbase.TagRewriteCell; -import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.ConnectionUtils; import org.apache.hadoop.hbase.client.Delete; @@ -82,6 +83,7 @@ import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager; import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination; import org.apache.hadoop.hbase.exceptions.RegionOpeningException; @@ -98,13 +100,11 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionStoreSequenceIds; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.StoreSequenceId; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.LastSequenceId; -// imports for things that haven't moved from regionserver.wal yet. import org.apache.hadoop.hbase.regionserver.wal.FSHLog; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec; @@ -123,14 +123,7 @@ import org.apache.hadoop.hbase.wal.WALProvider.Writer; import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; import org.apache.hadoop.io.MultipleIOException; -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import com.google.protobuf.ServiceException; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import com.google.protobuf.ServiceException; +// imports for things that haven't moved from regionserver.wal yet. /** * This class is responsible for splitting up a bunch of regionserver commit log @@ -155,8 +148,8 @@ public class WALSplitter { OutputSink outputSink; EntryBuffers entryBuffers; - private Set disablingOrDisabledTables = - new HashSet(); + private Map tableStatesCache = + new ConcurrentHashMap<>(); private BaseCoordinatedStateManager csm; private final WALFactory walFactory; @@ -311,15 +304,6 @@ public class WALSplitter { LOG.warn("Nothing to split in log file " + logPath); return true; } - if (csm != null) { - try { - TableStateManager tsm = csm.getTableStateManager(); - disablingOrDisabledTables = tsm.getTablesInStates( - ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING); - } catch (CoordinatedStateException e) { - throw new IOException("Can't get disabling/disabled tables", e); - } - } int numOpenedFilesBeforeReporting = conf.getInt("hbase.splitlog.report.openedfiles", 3); int numOpenedFilesLastCheck = 0; outputSink.setReporter(reporter); @@ -1635,7 +1619,7 @@ public class WALSplitter { } // check if current region in a disabling or disabled table - if (disablingOrDisabledTables.contains(buffer.tableName)) { + if (isTableDisabledOrDisabling(buffer.tableName)) { // need fall back to old way logRecoveredEditsOutputSink.append(buffer); hasEditsInDisablingOrDisabledTables = true; @@ -2119,6 +2103,26 @@ public class WALSplitter { } } + private boolean isTableDisabledOrDisabling(TableName tableName) { + if (csm == null) + return false; // we can't get state without CoordinatedStateManager + if (tableName.isSystemTable()) + return false; // assume that system tables never can be disabled + TableState tableState = tableStatesCache.get(tableName); + if (tableState == null) { + try { + tableState = + MetaTableAccessor.getTableState(csm.getServer().getConnection(), tableName); + if (tableState != null) + tableStatesCache.put(tableName, tableState); + } catch (IOException e) { + LOG.warn("State is not accessible for table " + tableName, e); + } + } + return tableState != null && tableState + .inStates(TableState.State.DISABLED, TableState.State.DISABLING); + } + /** * Private data structure that wraps a receiving RS and collecting statistics about the data * written to this newly assigned RS. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableStateManager.java deleted file mode 100644 index 7c8229e3e00..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableStateManager.java +++ /dev/null @@ -1,346 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.zookeeper; - -import com.google.protobuf.InvalidProtocolBufferException; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.CoordinatedStateException; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.TableStateManager; -import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; -import org.apache.zookeeper.KeeperException; - -import java.io.InterruptedIOException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -/** - * Implementation of TableStateManager which reads, caches and sets state - * up in ZooKeeper. If multiple read/write clients, will make for confusion. - * Code running on client side without consensus context should use - * {@link ZKTableStateClientSideReader} instead. - * - *

To save on trips to the zookeeper ensemble, internally we cache table - * state. - */ -@InterfaceAudience.Private -public class ZKTableStateManager implements TableStateManager { - // A znode will exist under the table directory if it is in any of the - // following states: {@link TableState#ENABLING} , {@link TableState#DISABLING}, - // or {@link TableState#DISABLED}. If {@link TableState#ENABLED}, there will - // be no entry for a table in zk. Thats how it currently works. - - private static final Log LOG = LogFactory.getLog(ZKTableStateManager.class); - private final ZooKeeperWatcher watcher; - - /** - * Cache of what we found in zookeeper so we don't have to go to zk ensemble - * for every query. Synchronize access rather than use concurrent Map because - * synchronization needs to span query of zk. - */ - private final Map cache = - new HashMap(); - - public ZKTableStateManager(final ZooKeeperWatcher zkw) throws KeeperException, - InterruptedException { - super(); - this.watcher = zkw; - populateTableStates(); - } - - /** - * Gets a list of all the tables set as disabled in zookeeper. - * @throws KeeperException, InterruptedException - */ - private void populateTableStates() throws KeeperException, InterruptedException { - synchronized (this.cache) { - List children = ZKUtil.listChildrenNoWatch(this.watcher, this.watcher.tableZNode); - if (children == null) return; - for (String child: children) { - TableName tableName = TableName.valueOf(child); - ZooKeeperProtos.Table.State state = getTableState(this.watcher, tableName); - if (state != null) this.cache.put(tableName, state); - } - } - } - - /** - * Sets table state in ZK. Sets no watches. - * - * {@inheritDoc} - */ - @Override - public void setTableState(TableName tableName, ZooKeeperProtos.Table.State state) - throws CoordinatedStateException { - synchronized (this.cache) { - LOG.warn("Moving table " + tableName + " state from " + this.cache.get(tableName) - + " to " + state); - try { - setTableStateInZK(tableName, state); - } catch (KeeperException e) { - throw new CoordinatedStateException(e); - } - } - } - - /** - * Checks and sets table state in ZK. Sets no watches. - * {@inheritDoc} - */ - @Override - public boolean setTableStateIfInStates(TableName tableName, - ZooKeeperProtos.Table.State newState, - ZooKeeperProtos.Table.State... states) - throws CoordinatedStateException { - synchronized (this.cache) { - // Transition ENABLED->DISABLING has to be performed with a hack, because - // we treat empty state as enabled in this case because 0.92- clusters. - if ( - (newState == ZooKeeperProtos.Table.State.DISABLING) && - this.cache.get(tableName) != null && !isTableState(tableName, states) || - (newState != ZooKeeperProtos.Table.State.DISABLING && - !isTableState(tableName, states) )) { - return false; - } - try { - setTableStateInZK(tableName, newState); - } catch (KeeperException e) { - throw new CoordinatedStateException(e); - } - return true; - } - } - - /** - * Checks and sets table state in ZK. Sets no watches. - * {@inheritDoc} - */ - @Override - public boolean setTableStateIfNotInStates(TableName tableName, - ZooKeeperProtos.Table.State newState, - ZooKeeperProtos.Table.State... states) - throws CoordinatedStateException { - synchronized (this.cache) { - if (isTableState(tableName, states)) { - // If the table is in the one of the states from the states list, the cache - // might be out-of-date, try to find it out from the master source (zookeeper server). - // - // Note: this adds extra zookeeper server calls and might have performance impact. - // However, this is not the happy path so we should not reach here often. Therefore, - // the performance impact should be minimal to none. - try { - ZooKeeperProtos.Table.State curstate = getTableState(watcher, tableName); - - if (isTableInState(Arrays.asList(states), curstate)) { - return false; - } - } catch (KeeperException e) { - throw new CoordinatedStateException(e); - } catch (InterruptedException e) { - throw new CoordinatedStateException(e); - } - } - try { - setTableStateInZK(tableName, newState); - } catch (KeeperException e) { - throw new CoordinatedStateException(e); - } - return true; - } - } - - private void setTableStateInZK(final TableName tableName, - final ZooKeeperProtos.Table.State state) - throws KeeperException { - String znode = ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString()); - if (ZKUtil.checkExists(this.watcher, znode) == -1) { - ZKUtil.createAndFailSilent(this.watcher, znode); - } - synchronized (this.cache) { - ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder(); - builder.setState(state); - byte [] data = ProtobufUtil.prependPBMagic(builder.build().toByteArray()); - ZKUtil.setData(this.watcher, znode, data); - this.cache.put(tableName, state); - } - } - - /** - * Checks if table is marked in specified state in ZK. - * - * {@inheritDoc} - */ - @Override - public boolean isTableState(final TableName tableName, - final ZooKeeperProtos.Table.State... states) { - synchronized (this.cache) { - ZooKeeperProtos.Table.State currentState = this.cache.get(tableName); - return isTableInState(Arrays.asList(states), currentState); - } - } - - /** - * Deletes the table in zookeeper. Fails silently if the - * table is not currently disabled in zookeeper. Sets no watches. - * - * {@inheritDoc} - */ - @Override - public void setDeletedTable(final TableName tableName) - throws CoordinatedStateException { - synchronized (this.cache) { - if (this.cache.remove(tableName) == null) { - LOG.warn("Moving table " + tableName + " state to deleted but was " + - "already deleted"); - } - try { - ZKUtil.deleteNodeFailSilent(this.watcher, - ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString())); - } catch (KeeperException e) { - throw new CoordinatedStateException(e); - } - } - } - - /** - * check if table is present. - * - * @param tableName table we're working on - * @return true if the table is present - */ - @Override - public boolean isTablePresent(final TableName tableName) { - synchronized (this.cache) { - ZooKeeperProtos.Table.State state = this.cache.get(tableName); - return !(state == null); - } - } - - /** - * Gets a list of all the tables set as disabling in zookeeper. - * @return Set of disabling tables, empty Set if none - * @throws CoordinatedStateException if error happened in underlying coordination engine - */ - @Override - public Set getTablesInStates(ZooKeeperProtos.Table.State... states) - throws InterruptedIOException, CoordinatedStateException { - try { - return getAllTables(states); - } catch (KeeperException e) { - throw new CoordinatedStateException(e); - } - } - - /** - * {@inheritDoc} - */ - @Override - public void checkAndRemoveTableState(TableName tableName, ZooKeeperProtos.Table.State states, - boolean deletePermanentState) - throws CoordinatedStateException { - synchronized (this.cache) { - if (isTableState(tableName, states)) { - this.cache.remove(tableName); - if (deletePermanentState) { - try { - ZKUtil.deleteNodeFailSilent(this.watcher, - ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString())); - } catch (KeeperException e) { - throw new CoordinatedStateException(e); - } - } - } - } - } - - /** - * Gets a list of all the tables of specified states in zookeeper. - * @return Set of tables of specified states, empty Set if none - * @throws KeeperException - */ - Set getAllTables(final ZooKeeperProtos.Table.State... states) - throws KeeperException, InterruptedIOException { - - Set allTables = new HashSet(); - List children = - ZKUtil.listChildrenNoWatch(watcher, watcher.tableZNode); - if(children == null) return allTables; - for (String child: children) { - TableName tableName = TableName.valueOf(child); - ZooKeeperProtos.Table.State state; - try { - state = getTableState(watcher, tableName); - } catch (InterruptedException e) { - throw new InterruptedIOException(); - } - for (ZooKeeperProtos.Table.State expectedState: states) { - if (state == expectedState) { - allTables.add(tableName); - break; - } - } - } - return allTables; - } - - /** - * Gets table state from ZK. - * @param zkw ZooKeeperWatcher instance to use - * @param tableName table we're checking - * @return Null or {@link ZooKeeperProtos.Table.State} found in znode. - * @throws KeeperException - */ - private ZooKeeperProtos.Table.State getTableState(final ZooKeeperWatcher zkw, - final TableName tableName) - throws KeeperException, InterruptedException { - String znode = ZKUtil.joinZNode(zkw.tableZNode, tableName.getNameAsString()); - byte [] data = ZKUtil.getData(zkw, znode); - if (data == null || data.length <= 0) return null; - try { - ProtobufUtil.expectPBMagicPrefix(data); - ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder(); - int magicLen = ProtobufUtil.lengthOfPBMagic(); - ZooKeeperProtos.Table t = builder.mergeFrom(data, magicLen, data.length - magicLen).build(); - return t.getState(); - } catch (InvalidProtocolBufferException e) { - KeeperException ke = new KeeperException.DataInconsistencyException(); - ke.initCause(e); - throw ke; - } catch (DeserializationException e) { - throw ZKUtil.convert(e); - } - } - - /** - * @return true if current state isn't null and is contained - * in the list of expected states. - */ - private boolean isTableInState(final List expectedStates, - final ZooKeeperProtos.Table.State currentState) { - return currentState != null && expectedStates.contains(currentState); - } -} 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 1eef9671863..a21d2bbecac 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 @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.io.OutputStream; @@ -40,6 +41,7 @@ import java.util.Set; import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -67,6 +69,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; @@ -2119,8 +2122,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { assertEquals(failMsg, result.getColumnCells(f, null).size(), 1); Cell cell = result.getColumnLatestCell(f, null); assertTrue(failMsg, - Bytes.equals(data, 0, data.length, cell.getValueArray(), cell.getValueOffset(), - cell.getValueLength())); + Bytes.equals(data, 0, data.length, cell.getValueArray(), cell.getValueOffset(), + cell.getValueLength())); } } @@ -2232,15 +2235,16 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { Table meta = new HTable(conf, TableName.META_TABLE_NAME); Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR); List newRegions = new ArrayList(startKeys.length); + TableName tableName = htd.getTableName(); // add custom ones for (int i = 0; i < startKeys.length; i++) { int j = (i + 1) % startKeys.length; - HRegionInfo hri = new HRegionInfo(htd.getTableName(), startKeys[i], + HRegionInfo hri = new HRegionInfo(tableName, startKeys[i], startKeys[j]); MetaTableAccessor.addRegionToMeta(meta, hri); newRegions.add(hri); } - + MetaTableAccessor.updateTableState(getConnection(), tableName, TableState.State.ENABLED); meta.close(); return newRegions; } @@ -2817,7 +2821,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { } public String explainTableAvailability(TableName tableName) throws IOException { - String msg = explainTableState(tableName) + ","; + String msg = explainTableState(tableName, TableState.State.ENABLED) + ", "; if (getHBaseCluster().getMaster().isAlive()) { Map assignments = getHBaseCluster().getMaster().getAssignmentManager().getRegionStates() @@ -2845,19 +2849,38 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { return msg; } - public String explainTableState(TableName tableName) throws IOException { - try { - if (getHBaseAdmin().isTableEnabled(tableName)) - return "table enabled in zk"; - else if (getHBaseAdmin().isTableDisabled(tableName)) - return "table disabled in zk"; - else - return "table in uknown state"; - } catch (TableNotFoundException e) { - return "table not exists"; + public String explainTableState(final TableName table, TableState.State state) + throws IOException { + TableState tableState = MetaTableAccessor.getTableState(connection, table); + if (tableState == null) { + return "TableState in META: No table state in META for table " + table + + " last state in meta (including deleted is " + findLastTableState(table) + ")"; + } else if (!tableState.inStates(state)) { + return "TableState in META: Not " + state + " state, but " + tableState; + } else { + return "TableState in META: OK"; } } + @Nullable + public TableState findLastTableState(final TableName table) throws IOException { + final AtomicReference lastTableState = new AtomicReference<>(null); + MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() { + @Override + public boolean visit(Result r) throws IOException { + if (!Arrays.equals(r.getRow(), table.getName())) + return false; + TableState state = MetaTableAccessor.getTableState(r); + if (state != null) + lastTableState.set(state); + return true; + } + }; + MetaTableAccessor + .fullScan(connection, visitor, table.getName(), MetaTableAccessor.QueryType.TABLE, true); + return lastTableState.get(); + } + /** * Wait until all regions in a table have been assigned * @param table Table to wait on. @@ -2875,8 +2898,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { long startWait = System.currentTimeMillis(); while (!admin.isTableAvailable(TableName.valueOf(table))) { assertTrue("Timed out waiting for table to become available " + - Bytes.toStringBinary(table), - System.currentTimeMillis() - startWait < timeoutMillis); + Bytes.toStringBinary(table), + System.currentTimeMillis() - startWait < timeoutMillis); Thread.sleep(200); } } @@ -3132,6 +3155,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { HRegionInfo info = HRegionInfo.parseFromOrNull(b); if (info != null && info.getTable().equals(tableName)) { b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER); + if (b == null) { + LOG.debug(info.getEncodedName() + " is not assigned yet"); + } allRegionsAssigned &= (b != null); } } @@ -3160,6 +3186,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { @Override public boolean evaluate() throws IOException { List hris = states.getRegionsOfTable(tableName); + LOG.debug("Regions are " + hris); return hris != null && !hris.isEmpty(); } }); @@ -3602,7 +3629,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { Collection hrs = rs.getOnlineRegionsLocalContext(); for (HRegion r: hrs) { assertTrue("Region should not be double assigned", - r.getRegionId() != hri.getRegionId()); + r.getRegionId() != hri.getRegionId()); } } return; // good, we are happy @@ -3612,7 +3639,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { Thread.sleep(10); } fail("Could not find region " + hri.getRegionNameAsString() - + " on server " + server); + + " on server " + server); } public HRegion createTestRegion(String tableName, HColumnDescriptor hcd) @@ -3682,7 +3709,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { return new ExplainingPredicate() { @Override public String explainFailure() throws IOException { - return explainTableState(tableName); + return explainTableState(tableName, TableState.State.ENABLED); } @Override @@ -3699,7 +3726,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { return new ExplainingPredicate() { @Override public String explainFailure() throws IOException { - return explainTableState(tableName); + return explainTableState(tableName, TableState.State.DISABLED); } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java index 66909901c43..cdce2509bc0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hbase; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager; import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.ExecutorService; @@ -28,9 +29,11 @@ import org.apache.hadoop.hbase.executor.ExecutorType; import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.LoadBalancer; +import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.RegionPlan; import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.ServerManager; +import org.apache.hadoop.hbase.master.TableStateManager; import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory; import org.apache.hadoop.hbase.regionserver.RegionOpeningState; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -88,8 +91,9 @@ public class TestDrainingServer { public void testAssignmentManagerDoesntUseDrainingServer() throws Exception { AssignmentManager am; Configuration conf = TEST_UTIL.getConfiguration(); - final HMaster master = Mockito.mock(HMaster.class); - final Server server = Mockito.mock(Server.class); + final MasterServices server = Mockito.mock(MasterServices.class); + final TableStateManager stateManager = new TableStateManager.InMemoryTableStateManager(); + Mockito.when(server.getTableStateManager()).thenReturn(stateManager); final ServerManager serverManager = Mockito.mock(ServerManager.class); final ServerName SERVERNAME_A = ServerName.valueOf("mockserver_a.org", 1000, 8000); final ServerName SERVERNAME_B = ServerName.valueOf("mockserver_b.org", 1001, 8000); @@ -135,13 +139,13 @@ public class TestDrainingServer { Mockito.when(serverManager.addServerToDrainList(sn)).thenReturn(true); } - Mockito.when(master.getServerManager()).thenReturn(serverManager); + Mockito.when(server.getServerManager()).thenReturn(serverManager); am = new AssignmentManager(server, serverManager, - balancer, startupMasterExecutor("mockExecutorService"), null, null); + balancer, startupMasterExecutor("mockExecutorService"), null, null, stateManager); - Mockito.when(master.getAssignmentManager()).thenReturn(am); - Mockito.when(master.getZooKeeper()).thenReturn(zkWatcher); + Mockito.when(server.getAssignmentManager()).thenReturn(am); + Mockito.when(server.getZooKeeper()).thenReturn(zkWatcher); am.addPlan(REGIONINFO.getEncodedName(), new RegionPlan(REGIONINFO, null, SERVERNAME_A)); @@ -165,9 +169,10 @@ public class TestDrainingServer { Configuration conf = TEST_UTIL.getConfiguration(); LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(conf); AssignmentManager am; - final HMaster master = Mockito.mock(HMaster.class); - final Server server = Mockito.mock(Server.class); + final MasterServices server = Mockito.mock(MasterServices.class); final ServerManager serverManager = Mockito.mock(ServerManager.class); + final TableStateManager stateManager = new TableStateManager.InMemoryTableStateManager(); + Mockito.when(server.getTableStateManager()).thenReturn(stateManager); final ServerName SERVERNAME_A = ServerName.valueOf("mockserverbulk_a.org", 1000, 8000); final ServerName SERVERNAME_B = ServerName.valueOf("mockserverbulk_b.org", 1001, 8000); final ServerName SERVERNAME_C = ServerName.valueOf("mockserverbulk_c.org", 1002, 8000); @@ -234,7 +239,7 @@ public class TestDrainingServer { Mockito.when(serverManager.addServerToDrainList(entry.getValue())).thenReturn(true); } - Mockito.when(master.getServerManager()).thenReturn(serverManager); + Mockito.when(server.getServerManager()).thenReturn(serverManager); drainedServers.add(SERVERNAME_A); drainedServers.add(SERVERNAME_B); @@ -242,10 +247,8 @@ public class TestDrainingServer { drainedServers.add(SERVERNAME_D); am = new AssignmentManager(server, serverManager, - balancer, startupMasterExecutor("mockExecutorServiceBulk"), null, null); + balancer, startupMasterExecutor("mockExecutorServiceBulk"), null, null, stateManager); - Mockito.when(master.getAssignmentManager()).thenReturn(am); - zkWatcher.registerListener(am); for (ServerName drained : drainedServers) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaMigrationConvertingToPB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaMigrationConvertingToPB.java index 3845bcd2f8e..f1c74026f5f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaMigrationConvertingToPB.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaMigrationConvertingToPB.java @@ -270,7 +270,7 @@ public class TestMetaMigrationConvertingToPB { */ void verifyMetaRowsAreUpdated(HConnection hConnection) throws IOException { - List results = MetaTableAccessor.fullScan(hConnection); + List results = MetaTableAccessor.fullScanRegions(hConnection); assertTrue(results.size() >= REGION_COUNT); for (Result result : results) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java index bf7a3cd29df..bf177597956 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java @@ -226,7 +226,12 @@ public class TestMetaTableAccessor { admin.deleteTable(name); assertFalse(MetaTableAccessor.tableExists(connection, name)); assertTrue(MetaTableAccessor.tableExists(connection, - TableName.META_TABLE_NAME)); + TableName.META_TABLE_NAME)); + UTIL.createTable(name, HConstants.CATALOG_FAMILY); + assertTrue(MetaTableAccessor.tableExists(connection, name)); + admin.disableTable(name); + admin.deleteTable(name); + assertFalse(MetaTableAccessor.tableExists(connection, name)); } @Test public void testGetRegion() throws IOException, InterruptedException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java index 400d92942a8..b679945f8ca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -50,9 +51,7 @@ import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.executor.EventHandler; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.exceptions.MergeRegionException; import org.apache.hadoop.hbase.master.HMaster; @@ -251,7 +250,7 @@ public class TestAdmin1 { this.admin.disableTable(ht.getName()); assertTrue("Table must be disabled.", TEST_UTIL.getHBaseCluster() .getMaster().getAssignmentManager().getTableStateManager().isTableState( - ht.getName(), ZooKeeperProtos.Table.State.DISABLED)); + ht.getName(), TableState.State.DISABLED)); // Test that table is disabled get = new Get(row); @@ -278,7 +277,7 @@ public class TestAdmin1 { this.admin.enableTable(table); assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster() .getMaster().getAssignmentManager().getTableStateManager().isTableState( - ht.getName(), ZooKeeperProtos.Table.State.ENABLED)); + ht.getName(), TableState.State.ENABLED)); // Test that table is enabled try { @@ -290,6 +289,13 @@ public class TestAdmin1 { ht.close(); } + private TableState.State getStateFromMeta(TableName table) throws IOException { + TableState state = + MetaTableAccessor.getTableState(TEST_UTIL.getConnection(), table); + assertNotNull(state); + return state.getState(); + } + @Test (timeout=300000) public void testDisableAndEnableTables() throws IOException { final byte [] row = Bytes.toBytes("row"); @@ -321,6 +327,10 @@ public class TestAdmin1 { ok = true; } + assertEquals(TableState.State.DISABLED, getStateFromMeta(table1)); + assertEquals(TableState.State.DISABLED, getStateFromMeta(table2)); + + assertTrue(ok); this.admin.enableTables("testDisableAndEnableTable.*"); @@ -350,7 +360,7 @@ public class TestAdmin1 { assertEquals(numTables + 1, tables.length); assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster() .getMaster().getAssignmentManager().getTableStateManager().isTableState( - TableName.valueOf("testCreateTable"), ZooKeeperProtos.Table.State.ENABLED)); + TableName.valueOf("testCreateTable"), TableState.State.ENABLED)); } @Test (timeout=300000) @@ -1299,10 +1309,7 @@ public class TestAdmin1 { ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL); TableName tableName = TableName.valueOf("testMasterAdmin"); TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close(); - while (!ZKTableStateClientSideReader.isEnabledTable(zkw, - TableName.valueOf("testMasterAdmin"))) { - Thread.sleep(10); - } + TEST_UTIL.waitTableAvailable(TableName.valueOf("testMasterAdmin")); this.admin.disableTable(tableName); try { new HTable(TEST_UTIL.getConfiguration(), tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java index f615df216ee..dcd4a5e2b0f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java @@ -18,10 +18,8 @@ */ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors; -import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck; -import static org.junit.Assert.*; - +import javax.annotation.Nullable; +import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -39,21 +37,28 @@ import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.HBaseFsck; -import org.apache.hadoop.hbase.util.HBaseFsckRepair; import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE; +import org.apache.hadoop.hbase.util.HBaseFsckRepair; import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil; import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.apache.zookeeper.KeeperException; import org.junit.After; import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; +import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors; +import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + /** * Tests the scenarios where replicas are enabled for the meta table */ @@ -225,7 +230,8 @@ public class TestMetaWithReplicas { stopMasterAndValidateReplicaCount(2, 3); } - private void stopMasterAndValidateReplicaCount(int originalReplicaCount, int newReplicaCount) + private void stopMasterAndValidateReplicaCount(final int originalReplicaCount, + final int newReplicaCount) throws Exception { ServerName sn = TEST_UTIL.getHBaseClusterInterface().getClusterStatus().getMaster(); TEST_UTIL.getHBaseClusterInterface().stopMaster(sn); @@ -236,16 +242,7 @@ public class TestMetaWithReplicas { newReplicaCount); TEST_UTIL.getHBaseClusterInterface().startMaster(sn.getHostname(), 0); TEST_UTIL.getHBaseClusterInterface().waitForActiveAndReadyMaster(); - int count = 0; - do { - metaZnodes = TEST_UTIL.getZooKeeperWatcher().getMetaReplicaNodes(); - Thread.sleep(10); - count++; - // wait for the count to be different from the originalReplicaCount. When the - // replica count is reduced, that will happen when the master unassigns excess - // replica, and deletes the excess znodes - } while (metaZnodes.size() == originalReplicaCount && count < 1000); - assert(metaZnodes.size() == newReplicaCount); + TEST_UTIL.waitFor(10000, predicateMetaHasReplicas(newReplicaCount)); // also check if hbck returns without errors TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, newReplicaCount); @@ -253,6 +250,46 @@ public class TestMetaWithReplicas { HbckTestingUtil.assertNoErrors(hbck); } + private Waiter.ExplainingPredicate predicateMetaHasReplicas( + final int newReplicaCount) { + return new Waiter.ExplainingPredicate() { + @Override + public String explainFailure() throws Exception { + return checkMetaLocationAndExplain(newReplicaCount); + } + + @Override + public boolean evaluate() throws Exception { + return checkMetaLocationAndExplain(newReplicaCount) == null; + } + }; + } + + @Nullable + private String checkMetaLocationAndExplain(int originalReplicaCount) + throws KeeperException, IOException { + List metaZnodes = TEST_UTIL.getZooKeeperWatcher().getMetaReplicaNodes(); + if (metaZnodes.size() == originalReplicaCount) { + RegionLocations rl = ((ClusterConnection) TEST_UTIL.getConnection()) + .locateRegion(TableName.META_TABLE_NAME, + HConstants.EMPTY_START_ROW, false, false); + for (HRegionLocation location : rl.getRegionLocations()) { + if (location == null) { + return "Null location found in " + rl.toString(); + } + if (location.getRegionInfo() == null) { + return "Null regionInfo for location " + location; + } + if (location.getHostname() == null) { + return "Null hostName for location " + location; + } + } + return null; // OK + } + return "Replica count is not as expected " + originalReplicaCount + " <> " + metaZnodes.size() + + "(" + metaZnodes.toString() + ")"; + } + @Test public void testHBaseFsckWithMetaReplicas() throws Exception { HBaseFsck hbck = HbckTestingUtil.doFsck(TEST_UTIL.getConfiguration(), false); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java index 03b3ddaffb7..972b00b7426 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java @@ -129,7 +129,8 @@ public class TestReplicaWithCluster { @AfterClass public static void afterClass() throws Exception { - HTU2.shutdownMiniCluster(); + if (HTU2 != null) + HTU2.shutdownMiniCluster(); HTU.shutdownMiniCluster(); } @@ -220,7 +221,6 @@ public class TestReplicaWithCluster { admin.disableTable(hdt.getTableName()); admin.deleteTable(hdt.getTableName()); - HTU.getHBaseCluster().startMaster(); admin.close(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java index 9e8097edc1e..d6118515f13 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java @@ -17,12 +17,6 @@ */ package org.apache.hadoop.hbase.master; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotSame; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -30,6 +24,8 @@ import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellScannable; import org.apache.hadoop.hbase.CellUtil; @@ -41,17 +37,19 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MetaMockingUtil; import org.apache.hadoop.hbase.RegionException; import org.apache.hadoop.hbase.RegionTransition; -import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.HConnectionTestingUtility; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager; import org.apache.hadoop.hbase.coordination.OpenRegionCoordination; import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager; @@ -74,7 +72,6 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table; import org.apache.hadoop.hbase.regionserver.RegionOpeningState; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -99,8 +96,11 @@ import org.mockito.internal.util.reflection.Whitebox; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** @@ -120,7 +120,7 @@ public class TestAssignmentManager { private static boolean enabling = false; // Mocked objects or; get redone for each test. - private Server server; + private MasterServices server; private ServerManager serverManager; private ZooKeeperWatcher watcher; private CoordinatedStateManager cp; @@ -128,6 +128,7 @@ public class TestAssignmentManager { private LoadBalancer balancer; private HMaster master; private ClusterConnection connection; + private TableStateManager tableStateManager; @BeforeClass public static void beforeClass() throws Exception { @@ -148,9 +149,11 @@ public class TestAssignmentManager { // Mock a Server. Have it return a legit Configuration and ZooKeeperWatcher. // If abort is called, be sure to fail the test (don't just swallow it // silently as is mockito default). - this.server = Mockito.mock(Server.class); + this.tableStateManager = new TableStateManager.InMemoryTableStateManager(); + this.server = Mockito.mock(MasterServices.class); Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("master,1,1")); Mockito.when(server.getConfiguration()).thenReturn(HTU.getConfiguration()); + Mockito.when(server.getTableStateManager()).thenReturn(tableStateManager); this.watcher = new ZooKeeperWatcher(HTU.getConfiguration(), "mockedServer", this.server, true); Mockito.when(server.getZooKeeper()).thenReturn(this.watcher); @@ -229,7 +232,8 @@ public class TestAssignmentManager { throws IOException, KeeperException, InterruptedException, ServiceException, DeserializationException, CoordinatedStateException { AssignmentManagerWithExtrasForTesting am = - setUpMockedAssignmentManager(this.server, this.serverManager); + setUpMockedAssignmentManager(this.server, this.serverManager, + tableStateManager); try { createRegionPlanAndBalance(am, SERVERNAME_A, SERVERNAME_B, REGIONINFO); startFakeFailedOverMasterAssignmentManager(am, this.watcher); @@ -278,7 +282,8 @@ public class TestAssignmentManager { throws IOException, KeeperException, InterruptedException, ServiceException, DeserializationException, CoordinatedStateException { AssignmentManagerWithExtrasForTesting am = - setUpMockedAssignmentManager(this.server, this.serverManager); + setUpMockedAssignmentManager(this.server, this.serverManager, + tableStateManager); try { createRegionPlanAndBalance(am, SERVERNAME_A, SERVERNAME_B, REGIONINFO); startFakeFailedOverMasterAssignmentManager(am, this.watcher); @@ -328,7 +333,8 @@ public class TestAssignmentManager { throws IOException, KeeperException, InterruptedException, ServiceException, DeserializationException, CoordinatedStateException { AssignmentManagerWithExtrasForTesting am = - setUpMockedAssignmentManager(this.server, this.serverManager); + setUpMockedAssignmentManager(this.server, this.serverManager, + tableStateManager); try { createRegionPlanAndBalance(am, SERVERNAME_A, SERVERNAME_B, REGIONINFO); startFakeFailedOverMasterAssignmentManager(am, this.watcher); @@ -402,7 +408,8 @@ public class TestAssignmentManager { .getConfiguration()); // Create an AM. AssignmentManager am = new AssignmentManager(this.server, - this.serverManager, balancer, executor, null, master.getTableLockManager()); + this.serverManager, balancer, executor, null, master.getTableLockManager(), + tableStateManager); am.failoverCleanupDone.set(true); try { // Make sure our new AM gets callbacks; once registered, can't unregister. @@ -473,7 +480,8 @@ public class TestAssignmentManager { // Create an AM. AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager( - this.server, this.serverManager); + this.server, this.serverManager, + tableStateManager); try { processServerShutdownHandler(am, false); } finally { @@ -496,8 +504,8 @@ public class TestAssignmentManager { @Test (timeout=180000) public void testSSHWhenDisableTableInProgress() throws KeeperException, IOException, CoordinatedStateException, ServiceException { - testCaseWithPartiallyDisabledState(Table.State.DISABLING); - testCaseWithPartiallyDisabledState(Table.State.DISABLED); + testCaseWithPartiallyDisabledState(TableState.State.DISABLING); + testCaseWithPartiallyDisabledState(TableState.State.DISABLED); } @@ -527,14 +535,15 @@ public class TestAssignmentManager { // Create an AM. AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager( - this.server, this.serverManager); + this.server, this.serverManager, + tableStateManager); // adding region to regions and servers maps. am.regionOnline(REGIONINFO, SERVERNAME_A); // adding region in pending close. am.getRegionStates().updateRegionState( REGIONINFO, State.SPLITTING, SERVERNAME_A); am.getTableStateManager().setTableState(REGIONINFO.getTable(), - Table.State.ENABLED); + TableState.State.ENABLED); RegionTransition data = RegionTransition.createRegionTransition(EventType.RS_ZK_REGION_SPLITTING, REGIONINFO.getRegionName(), SERVERNAME_A); String node = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName()); @@ -565,7 +574,7 @@ public class TestAssignmentManager { } } - private void testCaseWithPartiallyDisabledState(Table.State state) throws KeeperException, + private void testCaseWithPartiallyDisabledState(TableState.State state) throws KeeperException, IOException, CoordinatedStateException, ServiceException { // Create and startup an executor. This is used by AssignmentManager // handling zk callbacks. @@ -575,17 +584,18 @@ public class TestAssignmentManager { // Create an AM. AssignmentManager am = new AssignmentManager(this.server, - this.serverManager, balancer, executor, null, master.getTableLockManager()); + this.serverManager, balancer, executor, null, master.getTableLockManager(), + tableStateManager); // adding region to regions and servers maps. am.regionOnline(REGIONINFO, SERVERNAME_A); // adding region in pending close. am.getRegionStates().updateRegionState(REGIONINFO, State.PENDING_CLOSE); - if (state == Table.State.DISABLING) { + if (state == TableState.State.DISABLING) { am.getTableStateManager().setTableState(REGIONINFO.getTable(), - Table.State.DISABLING); + TableState.State.DISABLING); } else { am.getTableStateManager().setTableState(REGIONINFO.getTable(), - Table.State.DISABLED); + TableState.State.DISABLED); } RegionTransition data = RegionTransition.createRegionTransition(EventType.M_ZK_REGION_CLOSING, REGIONINFO.getRegionName(), SERVERNAME_A); @@ -604,7 +614,7 @@ public class TestAssignmentManager { // check whether in rit or not. In the DISABLING case also the below // assert will be true but the piece of code added for HBASE-5927 will not // do that. - if (state == Table.State.DISABLED) { + if (state == TableState.State.DISABLED) { assertFalse("Region state of region in pending close should be removed from rit.", am.getRegionStates().isRegionsInTransition()); } @@ -658,7 +668,7 @@ public class TestAssignmentManager { // Get a connection w/ mocked up common methods. ClusterConnection connection = HConnectionTestingUtility.getMockedConnectionAndDecorate(HTU.getConfiguration(), - null, implementation, SERVERNAME_B, REGIONINFO); + null, implementation, SERVERNAME_B, REGIONINFO); // These mocks were done up when all connections were managed. World is different now we // moved to unmanaged connections. It messes up the intercepts done in these tests. // Just mark connections as marked and then down in MetaTableAccessor, it will go the path @@ -726,7 +736,8 @@ public class TestAssignmentManager { .getConfiguration()); // Create an AM. AssignmentManager am = new AssignmentManager(this.server, - this.serverManager, balancer, null, null, master.getTableLockManager()); + this.serverManager, balancer, null, null, master.getTableLockManager(), + tableStateManager); try { // First make sure my mock up basically works. Unassign a region. unassign(am, SERVERNAME_A, hri); @@ -763,7 +774,8 @@ public class TestAssignmentManager { final RecoverableZooKeeper recoverableZk = Mockito .mock(RecoverableZooKeeper.class); AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager( - this.server, this.serverManager); + this.server, this.serverManager, + tableStateManager); Watcher zkw = new ZooKeeperWatcher(HBaseConfiguration.create(), "unittest", null) { @Override @@ -797,7 +809,8 @@ public class TestAssignmentManager { HConstants.HBASE_MASTER_LOADBALANCER_CLASS, MockedLoadBalancer.class, LoadBalancer.class); AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager( - this.server, this.serverManager); + this.server, this.serverManager, + tableStateManager); try { // Boolean variable used for waiting until randomAssignment is called and // new @@ -892,7 +905,8 @@ public class TestAssignmentManager { public void testRegionInOpeningStateOnDeadRSWhileMasterFailover() throws IOException, KeeperException, ServiceException, CoordinatedStateException, InterruptedException { AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager( - this.server, this.serverManager); + this.server, this.serverManager, + tableStateManager); ZKAssign.createNodeOffline(this.watcher, REGIONINFO, SERVERNAME_A); int version = ZKAssign.getVersion(this.watcher, REGIONINFO); ZKAssign.transitionNode(this.watcher, REGIONINFO, SERVERNAME_A, EventType.M_ZK_REGION_OFFLINE, @@ -916,7 +930,7 @@ public class TestAssignmentManager { zkOrd.setVersion(version); assertFalse(am.processRegionsInTransition(rt, REGIONINFO, orc, zkOrd)); - am.getTableStateManager().setTableState(REGIONINFO.getTable(), Table.State.ENABLED); + am.getTableStateManager().setTableState(REGIONINFO.getTable(), TableState.State.ENABLED); processServerShutdownHandler(am, false); // Waiting for the assignment to get completed. while (!am.gate.get()) { @@ -950,9 +964,12 @@ public class TestAssignmentManager { CoordinatedStateManager csm = CoordinatedStateManagerFactory.getCoordinatedStateManager( HTU.getConfiguration()); - Server server = new HMaster(HTU.getConfiguration(), csm); + HMaster server = new HMaster(HTU.getConfiguration(), csm); + Whitebox.setInternalState(server, "tableStateManager", tableStateManager); + AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server, - this.serverManager); + this.serverManager, + tableStateManager); Whitebox.setInternalState(server, "metaTableLocator", Mockito.mock(MetaTableLocator.class)); @@ -967,7 +984,7 @@ public class TestAssignmentManager { try{ // set table in disabling state. am.getTableStateManager().setTableState(REGIONINFO.getTable(), - Table.State.DISABLING); + TableState.State.DISABLING); am.joinCluster(); // should not call retainAssignment if we get empty regions in assignAllUserRegions. assertFalse( @@ -976,13 +993,13 @@ public class TestAssignmentManager { // need to change table state from disabling to disabled. assertTrue("Table should be disabled.", am.getTableStateManager().isTableState(REGIONINFO.getTable(), - Table.State.DISABLED)); + TableState.State.DISABLED)); } finally { this.server.getConfiguration().setClass( HConstants.HBASE_MASTER_LOADBALANCER_CLASS, SimpleLoadBalancer.class, LoadBalancer.class); am.getTableStateManager().setTableState(REGIONINFO.getTable(), - Table.State.ENABLED); + TableState.State.ENABLED); am.shutdown(); } } @@ -1004,10 +1021,16 @@ public class TestAssignmentManager { HTU.getConfiguration().setInt(HConstants.MASTER_PORT, 0); CoordinatedStateManager csm = CoordinatedStateManagerFactory.getCoordinatedStateManager( HTU.getConfiguration()); - Server server = new HMaster(HTU.getConfiguration(), csm); + HMaster server = new HMaster(HTU.getConfiguration(), csm); + Whitebox.setInternalState(server, "tableStateManager", this.tableStateManager); Whitebox.setInternalState(server, "serverManager", this.serverManager); + TableDescriptors tableDescriptors = Mockito.mock(TableDescriptors.class); + Mockito.when(tableDescriptors.get(REGIONINFO.getTable())) + .thenReturn(new HTableDescriptor(REGIONINFO.getTable())); + Whitebox.setInternalState(server, "tableDescriptors", tableDescriptors); AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server, - this.serverManager); + this.serverManager, + this.tableStateManager); Whitebox.setInternalState(server, "metaTableLocator", Mockito.mock(MetaTableLocator.class)); @@ -1018,64 +1041,24 @@ public class TestAssignmentManager { try { // set table in enabling state. am.getTableStateManager().setTableState(REGIONINFO.getTable(), - Table.State.ENABLING); + TableState.State.ENABLING); new EnableTableHandler(server, REGIONINFO.getTable(), am, new NullTableLockManager(), true).prepare() .process(); assertEquals("Number of assignments should be 1.", 1, assignmentCount); assertTrue("Table should be enabled.", am.getTableStateManager().isTableState(REGIONINFO.getTable(), - Table.State.ENABLED)); + TableState.State.ENABLED)); } finally { enabling = false; assignmentCount = 0; am.getTableStateManager().setTableState(REGIONINFO.getTable(), - Table.State.ENABLED); + TableState.State.ENABLED); am.shutdown(); ZKAssign.deleteAllNodes(this.watcher); } } - /** - * Test verifies whether stale znodes of unknown tables as for the hbase:meta will be removed or - * not. - * @throws KeeperException - * @throws IOException - * @throws Exception - */ - @Test (timeout=180000) - public void testMasterRestartShouldRemoveStaleZnodesOfUnknownTableAsForMeta() - throws Exception { - List destServers = new ArrayList(1); - destServers.add(SERVERNAME_A); - Mockito.when(this.serverManager.createDestinationServersList()).thenReturn(destServers); - Mockito.when(this.serverManager.isServerOnline(SERVERNAME_A)).thenReturn(true); - HTU.getConfiguration().setInt(HConstants.MASTER_PORT, 0); - CoordinatedStateManager csm = CoordinatedStateManagerFactory.getCoordinatedStateManager( - HTU.getConfiguration()); - Server server = new HMaster(HTU.getConfiguration(), csm); - Whitebox.setInternalState(server, "serverManager", this.serverManager); - AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server, - this.serverManager); - - Whitebox.setInternalState(server, "metaTableLocator", Mockito.mock(MetaTableLocator.class)); - - // Make it so we can get a catalogtracker from servermanager.. .needed - // down in guts of server shutdown handler. - Whitebox.setInternalState(server, "clusterConnection", am.getConnection()); - - try { - TableName tableName = TableName.valueOf("dummyTable"); - // set table in enabling state. - am.getTableStateManager().setTableState(tableName, - Table.State.ENABLING); - am.joinCluster(); - assertFalse("Table should not be present in zookeeper.", - am.getTableStateManager().isTablePresent(tableName)); - } finally { - am.shutdown(); - } - } /** * When a region is in transition, if the region server opening the region goes down, * the region assignment takes a long time normally (waiting for timeout monitor to trigger assign). @@ -1086,7 +1069,8 @@ public class TestAssignmentManager { throws KeeperException, IOException, CoordinatedStateException, ServiceException { // Create an AM. AssignmentManagerWithExtrasForTesting am = - setUpMockedAssignmentManager(this.server, this.serverManager); + setUpMockedAssignmentManager(this.server, this.serverManager, + tableStateManager); // adding region in pending open. RegionState state = new RegionState(REGIONINFO, State.OPENING, System.currentTimeMillis(), SERVERNAME_A); @@ -1096,7 +1080,7 @@ public class TestAssignmentManager { am.regionPlans.put(REGIONINFO.getEncodedName(), new RegionPlan(REGIONINFO, SERVERNAME_B, SERVERNAME_A)); am.getTableStateManager().setTableState(REGIONINFO.getTable(), - Table.State.ENABLED); + TableState.State.ENABLED); try { am.assignInvoked = false; @@ -1122,7 +1106,8 @@ public class TestAssignmentManager { public void testClosingFailureDuringRecovery() throws Exception { AssignmentManagerWithExtrasForTesting am = - setUpMockedAssignmentManager(this.server, this.serverManager); + setUpMockedAssignmentManager(this.server, this.serverManager, + tableStateManager); ZKAssign.createNodeClosing(this.watcher, REGIONINFO, SERVERNAME_A); try { am.getRegionStates().createRegionState(REGIONINFO); @@ -1196,8 +1181,10 @@ public class TestAssignmentManager { * @throws IOException * @throws KeeperException */ - private AssignmentManagerWithExtrasForTesting setUpMockedAssignmentManager(final Server server, - final ServerManager manager) throws IOException, KeeperException, + private AssignmentManagerWithExtrasForTesting setUpMockedAssignmentManager( + final MasterServices server, + final ServerManager manager, + final TableStateManager stateManager) throws IOException, KeeperException, ServiceException, CoordinatedStateException { // Make an RS Interface implementation. Make it so a scanner can go against // it and a get to return the single region, REGIONINFO, this test is @@ -1252,7 +1239,8 @@ public class TestAssignmentManager { ExecutorService executor = startupMasterExecutor("mockedAMExecutor"); this.balancer = LoadBalancerFactory.getLoadBalancer(server.getConfiguration()); AssignmentManagerWithExtrasForTesting am = new AssignmentManagerWithExtrasForTesting( - server, connection, manager, this.balancer, executor, new NullTableLockManager()); + server, connection, manager, this.balancer, executor, new NullTableLockManager(), + tableStateManager); return am; } @@ -1268,11 +1256,13 @@ public class TestAssignmentManager { private ClusterConnection connection; public AssignmentManagerWithExtrasForTesting( - final Server master, ClusterConnection connection, final ServerManager serverManager, + final MasterServices master, ClusterConnection connection, final ServerManager serverManager, final LoadBalancer balancer, - final ExecutorService service, final TableLockManager tableLockManager) + final ExecutorService service, final TableLockManager tableLockManager, + final TableStateManager tableStateManager) throws KeeperException, IOException, CoordinatedStateException { - super(master, serverManager, balancer, service, null, tableLockManager); + super(master, serverManager, balancer, service, null, tableLockManager, + tableStateManager); this.es = service; this.connection = connection; } @@ -1394,7 +1384,8 @@ public class TestAssignmentManager { server.getConfiguration()); // Create an AM. AssignmentManager am = new AssignmentManager(this.server, - this.serverManager, balancer, null, null, master.getTableLockManager()); + this.serverManager, balancer, null, null, master.getTableLockManager(), + tableStateManager); RegionStates regionStates = am.getRegionStates(); try { // First set the state of the region to merging @@ -1428,7 +1419,8 @@ public class TestAssignmentManager { final AtomicBoolean zkEventProcessed = new AtomicBoolean(false); // Create an AM. AssignmentManager am = new AssignmentManager(this.server, - this.serverManager, balancer, null, null, master.getTableLockManager()) { + this.serverManager, balancer, null, null, master.getTableLockManager(), + tableStateManager) { @Override void handleRegion(final RegionTransition rt, OpenRegionCoordination coordination, @@ -1470,7 +1462,8 @@ public class TestAssignmentManager { @Test (timeout=180000) public void testBalanceRegionOfDeletedTable() throws Exception { AssignmentManager am = new AssignmentManager(this.server, this.serverManager, - balancer, null, null, master.getTableLockManager()); + balancer, null, null, master.getTableLockManager(), + tableStateManager); RegionStates regionStates = am.getRegionStates(); HRegionInfo hri = REGIONINFO; regionStates.createRegionState(hri); @@ -1501,7 +1494,8 @@ public class TestAssignmentManager { server.getConfiguration()); // Create an AM. AssignmentManager am = new AssignmentManager(this.server, - this.serverManager, balancer, null, null, master.getTableLockManager()); + this.serverManager, balancer, null, null, master.getTableLockManager(), + tableStateManager); RegionStates regionStates = am.getRegionStates(); try { am.regionPlans.put(REGIONINFO.getEncodedName(), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java index 5de785a5329..022d8d33f65 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -723,7 +724,7 @@ public class TestAssignmentManagerOnCluster { } } - am.getTableStateManager().setTableState(table, ZooKeeperProtos.Table.State.DISABLING); + am.getTableStateManager().setTableState(table, TableState.State.DISABLING); List toAssignRegions = am.processServerShutdown(destServerName); assertTrue("Regions to be assigned should be empty.", toAssignRegions.isEmpty()); assertTrue("Regions to be assigned should be empty.", am.getRegionStates() @@ -732,7 +733,7 @@ public class TestAssignmentManagerOnCluster { if (hri != null && serverName != null) { am.regionOnline(hri, serverName); } - am.getTableStateManager().setTableState(table, ZooKeeperProtos.Table.State.DISABLED); + am.getTableStateManager().setTableState(table, TableState.State.DISABLED); TEST_UTIL.deleteTable(table); } } @@ -1267,7 +1268,7 @@ public class TestAssignmentManagerOnCluster { tableNameList.add(TableName.valueOf(name + "_" + i)); } } - List metaRows = MetaTableAccessor.fullScanOfMeta(admin.getConnection()); + List metaRows = MetaTableAccessor.fullScanRegions(admin.getConnection()); int count = 0; // Check all 100 rows are in meta for (Result result : metaRows) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java index b88c747f4eb..89dd9840cde 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java @@ -450,6 +450,11 @@ public class TestCatalogJanitor { // Auto-generated method stub return 0; } + + @Override + public TableStateManager getTableStateManager() { + return null; + } } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java index 35ba761f053..737846a9aa2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java @@ -807,6 +807,7 @@ public class TestDistributedLogSplitting { LOG.info("Disabling table\n"); TEST_UTIL.getHBaseAdmin().disableTable(TableName.valueOf("disableTable")); + TEST_UTIL.waitTableDisabled(TableName.valueOf("disableTable").getName()); // abort RS LOG.info("Aborting region server: " + hrs.getServerName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java index 70cb2fc2088..224315574eb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -42,7 +43,6 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.util.StringUtils; @@ -84,7 +84,7 @@ public class TestMaster { try (HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILYNAME)) { assertTrue(m.assignmentManager.getTableStateManager().isTableState(TABLENAME, - ZooKeeperProtos.Table.State.ENABLED)); + TableState.State.ENABLED)); TEST_UTIL.loadTable(ht, FAMILYNAME, false); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java index 8ae26a350b7..d24795a319a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java @@ -18,11 +18,6 @@ */ package org.apache.hadoop.hbase.master; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; - import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; @@ -43,24 +38,23 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.RegionTransition; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.TableStateManager; import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.master.RegionState.State; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.RequestConverter; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.RegionMergeTransaction; import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; +import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSUtils; @@ -70,12 +64,16 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKAssign; -import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.data.Stat; import org.junit.Test; import org.junit.experimental.categories.Category; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + @Category(LargeTests.class) public class TestMasterFailover { private static final Log LOG = LogFactory.getLog(TestMasterFailover.class); @@ -300,8 +298,8 @@ public class TestMasterFailover { log("Beginning to mock scenarios"); // Disable the disabledTable in ZK - TableStateManager zktable = new ZKTableStateManager(zkw); - zktable.setTableState(disabledTable, ZooKeeperProtos.Table.State.DISABLED); + MetaTableAccessor.updateTableState(TEST_UTIL.getConnection(), disabledTable, + TableState.State.DISABLED); /* * ZK = OFFLINE @@ -617,7 +615,7 @@ public class TestMasterFailover { assertTrue(" Table must be enabled.", master.getAssignmentManager() .getTableStateManager().isTableState(TableName.valueOf("enabledTable"), - ZooKeeperProtos.Table.State.ENABLED)); + TableState.State.ENABLED)); // we also need regions assigned out on the dead server List enabledAndOnDeadRegions = new ArrayList(); enabledAndOnDeadRegions.addAll(enabledRegions.subList(0, 6)); @@ -677,12 +675,15 @@ public class TestMasterFailover { log("Beginning to mock scenarios"); // Disable the disabledTable in ZK - TableStateManager zktable = new ZKTableStateManager(zkw); - zktable.setTableState(disabledTable, ZooKeeperProtos.Table.State.DISABLED); + MetaTableAccessor + .updateTableState(TEST_UTIL.getConnection(), disabledTable, TableState.State.DISABLED); - assertTrue(" The enabled table should be identified on master fail over.", - zktable.isTableState(TableName.valueOf("enabledTable"), - ZooKeeperProtos.Table.State.ENABLED)); + TableState enabledTableState = MetaTableAccessor + .getTableState(TEST_UTIL.getConnection(), TableName.valueOf("enabledTable")); + assertNotNull(" State should be accessible in META", enabledTableState); + assertEquals(" The enabled table should be identified on master fail over.", + enabledTableState.getState(), + TableState.State.ENABLED); /* * ZK = CLOSING diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java index 2c4b06e6dd5..0946edc626b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java @@ -133,7 +133,7 @@ public class TestMasterOperationsForRegionReplicas { } } - List metaRows = MetaTableAccessor.fullScanOfMeta(ADMIN.getConnection()); + List metaRows = MetaTableAccessor.fullScanRegions(ADMIN.getConnection()); int numRows = 0; for (Result result : metaRows) { RegionLocations locations = MetaTableAccessor.getRegionLocations(result); @@ -298,7 +298,7 @@ public class TestMasterOperationsForRegionReplicas { return true; } }; - MetaTableAccessor.fullScan(connection, visitor); + MetaTableAccessor.fullScanRegions(connection, visitor); assert(count.get() == numRegions); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java index a2ecfb4ead6..5af7b470afd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java @@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.RegionLocator; -import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; @@ -102,8 +102,8 @@ public class TestMasterRestartAfterDisablingTable { assertTrue("The table should not be in enabled state", cluster.getMaster() .getAssignmentManager().getTableStateManager().isTableState( - TableName.valueOf("tableRestart"), ZooKeeperProtos.Table.State.DISABLED, - ZooKeeperProtos.Table.State.DISABLING)); + TableName.valueOf("tableRestart"), TableState.State.DISABLED, + TableState.State.DISABLING)); log("Enabling table\n"); // Need a new Admin, the previous one is on the old master Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); @@ -118,7 +118,7 @@ public class TestMasterRestartAfterDisablingTable { 6, regions.size()); assertTrue("The table should be in enabled state", cluster.getMaster() .getAssignmentManager().getTableStateManager() - .isTableState(TableName.valueOf("tableRestart"), ZooKeeperProtos.Table.State.ENABLED)); + .isTableState(TableName.valueOf("tableRestart"), TableState.State.ENABLED)); ht.close(); TEST_UTIL.shutdownMiniCluster(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java index 4ae6b248438..c1482a70e1b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java @@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.MockServer; import org.apache.hadoop.hbase.zookeeper.ZKAssign; -import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -128,6 +127,7 @@ public class TestOpenedRegionHandler { region = HRegion.createHRegion(hri, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd); assertNotNull(region); AssignmentManager am = Mockito.mock(AssignmentManager.class); + TableStateManager tsm = Mockito.mock(TableStateManager.class); RegionStates rsm = Mockito.mock(RegionStates.class); Mockito.doReturn(rsm).when(am).getRegionStates(); when(rsm.isRegionInTransition(hri)).thenReturn(false); @@ -137,7 +137,7 @@ public class TestOpenedRegionHandler { // create a node with OPENED state zkw = HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL, region, server.getServerName()); - when(am.getTableStateManager()).thenReturn(new ZKTableStateManager(zkw)); + when(am.getTableStateManager()).thenReturn(tsm); Stat stat = new Stat(); String nodeName = ZKAssign.getNodeName(zkw, region.getRegionInfo() .getEncodedName()); @@ -166,6 +166,7 @@ public class TestOpenedRegionHandler { try { handler.process(); } catch (Exception e) { + e.printStackTrace(); expectedException = true; } assertFalse("The process method should not throw any exception.", diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java index 20cbd629aa1..a35e3594b39 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java @@ -19,10 +19,8 @@ package org.apache.hadoop.hbase.master; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.TableStateManager; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -43,7 +41,7 @@ public class TestRegionStates { @Test public void testWeDontReturnDrainingServersForOurBalancePlans() throws Exception { - Server server = mock(Server.class); + MasterServices server = mock(MasterServices.class); when(server.getServerName()).thenReturn(ServerName.valueOf("master,1,1")); Configuration configuration = mock(Configuration.class); when(server.getConfiguration()).thenReturn(configuration); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java index 39b347d8265..93b8755439b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java @@ -18,25 +18,32 @@ */ package org.apache.hadoop.hbase.master.handler; -import java.util.Collections; +import java.io.IOException; +import java.util.ArrayList; import java.util.List; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; +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.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.junit.After; import org.junit.Before; @@ -46,13 +53,6 @@ import org.junit.experimental.categories.Category; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import java.io.IOException; -import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.client.Table; - @Category({ MediumTests.class }) public class TestEnableTableHandler { private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -84,7 +84,6 @@ public class TestEnableTableHandler { admin.enableTable(tableName); TEST_UTIL.waitTableEnabled(tableName); - // disable once more admin.disableTable(tableName); @@ -94,30 +93,49 @@ public class TestEnableTableHandler { rs.getRegionServer().stop("stop"); cluster.waitForRegionServerToStop(rs.getRegionServer().getServerName(), 10000); - TEST_UTIL.waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME); + LOG.debug("Now enabling table " + tableName); admin.enableTable(tableName); assertTrue(admin.isTableEnabled(tableName)); JVMClusterUtil.RegionServerThread rs2 = cluster.startRegionServer(); - m.getAssignmentManager().assign(admin.getTableRegions(tableName)); + cluster.waitForRegionServerToStart(rs2.getRegionServer().getServerName().getHostname(), + rs2.getRegionServer().getServerName().getPort(), 60000); + + List regions = TEST_UTIL.getHBaseAdmin().getTableRegions(tableName); + assertEquals(1, regions.size()); + for (HRegionInfo region : regions) { + TEST_UTIL.getHBaseAdmin().assign(region.getEncodedNameAsBytes()); + } + LOG.debug("Waiting for table assigned " + tableName); TEST_UTIL.waitUntilAllRegionsAssigned(tableName); List onlineRegions = admin.getOnlineRegions( rs2.getRegionServer().getServerName()); - assertEquals(2, onlineRegions.size()); - assertEquals(tableName, onlineRegions.get(1).getTable()); + ArrayList tableRegions = filterTableRegions(tableName, onlineRegions); + assertEquals(1, tableRegions.size()); + } + + private ArrayList filterTableRegions(final TableName tableName, + List onlineRegions) { + return Lists.newArrayList(Iterables.filter(onlineRegions, new Predicate() { + @Override + public boolean apply(HRegionInfo input) { + return input.getTable().equals(tableName); + } + })); } /** * We were only clearing rows that had a hregioninfo column in hbase:meta. Mangled rows that * were missing the hregioninfo because of error were being left behind messing up any * subsequent table made with the same name. HBASE-12980 + * * @throws IOException * @throws InterruptedException */ - @Test(timeout=60000) + @Test(timeout = 60000) public void testDeleteForSureClearsAllTableRowsFromMeta() - throws IOException, InterruptedException { + throws IOException, InterruptedException { final TableName tableName = TableName.valueOf("testDeleteForSureClearsAllTableRowsFromMeta"); final MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); final HMaster m = cluster.getMaster(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/migration/TestUpgradeTo96.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/migration/TestUpgradeTo96.java index 2a8c9e388a6..dc4bc400857 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/migration/TestUpgradeTo96.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/migration/TestUpgradeTo96.java @@ -30,6 +30,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FsShell; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.io.FileLink; import org.apache.hadoop.hbase.io.HFileLink; @@ -136,7 +137,7 @@ public class TestUpgradeTo96 { // add table znode, data of its children would be protobuffized tableAZnode = ZKUtil.joinZNode(zkw.tableZNode, "a"); ZKUtil.createWithParents(zkw, tableAZnode, - Bytes.toBytes(ZooKeeperProtos.Table.State.ENABLED.toString())); + Bytes.toBytes(TableState.State.ENABLED.toString())); // add replication znodes, data of its children would be protobuffized String replicationZnode = ZKUtil.joinZNode(zkw.baseZNode, "replication"); replicationPeerZnode = ZKUtil.joinZNode(replicationZnode, "peers"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java index 2f99cd59e31..9054aa9beeb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java @@ -211,6 +211,7 @@ public class TestCoprocessorScanPolicy { // should be gone now assertEquals(0, r.size()); t.close(); + EnvironmentEdgeManager.reset(); } public static class ScanObserver extends BaseRegionObserver { 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 df01d710699..0bd412d5881 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 @@ -261,7 +261,7 @@ public class TestFSTableDescriptors { htds.createTableDescriptor(htd); } // add hbase:meta - HTableDescriptor htd = new HTableDescriptor(HTableDescriptor.META_TABLEDESC.getTableName()); + HTableDescriptor htd = new HTableDescriptor(TableName.META_TABLE_NAME); htds.createTableDescriptor(htd); assertTrue(htds.getAll().size() == count + 1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java index c4ed3eedc2e..4083beaa2a5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java @@ -37,8 +37,6 @@ import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Random; -import java.util.Map.Entry; import java.util.NavigableMap; import java.util.Set; import java.util.concurrent.Callable; @@ -69,7 +67,6 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.MiniHBaseCluster; -import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableAccessor; @@ -84,12 +81,10 @@ import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.MetaScanner; -import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.hfile.TestHFile; @@ -164,6 +159,7 @@ public class TestHBaseFsck { conf.setInt("hbase.hconnection.threads.max", 2 * POOL_SIZE); conf.setInt("hbase.hconnection.threads.core", POOL_SIZE); conf.setInt("hbase.hbck.close.timeout", 2 * REGION_ONLINE_TIMEOUT); + conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 2 * REGION_ONLINE_TIMEOUT); TEST_UTIL.startMiniCluster(3); tableExecutorService = new ThreadPoolExecutor(1, POOL_SIZE, 60, TimeUnit.SECONDS, @@ -1356,7 +1352,7 @@ public class TestHBaseFsck { HBaseFsck hbck = doFsck(conf, false); assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.NOT_IN_HDFS, ERROR_CODE.NOT_IN_HDFS, ERROR_CODE.NOT_IN_HDFS, - ERROR_CODE.NOT_IN_HDFS,}); + ERROR_CODE.NOT_IN_HDFS, ERROR_CODE.ORPHAN_TABLE_STATE, }); // holes are separate from overlap groups assertEquals(0, hbck.getOverlapGroups(table).size()); @@ -1398,6 +1394,34 @@ public class TestHBaseFsck { assertNoErrors(doFsck(conf, false)); } + /** + * when the hbase.version file missing, It is fix the fault. + */ + @Test (timeout=180000) + public void testNoTableState() throws Exception { + // delete the hbase.version file + TableName table = + TableName.valueOf("testNoTableState"); + try { + setupTable(table); + // make sure data in regions, if in wal only there is no data loss + admin.flush(table); + + MetaTableAccessor.deleteTableState(TEST_UTIL.getConnection(), table); + + // test + HBaseFsck hbck = doFsck(conf, false); + assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NO_TABLE_STATE }); + // fix table state missing + doFsck(conf, true); + + assertNoErrors(doFsck(conf, false)); + assertTrue(TEST_UTIL.getHBaseAdmin().isTableEnabled(table)); + } finally { + cleanupTable(table); + } + } + /** * The region is not deployed when the table is disabled. */ @@ -2405,55 +2429,6 @@ public class TestHBaseFsck { writeLock.release(); // release for clean state } - /** - * Test orphaned table ZNode (for table states) - */ - @Test - public void testOrphanedTableZNode() throws Exception { - TableName table = TableName.valueOf("testOrphanedZKTableEntry"); - - try { - TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getTableStateManager() - .setTableState(table, ZooKeeperProtos.Table.State.ENABLING); - - try { - setupTable(table); - Assert.fail( - "Create table should fail when its ZNode has already existed with ENABLING state."); - } catch(TableExistsException t) { - //Expected exception - } - // The setup table was interrupted in some state that needs to some cleanup. - try { - cleanupTable(table); - } catch (IOException e) { - // Because create table failed, it is expected that the cleanup table would - // throw some exception. Ignore and continue. - } - - HBaseFsck hbck = doFsck(conf, false); - assertTrue(hbck.getErrors().getErrorList().contains(ERROR_CODE.ORPHANED_ZK_TABLE_ENTRY)); - - // fix the orphaned ZK entry - hbck = doFsck(conf, true); - - // check that orpahned ZK table entry is gone. - hbck = doFsck(conf, false); - assertFalse(hbck.getErrors().getErrorList().contains(ERROR_CODE.ORPHANED_ZK_TABLE_ENTRY)); - // Now create table should succeed. - setupTable(table); - } finally { - // This code could be called that either a table was created successfully or set up - // table failed in some unknown state. Therefore, clean up can either succeed or fail. - try { - cleanupTable(table); - } catch (IOException e) { - // The cleanup table would throw some exception if create table failed in some state. - // Ignore this exception - } - } - } - @Test (timeout=180000) public void testMetaOffline() throws Exception { // check no errors diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java index 744240ece67..5be4b1e52a8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java @@ -35,6 +35,9 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.client.HConnectionManager; +import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ServerName; @@ -229,7 +232,6 @@ public class OfflineMetaRebuildTestCore { out.write(hri.toDelimitedByteArray()); out.close(); - // add to meta. MetaTableAccessor.addRegionToMeta(meta, hri); meta.close(); return hri; @@ -284,16 +286,9 @@ public class OfflineMetaRebuildTestCore { * @return # of entries in meta. */ protected int scanMeta() throws IOException { - int count = 0; - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); - ResultScanner scanner = meta.getScanner(new Scan()); - LOG.info("Table: " + Bytes.toString(meta.getTableName())); - for (Result res : scanner) { - LOG.info(Bytes.toString(res.getRow())); - count++; - } - meta.close(); - return count; + LOG.info("Scanning META"); + MetaTableAccessor.fullScanMetaAndPrint(TEST_UTIL.getConnection()); + return MetaTableAccessor.fullScanRegions(TEST_UTIL.getConnection()).size(); } protected HTableDescriptor[] getTables(final Configuration configuration) throws IOException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java index f4a035fbe35..a4ddc13a1dc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java @@ -20,11 +20,13 @@ package org.apache.hadoop.hbase.util.hbck; import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors; import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import java.util.Arrays; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; @@ -57,24 +59,29 @@ public class TestOfflineMetaRebuildBase extends OfflineMetaRebuildTestCore { // shutdown the minicluster TEST_UTIL.shutdownMiniHBaseCluster(); - TEST_UTIL.shutdownMiniZKCluster(); // rebuild meta table from scratch HBaseFsck fsck = new HBaseFsck(conf); assertTrue(fsck.rebuildMeta(false)); // bring up the minicluster - TEST_UTIL.startMiniZKCluster(); TEST_UTIL.restartHBaseCluster(3); try (Connection connection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) { Admin admin = connection.getAdmin(); - admin.enableTable(table); + if (admin.isTableDisabled(table)) + admin.enableTable(table); LOG.info("Waiting for no more RIT"); TEST_UTIL.waitUntilNoRegionsInTransition(60000); LOG.info("No more RIT in ZK, now doing final test verification"); // everything is good again. - assertEquals(5, scanMeta()); + assertEquals(5, scanMeta()); // including table state rows + TableName[] tableNames = TEST_UTIL.getHBaseAdmin().listTableNames(); + for (TableName tableName : tableNames) { + HTableDescriptor tableDescriptor = TEST_UTIL.getHBaseAdmin().getTableDescriptor(tableName); + assertNotNull(tableDescriptor); + assertTrue(TEST_UTIL.getHBaseAdmin().isTableEnabled(tableName)); + } HTableDescriptor[] htbls = admin.listTables(); LOG.info("Tables present after restart: " + Arrays.toString(htbls)); assertEquals(1, htbls.length); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java index f6fd4f1bd9e..7bdc6f2465a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java @@ -61,7 +61,6 @@ public class TestOfflineMetaRebuildHole extends OfflineMetaRebuildTestCore { // shutdown the minicluster TEST_UTIL.shutdownMiniHBaseCluster(); - TEST_UTIL.shutdownMiniZKCluster(); // attempt to rebuild meta table from scratch HBaseFsck fsck = new HBaseFsck(conf); @@ -69,7 +68,6 @@ public class TestOfflineMetaRebuildHole extends OfflineMetaRebuildTestCore { fsck.close(); // bring up the minicluster - TEST_UTIL.startMiniZKCluster(); // tables seem enabled by default TEST_UTIL.restartHBaseCluster(3); ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java index 69a962b1cfb..b8ca7d78bb8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java @@ -66,7 +66,6 @@ public class TestOfflineMetaRebuildOverlap extends OfflineMetaRebuildTestCore { // shutdown the minicluster TEST_UTIL.shutdownMiniHBaseCluster(); - TEST_UTIL.shutdownMiniZKCluster(); // attempt to rebuild meta table from scratch HBaseFsck fsck = new HBaseFsck(conf); @@ -77,7 +76,6 @@ public class TestOfflineMetaRebuildOverlap extends OfflineMetaRebuildTestCore { assertEquals(3, problems.size()); // bring up the minicluster - TEST_UTIL.startMiniZKCluster(); // tables seem enabled by default TEST_UTIL.restartHBaseCluster(3); ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTableStateManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTableStateManager.java deleted file mode 100644 index e81c89f0340..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTableStateManager.java +++ /dev/null @@ -1,114 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.zookeeper; - -import java.io.IOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.CoordinatedStateException; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.TableStateManager; -import org.apache.zookeeper.KeeperException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table; - -@Category(MediumTests.class) -public class TestZKTableStateManager { - private static final Log LOG = LogFactory.getLog(TestZKTableStateManager.class); - private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - TEST_UTIL.startMiniZKCluster(); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - TEST_UTIL.shutdownMiniZKCluster(); - } - - @Test - public void testTableStates() - throws CoordinatedStateException, IOException, KeeperException, InterruptedException { - final TableName name = - TableName.valueOf("testDisabled"); - Abortable abortable = new Abortable() { - @Override - public void abort(String why, Throwable e) { - LOG.info(why, e); - } - - @Override - public boolean isAborted() { - return false; - } - - }; - ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), - name.getNameAsString(), abortable, true); - TableStateManager zkt = new ZKTableStateManager(zkw); - assertFalse(zkt.isTableState(name, Table.State.ENABLED)); - assertFalse(zkt.isTableState(name, Table.State.DISABLING)); - assertFalse(zkt.isTableState(name, Table.State.DISABLED)); - assertFalse(zkt.isTableState(name, Table.State.ENABLING)); - assertFalse(zkt.isTableState(name, Table.State.DISABLED, Table.State.DISABLING)); - assertFalse(zkt.isTableState(name, Table.State.DISABLED, Table.State.ENABLING)); - assertFalse(zkt.isTablePresent(name)); - zkt.setTableState(name, Table.State.DISABLING); - assertTrue(zkt.isTableState(name, Table.State.DISABLING)); - assertTrue(zkt.isTableState(name, Table.State.DISABLED, Table.State.DISABLING)); - assertFalse(zkt.getTablesInStates(Table.State.DISABLED).contains(name)); - assertTrue(zkt.isTablePresent(name)); - zkt.setTableState(name, Table.State.DISABLED); - assertTrue(zkt.isTableState(name, Table.State.DISABLED)); - assertTrue(zkt.isTableState(name, Table.State.DISABLED, Table.State.DISABLING)); - assertFalse(zkt.isTableState(name, Table.State.DISABLING)); - assertTrue(zkt.getTablesInStates(Table.State.DISABLED).contains(name)); - assertTrue(zkt.isTablePresent(name)); - zkt.setTableState(name, Table.State.ENABLING); - assertTrue(zkt.isTableState(name, Table.State.ENABLING)); - assertTrue(zkt.isTableState(name, Table.State.DISABLED, Table.State.ENABLING)); - assertFalse(zkt.isTableState(name, Table.State.DISABLED)); - assertFalse(zkt.getTablesInStates(Table.State.DISABLED).contains(name)); - assertTrue(zkt.isTablePresent(name)); - zkt.setTableState(name, Table.State.ENABLED); - assertTrue(zkt.isTableState(name, Table.State.ENABLED)); - assertFalse(zkt.isTableState(name, Table.State.ENABLING)); - assertTrue(zkt.isTablePresent(name)); - zkt.setDeletedTable(name); - assertFalse(zkt.isTableState(name, Table.State.ENABLED)); - assertFalse(zkt.isTableState(name, Table.State.DISABLING)); - assertFalse(zkt.isTableState(name, Table.State.DISABLED)); - assertFalse(zkt.isTableState(name, Table.State.ENABLING)); - assertFalse(zkt.isTableState(name, Table.State.DISABLED, Table.State.DISABLING)); - assertFalse(zkt.isTableState(name, Table.State.DISABLED, Table.State.ENABLING)); - assertFalse(zkt.isTablePresent(name)); - } -}