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