View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.regionserver;
20  
21  import static org.apache.hadoop.util.StringUtils.humanReadableInt;
22  
23  import java.io.IOException;
24  import java.lang.Thread.UncaughtExceptionHandler;
25  import java.lang.management.MemoryUsage;
26  import java.util.ArrayList;
27  import java.util.ConcurrentModificationException;
28  import java.util.HashMap;
29  import java.util.HashSet;
30  import java.util.List;
31  import java.util.Map;
32  import java.util.Objects;
33  import java.util.Set;
34  import java.util.SortedMap;
35  import java.util.concurrent.BlockingQueue;
36  import java.util.concurrent.DelayQueue;
37  import java.util.concurrent.Delayed;
38  import java.util.concurrent.ThreadFactory;
39  import java.util.concurrent.TimeUnit;
40  import java.util.concurrent.atomic.AtomicBoolean;
41  import java.util.concurrent.locks.ReentrantReadWriteLock;
42  
43  import org.apache.commons.logging.Log;
44  import org.apache.commons.logging.LogFactory;
45  import org.apache.hadoop.conf.Configuration;
46  import org.apache.hadoop.hbase.DroppedSnapshotException;
47  import org.apache.hadoop.hbase.HConstants;
48  import org.apache.hadoop.hbase.RemoteExceptionHandler;
49  import org.apache.hadoop.hbase.classification.InterfaceAudience;
50  import org.apache.hadoop.hbase.client.RegionReplicaUtil;
51  import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
52  import org.apache.hadoop.hbase.regionserver.Region.FlushResult;
53  import org.apache.hadoop.hbase.util.Bytes;
54  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
55  import org.apache.hadoop.hbase.util.HasThread;
56  import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
57  import org.apache.hadoop.hbase.util.Threads;
58  import org.apache.hadoop.util.StringUtils;
59  import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
60  import org.apache.htrace.Trace;
61  import org.apache.htrace.TraceScope;
62  import org.apache.hadoop.hbase.util.Counter;
63  
64  import com.google.common.base.Preconditions;
65  
66  /**
67   * Thread that flushes cache on request
68   *
69   * NOTE: This class extends Thread rather than Chore because the sleep time
70   * can be interrupted when there is something to do, rather than the Chore
71   * sleep time which is invariant.
72   *
73   * @see FlushRequester
74   */
75  @InterfaceAudience.Private
76  class MemStoreFlusher implements FlushRequester {
77    private static final Log LOG = LogFactory.getLog(MemStoreFlusher.class);
78  
79    private Configuration conf;
80    // These two data members go together.  Any entry in the one must have
81    // a corresponding entry in the other.
82    private final BlockingQueue<FlushQueueEntry> flushQueue =
83      new DelayQueue<FlushQueueEntry>();
84    private final Map<Region, FlushRegionEntry> regionsInQueue =
85      new HashMap<Region, FlushRegionEntry>();
86    private AtomicBoolean wakeupPending = new AtomicBoolean();
87  
88    private final long threadWakeFrequency;
89    private final HRegionServer server;
90    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
91    private final Object blockSignal = new Object();
92  
93    protected long globalMemStoreLimit;
94    protected float globalMemStoreLimitLowMarkPercent;
95    protected long globalMemStoreLimitLowMark;
96  
97    private long blockingWaitTime;
98    private final Counter updatesBlockedMsHighWater = new Counter();
99  
100   private final FlushHandler[] flushHandlers;
101   private List<FlushRequestListener> flushRequestListeners = new ArrayList<FlushRequestListener>(1);
102 
103   /**
104    * @param conf
105    * @param server
106    */
107   public MemStoreFlusher(final Configuration conf,
108       final HRegionServer server) {
109     super();
110     this.conf = conf;
111     this.server = server;
112     this.threadWakeFrequency =
113       conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
114     long max = -1L;
115     final MemoryUsage usage = HeapMemorySizeUtil.safeGetHeapMemoryUsage();
116     if (usage != null) {
117       max = usage.getMax();
118     }
119     float globalMemStorePercent = HeapMemorySizeUtil.getGlobalMemStorePercent(conf, true);
120     this.globalMemStoreLimit = (long) (max * globalMemStorePercent);
121     this.globalMemStoreLimitLowMarkPercent =
122         HeapMemorySizeUtil.getGlobalMemStoreLowerMark(conf, globalMemStorePercent);
123     this.globalMemStoreLimitLowMark =
124         (long) (this.globalMemStoreLimit * this.globalMemStoreLimitLowMarkPercent);
125 
126     this.blockingWaitTime = conf.getInt("hbase.hstore.blockingWaitTime",
127       90000);
128     int handlerCount = conf.getInt("hbase.hstore.flusher.count", 2);
129     this.flushHandlers = new FlushHandler[handlerCount];
130     LOG.info("globalMemStoreLimit="
131         + TraditionalBinaryPrefix.long2String(this.globalMemStoreLimit, "", 1)
132         + ", globalMemStoreLimitLowMark="
133         + TraditionalBinaryPrefix.long2String(this.globalMemStoreLimitLowMark, "", 1)
134         + ", maxHeap=" + TraditionalBinaryPrefix.long2String(max, "", 1));
135   }
136 
137   public Counter getUpdatesBlockedMsHighWater() {
138     return this.updatesBlockedMsHighWater;
139   }
140 
141   /**
142    * The memstore across all regions has exceeded the low water mark. Pick
143    * one region to flush and flush it synchronously (this is called from the
144    * flush thread)
145    * @return true if successful
146    */
147   private boolean flushOneForGlobalPressure() {
148     SortedMap<Long, Region> regionsBySize = server.getCopyOfOnlineRegionsSortedBySize();
149     Set<Region> excludedRegions = new HashSet<Region>();
150 
151     double secondaryMultiplier
152       = ServerRegionReplicaUtil.getRegionReplicaStoreFileRefreshMultiplier(conf);
153 
154     boolean flushedOne = false;
155     while (!flushedOne) {
156       // Find the biggest region that doesn't have too many storefiles
157       // (might be null!)
158       Region bestFlushableRegion = getBiggestMemstoreRegion(regionsBySize, excludedRegions, true);
159       // Find the biggest region, total, even if it might have too many flushes.
160       Region bestAnyRegion = getBiggestMemstoreRegion(
161           regionsBySize, excludedRegions, false);
162       // Find the biggest region that is a secondary region
163       Region bestRegionReplica = getBiggestMemstoreOfRegionReplica(regionsBySize,
164         excludedRegions);
165 
166       if (bestAnyRegion == null && bestRegionReplica == null) {
167         LOG.error("Above memory mark but there are no flushable regions!");
168         return false;
169       }
170 
171       Region regionToFlush;
172       if (bestFlushableRegion != null &&
173           bestAnyRegion.getMemstoreSize() > 2 * bestFlushableRegion.getMemstoreSize()) {
174         // Even if it's not supposed to be flushed, pick a region if it's more than twice
175         // as big as the best flushable one - otherwise when we're under pressure we make
176         // lots of little flushes and cause lots of compactions, etc, which just makes
177         // life worse!
178         if (LOG.isDebugEnabled()) {
179           LOG.debug("Under global heap pressure: " + "Region "
180               + bestAnyRegion.getRegionInfo().getRegionNameAsString()
181               + " has too many " + "store files, but is "
182               + TraditionalBinaryPrefix.long2String(bestAnyRegion.getMemstoreSize(), "", 1)
183               + " vs best flushable region's "
184               + TraditionalBinaryPrefix.long2String(bestFlushableRegion.getMemstoreSize(), "", 1)
185               + ". Choosing the bigger.");
186         }
187         regionToFlush = bestAnyRegion;
188       } else {
189         if (bestFlushableRegion == null) {
190           regionToFlush = bestAnyRegion;
191         } else {
192           regionToFlush = bestFlushableRegion;
193         }
194       }
195 
196       Preconditions.checkState(
197         (regionToFlush != null && regionToFlush.getMemstoreSize() > 0) ||
198         (bestRegionReplica != null && bestRegionReplica.getMemstoreSize() > 0));
199 
200       if (regionToFlush == null ||
201           (bestRegionReplica != null &&
202            ServerRegionReplicaUtil.isRegionReplicaStoreFileRefreshEnabled(conf) &&
203            (bestRegionReplica.getMemstoreSize()
204                > secondaryMultiplier * regionToFlush.getMemstoreSize()))) {
205         LOG.info("Refreshing storefiles of region " + bestRegionReplica +
206           " due to global heap pressure. memstore size=" + StringUtils.humanReadableInt(
207             server.getRegionServerAccounting().getGlobalMemstoreSize()));
208         flushedOne = refreshStoreFilesAndReclaimMemory(bestRegionReplica);
209         if (!flushedOne) {
210           LOG.info("Excluding secondary region " + bestRegionReplica +
211               " - trying to find a different region to refresh files.");
212           excludedRegions.add(bestRegionReplica);
213         }
214       } else {
215         LOG.info("Flush of region " + regionToFlush + " due to global heap pressure. "
216             + "Total Memstore size="
217             + humanReadableInt(server.getRegionServerAccounting().getGlobalMemstoreSize())
218             + ", Region memstore size="
219             + humanReadableInt(regionToFlush.getMemstoreSize()));
220         flushedOne = flushRegion(regionToFlush, true, false);
221 
222         if (!flushedOne) {
223           LOG.info("Excluding unflushable region " + regionToFlush +
224               " - trying to find a different region to flush.");
225           excludedRegions.add(regionToFlush);
226         }
227       }
228     }
229     return true;
230   }
231 
232   private class FlushHandler extends HasThread {
233 
234     private FlushHandler(String name) {
235       super(name);
236     }
237 
238     @Override
239     public void run() {
240       while (!server.isStopped()) {
241         FlushQueueEntry fqe = null;
242         try {
243           wakeupPending.set(false); // allow someone to wake us up again
244           fqe = flushQueue.poll(threadWakeFrequency, TimeUnit.MILLISECONDS);
245           if (fqe == null || fqe instanceof WakeupFlushThread) {
246             if (isAboveLowWaterMark()) {
247               LOG.debug("Flush thread woke up because memory above low water="
248                   + TraditionalBinaryPrefix.long2String(globalMemStoreLimitLowMark, "", 1));
249               if (!flushOneForGlobalPressure()) {
250                 // Wasn't able to flush any region, but we're above low water mark
251                 // This is unlikely to happen, but might happen when closing the
252                 // entire server - another thread is flushing regions. We'll just
253                 // sleep a little bit to avoid spinning, and then pretend that
254                 // we flushed one, so anyone blocked will check again
255                 Thread.sleep(1000);
256                 wakeUpIfBlocking();
257               }
258               // Enqueue another one of these tokens so we'll wake up again
259               wakeupFlushThread();
260             }
261             continue;
262           }
263           FlushRegionEntry fre = (FlushRegionEntry) fqe;
264           if (!flushRegion(fre)) {
265             break;
266           }
267         } catch (InterruptedException ex) {
268           continue;
269         } catch (ConcurrentModificationException ex) {
270           continue;
271         } catch (Exception ex) {
272           LOG.error("Cache flusher failed for entry " + fqe, ex);
273           if (!server.checkFileSystem()) {
274             break;
275           }
276         }
277       }
278       synchronized (regionsInQueue) {
279         regionsInQueue.clear();
280         flushQueue.clear();
281       }
282 
283       // Signal anyone waiting, so they see the close flag
284       wakeUpIfBlocking();
285       LOG.info(getName() + " exiting");
286     }
287   }
288 
289 
290   private void wakeupFlushThread() {
291     if (wakeupPending.compareAndSet(false, true)) {
292       flushQueue.add(new WakeupFlushThread());
293     }
294   }
295 
296   private Region getBiggestMemstoreRegion(
297       SortedMap<Long, Region> regionsBySize,
298       Set<Region> excludedRegions,
299       boolean checkStoreFileCount) {
300     synchronized (regionsInQueue) {
301       for (Region region : regionsBySize.values()) {
302         if (excludedRegions.contains(region)) {
303           continue;
304         }
305 
306         if (((HRegion)region).writestate.flushing ||
307             !((HRegion)region).writestate.writesEnabled) {
308           continue;
309         }
310 
311         if (checkStoreFileCount && isTooManyStoreFiles(region)) {
312           continue;
313         }
314         return region;
315       }
316     }
317     return null;
318   }
319 
320   private Region getBiggestMemstoreOfRegionReplica(SortedMap<Long, Region> regionsBySize,
321       Set<Region> excludedRegions) {
322     synchronized (regionsInQueue) {
323       for (Region region : regionsBySize.values()) {
324         if (excludedRegions.contains(region)) {
325           continue;
326         }
327 
328         if (RegionReplicaUtil.isDefaultReplica(region.getRegionInfo())) {
329           continue;
330         }
331 
332         return region;
333       }
334     }
335     return null;
336   }
337 
338   private boolean refreshStoreFilesAndReclaimMemory(Region region) {
339     try {
340       return region.refreshStoreFiles();
341     } catch (IOException e) {
342       LOG.warn("Refreshing store files failed with exception", e);
343     }
344     return false;
345   }
346 
347   /**
348    * Return true if global memory usage is above the high watermark
349    */
350   private boolean isAboveHighWaterMark() {
351     return server.getRegionServerAccounting().
352       getGlobalMemstoreSize() >= globalMemStoreLimit;
353   }
354 
355   /**
356    * Return true if we're above the high watermark
357    */
358   private boolean isAboveLowWaterMark() {
359     return server.getRegionServerAccounting().
360       getGlobalMemstoreSize() >= globalMemStoreLimitLowMark;
361   }
362 
363   @Override
364   public boolean requestFlush(Region r, boolean forceFlushAllStores) {
365     synchronized (regionsInQueue) {
366       if (!regionsInQueue.containsKey(r)) {
367         // This entry has no delay so it will be added at the top of the flush
368         // queue.  It'll come out near immediately.
369         FlushRegionEntry fqe = new FlushRegionEntry(r, forceFlushAllStores);
370         this.regionsInQueue.put(r, fqe);
371         this.flushQueue.add(fqe);
372         ((HRegion)r).incrementFlushesQueuedCount();
373         return true;
374       }
375       return false;
376     }
377   }
378 
379   @Override
380   public boolean requestDelayedFlush(Region r, long delay, boolean forceFlushAllStores) {
381     synchronized (regionsInQueue) {
382       if (!regionsInQueue.containsKey(r)) {
383         // This entry has some delay
384         FlushRegionEntry fqe = new FlushRegionEntry(r, forceFlushAllStores);
385         fqe.requeue(delay);
386         this.regionsInQueue.put(r, fqe);
387         this.flushQueue.add(fqe);
388         ((HRegion)r).incrementFlushesQueuedCount();
389         return true;
390       }
391       return false;
392     }
393   }
394 
395   public int getFlushQueueSize() {
396     return flushQueue.size();
397   }
398 
399   /**
400    * Only interrupt once it's done with a run through the work loop.
401    */
402   void interruptIfNecessary() {
403     lock.writeLock().lock();
404     try {
405       for (FlushHandler flushHander : flushHandlers) {
406         if (flushHander != null) flushHander.interrupt();
407       }
408     } finally {
409       lock.writeLock().unlock();
410     }
411   }
412 
413   synchronized void start(UncaughtExceptionHandler eh) {
414     ThreadFactory flusherThreadFactory = Threads.newDaemonThreadFactory(
415         server.getServerName().toShortString() + "-MemStoreFlusher", eh);
416     for (int i = 0; i < flushHandlers.length; i++) {
417       flushHandlers[i] = new FlushHandler("MemStoreFlusher." + i);
418       flusherThreadFactory.newThread(flushHandlers[i]);
419       flushHandlers[i].start();
420     }
421   }
422 
423   boolean isAlive() {
424     for (FlushHandler flushHander : flushHandlers) {
425       if (flushHander != null && flushHander.isAlive()) {
426         return true;
427       }
428     }
429     return false;
430   }
431 
432   void join() {
433     for (FlushHandler flushHander : flushHandlers) {
434       if (flushHander != null) {
435         Threads.shutdown(flushHander.getThread());
436       }
437     }
438   }
439 
440   /**
441    * A flushRegion that checks store file count.  If too many, puts the flush
442    * on delay queue to retry later.
443    * @param fqe
444    * @return true if the region was successfully flushed, false otherwise. If
445    * false, there will be accompanying log messages explaining why the region was
446    * not flushed.
447    */
448   private boolean flushRegion(final FlushRegionEntry fqe) {
449     Region region = fqe.region;
450     if (!region.getRegionInfo().isMetaRegion() &&
451         isTooManyStoreFiles(region)) {
452       if (fqe.isMaximumWait(this.blockingWaitTime)) {
453         LOG.info("Waited " + (EnvironmentEdgeManager.currentTime() - fqe.createTime) +
454           "ms on a compaction to clean up 'too many store files'; waited " +
455           "long enough... proceeding with flush of " +
456           region.getRegionInfo().getRegionNameAsString());
457       } else {
458         // If this is first time we've been put off, then emit a log message.
459         if (fqe.getRequeueCount() <= 0) {
460           // Note: We don't impose blockingStoreFiles constraint on meta regions
461           LOG.warn("Region " + region.getRegionInfo().getRegionNameAsString() + " has too many " +
462             "store files; delaying flush up to " + this.blockingWaitTime + "ms");
463           if (!this.server.compactSplitThread.requestSplit(region)) {
464             try {
465               this.server.compactSplitThread.requestSystemCompaction(
466                   region, Thread.currentThread().getName());
467             } catch (IOException e) {
468               LOG.error("Cache flush failed for region " +
469                   Bytes.toStringBinary(region.getRegionInfo().getRegionName()),
470                 RemoteExceptionHandler.checkIOException(e));
471             }
472           }
473         }
474 
475         // Put back on the queue.  Have it come back out of the queue
476         // after a delay of this.blockingWaitTime / 100 ms.
477         this.flushQueue.add(fqe.requeue(this.blockingWaitTime / 100));
478         // Tell a lie, it's not flushed but it's ok
479         return true;
480       }
481     }
482     return flushRegion(region, false, fqe.isForceFlushAllStores());
483   }
484 
485   /**
486    * Flush a region.
487    * @param region Region to flush.
488    * @param emergencyFlush Set if we are being force flushed. If true the region
489    * needs to be removed from the flush queue. If false, when we were called
490    * from the main flusher run loop and we got the entry to flush by calling
491    * poll on the flush queue (which removed it).
492    * @param forceFlushAllStores whether we want to flush all store.
493    * @return true if the region was successfully flushed, false otherwise. If
494    * false, there will be accompanying log messages explaining why the region was
495    * not flushed.
496    */
497   private boolean flushRegion(final Region region, final boolean emergencyFlush,
498       boolean forceFlushAllStores) {
499     synchronized (this.regionsInQueue) {
500       FlushRegionEntry fqe = this.regionsInQueue.remove(region);
501       // Use the start time of the FlushRegionEntry if available
502       if (fqe != null && emergencyFlush) {
503         // Need to remove from region from delay queue.  When NOT an
504         // emergencyFlush, then item was removed via a flushQueue.poll.
505         flushQueue.remove(fqe);
506       }
507     }
508 
509     lock.readLock().lock();
510     try {
511       notifyFlushRequest(region, emergencyFlush);
512       FlushResult flushResult = region.flush(forceFlushAllStores);
513       boolean shouldCompact = flushResult.isCompactionNeeded();
514       // We just want to check the size
515       boolean shouldSplit = ((HRegion)region).checkSplit() != null;
516       if (shouldSplit) {
517         this.server.compactSplitThread.requestSplit(region);
518       } else if (shouldCompact) {
519         server.compactSplitThread.requestSystemCompaction(
520             region, Thread.currentThread().getName());
521       }
522     } catch (DroppedSnapshotException ex) {
523       // Cache flush can fail in a few places. If it fails in a critical
524       // section, we get a DroppedSnapshotException and a replay of wal
525       // is required. Currently the only way to do this is a restart of
526       // the server. Abort because hdfs is probably bad (HBASE-644 is a case
527       // where hdfs was bad but passed the hdfs check).
528       server.abort("Replay of WAL required. Forcing server shutdown", ex);
529       return false;
530     } catch (IOException ex) {
531       LOG.error("Cache flush failed" + (region != null ? (" for region " +
532           Bytes.toStringBinary(region.getRegionInfo().getRegionName())) : ""),
533         RemoteExceptionHandler.checkIOException(ex));
534       if (!server.checkFileSystem()) {
535         return false;
536       }
537     } finally {
538       lock.readLock().unlock();
539       wakeUpIfBlocking();
540     }
541     return true;
542   }
543 
544   private void notifyFlushRequest(Region region, boolean emergencyFlush) {
545     FlushType type = FlushType.NORMAL;
546     if (emergencyFlush) {
547       type = isAboveHighWaterMark() ? FlushType.ABOVE_HIGHER_MARK : FlushType.ABOVE_LOWER_MARK;
548     }
549     for (FlushRequestListener listener : flushRequestListeners) {
550       listener.flushRequested(type, region);
551     }
552   }
553 
554   private void wakeUpIfBlocking() {
555     synchronized (blockSignal) {
556       blockSignal.notifyAll();
557     }
558   }
559 
560   private boolean isTooManyStoreFiles(Region region) {
561 
562     // When compaction is disabled, the region is flushable
563     if (!region.getTableDesc().isCompactionEnabled()) {
564       return false;
565     }
566 
567     for (Store store : region.getStores()) {
568       if (store.hasTooManyStoreFiles()) {
569         return true;
570       }
571     }
572     return false;
573   }
574 
575   /**
576    * Check if the regionserver's memstore memory usage is greater than the
577    * limit. If so, flush regions with the biggest memstores until we're down
578    * to the lower limit. This method blocks callers until we're down to a safe
579    * amount of memstore consumption.
580    */
581   public void reclaimMemStoreMemory() {
582     try (TraceScope scope = Trace.startSpan("MemStoreFluser.reclaimMemStoreMemory")) {
583       if (isAboveHighWaterMark()) {
584         if (Trace.isTracing()) {
585           scope.getSpan().addTimelineAnnotation("Force Flush. We're above high water mark.");
586         }
587         long start = EnvironmentEdgeManager.currentTime();
588         long nextLogTimeMs = start;
589         synchronized (this.blockSignal) {
590           boolean blocked = false;
591           long startTime = 0;
592           boolean interrupted = false;
593           try {
594             while (isAboveHighWaterMark() && !server.isStopped()) {
595               if (!blocked) {
596                 startTime = EnvironmentEdgeManager.currentTime();
597                 LOG.info("Blocking updates on "
598                     + server.toString()
599                     + ": the global memstore size "
600                     + TraditionalBinaryPrefix.long2String(server.getRegionServerAccounting()
601                     .getGlobalMemstoreSize(), "", 1) + " is >= than blocking "
602                     + TraditionalBinaryPrefix.long2String(globalMemStoreLimit, "", 1) + " size");
603               }
604               blocked = true;
605               wakeupFlushThread();
606               try {
607                 // we should be able to wait forever, but we've seen a bug where
608                 // we miss a notify, so put a 5 second bound on it at least.
609                 blockSignal.wait(5 * 1000);
610               } catch (InterruptedException ie) {
611                 LOG.warn("Interrupted while waiting");
612                 interrupted = true;
613               }
614               long nowMs = EnvironmentEdgeManager.currentTime();
615               if (nowMs >= nextLogTimeMs) {
616                 LOG.warn("Memstore is above high water mark and block " + (nowMs - start) + " ms");
617                 nextLogTimeMs = nowMs + 1000;
618               }
619             }
620           } finally {
621             if (interrupted) {
622               Thread.currentThread().interrupt();
623             }
624           }
625 
626           if (blocked) {
627             final long totalTime = EnvironmentEdgeManager.currentTime() - startTime;
628             if (totalTime > 0) {
629               this.updatesBlockedMsHighWater.add(totalTime);
630             }
631             LOG.info("Unblocking updates for server " + server.toString());
632           }
633         }
634       } else if (isAboveLowWaterMark()) {
635         wakeupFlushThread();
636       }
637     }
638   }
639 
640   @Override
641   public String toString() {
642     return "flush_queue="
643         + flushQueue.size();
644   }
645 
646   public String dumpQueue() {
647     StringBuilder queueList = new StringBuilder();
648     queueList.append("Flush Queue Queue dump:\n");
649     queueList.append("  Flush Queue:\n");
650     java.util.Iterator<FlushQueueEntry> it = flushQueue.iterator();
651 
652     while(it.hasNext()){
653       queueList.append("    "+it.next().toString());
654       queueList.append("\n");
655     }
656 
657     return queueList.toString();
658   }
659 
660   /**
661    * Register a MemstoreFlushListener
662    * @param listener
663    */
664   @Override
665   public void registerFlushRequestListener(final FlushRequestListener listener) {
666     this.flushRequestListeners.add(listener);
667   }
668 
669   /**
670    * Unregister the listener from MemstoreFlushListeners
671    * @param listener
672    * @return true when passed listener is unregistered successfully.
673    */
674   @Override
675   public boolean unregisterFlushRequestListener(final FlushRequestListener listener) {
676     return this.flushRequestListeners.remove(listener);
677   }
678 
679   /**
680    * Sets the global memstore limit to a new size.
681    * @param globalMemStoreSize
682    */
683   @Override
684   public void setGlobalMemstoreLimit(long globalMemStoreSize) {
685     this.globalMemStoreLimit = globalMemStoreSize;
686     this.globalMemStoreLimitLowMark =
687         (long) (this.globalMemStoreLimitLowMarkPercent * globalMemStoreSize);
688     reclaimMemStoreMemory();
689   }
690 
691   public long getMemoryLimit() {
692     return this.globalMemStoreLimit;
693   }
694 
695   interface FlushQueueEntry extends Delayed {
696   }
697 
698   /**
699    * Token to insert into the flush queue that ensures that the flusher does not sleep
700    */
701   static class WakeupFlushThread implements FlushQueueEntry {
702     @Override
703     public long getDelay(TimeUnit unit) {
704       return 0;
705     }
706 
707     @Override
708     public int compareTo(Delayed o) {
709       return -1;
710     }
711 
712     @Override
713     public int hashCode() {
714       return System.identityHashCode(this);
715     }
716 
717     @Override
718     public boolean equals(Object obj) {
719       return Objects.equals(this, obj);
720     }
721   }
722 
723   /**
724    * Datastructure used in the flush queue.  Holds region and retry count.
725    * Keeps tabs on how old this object is.  Implements {@link Delayed}.  On
726    * construction, the delay is zero. When added to a delay queue, we'll come
727    * out near immediately.  Call {@link #requeue(long)} passing delay in
728    * milliseconds before readding to delay queue if you want it to stay there
729    * a while.
730    */
731   static class FlushRegionEntry implements FlushQueueEntry {
732     private final Region region;
733 
734     private final long createTime;
735     private long whenToExpire;
736     private int requeueCount = 0;
737 
738     private boolean forceFlushAllStores;
739 
740     FlushRegionEntry(final Region r, boolean forceFlushAllStores) {
741       this.region = r;
742       this.createTime = EnvironmentEdgeManager.currentTime();
743       this.whenToExpire = this.createTime;
744       this.forceFlushAllStores = forceFlushAllStores;
745     }
746 
747     /**
748      * @param maximumWait
749      * @return True if we have been delayed > <code>maximumWait</code> milliseconds.
750      */
751     public boolean isMaximumWait(final long maximumWait) {
752       return (EnvironmentEdgeManager.currentTime() - this.createTime) > maximumWait;
753     }
754 
755     /**
756      * @return Count of times {@link #requeue(long)} was called; i.e this is
757      * number of times we've been requeued.
758      */
759     public int getRequeueCount() {
760       return this.requeueCount;
761     }
762 
763     /**
764      * @return whether we need to flush all stores.
765      */
766     public boolean isForceFlushAllStores() {
767       return forceFlushAllStores;
768     }
769 
770     /**
771      * @param when When to expire, when to come up out of the queue.
772      * Specify in milliseconds.  This method adds EnvironmentEdgeManager.currentTime()
773      * to whatever you pass.
774      * @return This.
775      */
776     public FlushRegionEntry requeue(final long when) {
777       this.whenToExpire = EnvironmentEdgeManager.currentTime() + when;
778       this.requeueCount++;
779       return this;
780     }
781 
782     @Override
783     public long getDelay(TimeUnit unit) {
784       return unit.convert(this.whenToExpire - EnvironmentEdgeManager.currentTime(),
785           TimeUnit.MILLISECONDS);
786     }
787 
788     @Override
789     public int compareTo(Delayed other) {
790       // Delay is compared first. If there is a tie, compare region's hash code
791       int ret = Long.valueOf(getDelay(TimeUnit.MILLISECONDS) -
792         other.getDelay(TimeUnit.MILLISECONDS)).intValue();
793       if (ret != 0) {
794         return ret;
795       }
796       FlushQueueEntry otherEntry = (FlushQueueEntry) other;
797       return hashCode() - otherEntry.hashCode();
798     }
799 
800     @Override
801     public String toString() {
802       return "[flush region "+Bytes.toStringBinary(region.getRegionInfo().getRegionName())+"]";
803     }
804 
805     @Override
806     public int hashCode() {
807       int hash = (int) getDelay(TimeUnit.MILLISECONDS);
808       return hash ^ region.hashCode();
809     }
810 
811    @Override
812     public boolean equals(Object obj) {
813       if (this == obj) {
814         return true;
815       }
816       if (obj == null || getClass() != obj.getClass()) {
817         return false;
818       }
819       FlushRegionEntry other = (FlushRegionEntry) obj;
820       if (!Bytes.equals(this.region.getRegionInfo().getRegionName(),
821           other.region.getRegionInfo().getRegionName())) {
822         return false;
823       }
824       return compareTo(other) == 0;
825     }
826   }
827 }