diff --git a/bin/region_status.rb b/bin/region_status.rb index 878d58a..a016afd 100644 --- a/bin/region_status.rb +++ b/bin/region_status.rb @@ -54,7 +54,7 @@ import org.apache.hadoop.hbase.client.Scan import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter import org.apache.hadoop.hbase.util.Bytes import org.apache.hadoop.hbase.HRegionInfo -import org.apache.hadoop.hbase.client.MetaScanner +import org.apache.hadoop.hbase.MetaTableAccessor import org.apache.hadoop.hbase.HTableDescriptor import org.apache.hadoop.hbase.client.HConnectionManager @@ -139,7 +139,7 @@ while true server_count = admin.getClusterStatus().getRegionsCount() else connection = HConnectionManager::getConnection(config); - server_count = MetaScanner::allTableRegions(config, connection, $TableName ,false).size() + server_count = MetaTableAccessor::allTableRegions(connection, $TableName).size() end print "Region Status: #{server_count} / #{meta_count}\n" if SHOULD_WAIT and server_count < meta_count diff --git a/conf/log4j.properties b/conf/log4j.properties index 472fc03..40f47ba 100644 --- a/conf/log4j.properties +++ b/conf/log4j.properties @@ -91,4 +91,3 @@ log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher=INFO # Uncomment the below if you want to remove logging of client region caching' # and scan of hbase:meta messages # log4j.logger.org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation=INFO -# log4j.logger.org.apache.hadoop.hbase.client.MetaScanner=INFO 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 37ef788..b647836 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,25 @@ */ package org.apache.hadoop.hbase; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.io.Closeable; +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; @@ -27,10 +43,11 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Consistency; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.NeedUnmanagedConnectionException; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.RegionReplicaUtil; @@ -38,29 +55,17 @@ 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.exceptions.DeserializationException; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos; 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.PairOfSameType; 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 @@ -119,7 +124,55 @@ public class MetaTableAccessor { int len = HRegionInfo.FIRST_META_REGIONINFO.getRegionName().length - 2; META_REGION_PREFIX = new byte [len]; System.arraycopy(HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), 0, - META_REGION_PREFIX, 0, len); + META_REGION_PREFIX, 0, len); + } + + /** + * Lists all of the table regions currently in META. + * Deprecated, keep there until some test use this. + * @param connection what we will use + * @param tableName table to list + * @return Map of all user-space regions to servers + * @throws java.io.IOException + * @deprecated use {@link #getTableRegionsAndLocations}, region can have multiple locations + */ + @Deprecated + public static NavigableMap allTableRegions( + Connection connection, final TableName tableName) throws IOException { + final NavigableMap regions = + new TreeMap(); + Visitor visitor = new TableVisitorBase(tableName) { + @Override + public boolean visitInternal(Result result) throws IOException { + RegionLocations locations = getRegionLocations(result); + if (locations == null) return true; + for (HRegionLocation loc : locations.getRegionLocations()) { + if (loc != null) { + HRegionInfo regionInfo = loc.getRegionInfo(); + regions.put(regionInfo, loc.getServerName()); + } + } + return true; + } + }; + scanMetaForTableRegions(connection, visitor, tableName); + return regions; + } + + @InterfaceAudience.Private + public enum QueryType { + ALL(HConstants.CATALOG_FAMILY), + REGION(HConstants.CATALOG_FAMILY); + + private final byte[][] families; + + QueryType(byte[]... families) { + this.families = families; + } + + byte[][] getFamilies() { + return this.families; + } } /** The delimiter for meta columns for replicaIds > 0 */ @@ -127,46 +180,45 @@ public class MetaTableAccessor { /** A regex for parsing server columns from meta. See above javadoc for meta layout */ private static final Pattern SERVER_COLUMN_PATTERN - = Pattern.compile("^server(_[0-9a-fA-F]{4})?$"); + = Pattern.compile("^server(_[0-9a-fA-F]{4})?$"); //////////////////////// // 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 { + scanMeta(connection, null, null, QueryType.REGION, visitor); } /** - * Performs a full scan of hbase:meta. + * Performs a full scan of hbase:meta for regions. * @param connection connection we're using - * @param visitor Visitor invoked against each row. * @throws IOException */ - public static void fullScan(Connection connection, - final Visitor visitor) - throws IOException { - fullScan(connection, visitor, null); + public static List fullScanRegions(Connection connection) + throws IOException { + return fullScan(connection, QueryType.REGION); } /** * 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) - throws IOException { + public static List fullScan(Connection connection, QueryType type) + throws IOException { CollectAllVisitor v = new CollectAllVisitor(); - fullScan(connection, v, null); + scanMeta(connection, null, null, type, v); return v.getResults(); } @@ -177,7 +229,7 @@ public class MetaTableAccessor { * @throws IOException */ static Table getMetaHTable(final Connection connection) - throws IOException { + throws IOException { // We used to pass whole CatalogTracker in here, now we just pass in Connection if (connection == null || connection.isClosed()) { throw new NullPointerException("No connection"); @@ -192,7 +244,7 @@ public class MetaTableAccessor { // There should still be a way to use this method with an unmanaged connection. if (connection instanceof ClusterConnection) { if (((ClusterConnection) connection).isManaged()) { - return new HTable(TableName.META_TABLE_NAME, (ClusterConnection) connection); + throw new NeedUnmanagedConnectionException(); } } return connection.getTable(TableName.META_TABLE_NAME); @@ -204,6 +256,7 @@ public class MetaTableAccessor { * @throws IOException */ private static Result get(final Table t, final Get g) throws IOException { + if (t == null) return null; try { return t.get(g); } finally { @@ -221,11 +274,11 @@ public class MetaTableAccessor { */ @Deprecated public static Pair getRegion(Connection connection, byte [] regionName) - throws IOException { + throws IOException { HRegionLocation location = getRegionLocation(connection, regionName); return location == null - ? null - : new Pair(location.getRegionInfo(), location.getServerName()); + ? null + : new Pair(location.getRegionInfo(), location.getServerName()); } /** @@ -236,7 +289,7 @@ public class MetaTableAccessor { * @throws IOException */ public static HRegionLocation getRegionLocation(Connection connection, - byte[] regionName) throws IOException { + byte[] regionName) throws IOException { byte[] row = regionName; HRegionInfo parsedInfo = null; try { @@ -250,8 +303,8 @@ public class MetaTableAccessor { Result r = get(getMetaHTable(connection), get); RegionLocations locations = getRegionLocations(r); return locations == null - ? null - : locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId()); + ? null + : locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId()); } /** @@ -262,7 +315,7 @@ public class MetaTableAccessor { * @throws IOException */ public static HRegionLocation getRegionLocation(Connection connection, - HRegionInfo regionInfo) throws IOException { + HRegionInfo regionInfo) throws IOException { byte[] row = getMetaKeyForRegion(regionInfo); Get get = new Get(row); get.addFamily(HConstants.CATALOG_FAMILY); @@ -280,12 +333,12 @@ public class MetaTableAccessor { * in the regionName. */ protected static HRegionInfo parseRegionInfoFromRegionName(byte[] regionName) - throws IOException { + throws IOException { byte[][] fields = HRegionInfo.parseRegionName(regionName); long regionId = Long.parseLong(Bytes.toString(fields[2])); int replicaId = fields.length > 3 ? Integer.parseInt(Bytes.toString(fields[3]), 16) : 0; return new HRegionInfo( - TableName.valueOf(fields[0]), fields[1], fields[1], false, regionId, replicaId); + TableName.valueOf(fields[0]), fields[1], fields[1], false, regionId, replicaId); } /** @@ -307,6 +360,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); @@ -316,7 +370,7 @@ public class MetaTableAccessor { return null; } return new Pair(mergeA, mergeB); - } + } /** * Checks if the specified table exists. Looks at the hbase:meta table hosted on @@ -328,62 +382,58 @@ 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. + throws IOException { + // Catalog tables always exist. + if (tableName.equals(TableName.META_TABLE_NAME)) return true; - } - // Make a version of ResultCollectingVisitor that only collects the first - CollectingVisitor visitor = new CollectingVisitor() { - private HRegionInfo current = null; - + final boolean[] visited = { false }; + scanMetaForTableRegions(connection, new DefaultVisitorBase() { @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; + public boolean visitInternal(Result rowResult) throws IOException { + return visited[0] = true; } + }, tableName); + return visited[0]; + } + + /** + * Lists all of the regions currently in META. + * + * @param connection to connect with + * @param excludeOfflinedSplitParents False if we are to include offlined/splitparents regions, + * true and we'll leave out offlined regions from returned list + * @return List of all user-space regions. + * @throws IOException + */ + @VisibleForTesting + public static List getAllRegions(Connection connection, + boolean excludeOfflinedSplitParents) + throws IOException { + List> result; + + result = getTableRegionsAndLocations(connection, null, + excludeOfflinedSplitParents); + + return getListOfHRegionInfos(result); - @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; } /** - * Gets all of the regions of the specified table. - * @param zkw zookeeper connection to access meta table + * Gets all of the regions of the specified table. Do not use this method + * to get meta table regions, use methods in MetaTableLocator instead. * @param connection connection we're using * @param tableName table we're looking for * @return Ordered list of {@link HRegionInfo}. * @throws IOException */ - public static List getTableRegions(ZooKeeperWatcher zkw, - Connection connection, TableName tableName) - throws IOException { - return getTableRegions(zkw, connection, tableName, false); + public static List getTableRegions(Connection connection, TableName tableName) + throws IOException { + return getTableRegions(connection, tableName, false); } /** - * Gets all of the regions of the specified table. - * @param zkw zookeeper connection to access meta table + * Gets all of the regions of the specified table. Do not use this method + * to get meta table regions, use methods in MetaTableLocator instead. * @param connection connection we're using * @param tableName table we're looking for * @param excludeOfflinedSplitParents If true, do not include offlined split @@ -391,15 +441,18 @@ public class MetaTableAccessor { * @return Ordered list of {@link HRegionInfo}. * @throws IOException */ - public static List getTableRegions(ZooKeeperWatcher zkw, - Connection connection, TableName tableName, final boolean excludeOfflinedSplitParents) - throws IOException { - List> result = null; - result = getTableRegionsAndLocations(zkw, connection, tableName, + public static List getTableRegions(Connection connection, + TableName tableName, final boolean excludeOfflinedSplitParents) + throws IOException { + List> result; + + result = getTableRegionsAndLocations(connection, tableName, excludeOfflinedSplitParents); + return getListOfHRegionInfos(result); } + @Nullable static List getListOfHRegionInfos(final List> pairs) { if (pairs == null || pairs.isEmpty()) return null; List result = new ArrayList(pairs.size()); @@ -421,15 +474,45 @@ public class MetaTableAccessor { /** * @param tableName table we're working with - * @return Place to start Scan in hbase:meta when passed a - * tableName; returns <tableName&rt; <,&rt; <,&rt; + * @return start row for scanning META according to query type */ - static byte [] getTableStartRowForMeta(TableName tableName) { - byte [] startRow = new byte[tableName.getName().length + 2]; - System.arraycopy(tableName.getName(), 0, startRow, 0, tableName.getName().length); - startRow[startRow.length - 2] = HConstants.DELIMITER; - startRow[startRow.length - 1] = HConstants.DELIMITER; - return startRow; + public static byte[] getTableStartRowForMeta(TableName tableName, QueryType type) { + if (tableName == null) { + return null; + } + switch (type) { + case ALL: + default: + case REGION: + byte[] startRow = new byte[tableName.getName().length + 2]; + System.arraycopy(tableName.getName(), 0, startRow, 0, tableName.getName().length); + startRow[startRow.length - 2] = HConstants.DELIMITER; + startRow[startRow.length - 1] = HConstants.DELIMITER; + return startRow; + } + } + + /** + * @param tableName table we're working with + * @return stop row for scanning META according to query type + */ + public static byte[] getTableStopRowForMeta(TableName tableName, QueryType type) { + if (tableName == null) { + return null; + } + final byte[] stopRow; + switch (type) { + case ALL: + case REGION: + default: + stopRow = new byte[tableName.getName().length + 3]; + System.arraycopy(tableName.getName(), 0, stopRow, 0, tableName.getName().length); + stopRow[stopRow.length - 3] = ' '; + stopRow[stopRow.length - 2] = HConstants.DELIMITER; + stopRow[stopRow.length - 1] = HConstants.DELIMITER; + break; + } + return stopRow; } /** @@ -441,85 +524,102 @@ public class MetaTableAccessor { * @param tableName bytes of table's name * @return configured Scan object */ - public static Scan getScanForTableName(TableName tableName) { - String strName = tableName.getNameAsString(); + @Deprecated + public static Scan getScanForTableName(Connection connection, TableName tableName) { // Start key is just the table name with delimiters - byte[] startKey = Bytes.toBytes(strName + ",,"); + byte[] startKey = getTableStartRowForMeta(tableName, QueryType.REGION); // Stop key appends the smallest possible char to the table name - byte[] stopKey = Bytes.toBytes(strName + " ,,"); + byte[] stopKey = getTableStopRowForMeta(tableName, QueryType.REGION); - Scan scan = new Scan(startKey); + Scan scan = getMetaScan(connection); + scan.setStartRow(startKey); scan.setStopRow(stopKey); return scan; } + private static Scan getMetaScan(Connection connection) { + return getMetaScan(connection, Integer.MAX_VALUE); + } + + private static Scan getMetaScan(Connection connection, int rowUpperLimit) { + Scan scan = new Scan(); + int scannerCaching = connection.getConfiguration() + .getInt(HConstants.HBASE_META_SCANNER_CACHING, + HConstants.DEFAULT_HBASE_META_SCANNER_CACHING); + if (connection.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS, + HConstants.DEFAULT_USE_META_REPLICAS)) { + scan.setConsistency(Consistency.TIMELINE); + } + if (rowUpperLimit <= scannerCaching) { + scan.setSmall(true); + } + int rows = Math.min(rowUpperLimit, scannerCaching); + scan.setCaching(rows); + return scan; + } /** - * @param zkw zookeeper connection to access meta table + * Do not use this method to get meta table regions, use methods in MetaTableLocator instead. * @param connection connection we're using * @param tableName table we're looking for * @return Return list of regioninfos and server. * @throws IOException */ public static List> - getTableRegionsAndLocations(ZooKeeperWatcher zkw, - Connection connection, TableName tableName) - throws IOException { - return getTableRegionsAndLocations(zkw, connection, tableName, true); + getTableRegionsAndLocations(Connection connection, TableName tableName) + throws IOException { + return getTableRegionsAndLocations(connection, tableName, true); } /** - * @param zkw ZooKeeperWatcher instance we're using to get hbase:meta location + * Do not use this method to get meta table regions, use methods in MetaTableLocator instead. * @param connection connection we're using - * @param tableName table to work with + * @param tableName table to work with, can be null for getting all regions + * @param excludeOfflinedSplitParents don't return split parents * @return Return list of regioninfos and server addresses. * @throws IOException */ public static List> getTableRegionsAndLocations( - ZooKeeperWatcher zkw, Connection connection, final TableName tableName, + Connection connection, @Nullable final TableName tableName, final boolean excludeOfflinedSplitParents) throws IOException { - - if (tableName.equals(TableName.META_TABLE_NAME)) { - // If meta, do a bit of special handling. - ServerName serverName = new MetaTableLocator().getMetaRegionLocation(zkw); - List> list = - new ArrayList>(); - list.add(new Pair(HRegionInfo.FIRST_META_REGIONINFO, - serverName)); - return list; + if (tableName != null && tableName.equals(TableName.META_TABLE_NAME)) { + throw new IOException("This method can't be used to locate meta regions;" + + " use MetaTableLocator instead"); } // Make a version of CollectingVisitor that collects HRegionInfo and ServerAddress CollectingVisitor> visitor = - new CollectingVisitor>() { - private RegionLocations current = null; - - @Override - public boolean visit(Result r) throws IOException { - current = getRegionLocations(r); - if (current == null || current.getRegionLocation().getRegionInfo() == null) { - LOG.warn("No serialized HRegionInfo in " + r); - return true; + new CollectingVisitor>() { + private RegionLocations current = null; + + @Override + public boolean visit(Result r) throws IOException { + current = getRegionLocations(r); + if (current == null || current.getRegionLocation().getRegionInfo() == null) { + LOG.warn("No serialized HRegionInfo in " + r); + return true; + } + HRegionInfo hri = current.getRegionLocation().getRegionInfo(); + if (excludeOfflinedSplitParents && hri.isSplitParent()) return true; + // Else call super and add this Result to the collection. + return super.visit(r); } - HRegionInfo hri = current.getRegionLocation().getRegionInfo(); - if (!isInsideTable(hri, tableName)) return false; - if (excludeOfflinedSplitParents && hri.isSplitParent()) return true; - // Else call super and add this Result to the collection. - return super.visit(r); - } - @Override - void add(Result r) { - if (current == null) { - return; - } - for (HRegionLocation loc : current.getRegionLocations()) { - if (loc != null) { - this.results.add(new Pair( - loc.getRegionInfo(), loc.getServerName())); + @Override + void add(Result r) { + if (current == null) { + return; + } + for (HRegionLocation loc : current.getRegionLocations()) { + if (loc != null) { + this.results.add(new Pair( + loc.getRegionInfo(), loc.getServerName())); + } } } - } - }; - fullScan(connection, visitor, getTableStartRowForMeta(tableName)); + }; + scanMeta(connection, + getTableStartRowForMeta(tableName, QueryType.REGION), + getTableStopRowForMeta(tableName, QueryType.REGION), + QueryType.REGION, visitor); return visitor.getResults(); } @@ -532,7 +632,7 @@ public class MetaTableAccessor { */ public static NavigableMap getServerUserRegions(Connection connection, final ServerName serverName) - throws IOException { + throws IOException { final NavigableMap hris = new TreeMap(); // Fill the above hris map with entries from hbase:meta that have the passed // servername. @@ -551,12 +651,12 @@ public class MetaTableAccessor { } } }; - fullScan(connection, v); + scanMeta(connection, null, null, QueryType.REGION, v); return hris; } public static void fullScanMetaAndPrint(Connection connection) - throws IOException { + throws IOException { Visitor v = new Visitor() { @Override public boolean visit(Result r) throws IOException { @@ -572,42 +672,140 @@ public class MetaTableAccessor { return true; } }; - fullScan(connection, v); + scanMeta(connection, null, null, QueryType.ALL, v); + } + + public static void scanMetaForTableRegions(Connection connection, + Visitor visitor, TableName tableName) throws IOException { + scanMeta(connection, tableName, QueryType.REGION, Integer.MAX_VALUE, visitor); + } + + public static void scanMeta(Connection connection, TableName table, + QueryType type, int maxRows, final Visitor visitor) throws IOException { + scanMeta(connection, getTableStartRowForMeta(table, type), getTableStopRowForMeta(table, type), + type, maxRows, visitor); + } + + public static void scanMeta(Connection connection, + @Nullable final byte[] startRow, @Nullable final byte[] stopRow, + QueryType type, final Visitor visitor) throws IOException { + scanMeta(connection, startRow, stopRow, type, Integer.MAX_VALUE, visitor); } /** - * Performs a full scan of a catalog table. + * Performs a scan of META table for given table starting from + * given row. + * * @param connection connection we're using + * @param visitor visitor to call + * @param tableName table withing we scan + * @param row start scan from this row + * @param rowLimit max number of rows to return + * @throws IOException + */ + public static void scanMeta(Connection connection, + final Visitor visitor, final TableName tableName, + final byte[] row, final int rowLimit) + throws IOException { + + byte[] startRow = null; + byte[] stopRow = null; + if (tableName != null) { + startRow = + getTableStartRowForMeta(tableName, QueryType.REGION); + if (row != null) { + HRegionInfo closestRi = + getClosestRegionInfo(connection, tableName, row); + startRow = HRegionInfo + .createRegionName(tableName, closestRi.getStartKey(), HConstants.ZEROES, false); + } + stopRow = + getTableStopRowForMeta(tableName, QueryType.REGION); + } + scanMeta(connection, startRow, stopRow, QueryType.REGION, rowLimit, visitor); + } + + + /** + * Performs a scan of META table. + * @param connection connection we're using + * @param startRow Where to start the scan. Pass null if want to begin scan + * at first row. + * @param stopRow Where to stop the scan. Pass null if want to scan all rows + * from the start one + * @param type scanned part of meta + * @param maxRows maximum rows to return * @param visitor Visitor invoked against each row. - * @param startrow Where to start the scan. Pass null if want to begin scan - * at first row. - * hbase:meta, the default (pass false to scan hbase:meta) * @throws IOException */ - public static void fullScan(Connection connection, - final Visitor visitor, final byte [] startrow) - throws IOException { - Scan scan = new Scan(); - if (startrow != null) scan.setStartRow(startrow); - if (startrow == null) { - int caching = connection.getConfiguration() - .getInt(HConstants.HBASE_META_SCANNER_CACHING, 100); - scan.setCaching(caching); + public static void scanMeta(Connection connection, + @Nullable final byte[] startRow, @Nullable final byte[] stopRow, + QueryType type, int maxRows, final Visitor visitor) + throws IOException { + int rowUpperLimit = maxRows > 0 ? maxRows : Integer.MAX_VALUE; + Scan scan = getMetaScan(connection, rowUpperLimit); + + for (byte[] family : type.getFamilies()) { + scan.addFamily(family); } - scan.addFamily(HConstants.CATALOG_FAMILY); - Table metaTable = getMetaHTable(connection); - ResultScanner scanner = null; - try { - scanner = metaTable.getScanner(scan); - Result data; - while((data = scanner.next()) != null) { - if (data.isEmpty()) continue; - // Break if visit returns false. - if (!visitor.visit(data)) break; + if (startRow != null) scan.setStartRow(startRow); + if (stopRow != null) scan.setStopRow(stopRow); + + if (LOG.isTraceEnabled()) { + LOG.trace("Scanning META" + + " starting at row=" + Bytes.toStringBinary(startRow) + + " stopping at row=" + Bytes.toStringBinary(stopRow) + + " for max=" + rowUpperLimit + + " with caching=" + scan.getCaching()); + } + + int currentRow = 0; + try (Table metaTable = getMetaHTable(connection)) { + try (ResultScanner scanner = metaTable.getScanner(scan)) { + Result data; + while ((data = scanner.next()) != null) { + if (data.isEmpty()) continue; + // Break if visit returns false. + if (!visitor.visit(data)) break; + if (++currentRow >= rowUpperLimit) break; + } } - } finally { - if (scanner != null) scanner.close(); - metaTable.close(); + } + if (visitor != null && visitor instanceof Closeable) { + try { + ((Closeable) visitor).close(); + } catch (Throwable t) { + ExceptionUtil.rethrowIfInterrupt(t); + LOG.debug("Got exception in closing the meta scanner visitor", t); + } + } + } + + /** + * @return Get closest metatable region row to passed row + * @throws java.io.IOException + */ + @Nonnull + public static HRegionInfo getClosestRegionInfo(Connection connection, + @Nonnull final TableName tableName, + @Nonnull final byte[] row) + throws IOException { + byte[] searchRow = HRegionInfo.createRegionName(tableName, row, HConstants.NINES, false); + Scan scan = getMetaScan(connection, 1); + scan.setReversed(true); + scan.setStartRow(searchRow); + try (ResultScanner resultScanner = getMetaHTable(connection).getScanner(scan)) { + Result result = resultScanner.next(); + if (result == null) { + throw new TableNotFoundException("Cannot find row in META " + + " for table: " + tableName + ", row=" + Bytes.toStringBinary(row)); + } + HRegionInfo regionInfo = getHRegionInfo(result); + if (regionInfo == null) { + throw new IOException("HRegionInfo was null or empty in Meta for " + + tableName + ", row=" + Bytes.toStringBinary(row)); + } + return regionInfo; } } @@ -615,7 +813,7 @@ 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; } @@ -635,9 +833,9 @@ public class MetaTableAccessor { @VisibleForTesting public static byte[] getServerColumn(int replicaId) { return replicaId == 0 - ? HConstants.SERVER_QUALIFIER - : Bytes.toBytes(HConstants.SERVER_QUALIFIER_STR + META_REPLICA_ID_DELIMITER - + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId)); + ? HConstants.SERVER_QUALIFIER + : Bytes.toBytes(HConstants.SERVER_QUALIFIER_STR + META_REPLICA_ID_DELIMITER + + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId)); } /** @@ -648,9 +846,9 @@ public class MetaTableAccessor { @VisibleForTesting public static byte[] getStartCodeColumn(int replicaId) { return replicaId == 0 - ? HConstants.STARTCODE_QUALIFIER - : Bytes.toBytes(HConstants.STARTCODE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER - + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId)); + ? HConstants.STARTCODE_QUALIFIER + : Bytes.toBytes(HConstants.STARTCODE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER + + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId)); } /** @@ -661,9 +859,9 @@ public class MetaTableAccessor { @VisibleForTesting public static byte[] getSeqNumColumn(int replicaId) { return replicaId == 0 - ? HConstants.SEQNUM_QUALIFIER - : Bytes.toBytes(HConstants.SEQNUM_QUALIFIER_STR + META_REPLICA_ID_DELIMITER - + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId)); + ? HConstants.SEQNUM_QUALIFIER + : Bytes.toBytes(HConstants.SEQNUM_QUALIFIER_STR + META_REPLICA_ID_DELIMITER + + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId)); } /** @@ -693,17 +891,18 @@ 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()); + 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())); + Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength())); } /** @@ -713,7 +912,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 +922,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 +933,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 @@ -770,7 +970,7 @@ public class MetaTableAccessor { * @return HRegionLocation parsed from the given meta row Result for the given replicaId */ private static HRegionLocation getRegionLocation(final Result r, final HRegionInfo regionInfo, - final int replicaId) { + final int replicaId) { ServerName serverName = getServerName(r, replicaId); long seqNum = getSeqNumDuringOpen(r, replicaId); HRegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId); @@ -795,11 +995,12 @@ 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()); + cell.getValueOffset(), cell.getValueLength()); } /** @@ -844,6 +1045,12 @@ public class MetaTableAccessor { } /** + * Implementations 'visit' a catalog table row but with close() at the end. + */ + public interface CloseableVisitor extends Visitor, Closeable { + } + + /** * A {@link Visitor} that collects content out of passed {@link Result}. */ static abstract class CollectingVisitor implements Visitor { @@ -877,6 +1084,59 @@ public class MetaTableAccessor { } /** + * A Visitor that skips offline regions and split parents + */ + public static abstract class DefaultVisitorBase implements Visitor { + + public DefaultVisitorBase() { + super(); + } + + public abstract boolean visitInternal(Result rowResult) throws IOException; + + @Override + public boolean visit(Result rowResult) throws IOException { + HRegionInfo info = getHRegionInfo(rowResult); + if (info == null) { + return true; + } + + //skip over offline and split regions + if (!(info.isOffline() || info.isSplit())) { + return visitInternal(rowResult); + } + return true; + } + } + + /** + * A Visitor for a table. Provides a consistent view of the table's + * hbase:meta entries during concurrent splits (see HBASE-5986 for details). This class + * does not guarantee ordered traversal of meta entries, and can block until the + * hbase:meta entries for daughters are available during splits. + */ + public static abstract class TableVisitorBase extends DefaultVisitorBase { + private TableName tableName; + + public TableVisitorBase(TableName tableName) { + super(); + this.tableName = tableName; + } + + @Override + public final boolean visit(Result rowResult) throws IOException { + HRegionInfo info = getHRegionInfo(rowResult); + if (info == null) { + return true; + } + if (!(info.getTable().equals(tableName))) { + return false; + } + return super.visit(rowResult); + } + } + + /** * Count regions in hbase:meta for passed table. * @param c Configuration object * @param tableName table name to count regions for @@ -897,7 +1157,7 @@ public class MetaTableAccessor { * @throws IOException */ public static int getRegionCount(final Configuration c, final TableName tableName) - throws IOException { + throws IOException { try (Connection connection = ConnectionFactory.createConnection(c)) { return getRegionCount(connection, tableName); } @@ -911,7 +1171,7 @@ public class MetaTableAccessor { * @throws IOException */ public static int getRegionCount(final Connection connection, final TableName tableName) - throws IOException { + throws IOException { try (RegionLocator locator = connection.getRegionLocator(tableName)) { List locations = locator.getAllRegionLocations(); return locations == null? 0: locations.size(); @@ -926,8 +1186,9 @@ public class MetaTableAccessor { * Generates and returns a Put containing the region into for the catalog table */ public static Put makePutFromRegionInfo(HRegionInfo regionInfo) - throws IOException { - Put put = new Put(regionInfo.getRegionName()); + throws IOException { + long now = EnvironmentEdgeManager.currentTime(); + Put put = new Put(regionInfo.getRegionName(), now); addRegionInfo(put, regionInfo); return put; } @@ -940,7 +1201,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; } @@ -950,11 +1213,11 @@ public class MetaTableAccessor { public static Put addDaughtersToPut(Put put, HRegionInfo splitA, HRegionInfo splitB) { if (splitA != null) { put.addImmutable( - HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER, splitA.toByteArray()); + HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER, splitA.toByteArray()); } if (splitB != null) { put.addImmutable( - HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER, splitB.toByteArray()); + HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER, splitB.toByteArray()); } return put; } @@ -966,7 +1229,7 @@ public class MetaTableAccessor { * @throws IOException */ static void putToMetaTable(final Connection connection, final Put p) - throws IOException { + throws IOException { put(getMetaHTable(connection), p); } @@ -990,7 +1253,7 @@ public class MetaTableAccessor { * @throws IOException */ public static void putsToMetaTable(final Connection connection, final List ps) - throws IOException { + throws IOException { Table t = getMetaHTable(connection); try { t.put(ps); @@ -1006,7 +1269,7 @@ public class MetaTableAccessor { * @throws IOException */ static void deleteFromMetaTable(final Connection connection, final Delete d) - throws IOException { + throws IOException { List dels = new ArrayList(1); dels.add(d); deleteFromMetaTable(connection, dels); @@ -1019,7 +1282,7 @@ public class MetaTableAccessor { * @throws IOException */ public static void deleteFromMetaTable(final Connection connection, final List deletes) - throws IOException { + throws IOException { Table t = getMetaHTable(connection); try { t.delete(deletes); @@ -1037,18 +1300,19 @@ public class MetaTableAccessor { * @throws IOException */ public static void removeRegionReplicasFromMeta(Set metaRows, - int replicaIndexToDeleteFrom, int numReplicasToRemove, Connection connection) + int replicaIndexToDeleteFrom, int numReplicasToRemove, Connection connection) 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); } @@ -1061,8 +1325,8 @@ public class MetaTableAccessor { * @throws IOException */ public static void mutateMetaTable(final Connection connection, - final List mutations) - throws IOException { + final List mutations) + throws IOException { Table t = getMetaHTable(connection); try { t.batch(mutations); @@ -1082,8 +1346,8 @@ public class MetaTableAccessor { * @throws IOException if problem connecting or updating meta */ public static void addRegionToMeta(Connection connection, - HRegionInfo regionInfo) - throws IOException { + HRegionInfo regionInfo) + throws IOException { putToMetaTable(connection, makePutFromRegionInfo(regionInfo)); LOG.info("Added " + regionInfo.getRegionNameAsString()); } @@ -1112,7 +1376,7 @@ public class MetaTableAccessor { * @throws IOException if problem connecting or updating meta */ public static void addRegionToMeta(Table meta, HRegionInfo regionInfo, - HRegionInfo splitA, HRegionInfo splitB) throws IOException { + HRegionInfo splitA, HRegionInfo splitB) throws IOException { Put put = makePutFromRegionInfo(regionInfo); addDaughtersToPut(put, splitA, splitB); meta.put(put); @@ -1134,7 +1398,7 @@ public class MetaTableAccessor { * @throws IOException if problem connecting or updating meta */ public static void addRegionToMeta(Connection connection, HRegionInfo regionInfo, - HRegionInfo splitA, HRegionInfo splitB) throws IOException { + HRegionInfo splitA, HRegionInfo splitB) throws IOException { Table meta = getMetaHTable(connection); try { addRegionToMeta(meta, regionInfo, splitA, splitB); @@ -1150,8 +1414,8 @@ public class MetaTableAccessor { * @throws IOException if problem connecting or updating meta */ public static void addRegionsToMeta(Connection connection, - List regionInfos, int regionReplication) - throws IOException { + List regionInfos, int regionReplication) + throws IOException { List puts = new ArrayList(); for (HRegionInfo regionInfo : regionInfos) { if (RegionReplicaUtil.isDefaultReplica(regionInfo)) { @@ -1178,14 +1442,15 @@ 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()); } putToMetaTable(connection, put); LOG.info("Added daughter " + regionInfo.getEncodedName() + - (sn == null? ", serverName=null": ", serverName=" + sn.toString())); + (sn == null? ", serverName=null": ", serverName=" + sn.toString())); } /** @@ -1201,7 +1466,7 @@ public class MetaTableAccessor { */ public static void mergeRegions(final Connection connection, HRegionInfo mergedRegion, HRegionInfo regionA, HRegionInfo regionB, ServerName sn, int regionReplication) - throws IOException { + throws IOException { Table meta = getMetaHTable(connection); try { HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion); @@ -1209,9 +1474,9 @@ public class MetaTableAccessor { // Put for parent Put putOfMerged = makePutFromRegionInfo(copyOfMerged); putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER, - regionA.toByteArray()); + regionA.toByteArray()); putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER, - regionB.toByteArray()); + regionB.toByteArray()); // Deletes for merging regions Delete deleteA = makeDeleteFromRegionInfo(regionA); @@ -1227,7 +1492,7 @@ public class MetaTableAccessor { } byte[] tableRow = Bytes.toBytes(mergedRegion.getRegionNameAsString() - + HConstants.DELIMITER); + + HConstants.DELIMITER); multiMutate(meta, tableRow, putOfMerged, deleteA, deleteB); } finally { meta.close(); @@ -1246,8 +1511,8 @@ public class MetaTableAccessor { * @param sn the location of the region */ public static void splitRegion(final Connection connection, - HRegionInfo parent, HRegionInfo splitA, HRegionInfo splitB, - ServerName sn, int regionReplication) throws IOException { + HRegionInfo parent, HRegionInfo splitA, HRegionInfo splitB, + ServerName sn, int regionReplication) throws IOException { Table meta = getMetaHTable(connection); try { HRegionInfo copyOfParent = new HRegionInfo(parent); @@ -1286,22 +1551,22 @@ public class MetaTableAccessor { throws IOException { CoprocessorRpcChannel channel = table.coprocessorService(row); MultiRowMutationProtos.MutateRowsRequest.Builder mmrBuilder - = MultiRowMutationProtos.MutateRowsRequest.newBuilder(); + = MultiRowMutationProtos.MutateRowsRequest.newBuilder(); for (Mutation mutation : mutations) { if (mutation instanceof Put) { mmrBuilder.addMutationRequest(ProtobufUtil.toMutation( - ClientProtos.MutationProto.MutationType.PUT, mutation)); + ClientProtos.MutationProto.MutationType.PUT, mutation)); } else if (mutation instanceof Delete) { mmrBuilder.addMutationRequest(ProtobufUtil.toMutation( - ClientProtos.MutationProto.MutationType.DELETE, mutation)); + ClientProtos.MutationProto.MutationType.DELETE, mutation)); } else { throw new DoNotRetryIOException("multi in MetaEditor doesn't support " - + mutation.getClass().getName()); + + mutation.getClass().getName()); } } MultiRowMutationProtos.MultiRowMutationService.BlockingInterface service = - MultiRowMutationProtos.MultiRowMutationService.newBlockingStub(channel); + MultiRowMutationProtos.MultiRowMutationService.newBlockingStub(channel); try { service.mutateRows(null, mmrBuilder.build()); } catch (ServiceException ex) { @@ -1322,8 +1587,8 @@ public class MetaTableAccessor { * @throws IOException */ public static void updateRegionLocation(Connection connection, - HRegionInfo regionInfo, ServerName sn, long updateSeqNum) - throws IOException { + HRegionInfo regionInfo, ServerName sn, long updateSeqNum) + throws IOException { updateLocation(connection, regionInfo, sn, updateSeqNum); } @@ -1341,14 +1606,15 @@ public class MetaTableAccessor { * if the server is down on other end. */ private static void updateLocation(final Connection connection, - HRegionInfo regionInfo, ServerName sn, long openSeqNum) - throws IOException { + 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() + - " with server=" + sn); + " with server=" + sn); } /** @@ -1358,9 +1624,11 @@ public class MetaTableAccessor { * @throws IOException */ public static void deleteRegion(Connection connection, - HRegionInfo regionInfo) - throws IOException { + 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()); } @@ -1372,10 +1640,13 @@ public class MetaTableAccessor { * @throws IOException */ public static void deleteRegions(Connection connection, - List regionsInfo) throws IOException { + 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); @@ -1389,13 +1660,13 @@ public class MetaTableAccessor { * @throws IOException */ public static void mutateRegions(Connection connection, - final List regionsToRemove, - final List regionsToAdd) - throws IOException { + final List regionsToRemove, + final List regionsToAdd) + throws IOException { List mutation = new ArrayList(); if (regionsToRemove != null) { for (HRegionInfo hri: regionsToRemove) { - mutation.add(new Delete(hri.getRegionName())); + mutation.add(makeDeleteFromRegionInfo(hri)); } } if (regionsToAdd != null) { @@ -1437,21 +1708,22 @@ public class MetaTableAccessor { * @throws IOException */ public static void deleteMergeQualifiers(Connection connection, - final HRegionInfo mergedRegion) throws IOException { + 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=" - + Bytes.toStringBinary(HConstants.MERGEA_QUALIFIER) + " and qualifier=" - + Bytes.toStringBinary(HConstants.MERGEB_QUALIFIER)); + + mergedRegion.getRegionNameAsString() + ", qualifier=" + + Bytes.toStringBinary(HConstants.MERGEA_QUALIFIER) + " and qualifier=" + + Bytes.toStringBinary(HConstants.MERGEB_QUALIFIER)); } private static Put addRegionInfo(final Put p, final HRegionInfo hri) - throws IOException { - p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, - hri.toByteArray()); + throws IOException { + p.addImmutable(getCatalogFamily(), HConstants.REGIONINFO_QUALIFIER, + hri.toByteArray()); return p; } @@ -1459,20 +1731,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, - Bytes.toBytes(sn.getHostAndPort())); - p.addImmutable(HConstants.CATALOG_FAMILY, getStartCodeColumn(replicaId), now, - Bytes.toBytes(sn.getStartcode())); - p.addImmutable(HConstants.CATALOG_FAMILY, getSeqNumColumn(replicaId), now, - Bytes.toBytes(openSeqNum)); + p.addImmutable(getCatalogFamily(), getServerColumn(replicaId), now, + Bytes.toBytes(sn.getHostAndPort())); + p.addImmutable(getCatalogFamily(), getStartCodeColumn(replicaId), now, + Bytes.toBytes(sn.getStartcode())); + 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/ConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java index 8212e9f..044cbac 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 @@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableAccessor.DefaultVisitorBase; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.RegionTooBusyException; import org.apache.hadoop.hbase.ServerName; @@ -70,8 +71,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; @@ -959,13 +958,13 @@ class ConnectionManager { public boolean isTableAvailable(final TableName tableName) throws IOException { final AtomicBoolean available = new AtomicBoolean(true); final AtomicInteger regionCount = new AtomicInteger(0); - MetaScannerVisitor visitor = new MetaScannerVisitorBase() { + DefaultVisitorBase visitor = new DefaultVisitorBase() { @Override - public boolean processRow(Result row) throws IOException { - HRegionInfo info = MetaScanner.getHRegionInfo(row); + public boolean visitInternal(Result rowResult) throws IOException { + HRegionInfo info = MetaTableAccessor.getHRegionInfo(rowResult); if (info != null && !info.isSplitParent()) { if (tableName.equals(info.getTable())) { - ServerName server = HRegionInfo.getServerName(row); + ServerName server = HRegionInfo.getServerName(rowResult); if (server == null) { available.set(false); return false; @@ -979,7 +978,7 @@ class ConnectionManager { return true; } }; - MetaScanner.metaScan(this, visitor, tableName); + MetaTableAccessor.scanMetaForTableRegions(this, visitor, tableName); return available.get() && (regionCount.get() > 0); } @@ -993,10 +992,10 @@ class ConnectionManager { throws IOException { final AtomicBoolean available = new AtomicBoolean(true); final AtomicInteger regionCount = new AtomicInteger(0); - MetaScannerVisitor visitor = new MetaScannerVisitorBase() { + DefaultVisitorBase visitor = new DefaultVisitorBase() { @Override - public boolean processRow(Result row) throws IOException { - HRegionInfo info = MetaScanner.getHRegionInfo(row); + public boolean visitInternal(Result row) throws IOException { + HRegionInfo info = MetaTableAccessor.getHRegionInfo(row); if (info != null && !info.isSplitParent()) { if (tableName.equals(info.getTable())) { ServerName server = HRegionInfo.getServerName(row); @@ -1024,7 +1023,7 @@ class ConnectionManager { return true; } }; - MetaScanner.metaScan(this, visitor, tableName); + MetaTableAccessor.scanMetaForTableRegions(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); } @@ -1066,9 +1065,10 @@ class ConnectionManager { @Override public List locateRegions(final TableName tableName, final boolean useCache, final boolean offlined) throws IOException { - NavigableMap regions = MetaScanner.allTableRegions(this, tableName); + List regions = MetaTableAccessor + .getTableRegions(this, tableName, !offlined); final List locations = new ArrayList(); - for (HRegionInfo regionInfo : regions.keySet()) { + for (HRegionInfo regionInfo : regions) { RegionLocations list = locateRegion(tableName, regionInfo.getStartKey(), useCache, true); if (list != null) { for (HRegionLocation loc : list.getRegionLocations()) { 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 520b953..8ff44fb 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 @@ -60,8 +60,6 @@ import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; -import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; -import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.MasterCoprocessorRpcChannel; @@ -549,9 +547,9 @@ public class HBaseAdmin implements Admin { if (!doneWithMetaScan) { // Wait for new table to come on-line final AtomicInteger actualRegCount = new AtomicInteger(0); - MetaScannerVisitor visitor = new MetaScannerVisitorBase() { + MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() { @Override - public boolean processRow(Result rowResult) throws IOException { + public boolean visit(Result rowResult) throws IOException { RegionLocations list = MetaTableAccessor.getRegionLocations(rowResult); if (list == null) { LOG.warn("No serialized HRegionInfo in " + rowResult); @@ -577,7 +575,7 @@ public class HBaseAdmin implements Admin { return true; } }; - MetaScanner.metaScan(connection, visitor, desc.getTableName()); + MetaTableAccessor.scanMetaForTableRegions(connection, visitor, desc.getTableName()); if (actualRegCount.get() < numRegs) { if (tries == this.numRetries * this.retryLongerMultiplier - 1) { throw new RegionOfflineException("Only " + actualRegCount.get() + @@ -698,8 +696,8 @@ public class HBaseAdmin implements Admin { for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) { try { // Find whether all regions are deleted. - List regionLations = - MetaScanner.listTableRegionLocations(conf, connection, tableName); + List> regionLations = + MetaTableAccessor.getTableRegionsAndLocations(connection, tableName); // let us wait until hbase:meta table is updated and // HMaster removes the table from its HTableDescriptors @@ -1749,30 +1747,21 @@ public class HBaseAdmin implements Admin { */ private void compact(final TableName tableName, final byte[] columnFamily,final boolean major) throws IOException { - ZooKeeperWatcher zookeeper = null; - try { - checkTableExists(tableName); - zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(), - new ThrowableAbortable()); - List> pairs = - MetaTableAccessor.getTableRegionsAndLocations(zookeeper, connection, tableName); - for (Pair pair: pairs) { - if (pair.getFirst().isOffline()) continue; - if (pair.getSecond() == null) continue; - try { - compact(pair.getSecond(), pair.getFirst(), major, columnFamily); - } catch (NotServingRegionException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("Trying to" + (major ? " major" : "") + " compact " + - pair.getFirst() + ": " + - StringUtils.stringifyException(e)); - } + checkTableExists(tableName); + List> pairs = + MetaTableAccessor.getTableRegionsAndLocations(connection, tableName); + for (Pair pair: pairs) { + if (pair.getFirst().isOffline()) continue; + if (pair.getSecond() == null) continue; + try { + compact(pair.getSecond(), pair.getFirst(), major, columnFamily); + } catch (NotServingRegionException e) { + if (LOG.isDebugEnabled()) { + LOG.debug("Trying to" + (major ? " major" : "") + " compact " + + pair.getFirst() + ": " + + StringUtils.stringifyException(e)); } } - } finally { - if (zookeeper != null) { - zookeeper.close(); - } } } @@ -2087,7 +2076,7 @@ public class HBaseAdmin implements Admin { zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(), new ThrowableAbortable()); List> pairs = - MetaTableAccessor.getTableRegionsAndLocations(zookeeper, connection, tableName); + MetaTableAccessor.getTableRegionsAndLocations(connection, tableName); for (Pair pair: pairs) { // May not be a server for a particular row if (pair.getSecond() == null) continue; @@ -2221,9 +2210,9 @@ public class HBaseAdmin implements Admin { final AtomicReference> result = new AtomicReference>(null); final String encodedName = Bytes.toString(regionName); - MetaScannerVisitor visitor = new MetaScannerVisitorBase() { + MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() { @Override - public boolean processRow(Result data) throws IOException { + public boolean visit(Result data) throws IOException { HRegionInfo info = HRegionInfo.getHRegionInfo(data); if (info == null) { LOG.warn("No serialized HRegionInfo in " + data); @@ -2232,11 +2221,13 @@ public class HBaseAdmin implements Admin { RegionLocations rl = MetaTableAccessor.getRegionLocations(data); boolean matched = false; ServerName sn = null; - for (HRegionLocation h : rl.getRegionLocations()) { - if (h != null && encodedName.equals(h.getRegionInfo().getEncodedName())) { - sn = h.getServerName(); - info = h.getRegionInfo(); - matched = true; + if (rl != null) { + for (HRegionLocation h : rl.getRegionLocations()) { + if (h != null && encodedName.equals(h.getRegionInfo().getEncodedName())) { + sn = h.getServerName(); + info = h.getRegionInfo(); + matched = true; + } } } if (!matched) return true; @@ -2245,7 +2236,7 @@ public class HBaseAdmin implements Admin { } }; - MetaScanner.metaScan(connection, visitor, null); + MetaTableAccessor.fullScanRegions(connection, visitor); pair = result.get(); } return pair; @@ -2563,16 +2554,7 @@ public class HBaseAdmin implements Admin { @Override public List getTableRegions(final TableName tableName) throws IOException { - ZooKeeperWatcher zookeeper = - new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(), - new ThrowableAbortable()); - List Regions = null; - try { - Regions = MetaTableAccessor.getTableRegions(zookeeper, connection, tableName, true); - } finally { - zookeeper.close(); - } - return Regions; + return MetaTableAccessor.getTableRegions(connection, tableName, true); } public List getTableRegions(final byte[] tableName) @@ -2717,13 +2699,10 @@ public class HBaseAdmin implements Admin { public CompactionState getCompactionState(final TableName tableName) throws IOException { CompactionState state = CompactionState.NONE; - ZooKeeperWatcher zookeeper = - new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(), - new ThrowableAbortable()); try { checkTableExists(tableName); List> pairs = - MetaTableAccessor.getTableRegionsAndLocations(zookeeper, connection, tableName); + MetaTableAccessor.getTableRegionsAndLocations(connection, tableName); for (Pair pair: pairs) { if (pair.getFirst().isOffline()) continue; if (pair.getSecond() == null) continue; @@ -2770,8 +2749,6 @@ public class HBaseAdmin implements Admin { } } catch (ServiceException se) { throw ProtobufUtil.getRemoteException(se); - } finally { - zookeeper.close(); } return state; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java index fa85653..782ab66 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java @@ -21,12 +21,12 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.NavigableMap; -import java.util.Map.Entry; +import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -34,8 +34,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.util.Pair; -import com.google.common.annotations.VisibleForTesting; - /** * An implementation of {@link RegionLocator}. Used to view region location information for a single * HBase table. Lightweight. Get as needed and just close when done. Instances of this class SHOULD @@ -85,11 +83,11 @@ public class HRegionLocator implements RegionLocator { @Override public List getAllRegionLocations() throws IOException { - NavigableMap locations = - MetaScanner.allTableRegions(this.connection, getName()); + List> locations = + MetaTableAccessor.getTableRegionsAndLocations(this.connection, getName()); ArrayList regions = new ArrayList<>(locations.size()); - for (Entry entry : locations.entrySet()) { - regions.add(new HRegionLocation(entry.getKey(), entry.getValue())); + for (Pair entry : locations) { + regions.add(new HRegionLocation(entry.getFirst(), entry.getSecond())); } return regions; } @@ -139,7 +137,18 @@ public class HRegionLocator implements RegionLocator { @VisibleForTesting List listRegionLocations() throws IOException { - return MetaScanner.listTableRegionLocations(getConfiguration(), this.connection, getName()); + final List regions = new ArrayList(); + MetaTableAccessor.Visitor visitor = new MetaTableAccessor.TableVisitorBase(tableName) { + @Override + public boolean visitInternal(Result result) throws IOException { + RegionLocations locations = MetaTableAccessor.getRegionLocations(result); + if (locations == null) return true; + regions.add(locations); + return true; + } + }; + MetaTableAccessor.scanMetaForTableRegions(connection, visitor, tableName); + return regions; } public Configuration getConfiguration() { 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 e8518bd..b9bf89e 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 @@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; @@ -632,7 +633,7 @@ public class HTable implements HTableInterface { @Deprecated public NavigableMap getRegionLocations() throws IOException { // TODO: Odd that this returns a Map of HRI to SN whereas getRegionLocator, singular, returns an HRegionLocation. - return MetaScanner.allTableRegions(this.connection, getName()); + return MetaTableAccessor.allTableRegions(this.connection, getName()); } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java deleted file mode 100644 index 706085c..0000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java +++ /dev/null @@ -1,422 +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.client; - -import java.io.Closeable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.NavigableMap; -import java.util.TreeMap; - -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.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.RegionLocations; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.ExceptionUtil; - -import com.google.common.annotations.VisibleForTesting; - -/** - * Scanner class that contains the hbase:meta table scanning logic. - * Provided visitors will be called for each row. - * - * Although public visibility, this is not a public-facing API and may evolve in - * minor releases. - * - *

