View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.master.cleaner;
19  
20  import java.io.IOException;
21  import java.util.ArrayList;
22  import java.util.List;
23  import java.util.Map;
24  import java.util.concurrent.BlockingQueue;
25  import java.util.concurrent.TimeUnit;
26  import java.util.concurrent.atomic.AtomicLong;
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.fs.FileStatus;
31  import org.apache.hadoop.fs.FileSystem;
32  import org.apache.hadoop.fs.Path;
33  import org.apache.hadoop.hbase.Stoppable;
34  import org.apache.hadoop.hbase.classification.InterfaceAudience;
35  import org.apache.hadoop.hbase.conf.ConfigurationObserver;
36  import org.apache.hadoop.hbase.io.HFileLink;
37  import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
38  import org.apache.hadoop.hbase.util.StealJobQueue;
39  
40  /**
41   * This Chore, every time it runs, will clear the HFiles in the hfile archive
42   * folder that are deletable for each HFile cleaner in the chain.
43   */
44  @InterfaceAudience.Private
45  public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> implements
46      ConfigurationObserver {
47  
48    public static final String MASTER_HFILE_CLEANER_PLUGINS = "hbase.master.hfilecleaner.plugins";
49  
50    public HFileCleaner(final int period, final Stoppable stopper, Configuration conf, FileSystem fs,
51        Path directory, DirScanPool pool) {
52      this(period, stopper, conf, fs, directory, pool, null);
53    }
54  
55    // Configuration key for large/small throttle point
56    public final static String HFILE_DELETE_THROTTLE_THRESHOLD =
57        "hbase.regionserver.thread.hfilecleaner.throttle";
58    public final static int DEFAULT_HFILE_DELETE_THROTTLE_THRESHOLD = 64 * 1024 * 1024;// 64M
59  
60    // Configuration key for large queue initial size
61    public final static String LARGE_HFILE_QUEUE_INIT_SIZE =
62        "hbase.regionserver.hfilecleaner.large.queue.size";
63    public final static int DEFAULT_LARGE_HFILE_QUEUE_INIT_SIZE = 10240;
64  
65    // Configuration key for small queue initial size
66    public final static String SMALL_HFILE_QUEUE_INIT_SIZE =
67        "hbase.regionserver.hfilecleaner.small.queue.size";
68    public final static int DEFAULT_SMALL_HFILE_QUEUE_INIT_SIZE = 10240;
69  
70    // Configuration key for large file delete thread number
71    public final static String LARGE_HFILE_DELETE_THREAD_NUMBER =
72        "hbase.regionserver.hfilecleaner.large.thread.count";
73    public final static int DEFAULT_LARGE_HFILE_DELETE_THREAD_NUMBER = 1;
74  
75    // Configuration key for small file delete thread number
76    public final static String SMALL_HFILE_DELETE_THREAD_NUMBER =
77        "hbase.regionserver.hfilecleaner.small.thread.count";
78    public final static int DEFAULT_SMALL_HFILE_DELETE_THREAD_NUMBER = 1;
79  
80    public static final String HFILE_DELETE_THREAD_TIMEOUT_MSEC =
81        "hbase.regionserver.hfilecleaner.thread.timeout.msec";
82    static final long DEFAULT_HFILE_DELETE_THREAD_TIMEOUT_MSEC = 60 * 1000L;
83  
84    public static final String HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC =
85        "hbase.regionserver.hfilecleaner.thread.check.interval.msec";
86    static final long DEFAULT_HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC = 1000L;
87  
88    private static final Log LOG = LogFactory.getLog(HFileCleaner.class);
89  
90    StealJobQueue<HFileDeleteTask> largeFileQueue;
91    BlockingQueue<HFileDeleteTask> smallFileQueue;
92    private int throttlePoint;
93    private int largeQueueInitSize;
94    private int smallQueueInitSize;
95    private int largeFileDeleteThreadNumber;
96    private int smallFileDeleteThreadNumber;
97    private long cleanerThreadTimeoutMsec;
98    private long cleanerThreadCheckIntervalMsec;
99    private List<Thread> threads = new ArrayList<Thread>();
100   private boolean running;
101 
102   private AtomicLong deletedLargeFiles = new AtomicLong();
103   private AtomicLong deletedSmallFiles = new AtomicLong();
104 
105   /**
106    * @param period the period of time to sleep between each run
107    * @param stopper the stopper
108    * @param conf configuration to use
109    * @param fs handle to the FS
110    * @param directory directory to be cleaned
111    * @param params params could be used in subclass of BaseHFileCleanerDelegate
112    */
113   public HFileCleaner(final int period, final Stoppable stopper, Configuration conf, FileSystem fs,
114                       Path directory, DirScanPool pool, Map<String, Object> params) {
115     super("HFileCleaner", period, stopper, conf, fs,
116       directory, MASTER_HFILE_CLEANER_PLUGINS, pool, params);
117     throttlePoint =
118         conf.getInt(HFILE_DELETE_THROTTLE_THRESHOLD, DEFAULT_HFILE_DELETE_THROTTLE_THRESHOLD);
119     largeQueueInitSize =
120         conf.getInt(LARGE_HFILE_QUEUE_INIT_SIZE, DEFAULT_LARGE_HFILE_QUEUE_INIT_SIZE);
121     smallQueueInitSize =
122         conf.getInt(SMALL_HFILE_QUEUE_INIT_SIZE, DEFAULT_SMALL_HFILE_QUEUE_INIT_SIZE);
123     largeFileQueue = new StealJobQueue<>(largeQueueInitSize, smallQueueInitSize);
124     smallFileQueue = largeFileQueue.getStealFromQueue();
125     largeFileDeleteThreadNumber =
126         conf.getInt(LARGE_HFILE_DELETE_THREAD_NUMBER, DEFAULT_LARGE_HFILE_DELETE_THREAD_NUMBER);
127     smallFileDeleteThreadNumber =
128         conf.getInt(SMALL_HFILE_DELETE_THREAD_NUMBER, DEFAULT_SMALL_HFILE_DELETE_THREAD_NUMBER);
129     cleanerThreadTimeoutMsec =
130         conf.getLong(HFILE_DELETE_THREAD_TIMEOUT_MSEC, DEFAULT_HFILE_DELETE_THREAD_TIMEOUT_MSEC);
131     cleanerThreadCheckIntervalMsec =
132         conf.getLong(HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC,
133             DEFAULT_HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC);
134     startHFileDeleteThreads();
135   }
136 
137   @Override
138   protected boolean validate(Path file) {
139     if (HFileLink.isBackReferencesDir(file) || HFileLink.isBackReferencesDir(file.getParent())) {
140       return true;
141     }
142     return StoreFileInfo.validateStoreFileName(file.getName());
143   }
144 
145   /**
146    * Exposed for TESTING!
147    */
148   public List<BaseHFileCleanerDelegate> getDelegatesForTesting() {
149     return this.cleanersChain;
150   }
151 
152   @Override
153   public int deleteFiles(Iterable<FileStatus> filesToDelete) {
154     int deletedFiles = 0;
155     List<HFileDeleteTask> tasks = new ArrayList<HFileDeleteTask>();
156     // construct delete tasks and add into relative queue
157     for (FileStatus file : filesToDelete) {
158       HFileDeleteTask task = deleteFile(file);
159       if (task != null) {
160         tasks.add(task);
161       }
162     }
163     // wait for each submitted task to finish
164     for (HFileDeleteTask task : tasks) {
165       if (task.getResult(cleanerThreadCheckIntervalMsec)) {
166         deletedFiles++;
167       }
168     }
169     return deletedFiles;
170   }
171 
172   /**
173    * Construct an {@link HFileDeleteTask} for each file to delete and add into the correct queue
174    * @param file the file to delete
175    * @return HFileDeleteTask to track progress
176    */
177   private HFileDeleteTask deleteFile(FileStatus file) {
178     HFileDeleteTask task = new HFileDeleteTask(file, cleanerThreadTimeoutMsec);
179     boolean enqueued = dispatch(task);
180     return enqueued ? task : null;
181   }
182 
183   private boolean dispatch(HFileDeleteTask task) {
184     if (task.fileLength >= this.throttlePoint) {
185       if (!this.largeFileQueue.offer(task)) {
186         // should never arrive here as long as we use PriorityQueue
187         if (LOG.isTraceEnabled()) {
188           LOG.trace("Large file deletion queue is full");
189         }
190         return false;
191       }
192     } else {
193       if (!this.smallFileQueue.offer(task)) {
194         // should never arrive here as long as we use PriorityQueue
195         if (LOG.isTraceEnabled()) {
196           LOG.trace("Small file deletion queue is full");
197         }
198         return false;
199       }
200     }
201     return true;
202   }
203 
204   @Override
205   public void cleanup() {
206     super.cleanup();
207     stopHFileDeleteThreads();
208   }
209 
210   /**
211    * Start threads for hfile deletion
212    */
213   private void startHFileDeleteThreads() {
214     final String n = Thread.currentThread().getName();
215     running = true;
216     // start thread for large file deletion
217     for (int i = 0; i < largeFileDeleteThreadNumber; i++) {
218       Thread large = new Thread() {
219         @Override
220         public void run() {
221           consumerLoop(largeFileQueue);
222         }
223       };
224       large.setDaemon(true);
225       large.setName(n + "-HFileCleaner.large." + i + "-" + System.currentTimeMillis());
226       large.start();
227       LOG.debug("Starting hfile cleaner for large files: " + large.getName());
228       threads.add(large);
229     }
230 
231     // start thread for small file deletion
232     for (int i = 0; i < smallFileDeleteThreadNumber; i++) {
233       Thread small = new Thread() {
234         @Override
235         public void run() {
236           consumerLoop(smallFileQueue);
237         }
238       };
239       small.setDaemon(true);
240       small.setName(n + "-HFileCleaner.small." + i + "-" + System.currentTimeMillis());
241       small.start();
242       LOG.debug("Starting hfile cleaner for small files: " + small.getName());
243       threads.add(small);
244     }
245   }
246 
247   protected void consumerLoop(BlockingQueue<HFileDeleteTask> queue) {
248     try {
249       while (running) {
250         HFileDeleteTask task = null;
251         try {
252           task = queue.take();
253         } catch (InterruptedException e) {
254           if (LOG.isDebugEnabled()) {
255             LOG.debug("Interrupted while trying to take a task from queue", e);
256           }
257           break;
258         }
259         if (task != null) {
260           if (LOG.isDebugEnabled()) {
261             LOG.debug("Removing: " + task.filePath + " from archive");
262           }
263           boolean succeed;
264           try {
265             succeed = this.fs.delete(task.filePath, false);
266           } catch (IOException e) {
267             LOG.warn("Failed to delete file " + task.filePath, e);
268             succeed = false;
269           }
270           task.setResult(succeed);
271           if (succeed) {
272             countDeletedFiles(task.fileLength >= throttlePoint, queue == largeFileQueue);
273           }
274         }
275       }
276     } finally {
277       if (LOG.isDebugEnabled()) {
278         LOG.debug("Exit thread: " + Thread.currentThread());
279       }
280     }
281   }
282 
283   // Currently only for testing purpose
284   private void countDeletedFiles(boolean isLargeFile, boolean fromLargeQueue) {
285     if (isLargeFile) {
286       if (deletedLargeFiles.get() == Long.MAX_VALUE) {
287         LOG.info("Deleted more than Long.MAX_VALUE large files, reset counter to 0");
288         deletedLargeFiles.set(0L);
289       }
290       deletedLargeFiles.incrementAndGet();
291     } else {
292       if (deletedSmallFiles.get() == Long.MAX_VALUE) {
293         LOG.info("Deleted more than Long.MAX_VALUE small files, reset counter to 0");
294         deletedSmallFiles.set(0L);
295       }
296       if (fromLargeQueue && LOG.isTraceEnabled()) {
297         LOG.trace("Stolen a small file deletion task in large file thread");
298       }
299       deletedSmallFiles.incrementAndGet();
300     }
301   }
302 
303   /**
304    * Stop threads for hfile deletion
305    */
306   private void stopHFileDeleteThreads() {
307     running = false;
308     if (LOG.isDebugEnabled()) {
309       LOG.debug("Stopping file delete threads");
310     }
311     for(Thread thread: threads){
312       thread.interrupt();
313     }
314   }
315 
316   static class HFileDeleteTask implements Comparable<HFileDeleteTask> {
317 
318     boolean done = false;
319     boolean result;
320     final Path filePath;
321     final long fileLength;
322     final long timeoutMsec;
323 
324     public HFileDeleteTask(FileStatus file, long timeoutMsec) {
325       this.filePath = file.getPath();
326       this.fileLength = file.getLen();
327       this.timeoutMsec = timeoutMsec;
328     }
329 
330     public synchronized void setResult(boolean result) {
331       this.done = true;
332       this.result = result;
333       notify();
334     }
335 
336     public synchronized boolean getResult(long waitIfNotFinished) {
337       long waitTimeMsec = 0;
338       try {
339         while (!done) {
340           long startTimeNanos = System.nanoTime();
341           wait(waitIfNotFinished);
342           waitTimeMsec += TimeUnit.MILLISECONDS.convert(System.nanoTime() - startTimeNanos,
343               TimeUnit.NANOSECONDS);
344           if (done) {
345             return this.result;
346           }
347           if (waitTimeMsec > timeoutMsec) {
348             LOG.warn("Wait more than " + timeoutMsec + " ms for deleting " + this.filePath
349                 + ", exit...");
350             return false;
351           }
352         }
353       } catch (InterruptedException e) {
354         LOG.warn("Interrupted while waiting for result of deleting " + filePath
355             + ", will return false", e);
356         return false;
357       }
358       return this.result;
359     }
360 
361     @Override
362     public int compareTo(HFileDeleteTask o) {
363       long sub = this.fileLength - o.fileLength;
364       // smaller value with higher priority in PriorityQueue, and we intent to delete the larger
365       // file first.
366       return (sub > 0) ? -1 : (sub < 0 ? 1 : 0);
367     }
368 
369     @Override
370     public boolean equals(Object o) {
371       if (this == o) {
372         return true;
373       }
374       if (o == null || !(o instanceof HFileDeleteTask)) {
375         return false;
376       }
377       HFileDeleteTask otherTask = (HFileDeleteTask) o;
378       return this.filePath.equals(otherTask.filePath) && (this.fileLength == otherTask.fileLength);
379     }
380 
381     @Override
382     public int hashCode() {
383       return filePath.hashCode();
384     }
385   }
386 
387   public List<Thread> getCleanerThreads() {
388     return threads;
389   }
390 
391   public long getNumOfDeletedLargeFiles() {
392     return deletedLargeFiles.get();
393   }
394 
395   public long getNumOfDeletedSmallFiles() {
396     return deletedSmallFiles.get();
397   }
398 
399   public long getLargeQueueInitSize() {
400     return largeQueueInitSize;
401   }
402 
403   public long getSmallQueueInitSize() {
404     return smallQueueInitSize;
405   }
406 
407   public long getThrottlePoint() {
408     return throttlePoint;
409   }
410 
411   long getCleanerThreadTimeoutMsec() {
412     return cleanerThreadTimeoutMsec;
413   }
414 
415   long getCleanerThreadCheckIntervalMsec() {
416     return cleanerThreadCheckIntervalMsec;
417   }
418 
419   @Override
420   public void onConfigurationChange(Configuration conf) {
421     if (!checkAndUpdateConfigurations(conf)) {
422       LOG.debug("Update configuration triggered but nothing changed for this cleaner");
423       return;
424     }
425     stopHFileDeleteThreads();
426     // record the left over tasks
427     List<HFileDeleteTask> leftOverTasks =
428         new ArrayList<>(largeFileQueue.size() + smallFileQueue.size());
429     for (HFileDeleteTask task : largeFileQueue) {
430       leftOverTasks.add(task);
431     }
432     for (HFileDeleteTask task : smallFileQueue) {
433       leftOverTasks.add(task);
434     }
435     largeFileQueue = new StealJobQueue<>(largeQueueInitSize, smallQueueInitSize);
436     smallFileQueue = largeFileQueue.getStealFromQueue();
437     threads.clear();
438     startHFileDeleteThreads();
439     // re-dispatch the left over tasks
440     for (HFileDeleteTask task : leftOverTasks) {
441       dispatch(task);
442     }
443   }
444 
445   /**
446    * Check new configuration and update settings if value changed
447    * @param conf The new configuration
448    * @return true if any configuration for HFileCleaner changes, false if no change
449    */
450   private boolean checkAndUpdateConfigurations(Configuration conf) {
451     boolean updated = false;
452     int throttlePoint =
453         conf.getInt(HFILE_DELETE_THROTTLE_THRESHOLD, DEFAULT_HFILE_DELETE_THROTTLE_THRESHOLD);
454     if (throttlePoint != this.throttlePoint) {
455       LOG.debug("Updating throttle point, from " + this.throttlePoint + " to " + throttlePoint);
456       this.throttlePoint = throttlePoint;
457       updated = true;
458     }
459     int largeQueueInitSize =
460         conf.getInt(LARGE_HFILE_QUEUE_INIT_SIZE, DEFAULT_LARGE_HFILE_QUEUE_INIT_SIZE);
461     if (largeQueueInitSize != this.largeQueueInitSize) {
462       LOG.debug("Updating largeQueueInitSize, from " + this.largeQueueInitSize + " to "
463           + largeQueueInitSize);
464       this.largeQueueInitSize = largeQueueInitSize;
465       updated = true;
466     }
467     int smallQueueInitSize =
468         conf.getInt(SMALL_HFILE_QUEUE_INIT_SIZE, DEFAULT_SMALL_HFILE_QUEUE_INIT_SIZE);
469     if (smallQueueInitSize != this.smallQueueInitSize) {
470       LOG.debug("Updating smallQueueInitSize, from " + this.smallQueueInitSize + " to "
471           + smallQueueInitSize);
472       this.smallQueueInitSize = smallQueueInitSize;
473       updated = true;
474     }
475     int largeFileDeleteThreadNumber =
476         conf.getInt(LARGE_HFILE_DELETE_THREAD_NUMBER, DEFAULT_LARGE_HFILE_DELETE_THREAD_NUMBER);
477     if (largeFileDeleteThreadNumber != this.largeFileDeleteThreadNumber) {
478       LOG.debug("Updating largeFileDeleteThreadNumber, from " + this.largeFileDeleteThreadNumber
479           + " to " + largeFileDeleteThreadNumber);
480       this.largeFileDeleteThreadNumber = largeFileDeleteThreadNumber;
481       updated = true;
482     }
483     int smallFileDeleteThreadNumber =
484         conf.getInt(SMALL_HFILE_DELETE_THREAD_NUMBER, DEFAULT_SMALL_HFILE_DELETE_THREAD_NUMBER);
485     if (smallFileDeleteThreadNumber != this.smallFileDeleteThreadNumber) {
486       LOG.debug("Updating smallFileDeleteThreadNumber, from " + this.smallFileDeleteThreadNumber
487           + " to " + smallFileDeleteThreadNumber);
488       this.smallFileDeleteThreadNumber = smallFileDeleteThreadNumber;
489       updated = true;
490     }
491     long cleanerThreadTimeoutMsec =
492         conf.getLong(HFILE_DELETE_THREAD_TIMEOUT_MSEC, DEFAULT_HFILE_DELETE_THREAD_TIMEOUT_MSEC);
493     if (cleanerThreadTimeoutMsec != this.cleanerThreadTimeoutMsec) {
494       this.cleanerThreadTimeoutMsec = cleanerThreadTimeoutMsec;
495       updated = true;
496     }
497     long cleanerThreadCheckIntervalMsec =
498         conf.getLong(HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC,
499             DEFAULT_HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC);
500     if (cleanerThreadCheckIntervalMsec != this.cleanerThreadCheckIntervalMsec) {
501       this.cleanerThreadCheckIntervalMsec = cleanerThreadCheckIntervalMsec;
502       updated = true;
503     }
504     return updated;
505   }
506 }