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