View Javadoc

1   /**
2    * Copyright 2010 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  package org.apache.hadoop.hbase.master;
21  
22  import java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.List;
25  import java.util.Map;
26  import java.util.NavigableMap;
27  import java.util.TreeMap;
28  import java.util.concurrent.atomic.AtomicBoolean;
29  
30  import org.apache.commons.logging.Log;
31  import org.apache.commons.logging.LogFactory;
32  import org.apache.hadoop.fs.FileStatus;
33  import org.apache.hadoop.fs.Path;
34  import org.apache.hadoop.fs.PathFilter;
35  import org.apache.hadoop.hbase.Chore;
36  import org.apache.hadoop.hbase.HColumnDescriptor;
37  import org.apache.hadoop.hbase.HConstants;
38  import org.apache.hadoop.hbase.HRegionInfo;
39  import org.apache.hadoop.hbase.HServerInfo;
40  import org.apache.hadoop.hbase.RemoteExceptionHandler;
41  import org.apache.hadoop.hbase.UnknownScannerException;
42  import org.apache.hadoop.hbase.client.Delete;
43  import org.apache.hadoop.hbase.client.Get;
44  import org.apache.hadoop.hbase.client.Put;
45  import org.apache.hadoop.hbase.client.Result;
46  import org.apache.hadoop.hbase.client.Scan;
47  import org.apache.hadoop.hbase.ipc.HRegionInterface;
48  import org.apache.hadoop.hbase.regionserver.HRegion;
49  import org.apache.hadoop.hbase.regionserver.Store;
50  import org.apache.hadoop.hbase.regionserver.StoreFile;
51  import org.apache.hadoop.hbase.util.Bytes;
52  import org.apache.hadoop.hbase.util.Writables;
53  import org.apache.hadoop.io.BooleanWritable;
54  import org.apache.hadoop.ipc.RemoteException;
55  
56  
57  /**
58   * Base HRegion scanner class. Holds utilty common to <code>ROOT</code> and
59   * <code>META</code> HRegion scanners.
60   *
61   * <p>How do we know if all regions are assigned? After the initial scan of
62   * the <code>ROOT</code> and <code>META</code> regions, all regions known at
63   * that time will have been or are in the process of being assigned.</p>
64   *
65   * <p>When a region is split the region server notifies the master of the
66   * split and the new regions are assigned. But suppose the master loses the
67   * split message? We need to periodically rescan the <code>ROOT</code> and
68   * <code>META</code> regions.
69   *    <ul>
70   *    <li>If we rescan, any regions that are new but not assigned will have
71   *    no server info. Any regions that are not being served by the same
72   *    server will get re-assigned.</li>
73   *
74   *    <li>Thus a periodic rescan of the root region will find any new
75   *    <code>META</code> regions where we missed the <code>META</code> split
76   *    message or we failed to detect a server death and consequently need to
77   *    assign the region to a new server.</li>
78   *
79   *    <li>if we keep track of all the known <code>META</code> regions, then
80   *    we can rescan them periodically. If we do this then we can detect any
81   *    regions for which we missed a region split message.</li>
82   *    </ul>
83   *
84   * Thus just keeping track of all the <code>META</code> regions permits
85   * periodic rescanning which will detect unassigned regions (new or
86   * otherwise) without the need to keep track of every region.</p>
87   *
88   * <p>So the <code>ROOT</code> region scanner needs to wake up:
89   * <ol>
90   * <li>when the master receives notification that the <code>ROOT</code>
91   * region has been opened.</li>
92   * <li>periodically after the first scan</li>
93   * </ol>
94   *
95   * The <code>META</code>  scanner needs to wake up:
96   * <ol>
97   * <li>when a <code>META</code> region comes on line</li>
98   * </li>periodically to rescan the online <code>META</code> regions</li>
99   * </ol>
100  *
101  * <p>A <code>META</code> region is not 'online' until it has been scanned
102  * once.
103  */
104 abstract class BaseScanner extends Chore {
105   static final Log LOG = LogFactory.getLog(BaseScanner.class.getName());
106   // These are names of new columns in a meta region offlined parent row.  They
107   // are added by the metascanner after we verify that split daughter made it
108   // in.  Their value is 'true' if present.
109   private static final byte[] SPLITA_CHECKED =
110     Bytes.toBytes(Bytes.toString(HConstants.SPLITA_QUALIFIER) + "_checked");
111   private static final byte[] SPLITB_CHECKED =
112     Bytes.toBytes(Bytes.toString(HConstants.SPLITB_QUALIFIER) + "_checked");
113   // Make the 'true' Writable once only.
114   private static byte[] TRUE_WRITABLE_AS_BYTES;
115   static {
116     try {
117       TRUE_WRITABLE_AS_BYTES = Writables.getBytes(new BooleanWritable(true));
118     } catch (IOException e) {
119       e.printStackTrace();
120     }
121   }
122   private final boolean rootRegion;
123   protected final HMaster master;
124 
125   protected boolean initialScanComplete;
126 
127   protected abstract boolean initialScan();
128   protected abstract void maintenanceScan();
129 
130   // will use this variable to synchronize and make sure we aren't interrupted
131   // mid-scan
132   final Object scannerLock = new Object();
133 
134   BaseScanner(final HMaster master, final boolean rootRegion,
135       final AtomicBoolean stop) {
136     super("Scanner for " + (rootRegion ? "-ROOT-":".META.") + " table",
137         master.getConfiguration().
138         getInt("hbase.master.meta.thread.rescanfrequency", 60 * 1000), stop);
139     this.rootRegion = rootRegion;
140     this.master = master;
141     this.initialScanComplete = false;
142   }
143 
144   /** @return true if initial scan completed successfully */
145   public boolean isInitialScanComplete() {
146     return initialScanComplete;
147   }
148 
149   @Override
150   protected boolean initialChore() {
151     return initialScan();
152   }
153 
154   @Override
155   protected void chore() {
156     maintenanceScan();
157   }
158 
159   /**
160    * @param region Region to scan
161    * @throws IOException
162    */
163   protected void scanRegion(final MetaRegion region) throws IOException {
164     HRegionInterface regionServer = null;
165     long scannerId = -1L;
166     LOG.info(Thread.currentThread().getName() + " scanning meta region " +
167       region.toString());
168 
169     // Array to hold list of split parents found.  Scan adds to list.  After
170     // scan we go check if parents can be removed and that their daughters
171     // are in place.
172     NavigableMap<HRegionInfo, Result> splitParents =
173       new TreeMap<HRegionInfo, Result>();
174     List<byte []> emptyRows = new ArrayList<byte []>();
175     int rows = 0;
176     try {
177       regionServer =
178         this.master.getServerConnection().getHRegionConnection(region.getServer());
179       Scan s = new Scan().addFamily(HConstants.CATALOG_FAMILY);
180       // Make this scan do a row at a time otherwise, data can be stale.
181       s.setCaching(1);
182       scannerId = regionServer.openScanner(region.getRegionName(), s);
183       while (true) {
184         Result values = regionServer.next(scannerId);
185         if (values == null || values.size() == 0) {
186           break;
187         }
188         HRegionInfo info = master.getHRegionInfo(values.getRow(), values);
189         if (info == null) {
190           emptyRows.add(values.getRow());
191           continue;
192         }
193         String serverAddress = getServerAddress(values);
194         long startCode = getStartCode(values);
195 
196         // Note Region has been assigned.
197         checkAssigned(regionServer, region, info, serverAddress, startCode);
198         if (isSplitParent(info)) {
199           splitParents.put(info, values);
200         }
201         rows += 1;
202       }
203       if (rootRegion) {
204         this.master.getRegionManager().setNumMetaRegions(rows);
205       }
206     } catch (IOException e) {
207       if (e instanceof RemoteException) {
208         e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
209         if (e instanceof UnknownScannerException) {
210           // Reset scannerId so we do not try closing a scanner the other side
211           // has lost account of: prevents duplicated stack trace out of the
212           // below close in the finally.
213           scannerId = -1L;
214         }
215       }
216       throw e;
217     } finally {
218       try {
219         if (scannerId != -1L && regionServer != null) {
220           regionServer.close(scannerId);
221         }
222       } catch (IOException e) {
223         LOG.error("Closing scanner",
224             RemoteExceptionHandler.checkIOException(e));
225       }
226     }
227 
228     // Scan is finished.
229 
230     // First clean up any meta region rows which had null HRegionInfos
231     if (emptyRows.size() > 0) {
232       LOG.warn("Found " + emptyRows.size() + " rows with empty HRegionInfo " +
233         "while scanning meta region " + Bytes.toString(region.getRegionName()));
234       this.master.deleteEmptyMetaRows(regionServer, region.getRegionName(),
235           emptyRows);
236     }
237 
238     // Take a look at split parents to see if any we can clean up any and to
239     // make sure that daughter regions are in place.
240     if (splitParents.size() > 0) {
241       for (Map.Entry<HRegionInfo, Result> e : splitParents.entrySet()) {
242         HRegionInfo hri = e.getKey();
243         cleanupAndVerifySplits(region.getRegionName(), regionServer,
244           hri, e.getValue());
245       }
246     }
247     LOG.info(Thread.currentThread().getName() + " scan of " + rows +
248       " row(s) of meta region " + region.toString() + " complete");
249   }
250 
251   /*
252    * @param r
253    * @return Empty String or server address found in <code>r</code>
254    */
255   static String getServerAddress(final Result r) {
256     final byte[] val = r.getValue(HConstants.CATALOG_FAMILY,
257         HConstants.SERVER_QUALIFIER);
258     return val == null || val.length <= 0 ? "" : Bytes.toString(val);
259   }
260 
261   /*
262    * @param r
263    * @return Return 0L or server startcode found in <code>r</code>
264    */
265   static long getStartCode(final Result r) {
266     final byte[] val = r.getValue(HConstants.CATALOG_FAMILY,
267         HConstants.STARTCODE_QUALIFIER);
268     return val == null || val.length <= 0 ? 0L : Bytes.toLong(val);
269   }
270 
271   /*
272    * @param info Region to check.
273    * @return True if this is a split parent.
274    */
275   private boolean isSplitParent(final HRegionInfo info) {
276     if (!info.isSplit()) {
277       return false;
278     }
279     if (!info.isOffline()) {
280       LOG.warn("Region is split but not offline: " +
281         info.getRegionNameAsString());
282     }
283     return true;
284   }
285 
286   /*
287    * If daughters no longer hold reference to the parents, delete the parent.
288    * If the parent is lone without daughter splits AND there are references in
289    * the filesystem, then a daughters was not added to .META. -- must have been
290    * a crash before their addition.  Add them here.
291    * @param metaRegionName Meta region name: e.g. .META.,,1
292    * @param server HRegionInterface of meta server to talk to
293    * @param parent HRegionInfo of split offlined parent
294    * @param rowContent Content of <code>parent</code> row in
295    * <code>metaRegionName</code>
296    * @return True if we removed <code>parent</code> from meta table and from
297    * the filesystem.
298    * @throws IOException
299    */
300   private boolean cleanupAndVerifySplits(final byte [] metaRegionName,
301     final HRegionInterface srvr, final HRegionInfo parent,
302     Result rowContent)
303   throws IOException {
304     boolean result = false;
305     // Run checks on each daughter split.
306     boolean hasReferencesA = checkDaughter(metaRegionName, srvr,
307       parent, rowContent, HConstants.SPLITA_QUALIFIER);
308     boolean hasReferencesB = checkDaughter(metaRegionName, srvr,
309         parent, rowContent, HConstants.SPLITB_QUALIFIER);
310     if (!hasReferencesA && !hasReferencesB) {
311       LOG.info("Deleting region " + parent.getRegionNameAsString() +
312         " because daughter splits no longer hold references");
313       HRegion.deleteRegion(this.master.getFileSystem(),
314         this.master.getRootDir(), parent);
315       HRegion.removeRegionFromMETA(srvr, metaRegionName,
316         parent.getRegionName());
317       result = true;
318     }
319     return result;
320   }
321 
322 
323   /*
324    * See if the passed daughter has references in the filesystem to the parent
325    * and if not, remove the note of daughter region in the parent row: its
326    * column info:splitA or info:splitB.  Also make sure that daughter row is
327    * present in the .META. and mark the parent row when confirmed so we don't
328    * keep checking.  The mark will be info:splitA_checked and its value will be
329    * a true BooleanWritable.
330    * @param metaRegionName
331    * @param srvr
332    * @param parent
333    * @param rowContent
334    * @param qualifier
335    * @return True if this daughter still has references to the parent.
336    * @throws IOException
337    */
338   private boolean checkDaughter(final byte [] metaRegionName,
339     final HRegionInterface srvr, final HRegionInfo parent,
340     final Result rowContent, final byte [] qualifier)
341   throws IOException {
342     HRegionInfo hri = getDaughterRegionInfo(rowContent, qualifier);
343     boolean references = hasReferences(metaRegionName, srvr, parent, rowContent,
344         hri, qualifier);
345     // Return if no references.
346     if (!references) return references;
347     if (!verifyDaughterRowPresent(rowContent, qualifier, srvr, metaRegionName,
348         hri, parent)) {
349       // If we got here, then the parent row does not yet have the
350       // "daughter row verified present" marker present. Add it.
351       addDaughterRowChecked(metaRegionName, srvr, parent.getRegionName(), hri,
352         qualifier);
353     }
354     return references;
355   }
356 
357   /*
358    * Check the daughter of parent is present in meta table.  If not there,
359    * add it.
360    * @param rowContent
361    * @param daughter
362    * @param srvr
363    * @param metaRegionName
364    * @param daughterHRI
365    * @throws IOException
366    * @return True, if parent row has marker for "daughter row verified present"
367    * else, false (and will do fixup adding daughter if daughter not present).
368    */
369   private boolean verifyDaughterRowPresent(final Result rowContent,
370       final byte [] daughter, final HRegionInterface srvr,
371       final byte [] metaRegionName,
372       final HRegionInfo daughterHRI, final HRegionInfo parent)
373   throws IOException {
374     // See if the 'checked' column is in parent. If so, we're done.
375     boolean present = getDaughterRowChecked(rowContent, daughter);
376     if (present) return present;
377     // Parent is not carrying the splitA_checked/splitB_checked so this must
378     // be the first time through here checking splitA/splitB are in metatable.
379     byte [] daughterRowKey = daughterHRI.getRegionName();
380     Get g = new Get(daughterRowKey);
381     g.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
382     Result r = srvr.get(metaRegionName, g);
383     if (r == null || r.isEmpty()) {
384       // Daughter row not present.  Fixup kicks in.  Insert it.
385       LOG.warn("Fixup broke split: Add missing split daughter to meta," +
386        " daughter=" + daughterHRI.toString() + ", parent=" + parent.toString());
387       Put p = new Put(daughterRowKey);
388       p.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
389         Writables.getBytes(daughterHRI));
390       srvr.put(metaRegionName, p);
391     }
392     return present;
393   }
394 
395   /*
396    * Add to parent a marker that we verified the daughter exists.
397    * @param metaRegionName
398    * @param srvr
399    * @param parent
400    * @param split
401    * @param daughter
402    * @throws IOException
403    */
404   private void addDaughterRowChecked(final byte [] metaRegionName,
405     final HRegionInterface srvr, final byte [] parent,
406     final HRegionInfo split, final byte [] daughter)
407   throws IOException {
408     Put p = new Put(parent);
409     p.add(HConstants.CATALOG_FAMILY, getNameOfVerifiedDaughterColumn(daughter),
410       TRUE_WRITABLE_AS_BYTES);
411     srvr.put(metaRegionName, p);
412   }
413 
414   /*
415    * @param rowContent
416    * @param which
417    * @return True if the daughter row has already been verified present in
418    * metatable.
419    * @throws IOException
420    */
421   private boolean getDaughterRowChecked(final Result rowContent,
422     final byte[] which)
423   throws IOException {
424     final byte[] b = rowContent.getValue(HConstants.CATALOG_FAMILY,
425       getNameOfVerifiedDaughterColumn(which));
426     BooleanWritable bw = null;
427     if (b != null && b.length > 0) {
428       bw = (BooleanWritable)Writables.getWritable(b, new BooleanWritable());
429     }
430     return bw == null? false: bw.get();
431   }
432 
433   /*
434    * @param daughter
435    * @return Returns splitA_checked or splitB_checked dependent on what
436    * <code>daughter</code> is.
437    */
438   private static byte [] getNameOfVerifiedDaughterColumn(final byte [] daughter) {
439     return (Bytes.equals(HConstants.SPLITA_QUALIFIER, daughter)
440             ? SPLITA_CHECKED : SPLITB_CHECKED);
441   }
442 
443   /*
444    * Get daughter HRegionInfo out of parent info:splitA/info:splitB columns.
445    * @param rowContent
446    * @param which Whether "info:splitA" or "info:splitB" column
447    * @return Deserialized content of the info:splitA or info:splitB as a
448    * HRegionInfo
449    * @throws IOException
450    */
451   private HRegionInfo getDaughterRegionInfo(final Result rowContent,
452     final byte [] which)
453   throws IOException {
454     return Writables.getHRegionInfoOrNull(
455         rowContent.getValue(HConstants.CATALOG_FAMILY, which));
456   }
457 
458   /*
459    * Remove mention of daughter from parent row.
460    * parent row.
461    * @param metaRegionName
462    * @param srvr
463    * @param parent
464    * @param split
465    * @param qualifier
466    * @throws IOException
467    */
468   private void removeDaughterFromParent(final byte [] metaRegionName,
469     final HRegionInterface srvr, final HRegionInfo parent,
470     final HRegionInfo split, final byte [] qualifier)
471   throws IOException {
472     if (LOG.isDebugEnabled()) {
473       LOG.debug(split.getRegionNameAsString() +
474         " no longer has references to " + parent.getRegionNameAsString());
475     }
476     Delete delete = new Delete(parent.getRegionName());
477     delete.deleteColumns(HConstants.CATALOG_FAMILY, qualifier);
478     srvr.delete(metaRegionName, delete);
479   }
480 
481   /*
482    * Checks if a daughter region -- either splitA or splitB -- still holds
483    * references to parent.  If not, removes reference to the split from
484    * the parent meta region row so we don't check it any more.
485    * @param metaRegionName Name of meta region to look in.
486    * @param srvr Where region resides.
487    * @param parent Parent region name.
488    * @param rowContent Keyed content of the parent row in meta region.
489    * @param split Which column family.
490    * @param qualifier Which of the daughters to look at, splitA or splitB.
491    * @return True if still has references to parent.
492    * @throws IOException
493    */
494   private boolean hasReferences(final byte [] metaRegionName,
495     final HRegionInterface srvr, final HRegionInfo parent,
496     Result rowContent, final HRegionInfo split, byte [] qualifier)
497   throws IOException {
498     boolean result = false;
499     if (split == null) {
500       return result;
501     }
502     Path tabledir =
503       new Path(this.master.getRootDir(), split.getTableDesc().getNameAsString());
504     for (HColumnDescriptor family: split.getTableDesc().getFamilies()) {
505       Path p = Store.getStoreHomedir(tabledir, split.getEncodedName(),
506         family.getName());
507       if (!this.master.getFileSystem().exists(p)) continue;
508       // Look for reference files.  Call listStatus with an anonymous
509       // instance of PathFilter.
510       FileStatus [] ps =
511         this.master.getFileSystem().listStatus(p, new PathFilter () {
512             public boolean accept(Path path) {
513               return StoreFile.isReference(path);
514             }
515           }
516       );
517 
518       if (ps != null && ps.length > 0) {
519         result = true;
520         break;
521       }
522     }
523     if (!result) {
524       removeDaughterFromParent(metaRegionName, srvr, parent, split, qualifier);
525     }
526     return result;
527   }
528 
529   /*
530    * Check the passed region is assigned.  If not, add to unassigned.
531    * @param regionServer
532    * @param meta
533    * @param info
534    * @param hostnameAndPort hostname ':' port as it comes out of .META.
535    * @param startCode
536    * @throws IOException
537    */
538   protected void checkAssigned(final HRegionInterface regionServer,
539     final MetaRegion meta, final HRegionInfo info,
540     final String hostnameAndPort, final long startCode)
541   throws IOException {
542     String serverName = null;
543     String sa = hostnameAndPort;
544     long sc = startCode;
545     if (sa == null || sa.length() <= 0) {
546       // Scans are sloppy.  They cache a row internally so may have data that
547       // is a little stale.  Make sure that for sure this serverAddress is null.
548       // We are trying to avoid double-assignments.  See hbase-1784.
549       Get g = new Get(info.getRegionName());
550       g.addFamily(HConstants.CATALOG_FAMILY);
551       Result r = regionServer.get(meta.getRegionName(), g);
552       if (r != null && !r.isEmpty()) {
553         sa = getServerAddress(r);
554         sc = getStartCode(r);
555       }
556     }
557     if (sa != null && sa.length() > 0) {
558       serverName = HServerInfo.getServerName(sa, sc);
559     }
560     HServerInfo storedInfo = null;
561     synchronized (this.master.getRegionManager()) {
562       /* We don't assign regions that are offline, in transition or were on
563        * a dead server. Regions that were on a dead server will get reassigned
564        * by ProcessServerShutdown
565        */
566       if (info.isOffline() ||
567         this.master.getRegionManager().regionIsInTransition(info.getRegionNameAsString()) ||
568           (serverName != null && this.master.getServerManager().isDead(serverName))) {
569         return;
570       }
571       if (serverName != null) {
572         storedInfo = this.master.getServerManager().getServerInfo(serverName);
573       }
574 
575       // If we can't find the HServerInfo, then add it to the list of
576       //  unassigned regions.
577       if (storedInfo == null) {
578         // The current assignment is invalid
579         if (LOG.isDebugEnabled()) {
580           LOG.debug("Current assignment of " + info.getRegionNameAsString() +
581             " is not valid; " + " serverAddress=" + sa +
582             ", startCode=" + sc + " unknown.");
583         }
584         // Now get the region assigned
585         this.master.getRegionManager().setUnassigned(info, true);
586       }
587     }
588   }
589 
590   /**
591    * Interrupt thread regardless of what it's doing
592    */
593   public void interruptAndStop() {
594     synchronized(scannerLock){
595       if (isAlive()) {
596         super.interrupt();
597         LOG.info("Interrupted");
598       }
599     }
600   }
601 }