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  package org.apache.hadoop.hbase.master;
20  
21  import java.io.FileNotFoundException;
22  import java.io.IOException;
23  import java.util.Comparator;
24  import java.util.HashSet;
25  import java.util.Map;
26  import java.util.TreeMap;
27  import java.util.concurrent.atomic.AtomicBoolean;
28  import java.util.concurrent.atomic.AtomicInteger;
29  
30  import org.apache.commons.logging.Log;
31  import org.apache.commons.logging.LogFactory;
32  import org.apache.hadoop.fs.FileSystem;
33  import org.apache.hadoop.fs.Path;
34  import org.apache.hadoop.hbase.HColumnDescriptor;
35  import org.apache.hadoop.hbase.HConstants;
36  import org.apache.hadoop.hbase.HRegionInfo;
37  import org.apache.hadoop.hbase.HTableDescriptor;
38  import org.apache.hadoop.hbase.MetaTableAccessor;
39  import org.apache.hadoop.hbase.ScheduledChore;
40  import org.apache.hadoop.hbase.Server;
41  import org.apache.hadoop.hbase.TableName;
42  import org.apache.hadoop.hbase.backup.HFileArchiver;
43  import org.apache.hadoop.hbase.classification.InterfaceAudience;
44  import org.apache.hadoop.hbase.client.Connection;
45  import org.apache.hadoop.hbase.client.MetaScanner;
46  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
47  import org.apache.hadoop.hbase.client.Result;
48  import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
49  import org.apache.hadoop.hbase.util.Bytes;
50  import org.apache.hadoop.hbase.util.FSUtils;
51  import org.apache.hadoop.hbase.util.Pair;
52  import org.apache.hadoop.hbase.util.PairOfSameType;
53  import org.apache.hadoop.hbase.util.Threads;
54  import org.apache.hadoop.hbase.util.Triple;
55  
56  /**
57   * A janitor for the catalog tables.  Scans the <code>hbase:meta</code> catalog
58   * table on a period looking for unused regions to garbage collect.
59   */
60  @InterfaceAudience.Private
61  public class CatalogJanitor extends ScheduledChore {
62    private static final Log LOG = LogFactory.getLog(CatalogJanitor.class.getName());
63    private final Server server;
64    private final MasterServices services;
65    private AtomicBoolean enabled = new AtomicBoolean(true);
66    private AtomicBoolean alreadyRunning = new AtomicBoolean(false);
67    private final Connection connection;
68  
69    CatalogJanitor(final Server server, final MasterServices services) {
70      super("CatalogJanitor-" + server.getServerName().toShortString(), server, server
71          .getConfiguration().getInt("hbase.catalogjanitor.interval", 300000));
72      this.server = server;
73      this.services = services;
74      this.connection = server.getConnection();
75    }
76  
77    @Override
78    protected boolean initialChore() {
79      try {
80        if (this.enabled.get()) scan();
81      } catch (IOException e) {
82        LOG.warn("Failed initial scan of catalog table", e);
83        return false;
84      }
85      return true;
86    }
87  
88    /**
89     * @param enabled
90     */
91    public boolean setEnabled(final boolean enabled) {
92      boolean alreadyEnabled = this.enabled.getAndSet(enabled);
93      // If disabling is requested on an already enabled chore, we could have an active
94      // scan still going on, callers might not be aware of that and do further action thinkng
95      // that no action would be from this chore.  In this case, the right action is to wait for
96      // the active scan to complete before exiting this function.
97      if (!enabled && alreadyEnabled) {
98        while (alreadyRunning.get()) {
99          Threads.sleepWithoutInterrupt(100);
100       }
101     }
102     return alreadyEnabled;
103   }
104 
105   boolean getEnabled() {
106     return this.enabled.get();
107   }
108 
109   @Override
110   protected void chore() {
111     try {
112       AssignmentManager am = this.services.getAssignmentManager();
113       if (this.enabled.get()
114           && am != null
115           && am.isFailoverCleanupDone()
116           && am.getRegionStates().getRegionsInTransition().size() == 0) {
117         scan();
118       } else {
119         LOG.warn("CatalogJanitor disabled! Not running scan.");
120       }
121     } catch (IOException e) {
122       LOG.warn("Failed scan of catalog table", e);
123     }
124   }
125 
126   /**
127    * Scans hbase:meta and returns a number of scanned rows, and a map of merged
128    * regions, and an ordered map of split parents.
129    * @return triple of scanned rows, map of merged regions and map of split
130    *         parent regioninfos
131    * @throws IOException
132    */
133   Triple<Integer, Map<HRegionInfo, Result>, Map<HRegionInfo, Result>> getMergedRegionsAndSplitParents()
134       throws IOException {
135     return getMergedRegionsAndSplitParents(null);
136   }
137 
138   /**
139    * Scans hbase:meta and returns a number of scanned rows, and a map of merged
140    * regions, and an ordered map of split parents. if the given table name is
141    * null, return merged regions and split parents of all tables, else only the
142    * specified table
143    * @param tableName null represents all tables
144    * @return triple of scanned rows, and map of merged regions, and map of split
145    *         parent regioninfos
146    * @throws IOException
147    */
148   Triple<Integer, Map<HRegionInfo, Result>, Map<HRegionInfo, Result>> getMergedRegionsAndSplitParents(
149       final TableName tableName) throws IOException {
150     final boolean isTableSpecified = (tableName != null);
151     // TODO: Only works with single hbase:meta region currently.  Fix.
152     final AtomicInteger count = new AtomicInteger(0);
153     // Keep Map of found split parents.  There are candidates for cleanup.
154     // Use a comparator that has split parents come before its daughters.
155     final Map<HRegionInfo, Result> splitParents =
156       new TreeMap<HRegionInfo, Result>(new SplitParentFirstComparator());
157     final Map<HRegionInfo, Result> mergedRegions = new TreeMap<HRegionInfo, Result>();
158     // This visitor collects split parents and counts rows in the hbase:meta table
159 
160     MetaScannerVisitor visitor = new MetaScanner.MetaScannerVisitorBase() {
161       @Override
162       public boolean processRow(Result r) throws IOException {
163         if (r == null || r.isEmpty()) return true;
164         count.incrementAndGet();
165         HRegionInfo info = HRegionInfo.getHRegionInfo(r);
166         if (info == null) return true; // Keep scanning
167         if (isTableSpecified
168             && info.getTable().compareTo(tableName) > 0) {
169           // Another table, stop scanning
170           return false;
171         }
172         if (info.isSplitParent()) splitParents.put(info, r);
173         if (r.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER) != null) {
174           mergedRegions.put(info, r);
175         }
176         // Returning true means "keep scanning"
177         return true;
178       }
179     };
180 
181     // Run full scan of hbase:meta catalog table passing in our custom visitor with
182     // the start row
183     MetaScanner.metaScan(this.connection, visitor, tableName);
184 
185     return new Triple<Integer, Map<HRegionInfo, Result>, Map<HRegionInfo, Result>>(
186         count.get(), mergedRegions, splitParents);
187   }
188 
189   /**
190    * If merged region no longer holds reference to the merge regions, archive
191    * merge region on hdfs and perform deleting references in hbase:meta
192    * @param mergedRegion
193    * @param regionA
194    * @param regionB
195    * @return true if we delete references in merged region on hbase:meta and archive
196    *         the files on the file system
197    * @throws IOException
198    */
199   boolean cleanMergeRegion(final HRegionInfo mergedRegion,
200       final HRegionInfo regionA, final HRegionInfo regionB) throws IOException {
201     FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
202     Path rootdir = this.services.getMasterFileSystem().getRootDir();
203     Path tabledir = FSUtils.getTableDir(rootdir, mergedRegion.getTable());
204     HTableDescriptor htd = getTableDescriptor(mergedRegion.getTable());
205     HRegionFileSystem regionFs = null;
206     try {
207       regionFs = HRegionFileSystem.openRegionFromFileSystem(
208           this.services.getConfiguration(), fs, tabledir, mergedRegion, true);
209     } catch (IOException e) {
210       LOG.warn("Merged region does not exist: " + mergedRegion.getEncodedName());
211     }
212     if (regionFs == null || !regionFs.hasReferences(htd)) {
213       LOG.debug("Deleting region " + regionA.getRegionNameAsString() + " and "
214           + regionB.getRegionNameAsString()
215           + " from fs because merged region no longer holds references");
216       HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, regionA);
217       HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, regionB);
218       MetaTableAccessor.deleteMergeQualifiers(services.getConnection(), mergedRegion);
219       services.getServerManager().removeRegion(regionA);
220       services.getServerManager().removeRegion(regionB);
221       return true;
222     }
223     return false;
224   }
225 
226   /**
227    * Run janitorial scan of catalog <code>hbase:meta</code> table looking for
228    * garbage to collect.
229    * @return number of cleaned regions
230    * @throws IOException
231    */
232   int scan() throws IOException {
233     try {
234       if (!alreadyRunning.compareAndSet(false, true)) {
235         return 0;
236       }
237       Triple<Integer, Map<HRegionInfo, Result>, Map<HRegionInfo, Result>> scanTriple =
238         getMergedRegionsAndSplitParents();
239       int count = scanTriple.getFirst();
240       /**
241        * clean merge regions first
242        */
243       int mergeCleaned = 0;
244       Map<HRegionInfo, Result> mergedRegions = scanTriple.getSecond();
245       for (Map.Entry<HRegionInfo, Result> e : mergedRegions.entrySet()) {
246         HRegionInfo regionA = HRegionInfo.getHRegionInfo(e.getValue(),
247             HConstants.MERGEA_QUALIFIER);
248         HRegionInfo regionB = HRegionInfo.getHRegionInfo(e.getValue(),
249             HConstants.MERGEB_QUALIFIER);
250         if (regionA == null || regionB == null) {
251           LOG.warn("Unexpected references regionA="
252               + (regionA == null ? "null" : regionA.getRegionNameAsString())
253               + ",regionB="
254               + (regionB == null ? "null" : regionB.getRegionNameAsString())
255               + " in merged region " + e.getKey().getRegionNameAsString());
256         } else {
257           if (cleanMergeRegion(e.getKey(), regionA, regionB)) {
258             mergeCleaned++;
259           }
260         }
261       }
262       /**
263        * clean split parents
264        */
265       Map<HRegionInfo, Result> splitParents = scanTriple.getThird();
266 
267       // Now work on our list of found parents. See if any we can clean up.
268       int splitCleaned = 0;
269       // regions whose parents are still around
270       HashSet<String> parentNotCleaned = new HashSet<String>();
271       for (Map.Entry<HRegionInfo, Result> e : splitParents.entrySet()) {
272         if (!parentNotCleaned.contains(e.getKey().getEncodedName()) &&
273             cleanParent(e.getKey(), e.getValue())) {
274           splitCleaned++;
275         } else {
276           // We could not clean the parent, so it's daughters should not be cleaned either (HBASE-6160)
277           PairOfSameType<HRegionInfo> daughters = HRegionInfo.getDaughterRegions(e.getValue());
278           parentNotCleaned.add(daughters.getFirst().getEncodedName());
279           parentNotCleaned.add(daughters.getSecond().getEncodedName());
280         }
281       }
282       if ((mergeCleaned + splitCleaned) != 0) {
283         LOG.info("Scanned " + count + " catalog row(s), gc'd " + mergeCleaned
284             + " unreferenced merged region(s) and " + splitCleaned
285             + " unreferenced parent region(s)");
286       } else if (LOG.isTraceEnabled()) {
287         LOG.trace("Scanned " + count + " catalog row(s), gc'd " + mergeCleaned
288             + " unreferenced merged region(s) and " + splitCleaned
289             + " unreferenced parent region(s)");
290       }
291       return mergeCleaned + splitCleaned;
292     } finally {
293       alreadyRunning.set(false);
294     }
295   }
296 
297   /**
298    * Compare HRegionInfos in a way that has split parents sort BEFORE their
299    * daughters.
300    */
301   static class SplitParentFirstComparator implements Comparator<HRegionInfo> {
302     Comparator<byte[]> rowEndKeyComparator = new Bytes.RowEndKeyComparator();
303     @Override
304     public int compare(HRegionInfo left, HRegionInfo right) {
305       // This comparator differs from the one HRegionInfo in that it sorts
306       // parent before daughters.
307       if (left == null) return -1;
308       if (right == null) return 1;
309       // Same table name.
310       int result = left.getTable().compareTo(right.getTable());
311       if (result != 0) return result;
312       // Compare start keys.
313       result = Bytes.compareTo(left.getStartKey(), right.getStartKey());
314       if (result != 0) return result;
315       // Compare end keys, but flip the operands so parent comes first
316       result = rowEndKeyComparator.compare(right.getEndKey(), left.getEndKey());
317 
318       return result;
319     }
320   }
321 
322   /**
323    * If daughters no longer hold reference to the parents, delete the parent.
324    * @param parent HRegionInfo of split offlined parent
325    * @param rowContent Content of <code>parent</code> row in
326    * <code>metaRegionName</code>
327    * @return True if we removed <code>parent</code> from meta table and from
328    * the filesystem.
329    * @throws IOException
330    */
331   boolean cleanParent(final HRegionInfo parent, Result rowContent)
332   throws IOException {
333     boolean result = false;
334     // Check whether it is a merged region and not clean reference
335     // No necessary to check MERGEB_QUALIFIER because these two qualifiers will
336     // be inserted/deleted together
337     if (rowContent.getValue(HConstants.CATALOG_FAMILY,
338         HConstants.MERGEA_QUALIFIER) != null) {
339       // wait cleaning merge region first
340       return result;
341     }
342     // Run checks on each daughter split.
343     PairOfSameType<HRegionInfo> daughters = HRegionInfo.getDaughterRegions(rowContent);
344     Pair<Boolean, Boolean> a = checkDaughterInFs(parent, daughters.getFirst());
345     Pair<Boolean, Boolean> b = checkDaughterInFs(parent, daughters.getSecond());
346     if (hasNoReferences(a) && hasNoReferences(b)) {
347       LOG.debug("Deleting region " + parent.getRegionNameAsString() +
348         " because daughter splits no longer hold references");
349       FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
350       if (LOG.isTraceEnabled()) LOG.trace("Archiving parent region: " + parent);
351       HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, parent);
352       MetaTableAccessor.deleteRegion(this.connection, parent);
353       services.getServerManager().removeRegion(parent);
354       result = true;
355     }
356     return result;
357   }
358 
359   /**
360    * @param p A pair where the first boolean says whether or not the daughter
361    * region directory exists in the filesystem and then the second boolean says
362    * whether the daughter has references to the parent.
363    * @return True the passed <code>p</code> signifies no references.
364    */
365   private boolean hasNoReferences(final Pair<Boolean, Boolean> p) {
366     return !p.getFirst() || !p.getSecond();
367   }
368 
369   /**
370    * Checks if a daughter region -- either splitA or splitB -- still holds
371    * references to parent.
372    * @param parent Parent region
373    * @param daughter Daughter region
374    * @return A pair where the first boolean says whether or not the daughter
375    * region directory exists in the filesystem and then the second boolean says
376    * whether the daughter has references to the parent.
377    * @throws IOException
378    */
379   Pair<Boolean, Boolean> checkDaughterInFs(final HRegionInfo parent, final HRegionInfo daughter)
380   throws IOException {
381     if (daughter == null)  {
382       return new Pair<Boolean, Boolean>(Boolean.FALSE, Boolean.FALSE);
383     }
384 
385     FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
386     Path rootdir = this.services.getMasterFileSystem().getRootDir();
387     Path tabledir = FSUtils.getTableDir(rootdir, daughter.getTable());
388 
389     Path daughterRegionDir = new Path(tabledir, daughter.getEncodedName());
390 
391     HRegionFileSystem regionFs = null;
392 
393     try {
394       if (!FSUtils.isExists(fs, daughterRegionDir)) {
395         return new Pair<Boolean, Boolean>(Boolean.FALSE, Boolean.FALSE);
396       }
397     } catch (IOException ioe) {
398       LOG.error("Error trying to determine if daughter region exists, " +
399                "assuming exists and has references", ioe);
400       return new Pair<Boolean, Boolean>(Boolean.TRUE, Boolean.TRUE);
401     }
402 
403     boolean references = false;
404     HTableDescriptor parentDescriptor = getTableDescriptor(parent.getTable());
405     try {
406       regionFs = HRegionFileSystem.openRegionFromFileSystem(
407           this.services.getConfiguration(), fs, tabledir, daughter, true);
408       
409       for (HColumnDescriptor family: parentDescriptor.getFamilies()) {
410         if ((references = regionFs.hasReferences(family.getNameAsString()))) {
411           break;
412         }
413       }
414     } catch (IOException e) {
415       LOG.error("Error trying to determine referenced files from : " + daughter.getEncodedName()
416           + ", to: " + parent.getEncodedName() + " assuming has references", e);
417       return new Pair<Boolean, Boolean>(Boolean.TRUE, Boolean.TRUE);
418     }
419     return new Pair<Boolean, Boolean>(Boolean.TRUE, Boolean.valueOf(references));
420   }
421 
422   private HTableDescriptor getTableDescriptor(final TableName tableName)
423       throws FileNotFoundException, IOException {
424     return this.services.getTableDescriptors().get(tableName);
425   }
426 
427   /**
428    * Checks if the specified region has merge qualifiers, if so, try to clean
429    * them
430    * @param region
431    * @return true if the specified region doesn't have merge qualifier now
432    * @throws IOException
433    */
434   public boolean cleanMergeQualifier(final HRegionInfo region)
435       throws IOException {
436     // Get merge regions if it is a merged region and already has merge
437     // qualifier
438     Pair<HRegionInfo, HRegionInfo> mergeRegions = MetaTableAccessor
439         .getRegionsFromMergeQualifier(this.services.getConnection(),
440           region.getRegionName());
441     if (mergeRegions == null
442         || (mergeRegions.getFirst() == null && mergeRegions.getSecond() == null)) {
443       // It doesn't have merge qualifier, no need to clean
444       return true;
445     }
446     // It shouldn't happen, we must insert/delete these two qualifiers together
447     if (mergeRegions.getFirst() == null || mergeRegions.getSecond() == null) {
448       LOG.error("Merged region " + region.getRegionNameAsString()
449           + " has only one merge qualifier in META.");
450       return false;
451     }
452     return cleanMergeRegion(region, mergeRegions.getFirst(),
453         mergeRegions.getSecond());
454   }
455 }