View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  package org.apache.hadoop.hbase.client;
21  
22  import java.io.Closeable;
23  import java.io.IOException;
24  import java.util.ArrayList;
25  import java.util.List;
26  import java.util.NavigableMap;
27  import java.util.TreeMap;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.hbase.classification.InterfaceAudience;
32  import org.apache.hadoop.conf.Configuration;
33  import org.apache.hadoop.hbase.HConstants;
34  import org.apache.hadoop.hbase.HRegionInfo;
35  import org.apache.hadoop.hbase.HRegionLocation;
36  import org.apache.hadoop.hbase.MetaTableAccessor;
37  import org.apache.hadoop.hbase.RegionLocations;
38  import org.apache.hadoop.hbase.ServerName;
39  import org.apache.hadoop.hbase.TableName;
40  import org.apache.hadoop.hbase.TableNotFoundException;
41  import org.apache.hadoop.hbase.util.Bytes;
42  import org.apache.hadoop.hbase.util.ExceptionUtil;
43  
44  import com.google.common.annotations.VisibleForTesting;
45  
46  /**
47   * Scanner class that contains the <code>hbase:meta</code> table scanning logic.
48   * Provided visitors will be called for each row.
49   *
50   * Although public visibility, this is not a public-facing API and may evolve in
51   * minor releases.
52   *
53   * <p> Note that during concurrent region splits, the scanner might not see
54   * hbase:meta changes across rows (for parent and daughter entries) consistently.
55   * see HBASE-5986, and {@link DefaultMetaScannerVisitor} for details. </p>
56   */
57  @InterfaceAudience.Private
58  //TODO: merge this to MetaTableAccessor, get rid of it.
59  public class MetaScanner {
60    private static final Log LOG = LogFactory.getLog(MetaScanner.class);
61    /**
62     * Scans the meta table and calls a visitor on each RowResult and uses a empty
63     * start row value as table name.
64     * 
65     * <p>Visible for testing. Use {@link
66     * #metaScan(Connection, MetaScannerVisitor, TableName)} instead.
67     *
68     * @param visitor A custom visitor
69     * @throws IOException e
70     */
71    @VisibleForTesting // Do not use. Used by tests only and hbck.
72    public static void metaScan(Connection connection,
73        MetaScannerVisitor visitor) throws IOException {
74      metaScan(connection, visitor, null, null, Integer.MAX_VALUE);
75    }
76  
77    /**
78     * Scans the meta table and calls a visitor on each RowResult. Uses a table
79     * name to locate meta regions.
80     *
81     * @param connection connection to use internally (null to use a new instance)
82     * @param visitor visitor object
83     * @param userTableName User table name in meta table to start scan at.  Pass
84     * null if not interested in a particular table.
85     * @throws IOException e
86     */
87    public static void metaScan(Connection connection,
88        MetaScannerVisitor visitor, TableName userTableName) throws IOException {
89      metaScan(connection, visitor, userTableName, null, Integer.MAX_VALUE,
90          TableName.META_TABLE_NAME);
91    }
92  
93    /**
94     * Scans the meta table and calls a visitor on each RowResult. Uses a table
95     * name and a row name to locate meta regions. And it only scans at most
96     * <code>rowLimit</code> of rows.
97     * 
98     * <p>Visible for testing. Use {@link
99     * #metaScan(Connection, MetaScannerVisitor, TableName)} instead.
100    *
101    * @param connection to scan on
102    * @param visitor Visitor object.
103    * @param userTableName User table name in meta table to start scan at.  Pass
104    * null if not interested in a particular table.
105    * @param row Name of the row at the user table. The scan will start from
106    * the region row where the row resides.
107    * @param rowLimit Max of processed rows. If it is less than 0, it
108    * will be set to default value <code>Integer.MAX_VALUE</code>.
109    * @throws IOException e
110    */
111   @VisibleForTesting // Do not use. Used by Master but by a method that is used testing.
112   public static void metaScan(Connection connection,
113       MetaScannerVisitor visitor, TableName userTableName, byte[] row,
114       int rowLimit)
115   throws IOException {
116     metaScan(connection, visitor, userTableName, row, rowLimit, TableName
117         .META_TABLE_NAME);
118   }
119 
120   /**
121    * Scans the meta table and calls a visitor on each RowResult. Uses a table
122    * name and a row name to locate meta regions. And it only scans at most
123    * <code>rowLimit</code> of rows.
124    *
125    * @param connection connection to use internally (null to use a new instance)
126    * @param visitor Visitor object. Closes the visitor before returning.
127    * @param tableName User table name in meta table to start scan at.  Pass
128    * null if not interested in a particular table.
129    * @param row Name of the row at the user table. The scan will start from
130    * the region row where the row resides.
131    * @param rowLimit Max of processed rows. If it is less than 0, it
132    * will be set to default value <code>Integer.MAX_VALUE</code>.
133    * @param metaTableName Meta table to scan, root or meta.
134    * @throws IOException e
135    */
136   static void metaScan(Connection connection,
137       final MetaScannerVisitor visitor, final TableName tableName,
138       final byte[] row, final int rowLimit, final TableName metaTableName)
139     throws IOException {
140 
141     int rowUpperLimit = rowLimit > 0 ? rowLimit: Integer.MAX_VALUE;
142     // Calculate startrow for scan.
143     byte[] startRow;
144     // If the passed in 'connection' is 'managed' -- i.e. every second test uses
145     // an HTable or an HBaseAdmin with managed connections -- then doing
146     // connection.getTable will throw an exception saying you are NOT to use
147     // managed connections getting tables.  Leaving this as it is for now. Will
148     // revisit when inclined to change all tests.  User code probaby makes use of
149     // managed connections too so don't change it till post hbase 1.0.
150     try (Table metaTable = new HTable(TableName.META_TABLE_NAME, connection, null)) {
151       if (row != null) {
152         // Scan starting at a particular row in a particular table
153         Result startRowResult = getClosestRowOrBefore(metaTable, tableName, row,
154             connection.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS,
155                 HConstants.DEFAULT_USE_META_REPLICAS));
156         if (startRowResult == null) {
157           throw new TableNotFoundException("Cannot find row in " + metaTable.getName() +
158             " for table: " + tableName + ", row=" + Bytes.toStringBinary(row));
159         }
160         HRegionInfo regionInfo = getHRegionInfo(startRowResult);
161         if (regionInfo == null) {
162           throw new IOException("HRegionInfo was null or empty in Meta for " +
163             tableName + ", row=" + Bytes.toStringBinary(row));
164         }
165         byte[] rowBefore = regionInfo.getStartKey();
166         startRow = HRegionInfo.createRegionName(tableName, rowBefore, HConstants.ZEROES, false);
167       } else if (tableName == null || tableName.getName().length == 0) {
168         // Full hbase:meta scan
169         startRow = HConstants.EMPTY_START_ROW;
170       } else {
171         // Scan hbase:meta for an entire table
172         startRow = HRegionInfo.createRegionName(tableName, HConstants.EMPTY_START_ROW,
173           HConstants.ZEROES, false);
174       }
175       final Scan scan = new Scan(startRow).addFamily(HConstants.CATALOG_FAMILY);
176       int scannerCaching = connection.getConfiguration()
177           .getInt(HConstants.HBASE_META_SCANNER_CACHING,
178               HConstants.DEFAULT_HBASE_META_SCANNER_CACHING);
179       if (connection.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS,
180                 HConstants.DEFAULT_USE_META_REPLICAS)) {
181         scan.setConsistency(Consistency.TIMELINE);
182       }
183       if (rowUpperLimit <= scannerCaching) {
184           scan.setSmall(true);
185       }
186       int rows = Math.min(rowLimit, scannerCaching);
187       scan.setCaching(rows);
188       if (LOG.isTraceEnabled()) {
189         LOG.trace("Scanning " + metaTableName.getNameAsString() + " starting at row=" +
190           Bytes.toStringBinary(startRow) + " for max=" + rowUpperLimit + " with caching=" + rows);
191       }
192       // Run the scan
193       try (ResultScanner resultScanner = metaTable.getScanner(scan)) {
194         Result result;
195         int processedRows = 0;
196         while ((result = resultScanner.next()) != null) {
197           if (visitor != null) {
198             if (!visitor.processRow(result)) break;
199           }
200           processedRows++;
201           if (processedRows >= rowUpperLimit) break;
202         }
203       }
204     } finally {
205       if (visitor != null) {
206         try {
207           visitor.close();
208         } catch (Throwable t) {
209           ExceptionUtil.rethrowIfInterrupt(t);
210           LOG.debug("Got exception in closing the meta scanner visitor", t);
211         }
212       }
213     }
214   }
215 
216   /**
217    * @return Get closest metatable region row to passed <code>row</code>
218    * @throws IOException
219    */
220   private static Result getClosestRowOrBefore(final Table metaTable, final TableName userTableName,
221       final byte [] row, boolean useMetaReplicas)
222   throws IOException {
223     byte[] searchRow = HRegionInfo.createRegionName(userTableName, row, HConstants.NINES, false);
224     Scan scan = Scan.createGetClosestRowOrBeforeReverseScan(searchRow);
225     if (useMetaReplicas) {
226       scan.setConsistency(Consistency.TIMELINE);
227     }
228     try (ResultScanner resultScanner = metaTable.getScanner(scan)) {
229       return resultScanner.next();
230     }
231   }
232 
233   /**
234    * Returns HRegionInfo object from the column
235    * HConstants.CATALOG_FAMILY:HConstants.REGIONINFO_QUALIFIER of the catalog
236    * table Result.
237    * @param data a Result object from the catalog table scan
238    * @return HRegionInfo or null
239    * @deprecated Use {@link org.apache.hadoop.hbase.MetaTableAccessor#getRegionLocations(Result)}
240    */
241   @Deprecated
242   public static HRegionInfo getHRegionInfo(Result data) {
243     return HRegionInfo.getHRegionInfo(data);
244   }
245 
246   /**
247    * Lists all of the regions currently in META.
248    * @param conf configuration
249    * @param connection to connect with
250    * @param offlined True if we are to include offlined regions, false and we'll
251    * leave out offlined regions from returned list.
252    * @return List of all user-space regions.
253    * @throws IOException
254    */
255   @VisibleForTesting // And for hbck.
256   public static List<HRegionInfo> listAllRegions(Configuration conf, Connection connection,
257       final boolean offlined)
258   throws IOException {
259     final List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
260     MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
261         @Override
262         public boolean processRow(Result result) throws IOException {
263           if (result == null || result.isEmpty()) {
264             return true;
265           }
266 
267           RegionLocations locations = MetaTableAccessor.getRegionLocations(result);
268           if (locations == null) return true;
269           for (HRegionLocation loc : locations.getRegionLocations()) {
270             if (loc != null) {
271               HRegionInfo regionInfo = loc.getRegionInfo();
272               // If region offline AND we are not to include offlined regions, return.
273               if (regionInfo.isOffline() && !offlined) continue;
274               regions.add(regionInfo);
275             }
276           }
277           return true;
278         }
279     };
280     metaScan(connection, visitor);
281     return regions;
282   }
283 
284   /**
285    * Lists all of the table regions currently in META.
286    * @param conf
287    * @param offlined True if we are to include offlined regions, false and we'll
288    * leave out offlined regions from returned list.
289    * @return Map of all user-space regions to servers
290    * @throws IOException
291    * @deprecated Use {@link #allTableRegions(Connection, TableName)} instead
292    */
293   @Deprecated
294   public static NavigableMap<HRegionInfo, ServerName> allTableRegions(Configuration conf,
295       Connection connection, final TableName tableName, boolean offlined) throws IOException {
296     return allTableRegions(connection, tableName);
297   }
298 
299   /**
300    * Lists all of the table regions currently in META.
301    * @param connection
302    * @param tableName
303    * @return Map of all user-space regions to servers
304    * @throws IOException
305    */
306   public static NavigableMap<HRegionInfo, ServerName> allTableRegions(
307       Connection connection, final TableName tableName) throws IOException {
308     final NavigableMap<HRegionInfo, ServerName> regions =
309       new TreeMap<HRegionInfo, ServerName>();
310     MetaScannerVisitor visitor = new TableMetaScannerVisitor(tableName) {
311       @Override
312       public boolean processRowInternal(Result result) throws IOException {
313         RegionLocations locations = MetaTableAccessor.getRegionLocations(result);
314         if (locations == null) return true;
315         for (HRegionLocation loc : locations.getRegionLocations()) {
316           if (loc != null) {
317             HRegionInfo regionInfo = loc.getRegionInfo();
318             regions.put(new UnmodifyableHRegionInfo(regionInfo), loc.getServerName());
319           }
320         }
321         return true;
322       }
323     };
324     metaScan(connection, visitor, tableName);
325     return regions;
326   }
327 
328   /**
329    * Lists table regions and locations grouped by region range from META.
330    */
331   public static List<RegionLocations> listTableRegionLocations(Configuration conf,
332       Connection connection, final TableName tableName) throws IOException {
333     final List<RegionLocations> regions = new ArrayList<RegionLocations>();
334     MetaScannerVisitor visitor = new TableMetaScannerVisitor(tableName) {
335       @Override
336       public boolean processRowInternal(Result result) throws IOException {
337         RegionLocations locations = MetaTableAccessor.getRegionLocations(result);
338         if (locations == null) return true;
339         regions.add(locations);
340         return true;
341       }
342     };
343     metaScan(connection, visitor, tableName);
344     return regions;
345   }
346 
347   /**
348    * Visitor class called to process each row of the hbase:meta table
349    */
350   public interface MetaScannerVisitor extends Closeable {
351     /**
352      * Visitor method that accepts a RowResult and the meta region location.
353      * Implementations can return false to stop the region's loop if it becomes
354      * unnecessary for some reason.
355      *
356      * @param rowResult result
357      * @return A boolean to know if it should continue to loop in the region
358      * @throws IOException e
359      */
360     boolean processRow(Result rowResult) throws IOException;
361   }
362 
363   public static abstract class MetaScannerVisitorBase implements MetaScannerVisitor {
364     @Override
365     public void close() throws IOException {
366     }
367   }
368 
369   /**
370    * A MetaScannerVisitor that skips offline regions and split parents
371    */
372   public static abstract class DefaultMetaScannerVisitor
373     extends MetaScannerVisitorBase {
374 
375     public DefaultMetaScannerVisitor() {
376       super();
377     }
378 
379     public abstract boolean processRowInternal(Result rowResult) throws IOException;
380 
381     @Override
382     public boolean processRow(Result rowResult) throws IOException {
383       HRegionInfo info = getHRegionInfo(rowResult);
384       if (info == null) {
385         return true;
386       }
387 
388       //skip over offline and split regions
389       if (!(info.isOffline() || info.isSplit())) {
390         return processRowInternal(rowResult);
391       }
392       return true;
393     }
394   }
395 
396   /**
397    * A MetaScannerVisitor for a table. Provides a consistent view of the table's
398    * hbase:meta entries during concurrent splits (see HBASE-5986 for details). This class
399    * does not guarantee ordered traversal of meta entries, and can block until the
400    * hbase:meta entries for daughters are available during splits.
401    */
402   public static abstract class TableMetaScannerVisitor extends DefaultMetaScannerVisitor {
403     private TableName tableName;
404 
405     public TableMetaScannerVisitor(TableName tableName) {
406       super();
407       this.tableName = tableName;
408     }
409 
410     @Override
411     public final boolean processRow(Result rowResult) throws IOException {
412       HRegionInfo info = getHRegionInfo(rowResult);
413       if (info == null) {
414         return true;
415       }
416       if (!(info.getTable().equals(tableName))) {
417         return false;
418       }
419       return super.processRow(rowResult);
420     }
421   }
422 }