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.regionserver;
21  
22  import org.apache.commons.logging.Log;
23  import org.apache.commons.logging.LogFactory;
24  import org.apache.hadoop.conf.Configuration;
25  import org.apache.hadoop.hbase.DroppedSnapshotException;
26  import org.apache.hadoop.hbase.HConstants;
27  import org.apache.hadoop.hbase.RemoteExceptionHandler;
28  import org.apache.hadoop.hbase.util.Bytes;
29  import org.apache.hadoop.util.StringUtils;
30  
31  import com.google.common.base.Preconditions;
32  
33  import java.io.IOException;
34  import java.lang.management.ManagementFactory;
35  import java.util.ConcurrentModificationException;
36  import java.util.EnumSet;
37  import java.util.HashMap;
38  import java.util.Map;
39  import java.util.Set;
40  import java.util.SortedMap;
41  import java.util.TreeSet;
42  import java.util.concurrent.BlockingQueue;
43  import java.util.concurrent.DelayQueue;
44  import java.util.concurrent.Delayed;
45  import java.util.concurrent.TimeUnit;
46  import java.util.concurrent.atomic.AtomicBoolean;
47  import java.util.concurrent.locks.Condition;
48  import java.util.concurrent.locks.ReentrantLock;
49  
50  /**
51   * Thread that flushes cache on request
52   *
53   * NOTE: This class extends Thread rather than Chore because the sleep time
54   * can be interrupted when there is something to do, rather than the Chore
55   * sleep time which is invariant.
56   *
57   * @see FlushRequester
58   */
59  class MemStoreFlusher extends Thread implements FlushRequester {
60    static final Log LOG = LogFactory.getLog(MemStoreFlusher.class);
61    // These two data members go together.  Any entry in the one must have
62    // a corresponding entry in the other.
63    private final BlockingQueue<FlushQueueEntry> flushQueue =
64      new DelayQueue<FlushQueueEntry>();
65    private final Map<HRegion, FlushRegionEntry> regionsInQueue =
66      new HashMap<HRegion, FlushRegionEntry>();
67    private AtomicBoolean wakeupPending = new AtomicBoolean();
68  
69    private final long threadWakeFrequency;
70    private final HRegionServer server;
71    private final ReentrantLock lock = new ReentrantLock();
72    private final Condition flushOccurred = lock.newCondition();
73  
74    protected final long globalMemStoreLimit;
75    protected final long globalMemStoreLimitLowMark;
76  
77    private static final float DEFAULT_UPPER = 0.4f;
78    private static final float DEFAULT_LOWER = 0.35f;
79    private static final String UPPER_KEY =
80      "hbase.regionserver.global.memstore.upperLimit";
81    private static final String LOWER_KEY =
82      "hbase.regionserver.global.memstore.lowerLimit";
83    private long blockingStoreFilesNumber;
84    private long blockingWaitTime;
85  
86    /**
87     * @param conf
88     * @param server
89     */
90    public MemStoreFlusher(final Configuration conf,
91        final HRegionServer server) {
92      super();
93      this.server = server;
94      this.threadWakeFrequency =
95        conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
96      long max = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax();
97      this.globalMemStoreLimit = globalMemStoreLimit(max, DEFAULT_UPPER,
98        UPPER_KEY, conf);
99      long lower = globalMemStoreLimit(max, DEFAULT_LOWER, LOWER_KEY, conf);
100     if (lower > this.globalMemStoreLimit) {
101       lower = this.globalMemStoreLimit;
102       LOG.info("Setting globalMemStoreLimitLowMark == globalMemStoreLimit " +
103         "because supplied " + LOWER_KEY + " was > " + UPPER_KEY);
104     }
105     this.globalMemStoreLimitLowMark = lower;
106     this.blockingStoreFilesNumber =
107       conf.getInt("hbase.hstore.blockingStoreFiles", 7);
108     if (this.blockingStoreFilesNumber == -1) {
109       this.blockingStoreFilesNumber = 1 +
110         conf.getInt("hbase.hstore.compactionThreshold", 3);
111     }
112     this.blockingWaitTime = conf.getInt("hbase.hstore.blockingWaitTime",
113       90000);
114     LOG.info("globalMemStoreLimit=" +
115       StringUtils.humanReadableInt(this.globalMemStoreLimit) +
116       ", globalMemStoreLimitLowMark=" +
117       StringUtils.humanReadableInt(this.globalMemStoreLimitLowMark) +
118       ", maxHeap=" + StringUtils.humanReadableInt(max));
119   }
120 
121   /**
122    * Calculate size using passed <code>key</code> for configured
123    * percentage of <code>max</code>.
124    * @param max
125    * @param defaultLimit
126    * @param key
127    * @param c
128    * @return Limit.
129    */
130   static long globalMemStoreLimit(final long max,
131      final float defaultLimit, final String key, final Configuration c) {
132     float limit = c.getFloat(key, defaultLimit);
133     return getMemStoreLimit(max, limit, defaultLimit);
134   }
135 
136   static long getMemStoreLimit(final long max, final float limit,
137       final float defaultLimit) {
138     if (limit >= 0.9f || limit < 0.1f) {
139       LOG.warn("Setting global memstore limit to default of " + defaultLimit +
140         " because supplied value outside allowed range of 0.1 -> 0.9");
141     }
142     return (long)(max * limit);
143   }
144   
145   /**
146    * The memstore across all regions has exceeded the low water mark. Pick
147    * one region to flush and flush it synchronously (this is called from the
148    * flush thread)
149    * @return true if successful
150    */
151   private boolean flushOneForGlobalPressure() {
152     SortedMap<Long, HRegion> regionsBySize =
153         server.getCopyOfOnlineRegionsSortedBySize();
154 
155     // TODO: HBASE-3532 - we can't use Set<HRegion> here because it doesn't
156     // implement equals correctly. So, set of region names.
157     Set<byte[]> excludedRegionNames = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
158 
159     boolean flushedOne = false;
160     while (!flushedOne) {
161       // Find the biggest region that doesn't have too many storefiles
162       HRegion bestFlushableRegion = getBiggestMemstoreRegion(
163           regionsBySize, excludedRegionNames, true);
164       // Find the biggest region, total, even if it might have too many flushes.
165       HRegion bestAnyRegion = getBiggestMemstoreRegion(
166           regionsBySize, excludedRegionNames, false);
167 
168       if (bestAnyRegion == null) {
169         LOG.fatal("Above memory mark but there are no flushable regions!");
170         return false;
171       }
172       
173       HRegion regionToFlush;
174       if (bestAnyRegion.memstoreSize.get() > 2 * bestFlushableRegion.memstoreSize.get()) {
175         // Even if it's not supposed to be flushed, pick a region if it's more than twice
176         // as big as the best flushable one - otherwise when we're under pressure we make
177         // lots of little flushes and cause lots of compactions, etc, which just makes
178         // life worse!
179         LOG.info("Under global heap pressure: " +
180             "Region " + bestAnyRegion.getRegionNameAsString() + " has too many " +
181             "store files, but is " +
182             StringUtils.humanReadableInt(bestAnyRegion.memstoreSize.get()) +
183             " vs best flushable region's " +
184             StringUtils.humanReadableInt(bestFlushableRegion.memstoreSize.get()) +
185             ". Choosing the bigger.");
186         regionToFlush = bestAnyRegion;
187       } else {
188         regionToFlush = bestFlushableRegion;
189       }
190       
191       Preconditions.checkState(regionToFlush.memstoreSize.get() > 0);
192       
193       LOG.info("Flush of region " + regionToFlush + " due to global heap pressure");
194       flushedOne = flushRegion(regionToFlush, true);
195       if (!flushedOne) {
196         LOG.info("Excluding unflushable region " + regionToFlush +
197           " - trying to find a different region to flush.");
198         excludedRegionNames.add(regionToFlush.getRegionName());
199       }
200     }
201     return true;
202   }
203 
204   @Override
205   public void run() {
206     while (!this.server.isStopped()) {
207       FlushQueueEntry fqe = null;
208       try {
209         wakeupPending.set(false); // allow someone to wake us up again
210         fqe = flushQueue.poll(threadWakeFrequency, TimeUnit.MILLISECONDS);
211         if (fqe == null || fqe instanceof WakeupFlushThread) {
212           if (isAboveLowWaterMark()) {
213             LOG.info("Flush thread woke up with memory above low water.");
214             if (!flushOneForGlobalPressure()) {
215               // Wasn't able to flush any region, but we're above low water mark
216               // This is unlikely to happen, but might happen when closing the
217               // entire server - another thread is flushing regions. We'll just
218               // sleep a little bit to avoid spinning, and then pretend that
219               // we flushed one, so anyone blocked will check again
220               lock.lock();
221               try {
222                 Thread.sleep(1000);
223                 flushOccurred.signalAll();
224               } finally {
225                 lock.unlock();
226               }
227             }
228             // Enqueue another one of these tokens so we'll wake up again
229             wakeupFlushThread();            
230           }
231           continue;
232         }
233         FlushRegionEntry fre = (FlushRegionEntry)fqe;
234         if (!flushRegion(fre)) {
235           break;
236         }
237       } catch (InterruptedException ex) {
238         continue;
239       } catch (ConcurrentModificationException ex) {
240         continue;
241       } catch (Exception ex) {
242         LOG.error("Cache flusher failed for entry " + fqe);
243         if (!server.checkFileSystem()) {
244           break;
245         }
246       }
247     }
248     this.regionsInQueue.clear();
249     this.flushQueue.clear();
250 
251     // Signal anyone waiting, so they see the close flag
252     lock.lock();
253     try {
254       flushOccurred.signalAll();
255     } finally {
256       lock.unlock();
257     }
258     LOG.info(getName() + " exiting");
259   }
260 
261   private void wakeupFlushThread() {
262     if (wakeupPending.compareAndSet(false, true)) {
263       flushQueue.add(new WakeupFlushThread());
264     }
265   }
266 
267   private HRegion getBiggestMemstoreRegion(
268       SortedMap<Long, HRegion> regionsBySize,
269       Set<byte[]> excludedRegionNames,
270       boolean checkStoreFileCount) {
271     synchronized (regionsInQueue) {
272       for (HRegion region : regionsBySize.values()) {
273         if (excludedRegionNames.contains(region.getRegionName())) {
274           continue;
275         }
276 
277         if (checkStoreFileCount && isTooManyStoreFiles(region)) {
278           continue;
279         }
280         return region;
281       }
282     }
283     return null;
284   }
285   
286   /**
287    * Return true if global memory usage is above the high watermark
288    */
289   private boolean isAboveHighWaterMark() {
290     return server.getGlobalMemStoreSize() >= globalMemStoreLimit;
291   }
292   
293   /**
294    * Return true if we're above the high watermark
295    */
296   private boolean isAboveLowWaterMark() {
297     return server.getGlobalMemStoreSize() >= globalMemStoreLimitLowMark;
298   }
299 
300   public void requestFlush(HRegion r) {
301     synchronized (regionsInQueue) {
302       if (!regionsInQueue.containsKey(r)) {
303         // This entry has no delay so it will be added at the top of the flush
304         // queue.  It'll come out near immediately.
305         FlushRegionEntry fqe = new FlushRegionEntry(r);
306         this.regionsInQueue.put(r, fqe);
307         this.flushQueue.add(fqe);
308       }
309     }
310   }
311 
312   public int getFlushQueueSize() {
313     return flushQueue.size();
314   }
315 
316   /**
317    * Only interrupt once it's done with a run through the work loop.
318    */
319   void interruptIfNecessary() {
320     lock.lock();
321     try {
322       this.interrupt();
323     } finally {
324       lock.unlock();
325     }
326   }
327 
328   /*
329    * A flushRegion that checks store file count.  If too many, puts the flush
330    * on delay queue to retry later.
331    * @param fqe
332    * @return true if the region was successfully flushed, false otherwise. If 
333    * false, there will be accompanying log messages explaining why the log was
334    * not flushed.
335    */
336   private boolean flushRegion(final FlushRegionEntry fqe) {
337     HRegion region = fqe.region;
338     if (!fqe.region.getRegionInfo().isMetaRegion() &&
339         isTooManyStoreFiles(region)) {
340       if (fqe.isMaximumWait(this.blockingWaitTime)) {
341         LOG.info("Waited " + (System.currentTimeMillis() - fqe.createTime) +
342           "ms on a compaction to clean up 'too many store files'; waited " +
343           "long enough... proceeding with flush of " +
344           region.getRegionNameAsString());
345       } else {
346         // If this is first time we've been put off, then emit a log message.
347         if (fqe.getRequeueCount() <= 0) {
348           // Note: We don't impose blockingStoreFiles constraint on meta regions
349           LOG.warn("Region " + region.getRegionNameAsString() + " has too many " +
350             "store files; delaying flush up to " + this.blockingWaitTime + "ms");
351         }
352         this.server.compactSplitThread.requestCompaction(region, getName());
353         // Put back on the queue.  Have it come back out of the queue
354         // after a delay of this.blockingWaitTime / 100 ms.
355         this.flushQueue.add(fqe.requeue(this.blockingWaitTime / 100));
356         // Tell a lie, it's not flushed but it's ok
357         return true;
358       }
359     }
360     return flushRegion(region, false);
361   }
362 
363   /*
364    * Flush a region.
365    * @param region Region to flush.
366    * @param emergencyFlush Set if we are being force flushed. If true the region
367    * needs to be removed from the flush queue. If false, when we were called
368    * from the main flusher run loop and we got the entry to flush by calling
369    * poll on the flush queue (which removed it).
370    *
371    * @return true if the region was successfully flushed, false otherwise. If
372    * false, there will be accompanying log messages explaining why the log was
373    * not flushed.
374    */
375   private boolean flushRegion(final HRegion region, final boolean emergencyFlush) {
376     synchronized (this.regionsInQueue) {
377       FlushRegionEntry fqe = this.regionsInQueue.remove(region);
378       if (fqe != null && emergencyFlush) {
379         // Need to remove from region from delay queue.  When NOT an
380         // emergencyFlush, then item was removed via a flushQueue.poll.
381         flushQueue.remove(fqe);
382      }
383      lock.lock();
384     }
385     try {
386       if (region.flushcache()) {
387         server.compactSplitThread.requestCompaction(region, getName());
388       }
389       server.getMetrics().addFlush(region.getRecentFlushInfo());
390     } catch (DroppedSnapshotException ex) {
391       // Cache flush can fail in a few places. If it fails in a critical
392       // section, we get a DroppedSnapshotException and a replay of hlog
393       // is required. Currently the only way to do this is a restart of
394       // the server. Abort because hdfs is probably bad (HBASE-644 is a case
395       // where hdfs was bad but passed the hdfs check).
396       server.abort("Replay of HLog required. Forcing server shutdown", ex);
397       return false;
398     } catch (IOException ex) {
399       LOG.error("Cache flush failed" +
400         (region != null ? (" for region " + Bytes.toString(region.getRegionName())) : ""),
401         RemoteExceptionHandler.checkIOException(ex));
402       if (!server.checkFileSystem()) {
403         return false;
404       }
405     } finally {
406       flushOccurred.signalAll();
407       lock.unlock();
408     }
409     return true;
410   }
411 
412   private boolean isTooManyStoreFiles(HRegion region) {
413     for (Store hstore: region.stores.values()) {
414       if (hstore.getStorefilesCount() > this.blockingStoreFilesNumber) {
415         return true;
416       }
417     }
418     return false;
419   }
420 
421   /**
422    * Check if the regionserver's memstore memory usage is greater than the
423    * limit. If so, flush regions with the biggest memstores until we're down
424    * to the lower limit. This method blocks callers until we're down to a safe
425    * amount of memstore consumption.
426    */
427   public synchronized void reclaimMemStoreMemory() {
428     if (isAboveHighWaterMark()) {
429       lock.lock();
430       try {
431         while (isAboveHighWaterMark() && !server.isStopped()) {
432           wakeupFlushThread();
433           try {
434             // we should be able to wait forever, but we've seen a bug where
435             // we miss a notify, so put a 5 second bound on it at least.
436             flushOccurred.await(5, TimeUnit.SECONDS);
437           } catch (InterruptedException ie) {
438             Thread.currentThread().interrupt();
439           }
440         }
441       } finally {
442         lock.unlock();
443       }
444     } else if (isAboveLowWaterMark()) {
445       wakeupFlushThread();
446     }
447   }
448 
449   interface FlushQueueEntry extends Delayed {}
450   
451   /**
452    * Token to insert into the flush queue that ensures that the flusher does not sleep
453    */
454   static class WakeupFlushThread implements FlushQueueEntry {
455     @Override
456     public long getDelay(TimeUnit unit) {
457       return 0;
458     }
459 
460     @Override
461     public int compareTo(Delayed o) {
462       return -1;
463     }
464   }
465   
466   /**
467    * Datastructure used in the flush queue.  Holds region and retry count.
468    * Keeps tabs on how old this object is.  Implements {@link Delayed}.  On
469    * construction, the delay is zero. When added to a delay queue, we'll come
470    * out near immediately.  Call {@link #requeue(long)} passing delay in
471    * milliseconds before readding to delay queue if you want it to stay there
472    * a while.
473    */
474   static class FlushRegionEntry implements FlushQueueEntry {
475     private final HRegion region;
476     
477     private final long createTime;
478     private long whenToExpire;
479     private int requeueCount = 0;
480 
481     FlushRegionEntry(final HRegion r) {
482       this.region = r;
483       this.createTime = System.currentTimeMillis();
484       this.whenToExpire = this.createTime;
485     }
486 
487     /**
488      * @param maximumWait
489      * @return True if we have been delayed > <code>maximumWait</code> milliseconds.
490      */
491     public boolean isMaximumWait(final long maximumWait) {
492       return (System.currentTimeMillis() - this.createTime) > maximumWait;
493     }
494 
495     /**
496      * @return Count of times {@link #resetDelay()} was called; i.e this is
497      * number of times we've been requeued.
498      */
499     public int getRequeueCount() {
500       return this.requeueCount;
501     }
502  
503     /**
504      * @param when When to expire, when to come up out of the queue.
505      * Specify in milliseconds.  This method adds System.currentTimeMillis()
506      * to whatever you pass.
507      * @return This.
508      */
509     public FlushRegionEntry requeue(final long when) {
510       this.whenToExpire = System.currentTimeMillis() + when;
511       this.requeueCount++;
512       return this;
513     }
514 
515     @Override
516     public long getDelay(TimeUnit unit) {
517       return unit.convert(this.whenToExpire - System.currentTimeMillis(),
518           TimeUnit.MILLISECONDS);
519     }
520 
521     @Override
522     public int compareTo(Delayed other) {
523       return Long.valueOf(getDelay(TimeUnit.MILLISECONDS) -
524         other.getDelay(TimeUnit.MILLISECONDS)).intValue();
525     }
526     
527     @Override
528     public String toString() {
529       return "[flush region " + Bytes.toString(region.getRegionName()) + "]";
530     }
531   }
532 }