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.HasThread;
46  import org.apache.hadoop.util.StringUtils;
47  
48  import com.google.common.base.Preconditions;
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 HasThread 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     float effectiveLimit = limit;
139     if (limit >= 0.9f || limit < 0.1f) {
140       LOG.warn("Setting global memstore limit to default of " + defaultLimit +
141         " because supplied value outside allowed range of 0.1 -> 0.9");
142       effectiveLimit = defaultLimit;
143     }
144     return (long)(max * effectiveLimit);
145   }
146 
147   /**
148    * The memstore across all regions has exceeded the low water mark. Pick
149    * one region to flush and flush it synchronously (this is called from the
150    * flush thread)
151    * @return true if successful
152    */
153   private boolean flushOneForGlobalPressure() {
154     SortedMap<Long, HRegion> regionsBySize =
155         server.getCopyOfOnlineRegionsSortedBySize();
156 
157     Set<HRegion> excludedRegions = new HashSet<HRegion>();
158 
159     boolean flushedOne = false;
160     while (!flushedOne) {
161       // Find the biggest region that doesn't have too many storefiles
162       // (might be null!)
163       HRegion bestFlushableRegion = getBiggestMemstoreRegion(
164           regionsBySize, excludedRegions, true);
165       // Find the biggest region, total, even if it might have too many flushes.
166       HRegion bestAnyRegion = getBiggestMemstoreRegion(
167           regionsBySize, excludedRegions, false);
168 
169       if (bestAnyRegion == null) {
170         LOG.error("Above memory mark but there are no flushable regions!");
171         return false;
172       }
173 
174       HRegion regionToFlush;
175       if (bestFlushableRegion != null &&
176           bestAnyRegion.memstoreSize.get() > 2 * bestFlushableRegion.memstoreSize.get()) {
177         // Even if it's not supposed to be flushed, pick a region if it's more than twice
178         // as big as the best flushable one - otherwise when we're under pressure we make
179         // lots of little flushes and cause lots of compactions, etc, which just makes
180         // life worse!
181         if (LOG.isDebugEnabled()) {
182           LOG.debug("Under global heap pressure: " +
183             "Region " + bestAnyRegion.getRegionNameAsString() + " has too many " +
184             "store files, but is " +
185             StringUtils.humanReadableInt(bestAnyRegion.memstoreSize.get()) +
186             " vs best flushable region's " +
187             StringUtils.humanReadableInt(bestFlushableRegion.memstoreSize.get()) +
188             ". Choosing the bigger.");
189         }
190         regionToFlush = bestAnyRegion;
191       } else {
192         if (bestFlushableRegion == null) {
193           regionToFlush = bestAnyRegion;
194         } else {
195           regionToFlush = bestFlushableRegion;
196         }
197       }
198 
199       Preconditions.checkState(regionToFlush.memstoreSize.get() > 0);
200 
201       LOG.info("Flush of region " + regionToFlush + " due to global heap pressure");
202       flushedOne = flushRegion(regionToFlush, true);
203       if (!flushedOne) {
204         LOG.info("Excluding unflushable region " + regionToFlush +
205           " - trying to find a different region to flush.");
206         excludedRegions.add(regionToFlush);
207       }
208     }
209     return true;
210   }
211 
212   @Override
213   public void run() {
214     while (!this.server.isStopped()) {
215       FlushQueueEntry fqe = null;
216       try {
217         wakeupPending.set(false); // allow someone to wake us up again
218         fqe = flushQueue.poll(threadWakeFrequency, TimeUnit.MILLISECONDS);
219         if (fqe == null || fqe instanceof WakeupFlushThread) {
220           if (isAboveLowWaterMark()) {
221             LOG.debug("Flush thread woke up because memory above low water=" +
222               StringUtils.humanReadableInt(this.globalMemStoreLimitLowMark));
223             if (!flushOneForGlobalPressure()) {
224               // Wasn't able to flush any region, but we're above low water mark
225               // This is unlikely to happen, but might happen when closing the
226               // entire server - another thread is flushing regions. We'll just
227               // sleep a little bit to avoid spinning, and then pretend that
228               // we flushed one, so anyone blocked will check again
229               lock.lock();
230               try {
231                 Thread.sleep(1000);
232                 flushOccurred.signalAll();
233               } finally {
234                 lock.unlock();
235               }
236             }
237             // Enqueue another one of these tokens so we'll wake up again
238             wakeupFlushThread();
239           }
240           continue;
241         }
242         FlushRegionEntry fre = (FlushRegionEntry)fqe;
243         if (!flushRegion(fre)) {
244           break;
245         }
246       } catch (InterruptedException ex) {
247         continue;
248       } catch (ConcurrentModificationException ex) {
249         continue;
250       } catch (Exception ex) {
251         LOG.error("Cache flusher failed for entry " + fqe, ex);
252         if (!server.checkFileSystem()) {
253           break;
254         }
255       }
256     }
257     this.regionsInQueue.clear();
258     this.flushQueue.clear();
259 
260     // Signal anyone waiting, so they see the close flag
261     lock.lock();
262     try {
263       flushOccurred.signalAll();
264     } finally {
265       lock.unlock();
266     }
267     LOG.info(getName() + " exiting");
268   }
269 
270   private void wakeupFlushThread() {
271     if (wakeupPending.compareAndSet(false, true)) {
272       flushQueue.add(new WakeupFlushThread());
273     }
274   }
275 
276   private HRegion getBiggestMemstoreRegion(
277       SortedMap<Long, HRegion> regionsBySize,
278       Set<HRegion> excludedRegions,
279       boolean checkStoreFileCount) {
280     synchronized (regionsInQueue) {
281       for (HRegion region : regionsBySize.values()) {
282         if (excludedRegions.contains(region)) {
283           continue;
284         }
285 
286         if (checkStoreFileCount && isTooManyStoreFiles(region)) {
287           continue;
288         }
289         return region;
290       }
291     }
292     return null;
293   }
294 
295   /**
296    * Return true if global memory usage is above the high watermark
297    */
298   private boolean isAboveHighWaterMark() {
299     return server.getRegionServerAccounting().
300       getGlobalMemstoreSize() >= globalMemStoreLimit;
301   }
302 
303   /**
304    * Return true if we're above the high watermark
305    */
306   private boolean isAboveLowWaterMark() {
307     return server.getRegionServerAccounting().
308       getGlobalMemstoreSize() >= globalMemStoreLimitLowMark;
309   }
310 
311   public void requestFlush(HRegion r) {
312     synchronized (regionsInQueue) {
313       if (!regionsInQueue.containsKey(r)) {
314         // This entry has no delay so it will be added at the top of the flush
315         // queue.  It'll come out near immediately.
316         FlushRegionEntry fqe = new FlushRegionEntry(r);
317         this.regionsInQueue.put(r, fqe);
318         this.flushQueue.add(fqe);
319       }
320     }
321   }
322 
323   public int getFlushQueueSize() {
324     return flushQueue.size();
325   }
326 
327   /**
328    * Only interrupt once it's done with a run through the work loop.
329    */
330   void interruptIfNecessary() {
331     lock.lock();
332     try {
333       this.interrupt();
334     } finally {
335       lock.unlock();
336     }
337   }
338 
339   /*
340    * A flushRegion that checks store file count.  If too many, puts the flush
341    * on delay queue to retry later.
342    * @param fqe
343    * @return true if the region was successfully flushed, false otherwise. If
344    * false, there will be accompanying log messages explaining why the log was
345    * not flushed.
346    */
347   private boolean flushRegion(final FlushRegionEntry fqe) {
348     HRegion region = fqe.region;
349     if (!fqe.region.getRegionInfo().isMetaRegion() &&
350         isTooManyStoreFiles(region)) {
351       if (fqe.isMaximumWait(this.blockingWaitTime)) {
352         LOG.info("Waited " + (System.currentTimeMillis() - fqe.createTime) +
353           "ms on a compaction to clean up 'too many store files'; waited " +
354           "long enough... proceeding with flush of " +
355           region.getRegionNameAsString());
356       } else {
357         // If this is first time we've been put off, then emit a log message.
358         if (fqe.getRequeueCount() <= 0) {
359           // Note: We don't impose blockingStoreFiles constraint on meta regions
360           LOG.warn("Region " + region.getRegionNameAsString() + " has too many " +
361             "store files; delaying flush up to " + this.blockingWaitTime + "ms");
362           if (!this.server.compactSplitThread.requestSplit(region)) {
363             this.server.compactSplitThread.requestCompaction(region, getName());
364           }
365         }
366 
367         // Put back on the queue.  Have it come back out of the queue
368         // after a delay of this.blockingWaitTime / 100 ms.
369         this.flushQueue.add(fqe.requeue(this.blockingWaitTime / 100));
370         // Tell a lie, it's not flushed but it's ok
371         return true;
372       }
373     }
374     return flushRegion(region, false);
375   }
376 
377   /*
378    * Flush a region.
379    * @param region Region to flush.
380    * @param emergencyFlush Set if we are being force flushed. If true the region
381    * needs to be removed from the flush queue. If false, when we were called
382    * from the main flusher run loop and we got the entry to flush by calling
383    * poll on the flush queue (which removed it).
384    *
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 HRegion region, final boolean emergencyFlush) {
390     synchronized (this.regionsInQueue) {
391       FlushRegionEntry fqe = this.regionsInQueue.remove(region);
392       if (fqe != null && emergencyFlush) {
393         // Need to remove from region from delay queue.  When NOT an
394         // emergencyFlush, then item was removed via a flushQueue.poll.
395         flushQueue.remove(fqe);
396      }
397      lock.lock();
398     }
399     try {
400       if (region.flushcache()) {
401         server.compactSplitThread.requestCompaction(region, getName());
402       }
403       server.getMetrics().addFlush(region.getRecentFlushInfo());
404     } catch (DroppedSnapshotException ex) {
405       // Cache flush can fail in a few places. If it fails in a critical
406       // section, we get a DroppedSnapshotException and a replay of hlog
407       // is required. Currently the only way to do this is a restart of
408       // the server. Abort because hdfs is probably bad (HBASE-644 is a case
409       // where hdfs was bad but passed the hdfs check).
410       server.abort("Replay of HLog required. Forcing server shutdown", ex);
411       return false;
412     } catch (IOException ex) {
413       LOG.error("Cache flush failed" +
414         (region != null ? (" for region " + Bytes.toStringBinary(region.getRegionName())) : ""),
415         RemoteExceptionHandler.checkIOException(ex));
416       if (!server.checkFileSystem()) {
417         return false;
418       }
419     } finally {
420       flushOccurred.signalAll();
421       lock.unlock();
422     }
423     return true;
424   }
425 
426   private boolean isTooManyStoreFiles(HRegion region) {
427     for (Store hstore: region.stores.values()) {
428       if (hstore.getStorefilesCount() > this.blockingStoreFilesNumber) {
429         return true;
430       }
431     }
432     return false;
433   }
434 
435   /**
436    * Check if the regionserver's memstore memory usage is greater than the
437    * limit. If so, flush regions with the biggest memstores until we're down
438    * to the lower limit. This method blocks callers until we're down to a safe
439    * amount of memstore consumption.
440    */
441   public synchronized void reclaimMemStoreMemory() {
442     if (isAboveHighWaterMark()) {
443       lock.lock();
444       try {
445         while (isAboveHighWaterMark() && !server.isStopped()) {
446           wakeupFlushThread();
447           try {
448             // we should be able to wait forever, but we've seen a bug where
449             // we miss a notify, so put a 5 second bound on it at least.
450             flushOccurred.await(5, TimeUnit.SECONDS);
451           } catch (InterruptedException ie) {
452             Thread.currentThread().interrupt();
453           }
454         }
455       } finally {
456         lock.unlock();
457       }
458     } else if (isAboveLowWaterMark()) {
459       wakeupFlushThread();
460     }
461   }
462 
463   interface FlushQueueEntry extends Delayed {}
464 
465   /**
466    * Token to insert into the flush queue that ensures that the flusher does not sleep
467    */
468   static class WakeupFlushThread implements FlushQueueEntry {
469     @Override
470     public long getDelay(TimeUnit unit) {
471       return 0;
472     }
473 
474     @Override
475     public int compareTo(Delayed o) {
476       return -1;
477     }
478   }
479 
480   /**
481    * Datastructure used in the flush queue.  Holds region and retry count.
482    * Keeps tabs on how old this object is.  Implements {@link Delayed}.  On
483    * construction, the delay is zero. When added to a delay queue, we'll come
484    * out near immediately.  Call {@link #requeue(long)} passing delay in
485    * milliseconds before readding to delay queue if you want it to stay there
486    * a while.
487    */
488   static class FlushRegionEntry implements FlushQueueEntry {
489     private final HRegion region;
490 
491     private final long createTime;
492     private long whenToExpire;
493     private int requeueCount = 0;
494 
495     FlushRegionEntry(final HRegion r) {
496       this.region = r;
497       this.createTime = System.currentTimeMillis();
498       this.whenToExpire = this.createTime;
499     }
500 
501     /**
502      * @param maximumWait
503      * @return True if we have been delayed > <code>maximumWait</code> milliseconds.
504      */
505     public boolean isMaximumWait(final long maximumWait) {
506       return (System.currentTimeMillis() - this.createTime) > maximumWait;
507     }
508 
509     /**
510      * @return Count of times {@link #resetDelay()} was called; i.e this is
511      * number of times we've been requeued.
512      */
513     public int getRequeueCount() {
514       return this.requeueCount;
515     }
516 
517     /**
518      * @param when When to expire, when to come up out of the queue.
519      * Specify in milliseconds.  This method adds System.currentTimeMillis()
520      * to whatever you pass.
521      * @return This.
522      */
523     public FlushRegionEntry requeue(final long when) {
524       this.whenToExpire = System.currentTimeMillis() + when;
525       this.requeueCount++;
526       return this;
527     }
528 
529     @Override
530     public long getDelay(TimeUnit unit) {
531       return unit.convert(this.whenToExpire - System.currentTimeMillis(),
532           TimeUnit.MILLISECONDS);
533     }
534 
535     @Override
536     public int compareTo(Delayed other) {
537       return Long.valueOf(getDelay(TimeUnit.MILLISECONDS) -
538         other.getDelay(TimeUnit.MILLISECONDS)).intValue();
539     }
540 
541     @Override
542     public String toString() {
543       return "[flush region " + Bytes.toStringBinary(region.getRegionName()) + "]";
544     }
545   }
546 }