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