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