Note that during concurrent region splits, the scanner might not see - * hbase:meta changes across rows (for parent and daughter entries) consistently. - * see HBASE-5986, and {@link DefaultMetaScannerVisitor} for details.

- */ -@InterfaceAudience.Private -//TODO: merge this to MetaTableAccessor, get rid of it. -public class MetaScanner { - private static final Log LOG = LogFactory.getLog(MetaScanner.class); - /** - * Scans the meta table and calls a visitor on each RowResult and uses a empty - * start row value as table name. - * - *

Visible for testing. Use {@link - * #metaScan(Connection, MetaScannerVisitor, TableName)} instead. - * - * @param visitor A custom visitor - * @throws IOException e - */ - @VisibleForTesting // Do not use. Used by tests only and hbck. - public static void metaScan(Connection connection, - MetaScannerVisitor visitor) throws IOException { - metaScan(connection, visitor, null, null, Integer.MAX_VALUE); - } - - /** - * Scans the meta table and calls a visitor on each RowResult. Uses a table - * name to locate meta regions. - * - * @param connection connection to use internally (null to use a new instance) - * @param visitor visitor object - * @param userTableName User table name in meta table to start scan at. Pass - * null if not interested in a particular table. - * @throws IOException e - */ - public static void metaScan(Connection connection, - MetaScannerVisitor visitor, TableName userTableName) throws IOException { - metaScan(connection, visitor, userTableName, null, Integer.MAX_VALUE, - TableName.META_TABLE_NAME); - } - - /** - * Scans the meta table and calls a visitor on each RowResult. Uses a table - * name and a row name to locate meta regions. And it only scans at most - * rowLimit of rows. - * - *

Visible for testing. Use {@link - * #metaScan(Connection, MetaScannerVisitor, TableName)} instead. - * - * @param connection to scan on - * @param visitor Visitor object. - * @param userTableName User table name in meta table to start scan at. Pass - * null if not interested in a particular table. - * @param row Name of the row at the user table. The scan will start from - * the region row where the row resides. - * @param rowLimit Max of processed rows. If it is less than 0, it - * will be set to default value Integer.MAX_VALUE. - * @throws IOException e - */ - @VisibleForTesting // Do not use. Used by Master but by a method that is used testing. - public static void metaScan(Connection connection, - MetaScannerVisitor visitor, TableName userTableName, byte[] row, - int rowLimit) - throws IOException { - metaScan(connection, visitor, userTableName, row, rowLimit, TableName - .META_TABLE_NAME); - } - - /** - * Scans the meta table and calls a visitor on each RowResult. Uses a table - * name and a row name to locate meta regions. And it only scans at most - * rowLimit of rows. - * - * @param connection connection to use internally (null to use a new instance) - * @param visitor Visitor object. Closes the visitor before returning. - * @param tableName User table name in meta table to start scan at. Pass - * null if not interested in a particular table. - * @param row Name of the row at the user table. The scan will start from - * the region row where the row resides. - * @param rowLimit Max of processed rows. If it is less than 0, it - * will be set to default value Integer.MAX_VALUE. - * @param metaTableName Meta table to scan, root or meta. - * @throws IOException e - */ - static void metaScan(Connection connection, - final MetaScannerVisitor visitor, final TableName tableName, - final byte[] row, final int rowLimit, final TableName metaTableName) - throws IOException { - - int rowUpperLimit = rowLimit > 0 ? rowLimit: Integer.MAX_VALUE; - // Calculate startrow for scan. - byte[] startRow; - // If the passed in 'connection' is 'managed' -- i.e. every second test uses - // an HTable or an HBaseAdmin with managed connections -- then doing - // connection.getTable will throw an exception saying you are NOT to use - // managed connections getting tables. Leaving this as it is for now. Will - // revisit when inclined to change all tests. User code probaby makes use of - // managed connections too so don't change it till post hbase 1.0. - try (Table metaTable = new HTable(TableName.META_TABLE_NAME, connection, null)) { - if (row != null) { - // Scan starting at a particular row in a particular table - Result startRowResult = getClosestRowOrBefore(metaTable, tableName, row, - connection.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS, - HConstants.DEFAULT_USE_META_REPLICAS)); - if (startRowResult == null) { - throw new TableNotFoundException("Cannot find row in " + metaTable.getName() + - " for table: " + tableName + ", row=" + Bytes.toStringBinary(row)); - } - HRegionInfo regionInfo = getHRegionInfo(startRowResult); - if (regionInfo == null) { - throw new IOException("HRegionInfo was null or empty in Meta for " + - tableName + ", row=" + Bytes.toStringBinary(row)); - } - byte[] rowBefore = regionInfo.getStartKey(); - startRow = HRegionInfo.createRegionName(tableName, rowBefore, HConstants.ZEROES, false); - } else if (tableName == null || tableName.getName().length == 0) { - // Full hbase:meta scan - startRow = HConstants.EMPTY_START_ROW; - } else { - // Scan hbase:meta for an entire table - startRow = HRegionInfo.createRegionName(tableName, HConstants.EMPTY_START_ROW, - HConstants.ZEROES, false); - } - final Scan scan = new Scan(startRow).addFamily(HConstants.CATALOG_FAMILY); - int scannerCaching = connection.getConfiguration() - .getInt(HConstants.HBASE_META_SCANNER_CACHING, - HConstants.DEFAULT_HBASE_META_SCANNER_CACHING); - if (connection.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS, - HConstants.DEFAULT_USE_META_REPLICAS)) { - scan.setConsistency(Consistency.TIMELINE); - } - if (rowUpperLimit <= scannerCaching) { - scan.setSmall(true); - } - int rows = Math.min(rowLimit, scannerCaching); - scan.setCaching(rows); - if (LOG.isTraceEnabled()) { - LOG.trace("Scanning " + metaTableName.getNameAsString() + " starting at row=" + - Bytes.toStringBinary(startRow) + " for max=" + rowUpperLimit + " with caching=" + rows); - } - // Run the scan - try (ResultScanner resultScanner = metaTable.getScanner(scan)) { - Result result; - int processedRows = 0; - while ((result = resultScanner.next()) != null) { - if (visitor != null) { - if (!visitor.processRow(result)) break; - } - processedRows++; - if (processedRows >= rowUpperLimit) break; - } - } - } finally { - if (visitor != null) { - try { - visitor.close(); - } catch (Throwable t) { - ExceptionUtil.rethrowIfInterrupt(t); - LOG.debug("Got exception in closing the meta scanner visitor", t); - } - } - } - } - - /** - * @return Get closest metatable region row to passed row - * @throws IOException - */ - private static Result getClosestRowOrBefore(final Table metaTable, final TableName userTableName, - final byte [] row, boolean useMetaReplicas) - throws IOException { - byte[] searchRow = HRegionInfo.createRegionName(userTableName, row, HConstants.NINES, false); - Scan scan = Scan.createGetClosestRowOrBeforeReverseScan(searchRow); - if (useMetaReplicas) { - scan.setConsistency(Consistency.TIMELINE); - } - try (ResultScanner resultScanner = metaTable.getScanner(scan)) { - return resultScanner.next(); - } - } - - /** - * Returns HRegionInfo object from the column - * HConstants.CATALOG_FAMILY:HConstants.REGIONINFO_QUALIFIER of the catalog - * table Result. - * @param data a Result object from the catalog table scan - * @return HRegionInfo or null - * @deprecated Use {@link org.apache.hadoop.hbase.MetaTableAccessor#getRegionLocations(Result)} - */ - @Deprecated - public static HRegionInfo getHRegionInfo(Result data) { - return HRegionInfo.getHRegionInfo(data); - } - - /** - * Lists all of the regions currently in META. - * @param conf configuration - * @param connection to connect with - * @param offlined True if we are to include offlined regions, false and we'll - * leave out offlined regions from returned list. - * @return List of all user-space regions. - * @throws IOException - */ - @VisibleForTesting // And for hbck. - public static List listAllRegions(Configuration conf, Connection connection, - final boolean offlined) - throws IOException { - final List regions = new ArrayList(); - MetaScannerVisitor visitor = new MetaScannerVisitorBase() { - @Override - public boolean processRow(Result result) throws IOException { - if (result == null || result.isEmpty()) { - return true; - } - - RegionLocations locations = MetaTableAccessor.getRegionLocations(result); - if (locations == null) return true; - for (HRegionLocation loc : locations.getRegionLocations()) { - if (loc != null) { - HRegionInfo regionInfo = loc.getRegionInfo(); - // If region offline AND we are not to include offlined regions, return. - if (regionInfo.isOffline() && !offlined) continue; - regions.add(regionInfo); - } - } - return true; - } - }; - metaScan(connection, visitor); - return regions; - } - - /** - * Lists all of the table regions currently in META. - * @param conf - * @param offlined True if we are to include offlined regions, false and we'll - * leave out offlined regions from returned list. - * @return Map of all user-space regions to servers - * @throws IOException - * @deprecated Use {@link #allTableRegions(Connection, TableName)} instead - */ - @Deprecated - public static NavigableMap allTableRegions(Configuration conf, - Connection connection, final TableName tableName, boolean offlined) throws IOException { - return allTableRegions(connection, tableName); - } - - /** - * Lists all of the table regions currently in META. - * @param connection - * @param tableName - * @return Map of all user-space regions to servers - * @throws IOException - */ - public static NavigableMap allTableRegions( - Connection connection, final TableName tableName) throws IOException { - final NavigableMap regions = - new TreeMap(); - MetaScannerVisitor visitor = new TableMetaScannerVisitor(tableName) { - @Override - public boolean processRowInternal(Result result) throws IOException { - RegionLocations locations = MetaTableAccessor.getRegionLocations(result); - if (locations == null) return true; - for (HRegionLocation loc : locations.getRegionLocations()) { - if (loc != null) { - HRegionInfo regionInfo = loc.getRegionInfo(); - regions.put(new UnmodifyableHRegionInfo(regionInfo), loc.getServerName()); - } - } - return true; - } - }; - metaScan(connection, visitor, tableName); - return regions; - } - - /** - * Lists table regions and locations grouped by region range from META. - */ - public static List listTableRegionLocations(Configuration conf, - Connection connection, final TableName tableName) throws IOException { - final List regions = new ArrayList(); - MetaScannerVisitor visitor = new TableMetaScannerVisitor(tableName) { - @Override - public boolean processRowInternal(Result result) throws IOException { - RegionLocations locations = MetaTableAccessor.getRegionLocations(result); - if (locations == null) return true; - regions.add(locations); - return true; - } - }; - metaScan(connection, visitor, tableName); - return regions; - } - - /** - * Visitor class called to process each row of the hbase:meta table - */ - public interface MetaScannerVisitor extends Closeable { - /** - * Visitor method that accepts a RowResult and the meta region location. - * Implementations can return false to stop the region's loop if it becomes - * unnecessary for some reason. - * - * @param rowResult result - * @return A boolean to know if it should continue to loop in the region - * @throws IOException e - */ - boolean processRow(Result rowResult) throws IOException; - } - - public static abstract class MetaScannerVisitorBase implements MetaScannerVisitor { - @Override - public void close() throws IOException { - } - } - - /** - * A MetaScannerVisitor that skips offline regions and split parents - */ - public static abstract class DefaultMetaScannerVisitor - extends MetaScannerVisitorBase { - - public DefaultMetaScannerVisitor() { - super(); - } - - public abstract boolean processRowInternal(Result rowResult) throws IOException; - - @Override - public boolean processRow(Result rowResult) throws IOException { - HRegionInfo info = getHRegionInfo(rowResult); - if (info == null) { - return true; - } - - //skip over offline and split regions - if (!(info.isOffline() || info.isSplit())) { - return processRowInternal(rowResult); - } - return true; - } - } - - /** - * A MetaScannerVisitor for a table. Provides a consistent view of the table's - * hbase:meta entries during concurrent splits (see HBASE-5986 for details). This class - * does not guarantee ordered traversal of meta entries, and can block until the - * hbase:meta entries for daughters are available during splits. - */ - public static abstract class TableMetaScannerVisitor extends DefaultMetaScannerVisitor { - private TableName tableName; - - public TableMetaScannerVisitor(TableName tableName) { - super(); - this.tableName = tableName; - } - - @Override - public final boolean processRow(Result rowResult) throws IOException { - HRegionInfo info = getHRegionInfo(rowResult); - if (info == null) { - return true; - } - if (!(info.getTable().equals(tableName))) { - return false; - } - return super.processRow(rowResult); - } - } -} 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 2d50c1b..09089f1 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 @@ -33,6 +33,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.commons.lang.NotImplementedException; import org.apache.commons.logging.Log; @@ -206,11 +207,11 @@ public class TestClientNoCluster extends Configured implements Tool { } @Test - public void testDoNotRetryMetaScanner() throws IOException { + public void testDoNotRetryMetaTableAccessor() throws IOException { this.conf.set("hbase.client.connection.impl", RegionServerStoppedOnScannerOpenConnection.class.getName()); try (Connection connection = ConnectionFactory.createConnection(conf)) { - MetaScanner.metaScan(connection, null); + MetaTableAccessor.fullScanRegions(connection); } } diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java index 001c6b5..48721bb 100644 --- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java @@ -33,6 +33,7 @@ import javax.ws.rs.core.Response.ResponseBuilder; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HRegionInfo; @@ -40,7 +41,6 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.MetaScanner; import org.apache.hadoop.hbase.rest.model.TableInfoModel; import org.apache.hadoop.hbase.rest.model.TableRegionModel; @@ -80,7 +80,9 @@ public class RegionsResource extends ResourceBase { TableInfoModel model = new TableInfoModel(tableName.getNameAsString()); Connection connection = ConnectionFactory.createConnection(servlet.getConfiguration()); - Map regions = MetaScanner.allTableRegions(connection, tableName); + @SuppressWarnings("deprecation") + Map regions = MetaTableAccessor + .allTableRegions(connection, tableName); connection.close(); for (Map.Entry e: regions.entrySet()) { HRegionInfo hri = e.getKey(); 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 13bebd3..ba9397f 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/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java index 3e895b4..9b71f6e 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 @@ -413,8 +413,7 @@ public class AssignmentManager extends ZooKeeperListener { if (TableName.META_TABLE_NAME.equals(tableName)) { hris = new MetaTableLocator().getMetaRegions(server.getZooKeeper()); } else { - hris = MetaTableAccessor.getTableRegions(server.getZooKeeper(), - server.getConnection(), tableName, true); + hris = MetaTableAccessor.getTableRegions(server.getConnection(), tableName, true); } Integer pending = 0; @@ -2850,7 +2849,7 @@ public class AssignmentManager extends ZooKeeperListener { ZooKeeperProtos.Table.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 diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java index 9d18c98..84c285e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java @@ -42,8 +42,6 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.HFileArchiver; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.MetaScanner; -import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.util.Bytes; @@ -142,9 +140,9 @@ public class CatalogJanitor extends ScheduledChore { final Map mergedRegions = new TreeMap(); // This visitor collects split parents and counts rows in the hbase:meta table - MetaScannerVisitor visitor = new MetaScanner.MetaScannerVisitorBase() { + MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() { @Override - public boolean processRow(Result r) throws IOException { + public boolean visit(Result r) throws IOException { if (r == null || r.isEmpty()) return true; count.incrementAndGet(); HRegionInfo info = HRegionInfo.getHRegionInfo(r); @@ -165,7 +163,7 @@ public class CatalogJanitor extends ScheduledChore { // Run full scan of hbase:meta catalog table passing in our custom visitor with // the start row - MetaScanner.metaScan(this.connection, visitor, tableName); + MetaTableAccessor.scanMetaForTableRegions(this.connection, visitor, tableName); return new Triple, Map>( count.get(), mergedRegions, splitParents); 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 0e654b3..5138a14 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 @@ -18,6 +18,10 @@ */ package org.apache.hadoop.hbase.master; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; import java.io.IOException; import java.io.InterruptedIOException; import java.lang.reflect.Constructor; @@ -38,11 +42,10 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Pattern; -import javax.servlet.ServletException; -import javax.servlet.http.HttpServlet; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; - +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Maps; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Service; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -72,9 +75,6 @@ import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.client.MetaScanner; -import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; -import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; @@ -140,11 +140,6 @@ import org.mortbay.jetty.Connector; import org.mortbay.jetty.nio.SelectChannelConnector; import org.mortbay.jetty.servlet.Context; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Maps; -import com.google.protobuf.Descriptors; -import com.google.protobuf.Service; - /** * HMaster is the "master server" for HBase. An HBase cluster has one active * master. If many masters are started, all compete. Whichever wins goes on to @@ -1674,10 +1669,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server { final AtomicReference> result = new AtomicReference>(null); - MetaScannerVisitor visitor = - new MetaScannerVisitorBase() { + MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() { @Override - public boolean processRow(Result data) throws IOException { + public boolean visit(Result data) throws IOException { if (data == null || data.size() <= 0) { return true; } @@ -1693,7 +1687,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } }; - MetaScanner.metaScan(clusterConnection, visitor, tableName, rowKey, 1); + MetaTableAccessor.scanMeta(clusterConnection, visitor, tableName, rowKey, 1); return result.get(); } 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 b03611c..39beba8 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/handler/DeleteTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java index e5ea91f..19b81a5 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 @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.HFileArchiver; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -142,9 +143,10 @@ public class DeleteTableHandler extends TableEventHandler { * @throws IOException */ private void cleanAnyRemainingRows() throws IOException { - Scan tableScan = MetaTableAccessor.getScanForTableName(tableName); + ClusterConnection connection = this.masterServices.getConnection(); + Scan tableScan = MetaTableAccessor.getScanForTableName(connection, tableName); try (Table metaTable = - this.masterServices.getConnection().getTable(TableName.META_TABLE_NAME)) { + connection.getTable(TableName.META_TABLE_NAME)) { List deletes = new ArrayList(); try (ResultScanner resScanner = metaTable.getScanner(tableScan)) { for (Result result : resScanner) { 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 243ec2d..8434f8f 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 @@ -195,8 +195,8 @@ public class EnableTableHandler extends EventHandler { tableRegionsAndLocations = new MetaTableLocator().getMetaRegionsAndLocations( server.getZooKeeper()); } else { - tableRegionsAndLocations = MetaTableAccessor.getTableRegionsAndLocations( - server.getZooKeeper(), server.getConnection(), tableName, true); + tableRegionsAndLocations = MetaTableAccessor + .getTableRegionsAndLocations(server.getConnection(), tableName, true); } int countOfRegionsInTable = tableRegionsAndLocations.size(); 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 ae40ee5..50cc4a0 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 @@ -98,9 +98,9 @@ public class ModifyTableHandler extends TableEventHandler { TableName table) throws IOException { if (newReplicaCount >= oldReplicaCount) return; Set tableRows = new HashSet(); - Scan scan = MetaTableAccessor.getScanForTableName(table); - scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); Connection connection = this.masterServices.getConnection(); + Scan scan = MetaTableAccessor.getScanForTableName(connection, table); + scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); try (Table metaTable = connection.getTable(TableName.META_TABLE_NAME)) { ResultScanner resScanner = metaTable.getScanner(scan); for (Result result : resScanner) { 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 1b141fc..46a9c9c 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 @@ -130,8 +130,7 @@ public abstract class TableEventHandler extends EventHandler { if (TableName.META_TABLE_NAME.equals(tableName)) { hris = new MetaTableLocator().getMetaRegions(server.getZooKeeper()); } else { - hris = MetaTableAccessor.getTableRegions(server.getZooKeeper(), - server.getConnection(), tableName); + hris = MetaTableAccessor.getTableRegions(server.getConnection(), tableName); } handleTableOperation(hris); if (eventType.isOnlineSchemaChangeSupported() && this.masterServices. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java index 2d7fbb7..005a5e3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java @@ -155,8 +155,7 @@ public final class MasterSnapshotVerifier { if (TableName.META_TABLE_NAME.equals(tableName)) { regions = new MetaTableLocator().getMetaRegions(services.getZooKeeper()); } else { - regions = MetaTableAccessor.getTableRegions(services.getZooKeeper(), - services.getConnection(), tableName); + regions = MetaTableAccessor.getTableRegions(services.getConnection(), tableName); } // Remove the non-default regions RegionReplicaUtil.removeNonDefaultRegions(regions); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java index 5ac9cbc..20d328c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java @@ -173,8 +173,8 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh regionsAndLocations = new MetaTableLocator().getMetaRegionsAndLocations( server.getZooKeeper()); } else { - regionsAndLocations = MetaTableAccessor.getTableRegionsAndLocations( - server.getZooKeeper(), server.getConnection(), snapshotTable, false); + regionsAndLocations = MetaTableAccessor + .getTableRegionsAndLocations(server.getConnection(), snapshotTable, false); } // run the snapshot diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java index 6a48eb6..5d56254 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java @@ -132,8 +132,8 @@ public class MasterFlushTableProcedureManager extends MasterProcedureManager { regionsAndLocations = new MetaTableLocator().getMetaRegionsAndLocations( master.getZooKeeper()); } else { - regionsAndLocations = MetaTableAccessor.getTableRegionsAndLocations( - master.getZooKeeper(), master.getConnection(), tableName, false); + regionsAndLocations = MetaTableAccessor + .getTableRegionsAndLocations(master.getConnection(), tableName, false); } Set regionServers = new HashSet(regionsAndLocations.size()); 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 dcbb8f1..6baf812 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 @@ -92,8 +92,6 @@ 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; @@ -103,9 +101,6 @@ 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; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.Result; @@ -712,6 +707,11 @@ public class HBaseFsck extends Configured implements Closeable { @Override public void close() throws IOException { + try { + unlockHbck(); + } catch (Exception io) { + LOG.warn(io); + } IOUtils.cleanup(null, admin, meta, connection); } @@ -734,7 +734,7 @@ public class HBaseFsck extends Configured implements Closeable { public void checkRegionBoundaries() { try { ByteArrayComparator comparator = new ByteArrayComparator(); - List regions = MetaScanner.listAllRegions(getConf(), connection, false); + List regions = MetaTableAccessor.getAllRegions(connection, true); final RegionBoundariesInformation currentRegionBoundariesInformation = new RegionBoundariesInformation(); Path hbaseRoot = FSUtils.getRootDir(getConf()); @@ -3238,7 +3238,7 @@ public class HBaseFsck extends Configured implements Closeable { * @throws IOException if an error is encountered */ boolean loadMetaEntries() throws IOException { - MetaScannerVisitor visitor = new MetaScannerVisitorBase() { + MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() { int countRecord = 1; // comparator to sort KeyValues with latest modtime @@ -3250,7 +3250,7 @@ public class HBaseFsck extends Configured implements Closeable { }; @Override - public boolean processRow(Result result) throws IOException { + public boolean visit(Result result) throws IOException { try { // record the latest modification of this META record @@ -3322,7 +3322,7 @@ public class HBaseFsck extends Configured implements Closeable { }; if (!checkMetaOnly) { // Scan hbase:meta to pick up user regions - MetaScanner.metaScan(connection, visitor); + MetaTableAccessor.fullScanRegions(connection, visitor); } errors.print(""); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSizeCalculator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSizeCalculator.java index 4f7c0a5..9cd24f6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSizeCalculator.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSizeCalculator.java @@ -88,6 +88,11 @@ public class RegionSizeCalculator { return; } + if (regionLocator.getName().isSystemTable()) { + LOG.info("Region size calculation disabled for system tables."); + return; + } + LOG.info("Calculating region sizes for table \"" + regionLocator.getName() + "\"."); //get regions for table 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 ab4cf6e..926c357 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 @@ -124,6 +124,10 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + /** * Facility for testing HBase. Replacement for * old HBaseTestCase and HBaseClusterTestCase functionality. @@ -2867,8 +2871,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { getHBaseCluster().getMaster().getAssignmentManager().getRegionStates() .getRegionAssignments(); final List> metaLocations = - MetaTableAccessor - .getTableRegionsAndLocations(getZooKeeperWatcher(), connection, tableName); + MetaTableAccessor.getTableRegionsAndLocations(connection, tableName); for (Pair metaLocation : metaLocations) { HRegionInfo hri = metaLocation.getFirst(); ServerName sn = metaLocation.getSecond(); 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 3845bcd..f1c7402 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 bf7a3cd..ba37a90 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 @@ -23,6 +23,12 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import java.io.IOException; import java.util.List; @@ -208,10 +214,10 @@ public class TestMetaTableAccessor { @Test public void testGetRegionsCatalogTables() throws IOException, InterruptedException { List regions = - MetaTableAccessor.getTableRegions(UTIL.getZooKeeperWatcher(), + MetaTableAccessor.getTableRegions( connection, TableName.META_TABLE_NAME); assertTrue(regions.size() >= 1); - assertTrue(MetaTableAccessor.getTableRegionsAndLocations(UTIL.getZooKeeperWatcher(), + assertTrue(MetaTableAccessor.getTableRegionsAndLocations( connection,TableName.META_TABLE_NAME).size() >= 1); } @@ -259,16 +265,16 @@ public class TestMetaTableAccessor { // Now make sure we only get the regions from 1 of the tables at a time - assertEquals(1, MetaTableAccessor.getTableRegions(UTIL.getZooKeeperWatcher(), + assertEquals(1, MetaTableAccessor.getTableRegions( connection, name).size()); - assertEquals(1, MetaTableAccessor.getTableRegions(UTIL.getZooKeeperWatcher(), + assertEquals(1, MetaTableAccessor.getTableRegions( connection, greaterName).size()); } private static List testGettingTableRegions(final Connection connection, final TableName name, final int regionCount) throws IOException, InterruptedException { - List regions = MetaTableAccessor.getTableRegions(UTIL.getZooKeeperWatcher(), + List regions = MetaTableAccessor.getTableRegions( connection, name); assertEquals(regionCount, regions.size()); Pair pair = @@ -391,7 +397,7 @@ public class TestMetaTableAccessor { Get get = new Get(row); Result result = meta.get(get); Cell serverCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY, - MetaTableAccessor.getServerColumn(replicaId)); + MetaTableAccessor.getServerColumn(replicaId)); Cell startCodeCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(replicaId)); assertNotNull(serverCell); @@ -471,5 +477,49 @@ public class TestMetaTableAccessor { meta.close(); } } + + @Test + public void testMetaScanner() throws Exception { + LOG.info("Starting testMetaScanner"); + + final TableName TABLENAME = TableName.valueOf("testMetaScanner"); + final byte[] FAMILY = Bytes.toBytes("family"); + final byte[][] SPLIT_KEYS = + new byte[][] { Bytes.toBytes("region_a"), Bytes.toBytes("region_b") }; + + UTIL.createTable(TABLENAME, FAMILY, SPLIT_KEYS); + HTable table = (HTable) connection.getTable(TABLENAME); + // Make sure all the regions are deployed + UTIL.countRows(table); + + MetaTableAccessor.Visitor visitor = + mock(MetaTableAccessor.Visitor.class); + doReturn(true).when(visitor).visit((Result) anyObject()); + + // Scanning the entire table should give us three rows + MetaTableAccessor.scanMetaForTableRegions(connection, visitor, TABLENAME); + verify(visitor, times(3)).visit((Result) anyObject()); + + // Scanning the table with a specified empty start row should also + // give us three hbase:meta rows + reset(visitor); + doReturn(true).when(visitor).visit((Result) anyObject()); + MetaTableAccessor.scanMeta(connection, visitor, TABLENAME, null, 1000); + verify(visitor, times(3)).visit((Result) anyObject()); + + // Scanning the table starting in the middle should give us two rows: + // region_a and region_b + reset(visitor); + doReturn(true).when(visitor).visit((Result) anyObject()); + MetaTableAccessor.scanMeta(connection, visitor, TABLENAME, Bytes.toBytes("region_ac"), 1000); + verify(visitor, times(2)).visit((Result) anyObject()); + + // Scanning with a limit of 1 should only give us one row + reset(visitor); + doReturn(true).when(visitor).visit((Result) anyObject()); + MetaTableAccessor.scanMeta(connection, visitor, TABLENAME, Bytes.toBytes("region_ac"), 1); + verify(visitor, times(1)).visit((Result) anyObject()); + table.close(); + } } 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 400d929..ea1b3b2 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 @@ -1196,7 +1196,7 @@ public class TestAdmin1 { ht.flushCommits(); ht.close(); List> regions = - MetaTableAccessor.getTableRegionsAndLocations(TEST_UTIL.getZooKeeperWatcher(), + MetaTableAccessor.getTableRegionsAndLocations( TEST_UTIL.getConnection(), tableName); boolean gotException = false; // the element at index 1 would be a replica (since the metareader gives us ordered diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java index e907549..3890608 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java @@ -26,6 +26,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; @@ -63,21 +64,22 @@ public class TestHTableMultiplexerFlushCache { TEST_UTIL.shutdownMiniCluster(); } - private static void checkExistence(HTable htable, byte[] row, byte[] family, byte[] quality, - byte[] value) throws Exception { + private static void checkExistence(final HTable htable, final byte[] row, final byte[] family, + final byte[] quality, + final byte[] value) throws Exception { // verify that the Get returns the correct result - Result r; - Get get = new Get(row); - get.addColumn(family, quality); - int nbTry = 0; - do { - assertTrue("Fail to get from " + htable.getName() + " after " + nbTry + " tries", nbTry < 50); - nbTry++; - Thread.sleep(100); - r = htable.get(get); - } while (r == null || r.getValue(family, quality) == null); - assertEquals("value", Bytes.toStringBinary(value), - Bytes.toStringBinary(r.getValue(family, quality))); + TEST_UTIL.waitFor(30000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + Result r; + Get get = new Get(row); + get.addColumn(family, quality); + r = htable.get(get); + return r != null && r.getValue(family, quality) != null + && Bytes.toStringBinary(value).equals( + Bytes.toStringBinary(r.getValue(family, quality))); + } + }); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java deleted file mode 100644 index bff9c78..0000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java +++ /dev/null @@ -1,242 +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.client; - -import static org.mockito.Matchers.anyObject; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.reset; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; - -import java.math.BigDecimal; -import java.util.List; -import java.util.NavigableMap; -import java.util.Random; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.StoppableImplementation; -import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.util.StringUtils; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -@Category(MediumTests.class) -public class TestMetaScanner { - final Log LOG = LogFactory.getLog(getClass()); - private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private Connection connection; - - public void setUp() throws Exception { - TEST_UTIL.startMiniCluster(1); - this.connection = TEST_UTIL.getConnection(); - } - - @After - public void tearDown() throws Exception { - TEST_UTIL.shutdownMiniCluster(); - } - - @Test - public void testMetaScanner() throws Exception { - LOG.info("Starting testMetaScanner"); - - setUp(); - final TableName TABLENAME = TableName.valueOf("testMetaScanner"); - final byte[] FAMILY = Bytes.toBytes("family"); - final byte[][] SPLIT_KEYS = - new byte[][] { Bytes.toBytes("region_a"), Bytes.toBytes("region_b") }; - - TEST_UTIL.createTable(TABLENAME, FAMILY, SPLIT_KEYS); - HTable table = (HTable) connection.getTable(TABLENAME); - // Make sure all the regions are deployed - TEST_UTIL.countRows(table); - - MetaScanner.MetaScannerVisitor visitor = - mock(MetaScanner.MetaScannerVisitor.class); - doReturn(true).when(visitor).processRow((Result)anyObject()); - - // Scanning the entire table should give us three rows - MetaScanner.metaScan(connection, visitor, TABLENAME); - verify(visitor, times(3)).processRow((Result)anyObject()); - - // Scanning the table with a specified empty start row should also - // give us three hbase:meta rows - reset(visitor); - doReturn(true).when(visitor).processRow((Result)anyObject()); - MetaScanner.metaScan(connection, visitor, TABLENAME, HConstants.EMPTY_BYTE_ARRAY, 1000); - verify(visitor, times(3)).processRow((Result)anyObject()); - - // Scanning the table starting in the middle should give us two rows: - // region_a and region_b - reset(visitor); - doReturn(true).when(visitor).processRow((Result)anyObject()); - MetaScanner.metaScan(connection, visitor, TABLENAME, Bytes.toBytes("region_ac"), 1000); - verify(visitor, times(2)).processRow((Result)anyObject()); - - // Scanning with a limit of 1 should only give us one row - reset(visitor); - doReturn(true).when(visitor).processRow((Result) anyObject()); - MetaScanner.metaScan(connection, visitor, TABLENAME, Bytes.toBytes("region_ac"), 1); - verify(visitor, times(1)).processRow((Result) anyObject()); - table.close(); - } - - @Test - public void testConcurrentMetaScannerAndCatalogJanitor() throws Throwable { - /* TEST PLAN: start with only one region in a table. Have a splitter - * thread and metascanner threads that continously scan the meta table for regions. - * CatalogJanitor from master will run frequently to clean things up - */ - TEST_UTIL.getConfiguration().setLong("hbase.catalogjanitor.interval", 500); - setUp(); - - final long runtime = 30 * 1000; //30 sec - LOG.info("Starting testConcurrentMetaScannerAndCatalogJanitor"); - final TableName TABLENAME = - TableName.valueOf("testConcurrentMetaScannerAndCatalogJanitor"); - final byte[] FAMILY = Bytes.toBytes("family"); - TEST_UTIL.createTable(TABLENAME, FAMILY); - - class RegionMetaSplitter extends StoppableImplementation implements Runnable { - Random random = new Random(); - Throwable ex = null; - @Override - public void run() { - while (!isStopped()) { - try { - List regions = MetaScanner.listAllRegions(TEST_UTIL.getConfiguration(), - connection, false); - - //select a random region - HRegionInfo parent = regions.get(random.nextInt(regions.size())); - if (parent == null || !TABLENAME.equals(parent.getTable())) { - continue; - } - - long startKey = 0, endKey = Long.MAX_VALUE; - byte[] start = parent.getStartKey(); - byte[] end = parent.getEndKey(); - if (!Bytes.equals(HConstants.EMPTY_START_ROW, parent.getStartKey())) { - startKey = Bytes.toLong(parent.getStartKey()); - } - if (!Bytes.equals(HConstants.EMPTY_END_ROW, parent.getEndKey())) { - endKey = Bytes.toLong(parent.getEndKey()); - } - if (startKey == endKey) { - continue; - } - - long midKey = BigDecimal.valueOf(startKey).add(BigDecimal.valueOf(endKey)) - .divideToIntegralValue(BigDecimal.valueOf(2)).longValue(); - - HRegionInfo splita = new HRegionInfo(TABLENAME, - start, - Bytes.toBytes(midKey)); - HRegionInfo splitb = new HRegionInfo(TABLENAME, - Bytes.toBytes(midKey), - end); - - MetaTableAccessor.splitRegion(connection, - parent, splita, splitb, ServerName.valueOf("fooserver", 1, 0), 1); - - Threads.sleep(random.nextInt(200)); - } catch (Throwable e) { - ex = e; - Assert.fail(StringUtils.stringifyException(e)); - } - } - } - void rethrowExceptionIfAny() throws Throwable { - if (ex != null) { throw ex; } - } - } - - class MetaScannerVerifier extends StoppableImplementation implements Runnable { - Random random = new Random(); - Throwable ex = null; - @Override - public void run() { - while(!isStopped()) { - try { - NavigableMap regions = - MetaScanner.allTableRegions(connection, TABLENAME); - - LOG.info("-------"); - byte[] lastEndKey = HConstants.EMPTY_START_ROW; - for (HRegionInfo hri: regions.navigableKeySet()) { - long startKey = 0, endKey = Long.MAX_VALUE; - if (!Bytes.equals(HConstants.EMPTY_START_ROW, hri.getStartKey())) { - startKey = Bytes.toLong(hri.getStartKey()); - } - if (!Bytes.equals(HConstants.EMPTY_END_ROW, hri.getEndKey())) { - endKey = Bytes.toLong(hri.getEndKey()); - } - LOG.info("start:" + startKey + " end:" + endKey + " hri:" + hri); - Assert.assertTrue("lastEndKey=" + Bytes.toString(lastEndKey) + ", startKey=" + - Bytes.toString(hri.getStartKey()), Bytes.equals(lastEndKey, hri.getStartKey())); - lastEndKey = hri.getEndKey(); - } - Assert.assertTrue(Bytes.equals(lastEndKey, HConstants.EMPTY_END_ROW)); - LOG.info("-------"); - Threads.sleep(10 + random.nextInt(50)); - } catch (Throwable e) { - ex = e; - Assert.fail(StringUtils.stringifyException(e)); - } - } - } - void rethrowExceptionIfAny() throws Throwable { - if (ex != null) { throw ex; } - } - } - - RegionMetaSplitter regionMetaSplitter = new RegionMetaSplitter(); - MetaScannerVerifier metaScannerVerifier = new MetaScannerVerifier(); - - Thread regionMetaSplitterThread = new Thread(regionMetaSplitter); - Thread metaScannerVerifierThread = new Thread(metaScannerVerifier); - - regionMetaSplitterThread.start(); - metaScannerVerifierThread.start(); - - Threads.sleep(runtime); - - regionMetaSplitter.stop("test finished"); - metaScannerVerifier.stop("test finished"); - - regionMetaSplitterThread.join(); - metaScannerVerifierThread.join(); - - regionMetaSplitter.rethrowExceptionIfAny(); - metaScannerVerifier.rethrowExceptionIfAny(); - } - -} 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 354dc66..5ef56d6 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 @@ -168,7 +168,7 @@ public class TestMetaWithReplicas { Thread.sleep(conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 30000) * 6); Connection c = ConnectionFactory.createConnection(util.getConfiguration()); - List regions = MetaTableAccessor.getTableRegions(zkw, c, + List regions = MetaTableAccessor.getTableRegions(c, TableName.valueOf(TABLE)); HRegionLocation hrl = MetaTableAccessor.getRegionLocation(c, regions.get(0)); // Ensure that the primary server for test table is not the same one as the primary diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java index c4ac827..61560ce 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java @@ -493,7 +493,7 @@ public class TestLoadIncrementalHFilesSplitRecovery { dir = buildBulkFiles(tableName, 3); // Mess it up by leaving a hole in the hbase:meta - List regionInfos = MetaTableAccessor.getTableRegions(util.getZooKeeperWatcher(), + List regionInfos = MetaTableAccessor.getTableRegions( connection, tableName); for (HRegionInfo regionInfo : regionInfos) { if (Bytes.equals(regionInfo.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) { @@ -512,8 +512,7 @@ public class TestLoadIncrementalHFilesSplitRecovery { table.close(); // Make sure at least the one region that still exists can be found. - regionInfos = MetaTableAccessor.getTableRegions(util.getZooKeeperWatcher(), - connection, tableName); + regionInfos = MetaTableAccessor.getTableRegions(connection, tableName); assertTrue(regionInfos.size() >= 1); this.assertExpectedTable(connection, tableName, ROWCOUNT, 2); 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 5de785a..82376ef 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 @@ -1267,7 +1267,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/TestMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java index 70cb2fc..95002af 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 @@ -89,7 +89,6 @@ public class TestMaster { } List> tableRegions = MetaTableAccessor.getTableRegionsAndLocations( - m.getZooKeeper(), m.getConnection(), TABLENAME); LOG.info("Regions after load: " + Joiner.on(',').join(tableRegions)); assertEquals(1, tableRegions.size()); @@ -107,7 +106,7 @@ public class TestMaster { Thread.sleep(100); } LOG.info("Making sure we can call getTableRegions while opening"); - tableRegions = MetaTableAccessor.getTableRegionsAndLocations(m.getZooKeeper(), + tableRegions = MetaTableAccessor.getTableRegionsAndLocations( m.getConnection(), TABLENAME, false); 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 2c4b06e..362b27c 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 @@ -98,8 +98,7 @@ public class TestMasterOperationsForRegionReplicas { ADMIN.createTable(desc, Bytes.toBytes("A"), Bytes.toBytes("Z"), numRegions); validateNumberOfRowsInMeta(table, numRegions, ADMIN.getConnection()); - List hris = MetaTableAccessor.getTableRegions(TEST_UTIL.getZooKeeperWatcher(), - ADMIN.getConnection(), table); + List hris = MetaTableAccessor.getTableRegions(ADMIN.getConnection(), table); assert(hris.size() == numRegions * numReplica); } finally { ADMIN.disableTable(table); @@ -120,8 +119,7 @@ public class TestMasterOperationsForRegionReplicas { TEST_UTIL.waitTableEnabled(table); validateNumberOfRowsInMeta(table, numRegions, ADMIN.getConnection()); - List hris = MetaTableAccessor.getTableRegions( - TEST_UTIL.getZooKeeperWatcher(), ADMIN.getConnection(), table); + List hris = MetaTableAccessor.getTableRegions(ADMIN.getConnection(), table); assert(hris.size() == numRegions * numReplica); // check that the master created expected number of RegionState objects for (int i = 0; i < numRegions; i++) { @@ -133,7 +131,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); @@ -210,7 +208,7 @@ public class TestMasterOperationsForRegionReplicas { .getAssignmentManager().getRegionStates().getRegionsOfTable(table); assert(regions.size() == numRegions * numReplica); //also make sure the meta table has the replica locations removed - hris = MetaTableAccessor.getTableRegions(TEST_UTIL.getZooKeeperWatcher(), + hris = MetaTableAccessor.getTableRegions( ADMIN.getConnection(), table); assert(hris.size() == numRegions * numReplica); //just check that the number of default replica regions in the meta table are the same @@ -246,7 +244,7 @@ public class TestMasterOperationsForRegionReplicas { ADMIN.createTable(desc, Bytes.toBytes("A"), Bytes.toBytes("Z"), numRegions); TEST_UTIL.waitTableEnabled(table); Set tableRows = new HashSet(); - List hris = MetaTableAccessor.getTableRegions(TEST_UTIL.getZooKeeperWatcher(), + List hris = MetaTableAccessor.getTableRegions( ADMIN.getConnection(), table); for (HRegionInfo hri : hris) { tableRows.add(hri.getRegionName()); @@ -298,7 +296,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/TestRegionPlacement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java index 0899785..237bbfe 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.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.MetaTableAccessor; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.NamespaceDescriptor; @@ -49,11 +50,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.MetaScanner; -import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper; import org.apache.hadoop.hbase.master.balancer.FavoredNodeLoadBalancer; @@ -470,11 +467,11 @@ public class TestRegionPlacement { final AtomicInteger regionOnPrimaryNum = new AtomicInteger(0); final AtomicInteger totalRegionNum = new AtomicInteger(0); LOG.info("The start of region placement verification"); - MetaScannerVisitor visitor = new MetaScannerVisitor() { - public boolean processRow(Result result) throws IOException { + MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() { + public boolean visit(Result result) throws IOException { try { @SuppressWarnings("deprecation") - HRegionInfo info = MetaScanner.getHRegionInfo(result); + HRegionInfo info = MetaTableAccessor.getHRegionInfo(result); if(info.getTable().getNamespaceAsString() .equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) { return true; @@ -522,11 +519,8 @@ public class TestRegionPlacement { throw e; } } - - @Override - public void close() throws IOException {} }; - MetaScanner.metaScan(CONNECTION, visitor); + MetaTableAccessor.fullScanRegions(CONNECTION, visitor); LOG.info("There are " + regionOnPrimaryNum.intValue() + " out of " + totalRegionNum.intValue() + " regions running on the primary" + " region servers" ); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java index 504d98d..669f854 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java @@ -31,13 +31,12 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.MetaScanner; import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -114,7 +113,7 @@ public class TestRestartCluster { } List allRegions = - MetaScanner.listAllRegions(UTIL.getConfiguration(), connection, true); + MetaTableAccessor.getAllRegions(connection, false); assertEquals(4, allRegions.size()); LOG.info("\n\nShutting down cluster"); @@ -130,7 +129,7 @@ public class TestRestartCluster { // Otherwise we're reusing an HConnection that has gone stale because // the shutdown of the cluster also called shut of the connection. allRegions = - MetaScanner.listAllRegions(new Configuration(UTIL.getConfiguration()), connection, true); + MetaTableAccessor.getAllRegions(connection, false); assertEquals(4, allRegions.size()); LOG.info("\n\nWaiting for tables to be available"); for(TableName TABLE: TABLES) { 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 5266975..f24c827 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 @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.master.handler; import java.util.Collections; import java.util.List; +import java.io.IOException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -126,30 +127,25 @@ public class TestEnableTableHandler { // Now I have a nice table, mangle it by removing the HConstants.REGIONINFO_QUALIFIER_STR // content from a few of the rows. try (Table metaTable = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) { - try (ResultScanner scanner = - metaTable.getScanner(MetaTableAccessor.getScanForTableName(tableName))) { - for (Result result : scanner) { - // Just delete one row. - Delete d = new Delete(result.getRow()); - d.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); - LOG.info("Mangled: " + d); - metaTable.delete(d); - break; - } - } + MetaTableAccessor.scanMetaForTableRegions(TEST_UTIL.getConnection(), + new MetaTableAccessor.Visitor() { + @Override + public boolean visit(Result r) throws IOException { + // Just delete one row. + Delete d = new Delete(r.getRow()); + d.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); + LOG.info("Mangled: " + d); + metaTable.delete(d); + return false; + } + }, tableName); admin.disableTable(tableName); TEST_UTIL.waitTableDisabled(tableName.getName()); // Presume this synchronous all is. admin.deleteTable(tableName); - int rowCount = 0; - try (ResultScanner scanner = - metaTable.getScanner(MetaTableAccessor.getScanForTableName(tableName))) { - for (Result result : scanner) { - LOG.info("Found when none expected: " + result); - rowCount++; - } - } - assertEquals(0, rowCount); + List allRegions = + MetaTableAccessor.getAllRegions(TEST_UTIL.getConnection(), false); + assertEquals(0, allRegions.size()); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java index d35c23e..b0d6b66 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.ServerName; @@ -50,7 +51,6 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.MetaScanner; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; @@ -224,11 +224,11 @@ public class TestEndToEndSplitTransaction { stopper.stop(null); if (regionChecker.ex != null) { - throw regionChecker.ex; + throw new AssertionError("regionChecker", regionChecker.ex); } if (regionSplitter.ex != null) { - throw regionSplitter.ex; + throw new AssertionError("regionSplitter", regionSplitter.ex); } //one final check @@ -258,15 +258,15 @@ public class TestEndToEndSplitTransaction { try { Random random = new Random(); for (int i= 0; i< 5; i++) { - NavigableMap regions = - MetaScanner.allTableRegions(connection, tableName); + List regions = + MetaTableAccessor.getTableRegions(connection, tableName, true); if (regions.size() == 0) { continue; } int regionIndex = random.nextInt(regions.size()); //pick a random region and split it into two - HRegionInfo region = Iterators.get(regions.keySet().iterator(), regionIndex); + HRegionInfo region = Iterators.get(regions.iterator(), regionIndex); //pick the mid split point int start = 0, end = Integer.MAX_VALUE; @@ -313,7 +313,7 @@ public class TestEndToEndSplitTransaction { } /** - * Checks regions using MetaScanner, MetaTableAccessor and HTable methods + * Checks regions using MetaTableAccessor and HTable methods */ static class RegionChecker extends ScheduledChore { Connection connection; @@ -330,15 +330,13 @@ public class TestEndToEndSplitTransaction { } /** verify region boundaries obtained from MetaScanner */ - void verifyRegionsUsingMetaScanner() throws Exception { + void verifyRegionsUsingMetaTableAccessor() throws Exception { - //MetaScanner.allTableRegions() - NavigableMap regions = MetaScanner.allTableRegions(connection, + NavigableMap regions = MetaTableAccessor.allTableRegions(connection, tableName); verifyTableRegions(regions.keySet()); - //MetaScanner.listAllRegions() - List regionList = MetaScanner.listAllRegions(conf, connection, false); + List regionList = MetaTableAccessor.getAllRegions(connection, true); verifyTableRegions(Sets.newTreeSet(regionList)); } @@ -360,7 +358,7 @@ public class TestEndToEndSplitTransaction { } void verify() throws Exception { - verifyRegionsUsingMetaScanner(); + verifyRegionsUsingMetaTableAccessor(); verifyRegionsUsingHTable(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java index 8391782..d9b74ec 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java @@ -178,7 +178,7 @@ public class TestRegionMergeTransactionOnCluster { table.close(); List> tableRegions = MetaTableAccessor - .getTableRegionsAndLocations(master.getZooKeeper(), master.getConnection(), tableName); + .getTableRegionsAndLocations(master.getConnection(), tableName); HRegionInfo mergedRegionInfo = tableRegions.get(0).getFirst(); HTableDescriptor tableDescritor = master.getTableDescriptors().get( tableName); @@ -296,13 +296,13 @@ public class TestRegionMergeTransactionOnCluster { // Create table and load data. createTableAndLoadData(master, tableName, 5, 2); List> initialRegionToServers = - MetaTableAccessor.getTableRegionsAndLocations(master.getZooKeeper(), master.getConnection(), + MetaTableAccessor.getTableRegionsAndLocations(master.getConnection(), tableName); // Merge 1st and 2nd region PairOfSameType mergedRegions = mergeRegionsAndVerifyRegionNum(master, tableName, 0, 2, 5 * 2 - 2); List> currentRegionToServers = - MetaTableAccessor.getTableRegionsAndLocations(master.getZooKeeper(), master.getConnection(), + MetaTableAccessor.getTableRegionsAndLocations(master.getConnection(), tableName); List initialRegions = new ArrayList(); for (Pair p : initialRegionToServers) { @@ -342,7 +342,7 @@ public class TestRegionMergeTransactionOnCluster { HMaster master, TableName tablename, int regionAnum, int regionBnum) throws Exception { List> tableRegions = MetaTableAccessor - .getTableRegionsAndLocations(master.getZooKeeper(), + .getTableRegionsAndLocations( master.getConnection(), tablename); HRegionInfo regionA = tableRegions.get(regionAnum).getFirst(); HRegionInfo regionB = tableRegions.get(regionBnum).getFirst(); @@ -358,7 +358,7 @@ public class TestRegionMergeTransactionOnCluster { List tableRegionsInMaster; long timeout = System.currentTimeMillis() + waitTime; while (System.currentTimeMillis() < timeout) { - tableRegionsInMeta = MetaTableAccessor.getTableRegionsAndLocations(master.getZooKeeper(), + tableRegionsInMeta = MetaTableAccessor.getTableRegionsAndLocations( master.getConnection(), tablename); tableRegionsInMaster = master.getAssignmentManager().getRegionStates() .getRegionsOfTable(tablename); @@ -369,7 +369,7 @@ public class TestRegionMergeTransactionOnCluster { Thread.sleep(250); } - tableRegionsInMeta = MetaTableAccessor.getTableRegionsAndLocations(master.getZooKeeper(), + tableRegionsInMeta = MetaTableAccessor.getTableRegionsAndLocations( master.getConnection(), tablename); LOG.info("Regions after merge:" + Joiner.on(',').join(tableRegionsInMeta)); assertEquals(expectedRegionNum, tableRegionsInMeta.size()); @@ -399,7 +399,7 @@ public class TestRegionMergeTransactionOnCluster { long timeout = System.currentTimeMillis() + waitTime; List> tableRegions; while (System.currentTimeMillis() < timeout) { - tableRegions = MetaTableAccessor.getTableRegionsAndLocations(master.getZooKeeper(), + tableRegions = MetaTableAccessor.getTableRegionsAndLocations( master.getConnection(), tablename); if (tableRegions.size() == numRegions * replication) break; @@ -407,7 +407,6 @@ public class TestRegionMergeTransactionOnCluster { } tableRegions = MetaTableAccessor.getTableRegionsAndLocations( - master.getZooKeeper(), master.getConnection(), tablename); LOG.info("Regions after load: " + Joiner.on(',').join(tableRegions)); assertEquals(numRegions * replication, tableRegions.size()); 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 ef9e1c5..1b849bf 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; @@ -83,13 +80,10 @@ import org.apache.hadoop.hbase.client.Get; 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; @@ -1303,7 +1297,8 @@ public class TestHBaseFsck { HRegionInfo[] oldHris = new HRegionInfo[2]; setupTableWithRegionReplica(table, 2); assertEquals(ROWKEYS.length, countRows()); - NavigableMap map = MetaScanner.allTableRegions(TEST_UTIL.getConnection(), + NavigableMap map = MetaTableAccessor + .allTableRegions(TEST_UTIL.getConnection(), tbl.getName()); int i = 0; // store the HRIs of the regions we will mess up @@ -1336,7 +1331,7 @@ public class TestHBaseFsck { i = 0; HRegionInfo[] newHris = new HRegionInfo[2]; // get all table's regions from meta - map = MetaScanner.allTableRegions(TEST_UTIL.getConnection(), tbl.getName()); + map = MetaTableAccessor.allTableRegions(TEST_UTIL.getConnection(), tbl.getName()); // get the HRIs of the new regions (hbck created new regions for fixing the hdfs mess-up) for (Map.Entry m : map.entrySet()) { if (m.getKey().getStartKey().length > 0 && @@ -2233,10 +2228,10 @@ public class TestHBaseFsck { // Mess it up by removing the RegionInfo for one region. final List deletes = new LinkedList(); Table meta = connection.getTable(TableName.META_TABLE_NAME, hbfsckExecutorService); - MetaScanner.metaScan(connection, new MetaScanner.MetaScannerVisitor() { + MetaTableAccessor.fullScanRegions(connection, new MetaTableAccessor.Visitor() { @Override - public boolean processRow(Result rowResult) throws IOException { + public boolean visit(Result rowResult) throws IOException { HRegionInfo hri = MetaTableAccessor.getHRegionInfo(rowResult); if (hri != null && !hri.getTable().isSystemTable()) { Delete delete = new Delete(rowResult.getRow()); @@ -2245,10 +2240,6 @@ public class TestHBaseFsck { } return true; } - - @Override - public void close() throws IOException { - } }); meta.delete(deletes); @@ -2628,11 +2619,14 @@ public class TestHBaseFsck { HBaseFsck hbck = doFsck(conf, false); assertNoErrors(hbck); // no errors try { + hbck.connect(); // need connection to have access to META hbck.checkRegionBoundaries(); } catch (IllegalArgumentException e) { if (e.getMessage().endsWith("not a valid DFS filename.")) { fail("Table directory path is not valid." + e.getMessage()); } + } finally { + hbck.close(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java index 04fa5bf..efcfab3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java @@ -116,16 +116,14 @@ public class TestMergeTable { Connection connection = HConnectionManager.getConnection(c); List originalTableRegions = - MetaTableAccessor.getTableRegions(UTIL.getZooKeeperWatcher(), connection, - desc.getTableName()); + MetaTableAccessor.getTableRegions(connection, desc.getTableName()); LOG.info("originalTableRegions size=" + originalTableRegions.size() + "; " + originalTableRegions); Admin admin = new HBaseAdmin(c); admin.disableTable(desc.getTableName()); HMerge.merge(c, FileSystem.get(c), desc.getTableName()); List postMergeTableRegions = - MetaTableAccessor.getTableRegions(UTIL.getZooKeeperWatcher(), connection, - desc.getTableName()); + MetaTableAccessor.getTableRegions(connection, desc.getTableName()); LOG.info("postMergeTableRegions size=" + postMergeTableRegions.size() + "; " + postMergeTableRegions); assertTrue("originalTableRegions=" + originalTableRegions.size() + diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java index 1577191..bb61a53 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java @@ -51,24 +51,27 @@ public class HbckTestingUtil { TableName table) throws Exception { HBaseFsck fsck = new HBaseFsck(conf, exec); fsck.connect(); - fsck.setDisplayFullReport(); // i.e. -details - fsck.setTimeLag(0); - fsck.setFixAssignments(fixAssignments); - fsck.setFixMeta(fixMeta); - fsck.setFixHdfsHoles(fixHdfsHoles); - fsck.setFixHdfsOverlaps(fixHdfsOverlaps); - fsck.setFixHdfsOrphans(fixHdfsOrphans); - fsck.setFixTableOrphans(fixTableOrphans); - fsck.setFixVersionFile(fixVersionFile); - fsck.setFixReferenceFiles(fixReferenceFiles); - fsck.setFixEmptyMetaCells(fixEmptyMetaRegionInfo); - fsck.setFixTableLocks(fixTableLocks); - fsck.setFixTableZNodes(fixTableZnodes); - if (table != null) { - fsck.includeTable(table); + try { + fsck.setDisplayFullReport(); // i.e. -details + fsck.setTimeLag(0); + fsck.setFixAssignments(fixAssignments); + fsck.setFixMeta(fixMeta); + fsck.setFixHdfsHoles(fixHdfsHoles); + fsck.setFixHdfsOverlaps(fixHdfsOverlaps); + fsck.setFixHdfsOrphans(fixHdfsOrphans); + fsck.setFixTableOrphans(fixTableOrphans); + fsck.setFixVersionFile(fixVersionFile); + fsck.setFixReferenceFiles(fixReferenceFiles); + fsck.setFixEmptyMetaCells(fixEmptyMetaRegionInfo); + fsck.setFixTableLocks(fixTableLocks); + fsck.setFixTableZNodes(fixTableZnodes); + if (table != null) { + fsck.includeTable(table); + } + fsck.onlineHbck(); + } finally { + fsck.close(); } - fsck.onlineHbck(); - fsck.close(); return fsck; }