diff --git a/bin/region_status.rb b/bin/region_status.rb index 878d58a..f4c3cf9 100644 --- a/bin/region_status.rb +++ b/bin/region_status.rb @@ -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::gatTableRegions(connection, $TableName).size() end print "Region Status: #{server_count} / #{meta_count}\n" if SHOULD_WAIT and server_count < meta_count 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 2e6723a..0fc6015 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,7 +17,9 @@ */ 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; @@ -41,6 +43,7 @@ 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; @@ -60,6 +63,7 @@ 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; @@ -167,7 +171,7 @@ public class MetaTableAccessor { public static void fullScanRegions(Connection connection, final Visitor visitor) throws IOException { - fullScan(connection, visitor, null, QueryType.REGION); + scanMeta(connection, null, null, QueryType.REGION, visitor); } /** @@ -189,20 +193,7 @@ public class MetaTableAccessor { public static void fullScanTables(Connection connection, final Visitor visitor) throws IOException { - fullScan(connection, visitor, null, QueryType.TABLE); - } - - /** - * Performs a full scan of hbase:meta. - * @param connection connection we're using - * @param visitor Visitor invoked against each row. - * @param type scanned part of meta - * @throws IOException - */ - public static void fullScan(Connection connection, - final Visitor visitor, QueryType type) - throws IOException { - fullScan(connection, visitor, null, type); + scanMeta(connection, null, null, QueryType.TABLE, visitor); } /** @@ -215,7 +206,7 @@ public class MetaTableAccessor { public static List fullScan(Connection connection, QueryType type) throws IOException { CollectAllVisitor v = new CollectAllVisitor(); - fullScan(connection, v, null, type); + scanMeta(connection, null, null, type, v); return v.getResults(); } @@ -441,15 +432,52 @@ 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 + */ + public static byte[] getTableStartRowForMeta(TableName tableName, QueryType type) { + if (tableName == null) { + return null; + } + switch (type) { + 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; + case ALL: + case TABLE: + default: + return tableName.getName(); + } + } + + /** + * @param tableName table we're working with + * @return stop 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[] getTableStopRowForMeta(TableName tableName, QueryType type) { + if (tableName == null) { + return null; + } + final byte[] stopRow; + switch (type) { + case REGION: + 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; + case ALL: + case TABLE: + default: + stopRow = new byte[tableName.getName().length + 1]; + System.arraycopy(tableName.getName(), 0, stopRow, 0, tableName.getName().length); + stopRow[stopRow.length - 1] = ' '; + break; + } + return stopRow; } /** @@ -461,18 +489,39 @@ 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; + } /** * Do not use this method to get meta table regions, use methods in MetaTableLocator instead. * @param connection connection we're using @@ -514,7 +563,6 @@ public class MetaTableAccessor { return true; } 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); @@ -533,11 +581,32 @@ public class MetaTableAccessor { } } }; - fullScan(connection, visitor, getTableStartRowForMeta(tableName), QueryType.REGION); + scanMeta(connection, + getTableStartRowForMeta(tableName, QueryType.REGION), + getTableStopRowForMeta(tableName, QueryType.REGION), + QueryType.REGION, visitor); return visitor.getResults(); } /** + * Get list of table regions and locations grouped by region range from META. + */ + public static List getTableRegionLocations( + Connection connection, final TableName tableName) throws IOException { + final List regions = new ArrayList(); + Visitor visitor = new ConsistentViewVisitor() { + @Override + public boolean visitInternal(Result result) throws IOException { + RegionLocations locations = MetaTableAccessor.getRegionLocations(result); + if (locations == null) return true; + regions.add(locations); + return true; + } + }; + scanTableRegions(connection, tableName, visitor); + return regions; + } + /** * @param connection connection we're using * @param serverName server whose regions we're interested in * @return List of user regions installed on this server (does not include @@ -565,7 +634,7 @@ public class MetaTableAccessor { } } }; - fullScan(connection, v, QueryType.REGION); + scanMeta(connection, null, null, QueryType.REGION, v); return hris; } @@ -591,62 +660,131 @@ public class MetaTableAccessor { return true; } }; - fullScan(connection, v, QueryType.ALL); + scanMeta(connection, null, null, QueryType.ALL, v); } - /** - * Performs a full scan of a catalog table. - * @param connection connection we're using - * @param visitor Visitor invoked against each row. - * @param startrow Where to start the scan. Pass null if want to begin scan - * at first row. - * @param type scanned part of meta - * hbase:meta, the default (pass false to scan hbase:meta) - * @throws IOException - */ - public static void fullScan(Connection connection, - final Visitor visitor, @Nullable final byte[] startrow, QueryType type) throws IOException { - fullScan(connection, visitor, startrow, type, false); + public static void scanTableRegions(Connection connection, TableName table, final Visitor visitor) + throws IOException { + scanMeta(connection, table, 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); + } + + public static void scanTableRegionsForRow(Connection connection, + final TableName tableName, + final byte[] row, + final int rowLimit, + final Visitor visitor) + throws IOException { + byte[] startRow = null; + byte[] stopRow = null; + if (tableName != null) { + startRow = + MetaTableAccessor.getTableStartRowForMeta(tableName, MetaTableAccessor.QueryType.REGION); + if (row != null) { + HRegionInfo closestRi = + MetaTableAccessor.getClosestRegionInfo(connection, tableName, row); + startRow = HRegionInfo + .createRegionName(tableName, closestRi.getStartKey(), HConstants.ZEROES, false); + } + stopRow = + MetaTableAccessor.getTableStopRowForMeta(tableName, MetaTableAccessor.QueryType.REGION); + } + MetaTableAccessor + .scanMeta(connection, startRow, stopRow, MetaTableAccessor.QueryType.REGION, rowLimit, + visitor); } /** - * Performs a full scan of a catalog table. + * Performs a scan of META table. * @param connection connection we're using - * @param visitor Visitor invoked against each row. - * @param startrow Where to start the scan. Pass null if want to begin scan - * at first row. + * @param 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 raw read raw data including Delete tumbstones - * hbase:meta, the default (pass false to scan hbase:meta) + * @param maxRows maximum rows to return + * @param visitor Visitor invoked against each row. * @throws IOException */ - public static void fullScan(Connection connection, - final Visitor visitor, @Nullable final byte[] startrow, QueryType type, boolean raw) + public static void scanMeta(Connection connection, + @Nullable final byte[] startRow, @Nullable final byte[] stopRow, + QueryType type, int maxRows, final Visitor visitor) throws IOException { - Scan scan = new Scan(); - scan.setRaw(raw); - if (startrow != null) scan.setStartRow(startrow); - if (startrow == null) { - int caching = connection.getConfiguration() - .getInt(HConstants.HBASE_META_SCANNER_CACHING, 100); - scan.setCaching(caching); - } + int rowUpperLimit = maxRows > 0 ? maxRows : Integer.MAX_VALUE; + Scan scan = getMetaScan(connection, rowUpperLimit); + for (byte[] family : type.getFamilies()) { scan.addFamily(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; + } + } + } + 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); } - } finally { - if (scanner != null) scanner.close(); - metaTable.close(); + } + } + + /** + * @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; } } @@ -977,9 +1115,15 @@ 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 { + public static abstract class CollectingVisitor implements Visitor { final List results = new ArrayList(); @Override public boolean visit(Result r) throws IOException { @@ -994,12 +1138,45 @@ public class MetaTableAccessor { * @return Collected results; wait till visits complete to collect all * possible results */ - List getResults() { + public List getResults() { return this.results; } } /** + * A {@link Visitor} that collects HRegionInfo + */ + public static class CollectingRegionsVisitor extends CollectingVisitor { + + boolean onlineOnly; + boolean noSplits; + + public CollectingRegionsVisitor(boolean onlineOnly, boolean noSplits) { + this.onlineOnly = onlineOnly; + this.noSplits = noSplits; + } + + public CollectingRegionsVisitor() { + this(true, true); + } + + @Override + void add(Result r) { + HRegionInfo hRegionInfo = getHRegionInfo(r); + if (hRegionInfo == null) { + return; + } + if (onlineOnly && hRegionInfo.isOffline()) { + return; + } + if (noSplits && hRegionInfo.isSplit()) { + return; + } + results.add(hRegionInfo); + } + } + + /** * Collects all returned. */ static class CollectAllVisitor extends CollectingVisitor { @@ -1010,6 +1187,23 @@ public class MetaTableAccessor { } /** + * Visit non splitting and online only regions + */ + public static abstract class ConsistentViewVisitor implements Visitor { + @Override + final public boolean visit(Result r) throws IOException { + HRegionInfo hri = getHRegionInfo(r); + if (hri == null || hri.isOffline() || hri.isSplit()) { + return true; + } else { + return visitInternal(r); + } + } + + public abstract boolean visitInternal(Result r) throws IOException; + } + + /** * Count regions in hbase:meta for passed table. * @param c Configuration object * @param tableName table name to count regions for 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 b1f43d4..e8c5310 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 @@ -30,7 +30,6 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.NavigableMap; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -1065,10 +1064,13 @@ final 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.getTableRegionsAndLocations(this, + tableName); final List locations = new ArrayList(); - for (HRegionInfo regionInfo : regions.keySet()) { - RegionLocations list = locateRegion(tableName, regionInfo.getStartKey(), useCache, true); + for (Pair pair : regions) { + RegionLocations list = + locateRegion(tableName, pair.getFirst().getStartKey(), useCache, true); if (list != null) { for (HRegionLocation loc : list.getRegionLocations()) { if (loc != null) { 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 3acaaf9..c3914d4 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 @@ -61,8 +61,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; @@ -559,9 +557,9 @@ public class HBaseAdmin implements Admin { if (tableWasEnabled) { // Wait all table regions comes online 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); @@ -587,7 +585,7 @@ public class HBaseAdmin implements Admin { return true; } }; - MetaScanner.metaScan(connection, visitor, desc.getTableName()); + MetaTableAccessor.scanTableRegions(connection, desc.getTableName(), visitor); if (actualRegCount.get() < numRegs) { if (tries == this.numRetries * this.retryLongerMultiplier - 1) { throw new RegionOfflineException("Only " + actualRegCount.get() + @@ -2243,15 +2241,17 @@ 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 { - HRegionInfo info = HRegionInfo.getHRegionInfo(data); + public boolean visit(Result data) throws IOException { + HRegionInfo info = MetaTableAccessor.getHRegionInfo(data); if (info == null) { LOG.warn("No serialized HRegionInfo in " + data); return true; } RegionLocations rl = MetaTableAccessor.getRegionLocations(data); + if (rl == null) + return true; boolean matched = false; ServerName sn = null; for (HRegionLocation h : rl.getRegionLocations()) { @@ -2267,7 +2267,7 @@ public class HBaseAdmin implements Admin { } }; - MetaScanner.metaScan(connection, visitor, null); + MetaTableAccessor.fullScanRegions(connection, visitor); pair = result.get(); } return pair; 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..4d49b21 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 @@ -27,6 +27,7 @@ import java.util.Map.Entry; 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; @@ -85,11 +86,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 +140,7 @@ public class HRegionLocator implements RegionLocator { @VisibleForTesting List listRegionLocations() throws IOException { - return MetaScanner.listTableRegionLocations(getConfiguration(), this.connection, getName()); + return MetaTableAccessor.getTableRegionLocations(this.connection, getName()); } 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 8ba10bf..cf43e18 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 @@ -34,6 +34,12 @@ import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import com.google.common.annotations.VisibleForTesting; +import com.google.protobuf.Descriptors; +import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.Message; +import com.google.protobuf.Service; +import com.google.protobuf.ServiceException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -44,6 +50,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; @@ -72,13 +79,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Threads; -import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.Descriptors; -import com.google.protobuf.InvalidProtocolBufferException; -import com.google.protobuf.Message; -import com.google.protobuf.Service; -import com.google.protobuf.ServiceException; - /** * An implementation of {@link Table}. Used to communicate with a single HBase table. * Lightweight. Get as needed and just close when done. @@ -629,7 +629,15 @@ public class HTable implements HTableInterface { 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()); + List> list = + MetaTableAccessor.getTableRegionsAndLocations(this.connection, getName()); + TreeMap tm = + new TreeMap(); + for (Pair pair : list) { + if (!tm.containsKey(pair.getFirst())) + tm.put(pair.getFirst(), pair.getSecond()); + } + return tm; } /** 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 7d91dbb..0000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java +++ /dev/null @@ -1,425 +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.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.classification.InterfaceAudience; -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 final class MetaScanner { - private static final Log LOG = LogFactory.getLog(MetaScanner.class); - - private MetaScanner() {} - - /** - * 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 d155fd7..4d0af5c 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 @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.RegionTooBusyException; import org.apache.hadoop.hbase.ServerName; @@ -209,7 +210,7 @@ public class TestClientNoCluster extends Configured implements Tool { 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..fcf4b72 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 @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.rest; import java.io.IOException; +import java.util.List; import java.util.Map; import javax.ws.rs.GET; @@ -33,6 +34,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,9 +42,9 @@ 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; +import org.apache.hadoop.hbase.util.Pair; @InterfaceAudience.Private public class RegionsResource extends ResourceBase { @@ -80,11 +82,13 @@ public class RegionsResource extends ResourceBase { TableInfoModel model = new TableInfoModel(tableName.getNameAsString()); Connection connection = ConnectionFactory.createConnection(servlet.getConfiguration()); - Map regions = MetaScanner.allTableRegions(connection, tableName); + List> + regions = MetaTableAccessor.getTableRegionsAndLocations(connection, + tableName); connection.close(); - for (Map.Entry e: regions.entrySet()) { - HRegionInfo hri = e.getKey(); - ServerName addr = e.getValue(); + for (Pair e: regions) { + HRegionInfo hri = e.getFirst(); + ServerName addr = e.getSecond(); model.add( new TableRegionModel(tableName.getNameAsString(), hri.getRegionId(), hri.getStartKey(), hri.getEndKey(), addr.getHostAndPort())); 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..3589507 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.scanTableRegions(this.connection, tableName, visitor); 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 61a1c66..8bc7566 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 @@ -57,14 +57,12 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NamespaceNotFoundException; import org.apache.hadoop.hbase.PleaseHoldException; -import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerName; @@ -74,9 +72,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.client.TableState; @@ -109,7 +104,6 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost; import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; import org.apache.hadoop.hbase.quotas.MasterQuotaManager; @@ -125,7 +119,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CompressionTest; import org.apache.hadoop.hbase.util.EncryptionTest; import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.util.HBaseFsckRepair; import org.apache.hadoop.hbase.util.HFileArchiveUtil; import org.apache.hadoop.hbase.util.HasThread; import org.apache.hadoop.hbase.util.Pair; @@ -1667,10 +1660,10 @@ 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; } @@ -1686,7 +1679,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } }; - MetaScanner.metaScan(clusterConnection, visitor, tableName, rowKey, 1); + MetaTableAccessor.scanTableRegionsForRow(clusterConnection, tableName, rowKey, 1, visitor); return result.get(); } 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 7fcda15..b5b7555 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; @@ -144,9 +145,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/ModifyTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java index b35de6a..1cb0643 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 @@ -97,9 +97,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/namespace/NamespaceStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java index c34a123..512c951 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java @@ -28,13 +28,14 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.MetaScanner; import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.TableNamespaceManager; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Pair; /** * NamespaceStateManager manages state (in terms of quota) of all the namespaces. It contains @@ -193,9 +194,10 @@ class NamespaceStateManager { List tables = this.master.listTableNamesByNamespace(namespace.getName()); for (TableName table : tables) { int regionCount = 0; - Map regions = MetaScanner.allTableRegions( - this.master.getConnection(), table); - for (HRegionInfo info : regions.keySet()) { + List + regions = MetaTableAccessor.getTableRegions( + this.master.getConnection(), table); + for (HRegionInfo info : regions) { if (!info.isSplit()) { regionCount++; } 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 a8b60cd..4dd9d5c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.util; +import javax.annotation.Nonnull; import java.io.Closeable; import java.io.FileNotFoundException; import java.io.IOException; @@ -98,9 +99,6 @@ import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; -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; @@ -640,6 +638,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); } @@ -661,81 +664,88 @@ public class HBaseFsck extends Configured implements Closeable { public void checkRegionBoundaries() { try { - ByteArrayComparator comparator = new ByteArrayComparator(); - List regions = MetaScanner.listAllRegions(getConf(), connection, false); + final ByteArrayComparator comparator = new ByteArrayComparator(); final RegionBoundariesInformation currentRegionBoundariesInformation = new RegionBoundariesInformation(); - Path hbaseRoot = FSUtils.getRootDir(getConf()); - for (HRegionInfo regionInfo : regions) { - Path tableDir = FSUtils.getTableDir(hbaseRoot, regionInfo.getTable()); - currentRegionBoundariesInformation.regionName = regionInfo.getRegionName(); - // For each region, get the start and stop key from the META and compare them to the - // same information from the Stores. - Path path = new Path(tableDir, regionInfo.getEncodedName()); - FileSystem fs = path.getFileSystem(getConf()); - FileStatus[] files = fs.listStatus(path); - // For all the column families in this region... - byte[] storeFirstKey = null; - byte[] storeLastKey = null; - for (FileStatus file : files) { - String fileName = file.getPath().toString(); - fileName = fileName.substring(fileName.lastIndexOf("/") + 1); - if (!fileName.startsWith(".") && !fileName.endsWith("recovered.edits")) { - FileStatus[] storeFiles = fs.listStatus(file.getPath()); - // For all the stores in this column family. - for (FileStatus storeFile : storeFiles) { - HFile.Reader reader = HFile.createReader(fs, storeFile.getPath(), new CacheConfig( - getConf()), getConf()); - if ((reader.getFirstKey() != null) - && ((storeFirstKey == null) || (comparator.compare(storeFirstKey, - reader.getFirstKey()) > 0))) { - storeFirstKey = reader.getFirstKey(); + final Path hbaseRoot = FSUtils.getRootDir(getConf()); + MetaTableAccessor.fullScanRegions(connection, + new MetaTableAccessor.ConsistentViewVisitor() { + @Override + public boolean visitInternal(@Nonnull Result r) throws IOException { + HRegionInfo regionInfo = MetaTableAccessor.getHRegionInfo(r); + Path tableDir = FSUtils.getTableDir(hbaseRoot, regionInfo.getTable()); + currentRegionBoundariesInformation.regionName = regionInfo.getRegionName(); + // For each region, get the start and stop key from the META and compare them to the + // same information from the Stores. + Path path = new Path(tableDir, regionInfo.getEncodedName()); + FileSystem fs = path.getFileSystem(getConf()); + FileStatus[] files = fs.listStatus(path); + // For all the column families in this region... + byte[] storeFirstKey = null; + byte[] storeLastKey = null; + for (FileStatus file : files) { + String fileName = file.getPath().toString(); + fileName = fileName.substring(fileName.lastIndexOf("/") + 1); + if (!fileName.startsWith(".") && !fileName.endsWith("recovered.edits")) { + FileStatus[] storeFiles = fs.listStatus(file.getPath()); + // For all the stores in this column family. + for (FileStatus storeFile : storeFiles) { + HFile.Reader reader = + HFile.createReader(fs, storeFile.getPath(), new CacheConfig( + getConf()), getConf()); + if ((reader.getFirstKey() != null) + && ((storeFirstKey == null) || (comparator.compare(storeFirstKey, + reader.getFirstKey()) > 0))) { + storeFirstKey = reader.getFirstKey(); + } + if ((reader.getLastKey() != null) + && ((storeLastKey == null) || (comparator.compare(storeLastKey, + reader.getLastKey())) < 0)) { + storeLastKey = reader.getLastKey(); + } + reader.close(); + } + } } - if ((reader.getLastKey() != null) - && ((storeLastKey == null) || (comparator.compare(storeLastKey, - reader.getLastKey())) < 0)) { - storeLastKey = reader.getLastKey(); + currentRegionBoundariesInformation.metaFirstKey = regionInfo.getStartKey(); + currentRegionBoundariesInformation.metaLastKey = regionInfo.getEndKey(); + currentRegionBoundariesInformation.storesFirstKey = keyOnly(storeFirstKey); + currentRegionBoundariesInformation.storesLastKey = keyOnly(storeLastKey); + if (currentRegionBoundariesInformation.metaFirstKey.length == 0) + currentRegionBoundariesInformation.metaFirstKey = null; + if (currentRegionBoundariesInformation.metaLastKey.length == 0) + currentRegionBoundariesInformation.metaLastKey = null; + + // For a region to be correct, we need the META start key to be smaller or equal to the + // smallest start key from all the stores, and the start key from the next META entry to + // be bigger than the last key from all the current stores. First region start key is null; + // Last region end key is null; some regions can be empty and not have any store. + + boolean valid = true; + // Checking start key. + if ((currentRegionBoundariesInformation.storesFirstKey != null) + && (currentRegionBoundariesInformation.metaFirstKey != null)) { + valid = valid + && comparator.compare(currentRegionBoundariesInformation.storesFirstKey, + currentRegionBoundariesInformation.metaFirstKey) >= 0; } - reader.close(); + // Checking stop key. + if ((currentRegionBoundariesInformation.storesLastKey != null) + && (currentRegionBoundariesInformation.metaLastKey != null)) { + valid = valid + && comparator.compare(currentRegionBoundariesInformation.storesLastKey, + currentRegionBoundariesInformation.metaLastKey) < 0; + } + if (!valid) { + errors.reportError(ERROR_CODE.BOUNDARIES_ERROR, + "Found issues with regions boundaries", + tablesInfo.get(regionInfo.getTable())); + LOG.warn("Region's boundaries not alligned between stores and META for:"); + LOG.warn(currentRegionBoundariesInformation); + } + return true; } - } - } - currentRegionBoundariesInformation.metaFirstKey = regionInfo.getStartKey(); - currentRegionBoundariesInformation.metaLastKey = regionInfo.getEndKey(); - currentRegionBoundariesInformation.storesFirstKey = keyOnly(storeFirstKey); - currentRegionBoundariesInformation.storesLastKey = keyOnly(storeLastKey); - if (currentRegionBoundariesInformation.metaFirstKey.length == 0) - currentRegionBoundariesInformation.metaFirstKey = null; - if (currentRegionBoundariesInformation.metaLastKey.length == 0) - currentRegionBoundariesInformation.metaLastKey = null; - - // For a region to be correct, we need the META start key to be smaller or equal to the - // smallest start key from all the stores, and the start key from the next META entry to - // be bigger than the last key from all the current stores. First region start key is null; - // Last region end key is null; some regions can be empty and not have any store. - - boolean valid = true; - // Checking start key. - if ((currentRegionBoundariesInformation.storesFirstKey != null) - && (currentRegionBoundariesInformation.metaFirstKey != null)) { - valid = valid - && comparator.compare(currentRegionBoundariesInformation.storesFirstKey, - currentRegionBoundariesInformation.metaFirstKey) >= 0; - } - // Checking stop key. - if ((currentRegionBoundariesInformation.storesLastKey != null) - && (currentRegionBoundariesInformation.metaLastKey != null)) { - valid = valid - && comparator.compare(currentRegionBoundariesInformation.storesLastKey, - currentRegionBoundariesInformation.metaLastKey) < 0; - } - if (!valid) { - errors.reportError(ERROR_CODE.BOUNDARIES_ERROR, "Found issues with regions boundaries", - tablesInfo.get(regionInfo.getTable())); - LOG.warn("Region's boundaries not alligned between stores and META for:"); - LOG.warn(currentRegionBoundariesInformation); - } - } + }); } catch (IOException e) { LOG.error(e); } @@ -3164,7 +3174,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 @@ -3176,7 +3186,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 @@ -3248,7 +3258,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/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 1a377fc..6912ea2 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 @@ -18,11 +18,6 @@ package org.apache.hadoop.hbase; import javax.annotation.Nullable; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - import java.io.File; import java.io.IOException; import java.io.OutputStream; @@ -127,6 +122,10 @@ import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.ZooKeeper.States; +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. @@ -2982,7 +2981,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { } }; MetaTableAccessor - .fullScan(connection, visitor, table.getName(), MetaTableAccessor.QueryType.TABLE, true); + .scanMeta(connection, null, null, + MetaTableAccessor.QueryType.TABLE, + Integer.MAX_VALUE, visitor); return lastTableState.get(); } 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 eefb974..fb62478 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 @@ -18,16 +18,11 @@ */ package org.apache.hadoop.hbase; -import static org.junit.Assert.assertEquals; -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 java.io.IOException; import java.util.List; import java.util.Random; +import com.google.common.collect.Lists; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -50,7 +45,17 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import com.google.common.collect.Lists; +import static org.junit.Assert.assertEquals; +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; /** * Test {@link org.apache.hadoop.hbase.MetaTableAccessor}. @@ -475,5 +480,53 @@ 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 + .scanMeta(connection, TABLENAME, MetaTableAccessor.QueryType.REGION, 10, visitor); + 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 + .scanTableRegionsForRow(connection, TABLENAME, null, 1000, visitor); + 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.scanTableRegionsForRow(connection, TABLENAME, Bytes.toBytes("region_ac"), + 1000, visitor); + 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.scanTableRegionsForRow(connection, TABLENAME, Bytes.toBytes("region_ac"), 1, + visitor); + verify(visitor, times(1)).visit((Result) anyObject()); + table.close(); + } } 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 2898369..f6c73ca 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 @@ -19,15 +19,13 @@ */ package org.apache.hadoop.hbase.client; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - 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.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -36,6 +34,8 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; +import static org.junit.Assert.assertTrue; + @Category({ LargeTests.class, ClientTests.class }) public class TestHTableMultiplexerFlushCache { final Log LOG = LogFactory.getLog(getClass()); @@ -64,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 e195baf..0000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java +++ /dev/null @@ -1,243 +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.ClientTests; -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, ClientTests.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/master/TestRegionPlacement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java index 25dd13e..ffb9e59 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.MiniHBaseCluster; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ServerName; @@ -51,8 +52,6 @@ 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 +469,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 +521,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 b51f7c7..47c86ce 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,11 +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.MetaScanner; +import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.util.Bytes; @@ -75,8 +76,8 @@ public class TestRestartCluster { UTIL.waitTableEnabled(TABLE); } - List allRegions = - MetaScanner.listAllRegions(UTIL.getConfiguration(), UTIL.getConnection(), true); + List allRegions = + MetaTableAccessor.fullScanRegions(UTIL.getConnection()); assertEquals(4, allRegions.size()); LOG.info("\n\nShutting down cluster"); @@ -91,8 +92,8 @@ public class TestRestartCluster { // Need to use a new 'Configuration' so we make a new HConnection. // 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()), UTIL.getConnection(), true); + allRegions = + MetaTableAccessor.fullScanRegions(UTIL.getConnection()); 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 6a60fc0..2db658b 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.ArrayList; import java.util.List; +import java.io.IOException; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; @@ -27,15 +28,20 @@ import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -49,13 +55,6 @@ import org.junit.experimental.categories.Category; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import java.io.IOException; -import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.client.Table; - @Category({ MasterTests.class, MediumTests.class }) public class TestEnableTableHandler { private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -146,7 +145,8 @@ public class TestEnableTableHandler { admin.createTable(desc, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); // Now I have a nice table, mangle it by removing the HConstants.REGIONINFO_QUALIFIER_STR // content from a few of the rows. - Scan metaScannerForMyTable = MetaTableAccessor.getScanForTableName(tableName); + Scan metaScannerForMyTable = + MetaTableAccessor.getScanForTableName(TEST_UTIL.getConnection(), tableName); try (Table metaTable = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) { try (ResultScanner scanner = metaTable.getScanner(metaScannerForMyTable)) { for (Result result : scanner) { 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 f29601c..ea155e0 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 @@ -17,11 +17,6 @@ */ package org.apache.hadoop.hbase.regionserver; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -29,7 +24,11 @@ import java.util.Map; import java.util.NavigableMap; import java.util.Random; import java.util.Set; +import java.util.TreeMap; +import com.google.common.collect.Iterators; +import com.google.common.collect.Sets; +import com.google.protobuf.ServiceException; import org.apache.commons.io.IOUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -38,7 +37,10 @@ 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.HRegionLocation; +import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.NotServingRegionException; +import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.Stoppable; @@ -50,7 +52,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; @@ -73,9 +74,10 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import com.google.common.collect.Iterators; -import com.google.common.collect.Sets; -import com.google.protobuf.ServiceException; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; @Category({FlakeyTests.class, LargeTests.class}) @SuppressWarnings("deprecation") @@ -244,7 +246,7 @@ public class TestEndToEndSplitTransaction { Random random = new Random(); for (int i= 0; i< 5; i++) { NavigableMap regions = - MetaScanner.allTableRegions(connection, tableName); + allTableRegions(connection, tableName); if (regions.size() == 0) { continue; } @@ -318,13 +320,15 @@ public class TestEndToEndSplitTransaction { void verifyRegionsUsingMetaScanner() throws Exception { //MetaScanner.allTableRegions() - NavigableMap regions = MetaScanner.allTableRegions(connection, + NavigableMap regions = allTableRegions(connection, tableName); verifyTableRegions(regions.keySet()); //MetaScanner.listAllRegions() - List regionList = MetaScanner.listAllRegions(conf, connection, false); - verifyTableRegions(Sets.newTreeSet(regionList)); + MetaTableAccessor.CollectingRegionsVisitor visitor = + new MetaTableAccessor.CollectingRegionsVisitor(true, true); + MetaTableAccessor.fullScanRegions(connection, visitor); + verifyTableRegions(Sets.newTreeSet(visitor.getResults())); } /** verify region boundaries obtained from HTable.getStartEndKeys() */ @@ -530,4 +534,28 @@ public class TestEndToEndSplitTransaction { IOUtils.closeQuietly(connection); } } + + public static NavigableMap allTableRegions( + Connection connection, final TableName tableName) throws IOException { + final NavigableMap regions = + new TreeMap(); + MetaTableAccessor.Visitor visitor = new MetaTableAccessor.ConsistentViewVisitor() { + @Override + public boolean visitInternal(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 HRegionInfo(regionInfo), loc.getServerName()); + } + } + return true; + } + }; + MetaTableAccessor.scanMeta(connection, tableName, MetaTableAccessor.QueryType.REGION, + Integer.MAX_VALUE, visitor); + return regions; + } + } 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 0d3a94e..978c092 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 @@ -18,16 +18,6 @@ */ package org.apache.hadoop.hbase.util; -import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors; -import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertNoErrors; -import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -37,7 +27,6 @@ import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.NavigableMap; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; @@ -50,6 +39,7 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import com.google.common.collect.Multimap; import org.apache.commons.io.IOUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -64,10 +54,10 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.Connection; @@ -78,7 +68,6 @@ 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.Put; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.Result; @@ -118,7 +107,15 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import com.google.common.collect.Multimap; +import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors; +import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertNoErrors; +import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** * This tests HBaseFsck's ability to detect reasons for inconsistent tables. @@ -1313,16 +1310,16 @@ public class TestHBaseFsck { HRegionInfo[] oldHris = new HRegionInfo[2]; setupTableWithRegionReplica(table, 2); assertEquals(ROWKEYS.length, countRows()); - NavigableMap map = - MetaScanner.allTableRegions(TEST_UTIL.getConnection(), - tbl.getName()); + List> map = + MetaTableAccessor.getTableRegionsAndLocations(TEST_UTIL.getConnection(), + tbl.getName()); int i = 0; // store the HRIs of the regions we will mess up - for (Map.Entry m : map.entrySet()) { - if (m.getKey().getStartKey().length > 0 && - m.getKey().getStartKey()[0] == Bytes.toBytes("B")[0]) { - LOG.debug("Initially server hosting " + m.getKey() + " is " + m.getValue()); - oldHris[i++] = m.getKey(); + for (Pair m : map) { + if (m.getFirst().getStartKey().length > 0 && + m.getFirst().getStartKey()[0] == Bytes.toBytes("B")[0]) { + LOG.debug("Initially server hosting " + m.getFirst() + " is " + m.getSecond()); + oldHris[i++] = m.getFirst(); } } // make sure data in regions @@ -1347,12 +1344,12 @@ 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.getTableRegionsAndLocations(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 && - m.getKey().getStartKey()[0] == Bytes.toBytes("B")[0]) { - newHris[i++] = m.getKey(); + for (Pair m : map) { + if (m.getFirst().getStartKey().length > 0 && + m.getFirst().getStartKey()[0] == Bytes.toBytes("B")[0]) { + newHris[i++] = m.getFirst(); } } // get all the online regions in the regionservers @@ -2270,23 +2267,19 @@ 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() { - - @Override - public boolean processRow(Result rowResult) throws IOException { - HRegionInfo hri = MetaTableAccessor.getHRegionInfo(rowResult); - if (hri != null && !hri.getTable().isSystemTable()) { - Delete delete = new Delete(rowResult.getRow()); - delete.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); - deletes.add(delete); - } - return true; - } - - @Override - public void close() throws IOException { - } - }); + MetaTableAccessor.fullScanRegions(connection, + new MetaTableAccessor.Visitor() { + @Override + public boolean visit(Result rowResult) throws IOException { + HRegionInfo hri = MetaTableAccessor.getHRegionInfo(rowResult); + if (hri != null && !hri.getTable().isSystemTable()) { + Delete delete = new Delete(rowResult.getRow()); + delete.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); + deletes.add(delete); + } + return true; + } + }); meta.delete(deletes); // Mess it up by creating a fake hbase:meta entry with no associated RegionInfo @@ -2616,11 +2609,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/hbck/HbckTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java index 217f60b..bce8938 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 @@ -49,24 +49,27 @@ public class HbckTestingUtil { boolean fixReferenceFiles, boolean fixEmptyMetaRegionInfo, boolean fixTableLocks, 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); - if (table != null) { - fsck.includeTable(table); + try { + 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); + if (table != null) { + fsck.includeTable(table); + } + fsck.onlineHbck(); + } finally { + fsck.close(); } - fsck.onlineHbck(); - fsck.close(); return fsck; }