1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
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
68
69
70
71
72
73
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
81
82 private final BlockingQueue<FlushQueueEntry> flushQueue =
83 new DelayQueue<FlushQueueEntry>();
84 protected 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
105
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
143
144
145
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
157
158 Region bestFlushableRegion = getBiggestMemstoreRegion(regionsBySize, excludedRegions, true);
159
160 Region bestAnyRegion = getBiggestMemstoreRegion(
161 regionsBySize, excludedRegions, false);
162
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
175
176
177
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);
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
251
252
253
254
255 Thread.sleep(1000);
256 wakeUpIfBlocking();
257 }
258
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
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
349
350 private boolean isAboveHighWaterMark() {
351 return server.getRegionServerAccounting().
352 getGlobalMemstoreSize() >= globalMemStoreLimit;
353 }
354
355
356
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 FlushRegionEntry existFqe = regionsInQueue.get(r);
367 if (existFqe != null) {
368
369 if (existFqe.isDelay() && existFqe.whenToExpire > EnvironmentEdgeManager.currentTime()) {
370 LOG.info("Remove the existing delayed flush entry for " + r + ", "
371 + "because we need to flush it immediately");
372 this.regionsInQueue.remove(r);
373 this.flushQueue.remove(existFqe);
374 ((HRegion)r).decrementFlushesQueuedCount();
375 } else {
376 LOG.info("Flush already requested on " + r);
377 return false;
378 }
379 }
380
381
382
383 FlushRegionEntry fqe = new FlushRegionEntry(r, forceFlushAllStores);
384 this.regionsInQueue.put(r, fqe);
385 this.flushQueue.add(fqe);
386 ((HRegion)r).incrementFlushesQueuedCount();
387 return true;
388 }
389 }
390
391 @Override
392 public boolean requestDelayedFlush(Region r, long delay, boolean forceFlushAllStores) {
393 synchronized (regionsInQueue) {
394 if (!regionsInQueue.containsKey(r)) {
395
396 FlushRegionEntry fqe = new FlushRegionEntry(r, forceFlushAllStores);
397 fqe.requeue(delay);
398 this.regionsInQueue.put(r, fqe);
399 this.flushQueue.add(fqe);
400 ((HRegion)r).incrementFlushesQueuedCount();
401 return true;
402 }
403 return false;
404 }
405 }
406
407 public int getFlushQueueSize() {
408 return flushQueue.size();
409 }
410
411
412
413
414 void interruptIfNecessary() {
415 lock.writeLock().lock();
416 try {
417 for (FlushHandler flushHander : flushHandlers) {
418 if (flushHander != null) flushHander.interrupt();
419 }
420 } finally {
421 lock.writeLock().unlock();
422 }
423 }
424
425 synchronized void start(UncaughtExceptionHandler eh) {
426 ThreadFactory flusherThreadFactory = Threads.newDaemonThreadFactory(
427 server.getServerName().toShortString() + "-MemStoreFlusher", eh);
428 for (int i = 0; i < flushHandlers.length; i++) {
429 flushHandlers[i] = new FlushHandler("MemStoreFlusher." + i);
430 flusherThreadFactory.newThread(flushHandlers[i]);
431 flushHandlers[i].start();
432 }
433 }
434
435 boolean isAlive() {
436 for (FlushHandler flushHander : flushHandlers) {
437 if (flushHander != null && flushHander.isAlive()) {
438 return true;
439 }
440 }
441 return false;
442 }
443
444 void join() {
445 for (FlushHandler flushHander : flushHandlers) {
446 if (flushHander != null) {
447 Threads.shutdown(flushHander.getThread());
448 }
449 }
450 }
451
452
453
454
455
456
457
458
459
460 private boolean flushRegion(final FlushRegionEntry fqe) {
461 Region region = fqe.region;
462 if (!region.getRegionInfo().isMetaRegion() &&
463 isTooManyStoreFiles(region)) {
464 if (fqe.isMaximumWait(this.blockingWaitTime)) {
465 LOG.info("Waited " + (EnvironmentEdgeManager.currentTime() - fqe.createTime) +
466 "ms on a compaction to clean up 'too many store files'; waited " +
467 "long enough... proceeding with flush of " +
468 region.getRegionInfo().getRegionNameAsString());
469 } else {
470
471 if (fqe.getRequeueCount() <= 0) {
472
473 LOG.warn("Region " + region.getRegionInfo().getRegionNameAsString() + " has too many " +
474 "store files; delaying flush up to " + this.blockingWaitTime + "ms");
475 if (!this.server.compactSplitThread.requestSplit(region)) {
476 try {
477 this.server.compactSplitThread.requestSystemCompaction(
478 region, Thread.currentThread().getName());
479 } catch (IOException e) {
480 LOG.error("Cache flush failed for region " +
481 Bytes.toStringBinary(region.getRegionInfo().getRegionName()),
482 RemoteExceptionHandler.checkIOException(e));
483 }
484 }
485 }
486
487
488
489 this.flushQueue.add(fqe.requeue(this.blockingWaitTime / 100));
490
491 return true;
492 }
493 }
494 return flushRegion(region, false, fqe.isForceFlushAllStores());
495 }
496
497
498
499
500
501
502
503
504
505
506
507
508
509 private boolean flushRegion(final Region region, final boolean emergencyFlush,
510 boolean forceFlushAllStores) {
511 synchronized (this.regionsInQueue) {
512 FlushRegionEntry fqe = this.regionsInQueue.remove(region);
513
514 if (fqe != null && emergencyFlush) {
515
516
517 flushQueue.remove(fqe);
518 }
519 }
520
521 lock.readLock().lock();
522 try {
523 notifyFlushRequest(region, emergencyFlush);
524 FlushResult flushResult = region.flush(forceFlushAllStores);
525 boolean shouldCompact = flushResult.isCompactionNeeded();
526
527 boolean shouldSplit = ((HRegion)region).checkSplit() != null;
528 if (shouldSplit) {
529 this.server.compactSplitThread.requestSplit(region);
530 } else if (shouldCompact) {
531 server.compactSplitThread.requestSystemCompaction(
532 region, Thread.currentThread().getName());
533 }
534 } catch (DroppedSnapshotException ex) {
535
536
537
538
539
540 server.abort("Replay of WAL required. Forcing server shutdown", ex);
541 return false;
542 } catch (IOException ex) {
543 LOG.error("Cache flush failed" + (region != null ? (" for region " +
544 Bytes.toStringBinary(region.getRegionInfo().getRegionName())) : ""),
545 RemoteExceptionHandler.checkIOException(ex));
546 if (!server.checkFileSystem()) {
547 return false;
548 }
549 } finally {
550 lock.readLock().unlock();
551 wakeUpIfBlocking();
552 }
553 return true;
554 }
555
556 private void notifyFlushRequest(Region region, boolean emergencyFlush) {
557 FlushType type = FlushType.NORMAL;
558 if (emergencyFlush) {
559 type = isAboveHighWaterMark() ? FlushType.ABOVE_HIGHER_MARK : FlushType.ABOVE_LOWER_MARK;
560 }
561 for (FlushRequestListener listener : flushRequestListeners) {
562 listener.flushRequested(type, region);
563 }
564 }
565
566 private void wakeUpIfBlocking() {
567 synchronized (blockSignal) {
568 blockSignal.notifyAll();
569 }
570 }
571
572 private boolean isTooManyStoreFiles(Region region) {
573
574
575 if (!region.getTableDesc().isCompactionEnabled()) {
576 return false;
577 }
578
579 for (Store store : region.getStores()) {
580 if (store.hasTooManyStoreFiles()) {
581 return true;
582 }
583 }
584 return false;
585 }
586
587
588
589
590
591
592
593 public void reclaimMemStoreMemory() {
594 try (TraceScope scope = Trace.startSpan("MemStoreFluser.reclaimMemStoreMemory")) {
595 if (isAboveHighWaterMark()) {
596 if (Trace.isTracing()) {
597 scope.getSpan().addTimelineAnnotation("Force Flush. We're above high water mark.");
598 }
599 long start = EnvironmentEdgeManager.currentTime();
600 long nextLogTimeMs = start;
601 synchronized (this.blockSignal) {
602 boolean blocked = false;
603 long startTime = 0;
604 boolean interrupted = false;
605 try {
606 while (isAboveHighWaterMark() && !server.isStopped()) {
607 if (!blocked) {
608 startTime = EnvironmentEdgeManager.currentTime();
609 LOG.info("Blocking updates on "
610 + server.toString()
611 + ": the global memstore size "
612 + TraditionalBinaryPrefix.long2String(server.getRegionServerAccounting()
613 .getGlobalMemstoreSize(), "", 1) + " is >= than blocking "
614 + TraditionalBinaryPrefix.long2String(globalMemStoreLimit, "", 1) + " size");
615 }
616 blocked = true;
617 wakeupFlushThread();
618 try {
619
620
621 blockSignal.wait(5 * 1000);
622 } catch (InterruptedException ie) {
623 LOG.warn("Interrupted while waiting");
624 interrupted = true;
625 }
626 long nowMs = EnvironmentEdgeManager.currentTime();
627 if (nowMs >= nextLogTimeMs) {
628 LOG.warn("Memstore is above high water mark and block " + (nowMs - start) + " ms");
629 nextLogTimeMs = nowMs + 1000;
630 }
631 }
632 } finally {
633 if (interrupted) {
634 Thread.currentThread().interrupt();
635 }
636 }
637
638 if (blocked) {
639 final long totalTime = EnvironmentEdgeManager.currentTime() - startTime;
640 if (totalTime > 0) {
641 this.updatesBlockedMsHighWater.add(totalTime);
642 }
643 LOG.info("Unblocking updates for server " + server.toString());
644 }
645 }
646 } else if (isAboveLowWaterMark()) {
647 wakeupFlushThread();
648 }
649 }
650 }
651
652 @Override
653 public String toString() {
654 return "flush_queue="
655 + flushQueue.size();
656 }
657
658 public String dumpQueue() {
659 StringBuilder queueList = new StringBuilder();
660 queueList.append("Flush Queue Queue dump:\n");
661 queueList.append(" Flush Queue:\n");
662 java.util.Iterator<FlushQueueEntry> it = flushQueue.iterator();
663
664 while(it.hasNext()){
665 queueList.append(" "+it.next().toString());
666 queueList.append("\n");
667 }
668
669 return queueList.toString();
670 }
671
672
673
674
675
676 @Override
677 public void registerFlushRequestListener(final FlushRequestListener listener) {
678 this.flushRequestListeners.add(listener);
679 }
680
681
682
683
684
685
686 @Override
687 public boolean unregisterFlushRequestListener(final FlushRequestListener listener) {
688 return this.flushRequestListeners.remove(listener);
689 }
690
691
692
693
694
695 @Override
696 public void setGlobalMemstoreLimit(long globalMemStoreSize) {
697 this.globalMemStoreLimit = globalMemStoreSize;
698 this.globalMemStoreLimitLowMark =
699 (long) (this.globalMemStoreLimitLowMarkPercent * globalMemStoreSize);
700 reclaimMemStoreMemory();
701 }
702
703 public long getMemoryLimit() {
704 return this.globalMemStoreLimit;
705 }
706
707 interface FlushQueueEntry extends Delayed {
708 }
709
710
711
712
713 static class WakeupFlushThread implements FlushQueueEntry {
714 @Override
715 public long getDelay(TimeUnit unit) {
716 return 0;
717 }
718
719 @Override
720 public int compareTo(Delayed o) {
721 return -1;
722 }
723
724 @Override
725 public int hashCode() {
726 return System.identityHashCode(this);
727 }
728
729 @Override
730 public boolean equals(Object obj) {
731 return Objects.equals(this, obj);
732 }
733 }
734
735
736
737
738
739
740
741
742
743 static class FlushRegionEntry implements FlushQueueEntry {
744 private final Region region;
745
746 private final long createTime;
747 private long whenToExpire;
748 private int requeueCount = 0;
749
750 private boolean forceFlushAllStores;
751
752 FlushRegionEntry(final Region r, boolean forceFlushAllStores) {
753 this.region = r;
754 this.createTime = EnvironmentEdgeManager.currentTime();
755 this.whenToExpire = this.createTime;
756 this.forceFlushAllStores = forceFlushAllStores;
757 }
758
759
760
761
762
763 public boolean isMaximumWait(final long maximumWait) {
764 return (EnvironmentEdgeManager.currentTime() - this.createTime) > maximumWait;
765 }
766
767
768
769
770 protected boolean isDelay() {
771 return this.whenToExpire > this.createTime;
772 }
773
774
775
776
777
778 public int getRequeueCount() {
779 return this.requeueCount;
780 }
781
782
783
784
785 public boolean isForceFlushAllStores() {
786 return forceFlushAllStores;
787 }
788
789
790
791
792
793
794
795 public FlushRegionEntry requeue(final long when) {
796 this.whenToExpire = EnvironmentEdgeManager.currentTime() + when;
797 this.requeueCount++;
798 return this;
799 }
800
801 @Override
802 public long getDelay(TimeUnit unit) {
803 return unit.convert(this.whenToExpire - EnvironmentEdgeManager.currentTime(),
804 TimeUnit.MILLISECONDS);
805 }
806
807 @Override
808 public int compareTo(Delayed other) {
809
810 int ret = Long.valueOf(getDelay(TimeUnit.MILLISECONDS) -
811 other.getDelay(TimeUnit.MILLISECONDS)).intValue();
812 if (ret != 0) {
813 return ret;
814 }
815 FlushQueueEntry otherEntry = (FlushQueueEntry) other;
816 return hashCode() - otherEntry.hashCode();
817 }
818
819 @Override
820 public String toString() {
821 return "[flush region "+Bytes.toStringBinary(region.getRegionInfo().getRegionName())+"]";
822 }
823
824 @Override
825 public int hashCode() {
826 int hash = (int) getDelay(TimeUnit.MILLISECONDS);
827 return hash ^ region.hashCode();
828 }
829
830 @Override
831 public boolean equals(Object obj) {
832 if (this == obj) {
833 return true;
834 }
835 if (obj == null || getClass() != obj.getClass()) {
836 return false;
837 }
838 FlushRegionEntry other = (FlushRegionEntry) obj;
839 if (!Bytes.equals(this.region.getRegionInfo().getRegionName(),
840 other.region.getRegionInfo().getRegionName())) {
841 return false;
842 }
843 return compareTo(other) == 0;
844 }
845 }
846 }