View Javadoc

1   /**
2    * Copyright 2009 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  
21  package org.apache.hadoop.hbase.client;
22  
23  import java.io.Closeable;
24  import java.io.IOException;
25  import java.util.ArrayList;
26  import java.util.List;
27  import java.util.NavigableMap;
28  import java.util.TreeMap;
29  import java.util.TreeSet;
30  
31  import org.apache.commons.logging.Log;
32  import org.apache.commons.logging.LogFactory;
33  import org.apache.hadoop.conf.Configuration;
34  import org.apache.hadoop.hbase.HConstants;
35  import org.apache.hadoop.hbase.HRegionInfo;
36  import org.apache.hadoop.hbase.ServerName;
37  import org.apache.hadoop.hbase.TableNotFoundException;
38  import org.apache.hadoop.hbase.errorhandling.TimeoutException;
39  import org.apache.hadoop.hbase.util.Bytes;
40  import org.apache.hadoop.hbase.util.Writables;
41  
42  /**
43   * Scanner class that contains the <code>.META.</code> table scanning logic
44   * and uses a Retryable scanner. Provided visitors will be called
45   * for each row.
46   *
47   * Although public visibility, this is not a public-facing API and may evolve in
48   * minor releases.
49   *
50   * <p> Note that during concurrent region splits, the scanner might not see
51   * META changes across rows (for parent and daughter entries) consistently.
52   * see HBASE-5986, and {@link BlockingMetaScannerVisitor} for details. </p>
53   */
54  public class MetaScanner {
55    private static final Log LOG = LogFactory.getLog(MetaScanner.class);
56    /**
57     * Scans the meta table and calls a visitor on each RowResult and uses a empty
58     * start row value as table name.
59     *
60     * @param configuration conf
61     * @param visitor A custom visitor
62     * @throws IOException e
63     */
64    public static void metaScan(Configuration configuration,
65        MetaScannerVisitor visitor)
66    throws IOException {
67      metaScan(configuration, null, visitor, null);
68    }
69  
70    /**
71     * Scans the meta table and calls a visitor on each RowResult and uses a empty
72     * start row value as table name.
73     *
74     * @param configuration conf
75     * @param connection connection to be used internally (null not allowed)
76     * @param visitor A custom visitor
77     * @throws IOException e
78     */
79    public static void metaScan(Configuration configuration, HConnection connection,
80        MetaScannerVisitor visitor, byte [] userTableName)
81    throws IOException {
82      metaScan(configuration, connection, visitor, userTableName, null, Integer.MAX_VALUE,
83          HConstants.META_TABLE_NAME);
84    }
85  
86    /**
87     * Scans the meta table and calls a visitor on each RowResult. Uses a table
88     * name and a row name to locate meta regions. And it only scans at most
89     * <code>rowLimit</code> of rows.
90     *
91     * @param configuration HBase configuration.
92     * @param visitor Visitor object.
93     * @param userTableName User table name in meta table to start scan at.  Pass
94     * null if not interested in a particular table.
95     * @param row Name of the row at the user table. The scan will start from
96     * the region row where the row resides.
97     * @param rowLimit Max of processed rows. If it is less than 0, it
98     * will be set to default value <code>Integer.MAX_VALUE</code>.
99     * @throws IOException e
100    */
101   public static void metaScan(Configuration configuration,
102       MetaScannerVisitor visitor, byte [] userTableName, byte[] row,
103       int rowLimit)
104   throws IOException {
105     metaScan(configuration, null, visitor, userTableName, row, rowLimit,
106       HConstants.META_TABLE_NAME);
107   }
108 
109   /**
110    * Scans the meta table and calls a visitor on each RowResult. Uses a table
111    * name and a row name to locate meta regions. And it only scans at most
112    * <code>rowLimit</code> of rows.
113    *
114    * @param configuration HBase configuration.
115    * @param connection connection to be used internally (null not allowed)
116    * @param visitor Visitor object. Closes the visitor before returning.
117    * @param tableName User table name in meta table to start scan at.  Pass
118    * null if not interested in a particular table.
119    * @param row Name of the row at the user table. The scan will start from
120    * the region row where the row resides.
121    * @param rowLimit Max of processed rows. If it is less than 0, it
122    * will be set to default value <code>Integer.MAX_VALUE</code>.
123    * @param metaTableName Meta table to scan, root or meta.
124    * @throws IOException e
125    */
126   public static void metaScan(Configuration configuration, HConnection connection,
127       MetaScannerVisitor visitor, byte [] tableName, byte[] row,
128       int rowLimit, final byte [] metaTableName)
129   throws IOException {
130     HTable metaTable = null;
131     try {
132       if (connection == null) {
133         metaTable = new HTable(configuration, HConstants.META_TABLE_NAME, null);
134       } else {
135         metaTable = new HTable(HConstants.META_TABLE_NAME, connection, null);
136       }
137       int rowUpperLimit = rowLimit > 0 ? rowLimit: Integer.MAX_VALUE;
138 
139       // if row is not null, we want to use the startKey of the row's region as
140       // the startRow for the meta scan.
141       byte[] startRow;
142       if (row != null) {
143         // Scan starting at a particular row in a particular table
144         assert tableName != null;
145         byte[] searchRow =
146           HRegionInfo.createRegionName(tableName, row, HConstants.NINES,
147             false);
148         Result startRowResult = metaTable.getRowOrBefore(searchRow,
149             HConstants.CATALOG_FAMILY);
150         if (startRowResult == null) {
151           throw new TableNotFoundException("Cannot find row in .META. for table: "
152               + Bytes.toString(tableName) + ", row=" + Bytes.toStringBinary(searchRow));
153         }
154         byte[] value = startRowResult.getValue(HConstants.CATALOG_FAMILY,
155             HConstants.REGIONINFO_QUALIFIER);
156         if (value == null || value.length == 0) {
157           throw new IOException("HRegionInfo was null or empty in Meta for " +
158             Bytes.toString(tableName) + ", row=" + Bytes.toStringBinary(searchRow));
159         }
160         HRegionInfo regionInfo = Writables.getHRegionInfo(value);
161 
162         byte[] rowBefore = regionInfo.getStartKey();
163         startRow = HRegionInfo.createRegionName(tableName, rowBefore,
164             HConstants.ZEROES, false);
165       } else if (tableName == null || tableName.length == 0) {
166         // Full META scan
167         startRow = HConstants.EMPTY_START_ROW;
168       } else {
169         // Scan META for an entire table
170         startRow = HRegionInfo.createRegionName(
171             tableName, HConstants.EMPTY_START_ROW, HConstants.ZEROES, false);
172       }
173 
174       // Scan over each meta region
175       ScannerCallable callable;
176       int rows = Math.min(rowLimit, configuration.getInt(
177           HConstants.HBASE_META_SCANNER_CACHING,
178           HConstants.DEFAULT_HBASE_META_SCANNER_CACHING));
179       do {
180         final Scan scan = new Scan(startRow).addFamily(HConstants.CATALOG_FAMILY);
181         if (LOG.isDebugEnabled()) {
182           LOG.debug("Scanning " + Bytes.toString(metaTableName) +
183             " starting at row=" + Bytes.toStringBinary(startRow) + " for max=" +
184             rowUpperLimit + " rows using " + metaTable.getConnection().toString());
185         }
186         callable = new ScannerCallable(metaTable.getConnection(), metaTableName, scan, null);
187         // Open scanner
188         callable.withRetries();
189 
190         int processedRows = 0;
191         try {
192           callable.setCaching(rows);
193           done: do {
194             if (processedRows >= rowUpperLimit) {
195               break;
196             }
197             //we have all the rows here
198             Result [] rrs = callable.withRetries();
199             if (rrs == null || rrs.length == 0 || rrs[0].size() == 0) {
200               break; //exit completely
201             }
202             for (Result rr : rrs) {
203               if (processedRows >= rowUpperLimit) {
204                 break done;
205               }
206               if (!visitor.processRow(rr))
207                 break done; //exit completely
208               processedRows++;
209             }
210             //here, we didn't break anywhere. Check if we have more rows
211           } while(true);
212           // Advance the startRow to the end key of the current region
213           startRow = callable.getHRegionInfo().getEndKey();
214         } finally {
215           // Close scanner
216           callable.setClose();
217           callable.withRetries();
218         }
219       } while (Bytes.compareTo(startRow, HConstants.LAST_ROW) != 0);
220     } finally {
221       visitor.close();
222       if (metaTable != null) {
223         metaTable.close();
224       }
225     }
226   }
227 
228   /**
229    * Used in tests.
230    *
231    * Lists all of the regions currently in META.
232    * @param conf
233    * @param offlined True if we are to include offlined regions, false and we'll
234    * leave out offlined regions from returned list.
235    * @return List of all user-space regions.
236    * @throws IOException
237    */
238   public static List<HRegionInfo> listAllRegions(Configuration conf, final boolean offlined)
239   throws IOException {
240     final List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
241     MetaScannerVisitor visitor = new BlockingMetaScannerVisitor(conf) {
242         @Override
243         public boolean processRowInternal(Result result) throws IOException {
244           if (result == null || result.isEmpty()) {
245             return true;
246           }
247           byte [] bytes = result.getValue(HConstants.CATALOG_FAMILY,
248             HConstants.REGIONINFO_QUALIFIER);
249           if (bytes == null) {
250             LOG.warn("Null REGIONINFO_QUALIFIER: " + result);
251             return true;
252           }
253           HRegionInfo regionInfo = Writables.getHRegionInfo(bytes);
254           // If region offline AND we are not to include offlined regions, return.
255           if (regionInfo.isOffline() && !offlined) return true;
256           regions.add(regionInfo);
257           return true;
258         }
259     };
260     metaScan(conf, visitor);
261     return regions;
262   }
263 
264   /**
265    * Lists all of the table regions currently in META.
266    * @param conf
267    * @param connection connection to be used internally (null to create a new connection)
268    * @param offlined True if we are to include offlined regions, false and we'll
269    * leave out offlined regions from returned list.
270    * @return Map of all user-space regions to servers
271    * @throws IOException
272    */
273   public static NavigableMap<HRegionInfo, ServerName> allTableRegions(Configuration conf,
274       HConnection connection, final byte[] tablename, final boolean offlined) throws IOException {
275     final NavigableMap<HRegionInfo, ServerName> regions =
276       new TreeMap<HRegionInfo, ServerName>();
277     MetaScannerVisitor visitor = new TableMetaScannerVisitor(conf, tablename) {
278       @Override
279       public boolean processRowInternal(Result rowResult) throws IOException {
280         HRegionInfo info = Writables.getHRegionInfo(
281             rowResult.getValue(HConstants.CATALOG_FAMILY,
282                 HConstants.REGIONINFO_QUALIFIER));
283         byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY,
284           HConstants.SERVER_QUALIFIER);
285         String hostAndPort = null;
286         if (value != null && value.length > 0) {
287           hostAndPort = Bytes.toString(value);
288         }
289         value = rowResult.getValue(HConstants.CATALOG_FAMILY,
290           HConstants.STARTCODE_QUALIFIER);
291         long startcode = -1L;
292         if (value != null && value.length > 0) startcode = Bytes.toLong(value);
293         if (!(info.isOffline() || info.isSplit())) {
294           ServerName sn = null;
295           if (hostAndPort != null && hostAndPort.length() > 0) {
296             sn = new ServerName(hostAndPort, startcode);
297           }
298           regions.put(new UnmodifyableHRegionInfo(info), sn);
299         }
300         return true;
301       }
302     };
303     metaScan(conf, connection, visitor, tablename);
304     return regions;
305   }
306 
307   /**
308    * Visitor class called to process each row of the .META. table
309    */
310   public interface MetaScannerVisitor extends Closeable {
311     /**
312      * Visitor method that accepts a RowResult and the meta region location.
313      * Implementations can return false to stop the region's loop if it becomes
314      * unnecessary for some reason.
315      *
316      * @param rowResult result
317      * @return A boolean to know if it should continue to loop in the region
318      * @throws IOException e
319      */
320     public boolean processRow(Result rowResult) throws IOException;
321   }
322 
323   public static abstract class MetaScannerVisitorBase implements MetaScannerVisitor {
324     @Override
325     public void close() throws IOException {
326     }
327   }
328 
329   /**
330    * A MetaScannerVisitor that provides a consistent view of the table's
331    * META entries during concurrent splits (see HBASE-5986 for details). This class
332    * does not guarantee ordered traversal of meta entries, and can block until the
333    * META entries for daughters are available during splits.
334    */
335   public static abstract class BlockingMetaScannerVisitor
336     extends MetaScannerVisitorBase {
337 
338     private static final int DEFAULT_BLOCKING_TIMEOUT = 10000;
339     private Configuration conf;
340     private TreeSet<byte[]> daughterRegions = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
341     private int blockingTimeout;
342     private HTable metaTable;
343 
344     public BlockingMetaScannerVisitor(Configuration conf) {
345       this.conf = conf;
346       this.blockingTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
347           DEFAULT_BLOCKING_TIMEOUT);
348     }
349 
350     public abstract boolean processRowInternal(Result rowResult) throws IOException;
351 
352     @Override
353     public void close() throws IOException {
354       super.close();
355       if (metaTable != null) {
356         metaTable.close();
357         metaTable = null;
358       }
359     }
360 
361     public HTable getMetaTable() throws IOException {
362       if (metaTable == null) {
363         metaTable = new HTable(conf, HConstants.META_TABLE_NAME);
364       }
365       return metaTable;
366     }
367 
368     @Override
369     public boolean processRow(Result rowResult) throws IOException {
370       HRegionInfo info = Writables.getHRegionInfoOrNull(
371           rowResult.getValue(HConstants.CATALOG_FAMILY,
372               HConstants.REGIONINFO_QUALIFIER));
373       if (info == null) {
374         return true;
375       }
376 
377       if (daughterRegions.remove(info.getRegionName())) {
378         return true; //we have already processed this row
379       }
380 
381       if (info.isSplitParent()) {
382         /* we have found a parent region which was split. We have to ensure that it's daughters are
383          * seen by this scanner as well, so we block until they are added to the META table. Even
384          * though we are waiting for META entries, ACID semantics in HBase indicates that this
385          * scanner might not see the new rows. So we manually query the daughter rows */
386         HRegionInfo splitA = Writables.getHRegionInfoOrNull(rowResult.getValue(HConstants.CATALOG_FAMILY,
387             HConstants.SPLITA_QUALIFIER));
388         HRegionInfo splitB = Writables.getHRegionInfoOrNull(rowResult.getValue(HConstants.CATALOG_FAMILY,
389             HConstants.SPLITB_QUALIFIER));
390 
391         HTable metaTable = getMetaTable();
392         long start = System.currentTimeMillis();
393         if (splitA != null) {
394           try {
395             Result resultA = getRegionResultBlocking(metaTable, blockingTimeout,
396               info.getRegionName(), splitA.getRegionName());
397             if (resultA != null) {
398               processRow(resultA);
399               daughterRegions.add(splitA.getRegionName());
400             }
401             // else parent is gone, so skip this daughter
402           } catch (TimeoutException e) {
403             throw new RegionOfflineException("Split daughter region " +
404                 splitA.getRegionNameAsString() + " cannot be found in META. Parent:" +
405                 info.getRegionNameAsString());
406           }
407         }
408         long rem = blockingTimeout - (System.currentTimeMillis() - start);
409 
410         if (splitB != null) {
411           try {
412             Result resultB = getRegionResultBlocking(metaTable, rem,
413               info.getRegionName(), splitB.getRegionName());
414             if (resultB != null) {
415               processRow(resultB);
416               daughterRegions.add(splitB.getRegionName());
417             }
418             // else parent is gone, so skip this daughter
419           } catch (TimeoutException e) {
420             throw new RegionOfflineException("Split daughter region " +
421                 splitB.getRegionNameAsString() + " cannot be found in META. Parent:" +
422                 info.getRegionNameAsString());
423           }
424         }
425       }
426 
427       return processRowInternal(rowResult);
428     }
429 
430     /**
431      * Returns region Result by querying the META table for regionName. It will block until
432      * the region is found in META. It will also check for parent in META to make sure that
433      * if parent is deleted, we no longer have to wait, and should continue (HBASE-8590)
434      * @return Result object is daughter is found, or null if parent is gone from META
435      * @throws TimeoutException if timeout is reached
436      */
437     private Result getRegionResultBlocking(HTable metaTable, long timeout, byte[] parentRegionName, byte[] regionName)
438         throws IOException, TimeoutException {
439       boolean logged = false;
440       long start = System.currentTimeMillis();
441       while (System.currentTimeMillis() - start < timeout) {
442         Get get = new Get(regionName);
443         Result result = metaTable.get(get);
444         HRegionInfo info = Writables.getHRegionInfoOrNull(
445             result.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER));
446         if (info != null) {
447           return result;
448         }
449 
450         // check whether parent is still there, if not it means we do not need to wait
451         Get parentGet = new Get(parentRegionName);
452         Result parentResult = metaTable.get(parentGet);
453         HRegionInfo parentInfo = Writables.getHRegionInfoOrNull(
454             parentResult.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER));
455         if (parentInfo == null) {
456           // this means that parent is no more (catalog janitor or somebody else deleted it)
457           return null;
458         }
459 
460         try {
461           if (!logged) {
462             if (LOG.isDebugEnabled()) {
463               LOG.debug("blocking until region is in META: " + Bytes.toStringBinary(regionName));
464             }
465             logged = true;
466           }
467           Thread.sleep(10);
468         } catch (InterruptedException ex) {
469           Thread.currentThread().interrupt();
470           break;
471         }
472       }
473       throw new TimeoutException("getRegionResultBlocking", start, System.currentTimeMillis(),
474         timeout);
475     }
476   }
477 
478   /**
479    * A MetaScannerVisitor for a table. Provides a consistent view of the table's
480    * META entries during concurrent splits (see HBASE-5986 for details). This class
481    * does not guarantee ordered traversal of meta entries, and can block until the
482    * META entries for daughters are available during splits.
483    */
484   public static abstract class TableMetaScannerVisitor extends BlockingMetaScannerVisitor {
485     private byte[] tableName;
486 
487     public TableMetaScannerVisitor(Configuration conf, byte[] tableName) {
488       super(conf);
489       this.tableName = tableName;
490     }
491 
492     @Override
493     public final boolean processRow(Result rowResult) throws IOException {
494       HRegionInfo info = Writables.getHRegionInfoOrNull(
495           rowResult.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER));
496       if (info == null) {
497         return true;
498       }
499       if (!(Bytes.equals(info.getTableName(), tableName))) {
500         return false;
501       }
502       return super.processRow(rowResult);
503     }
504 
505   }
506 }