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