1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21 package org.apache.hadoop.hbase.regionserver;
22
23 import java.lang.management.ManagementFactory;
24 import java.lang.management.RuntimeMXBean;
25 import java.rmi.UnexpectedException;
26 import java.util.Collections;
27 import java.util.Iterator;
28 import java.util.List;
29 import java.util.NavigableSet;
30 import java.util.SortedSet;
31 import java.util.concurrent.atomic.AtomicLong;
32 import java.util.concurrent.locks.ReentrantReadWriteLock;
33
34 import org.apache.commons.logging.Log;
35 import org.apache.commons.logging.LogFactory;
36 import org.apache.hadoop.hbase.HConstants;
37 import org.apache.hadoop.hbase.KeyValue;
38 import org.apache.hadoop.hbase.client.Scan;
39 import org.apache.hadoop.hbase.io.HeapSize;
40 import org.apache.hadoop.hbase.util.Bytes;
41 import org.apache.hadoop.hbase.util.ClassSize;
42
43
44
45
46
47
48
49
50
51
52
53
54 public class MemStore implements HeapSize {
55 private static final Log LOG = LogFactory.getLog(MemStore.class);
56
57
58
59
60
61
62 volatile KeyValueSkipListSet kvset;
63
64
65 volatile KeyValueSkipListSet snapshot;
66
67 final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
68
69 final KeyValue.KVComparator comparator;
70
71
72 final KeyValue.KVComparator comparatorIgnoreType;
73
74
75 final KeyValue.KVComparator comparatorIgnoreTimestamp;
76
77
78 final AtomicLong size;
79
80 TimeRangeTracker timeRangeTracker;
81 TimeRangeTracker snapshotTimeRangeTracker;
82
83
84
85
86 public MemStore() {
87 this(KeyValue.COMPARATOR);
88 }
89
90
91
92
93
94 public MemStore(final KeyValue.KVComparator c) {
95 this.comparator = c;
96 this.comparatorIgnoreTimestamp =
97 this.comparator.getComparatorIgnoringTimestamps();
98 this.comparatorIgnoreType = this.comparator.getComparatorIgnoringType();
99 this.kvset = new KeyValueSkipListSet(c);
100 this.snapshot = new KeyValueSkipListSet(c);
101 timeRangeTracker = new TimeRangeTracker();
102 snapshotTimeRangeTracker = new TimeRangeTracker();
103 this.size = new AtomicLong(DEEP_OVERHEAD);
104 }
105
106 void dump() {
107 for (KeyValue kv: this.kvset) {
108 LOG.info(kv);
109 }
110 for (KeyValue kv: this.snapshot) {
111 LOG.info(kv);
112 }
113 }
114
115
116
117
118
119
120 void snapshot() {
121 this.lock.writeLock().lock();
122 try {
123
124
125 if (!this.snapshot.isEmpty()) {
126 LOG.warn("Snapshot called again without clearing previous. " +
127 "Doing nothing. Another ongoing flush or did we fail last attempt?");
128 } else {
129 if (!this.kvset.isEmpty()) {
130 this.snapshot = this.kvset;
131 this.kvset = new KeyValueSkipListSet(this.comparator);
132 this.snapshotTimeRangeTracker = this.timeRangeTracker;
133 this.timeRangeTracker = new TimeRangeTracker();
134
135 this.size.set(DEEP_OVERHEAD);
136 }
137 }
138 } finally {
139 this.lock.writeLock().unlock();
140 }
141 }
142
143
144
145
146
147
148
149
150
151 KeyValueSkipListSet getSnapshot() {
152 return this.snapshot;
153 }
154
155
156
157
158
159
160
161 void clearSnapshot(final SortedSet<KeyValue> ss)
162 throws UnexpectedException {
163 this.lock.writeLock().lock();
164 try {
165 if (this.snapshot != ss) {
166 throw new UnexpectedException("Current snapshot is " +
167 this.snapshot + ", was passed " + ss);
168 }
169
170
171 if (!ss.isEmpty()) {
172 this.snapshot = new KeyValueSkipListSet(this.comparator);
173 this.snapshotTimeRangeTracker = new TimeRangeTracker();
174 }
175 } finally {
176 this.lock.writeLock().unlock();
177 }
178 }
179
180
181
182
183
184
185 long add(final KeyValue kv) {
186 long s = -1;
187 this.lock.readLock().lock();
188 try {
189 s = heapSizeChange(kv, this.kvset.add(kv));
190 timeRangeTracker.includeTimestamp(kv);
191 this.size.addAndGet(s);
192 } finally {
193 this.lock.readLock().unlock();
194 }
195 return s;
196 }
197
198
199
200
201
202
203 long delete(final KeyValue delete) {
204 long s = 0;
205 this.lock.readLock().lock();
206 try {
207 s += heapSizeChange(delete, this.kvset.add(delete));
208 timeRangeTracker.includeTimestamp(delete);
209 } finally {
210 this.lock.readLock().unlock();
211 }
212 this.size.addAndGet(s);
213 return s;
214 }
215
216
217
218
219
220
221 KeyValue getNextRow(final KeyValue kv) {
222 this.lock.readLock().lock();
223 try {
224 return getLowest(getNextRow(kv, this.kvset), getNextRow(kv, this.snapshot));
225 } finally {
226 this.lock.readLock().unlock();
227 }
228 }
229
230
231
232
233
234
235 private KeyValue getLowest(final KeyValue a, final KeyValue b) {
236 if (a == null) {
237 return b;
238 }
239 if (b == null) {
240 return a;
241 }
242 return comparator.compareRows(a, b) <= 0? a: b;
243 }
244
245
246
247
248
249
250
251 private KeyValue getNextRow(final KeyValue key,
252 final NavigableSet<KeyValue> set) {
253 KeyValue result = null;
254 SortedSet<KeyValue> tail = key == null? set: set.tailSet(key);
255
256 for (KeyValue kv: tail) {
257 if (comparator.compareRows(kv, key) <= 0)
258 continue;
259
260
261 result = kv;
262 break;
263 }
264 return result;
265 }
266
267
268
269
270 void getRowKeyAtOrBefore(final GetClosestRowBeforeTracker state) {
271 this.lock.readLock().lock();
272 try {
273 getRowKeyAtOrBefore(kvset, state);
274 getRowKeyAtOrBefore(snapshot, state);
275 } finally {
276 this.lock.readLock().unlock();
277 }
278 }
279
280
281
282
283
284 private void getRowKeyAtOrBefore(final NavigableSet<KeyValue> set,
285 final GetClosestRowBeforeTracker state) {
286 if (set.isEmpty()) {
287 return;
288 }
289 if (!walkForwardInSingleRow(set, state.getTargetKey(), state)) {
290
291 getRowKeyBefore(set, state);
292 }
293 }
294
295
296
297
298
299
300
301
302
303
304
305 private boolean walkForwardInSingleRow(final SortedSet<KeyValue> set,
306 final KeyValue firstOnRow, final GetClosestRowBeforeTracker state) {
307 boolean foundCandidate = false;
308 SortedSet<KeyValue> tail = set.tailSet(firstOnRow);
309 if (tail.isEmpty()) return foundCandidate;
310 for (Iterator<KeyValue> i = tail.iterator(); i.hasNext();) {
311 KeyValue kv = i.next();
312
313 if (state.isTooFar(kv, firstOnRow)) break;
314 if (state.isExpired(kv)) {
315 i.remove();
316 continue;
317 }
318
319 if (state.handle(kv)) {
320 foundCandidate = true;
321 break;
322 }
323 }
324 return foundCandidate;
325 }
326
327
328
329
330
331
332
333 private void getRowKeyBefore(NavigableSet<KeyValue> set,
334 final GetClosestRowBeforeTracker state) {
335 KeyValue firstOnRow = state.getTargetKey();
336 for (Member p = memberOfPreviousRow(set, state, firstOnRow);
337 p != null; p = memberOfPreviousRow(p.set, state, firstOnRow)) {
338
339 if (!state.isTargetTable(p.kv)) break;
340
341 if (!state.isBetterCandidate(p.kv)) break;
342
343 firstOnRow = new KeyValue(p.kv.getRow(), HConstants.LATEST_TIMESTAMP);
344
345 if (walkForwardInSingleRow(p.set, firstOnRow, state)) break;
346 }
347 }
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364 public long updateColumnValue(byte[] row,
365 byte[] family,
366 byte[] qualifier,
367 long newValue,
368 long now) {
369 this.lock.readLock().lock();
370 try {
371 KeyValue firstKv = KeyValue.createFirstOnRow(
372 row, family, qualifier);
373
374 KeyValue newKv;
375
376 SortedSet<KeyValue> snSs = snapshot.tailSet(firstKv);
377 if (!snSs.isEmpty()) {
378 KeyValue snKv = snSs.first();
379
380 if (snKv.matchingRow(firstKv) && snKv.matchingQualifier(firstKv)) {
381 if (snKv.getTimestamp() == now) {
382
383 now += 1;
384 }
385 }
386 }
387
388
389
390
391
392
393
394 SortedSet<KeyValue> ss = kvset.tailSet(firstKv);
395 Iterator<KeyValue> it = ss.iterator();
396 while ( it.hasNext() ) {
397 KeyValue kv = it.next();
398
399
400 if (!firstKv.matchingColumn(family,qualifier) || !firstKv.matchingRow(kv) ) {
401 break;
402 }
403
404
405 if (firstKv.matchingQualifier(kv)) {
406
407 if (kv.getType() == KeyValue.Type.Put.getCode()) {
408 now = Math.max(now, kv.getTimestamp());
409 }
410 }
411 }
412
413
414
415
416 newKv = new KeyValue(row, family, qualifier,
417 now,
418 Bytes.toBytes(newValue));
419 long addedSize = add(newKv);
420
421
422 ss = kvset.tailSet(firstKv);
423 it = ss.iterator();
424 while ( it.hasNext() ) {
425 KeyValue kv = it.next();
426
427 if (kv == newKv) {
428
429 continue;
430 }
431
432
433 if (!firstKv.matchingColumn(family,qualifier) || !firstKv.matchingRow(kv)) {
434 break;
435 }
436
437
438 if (firstKv.matchingQualifier(kv)) {
439
440 if (kv.getType() == KeyValue.Type.Put.getCode()) {
441
442 addedSize -= heapSizeChange(kv, true);
443
444 it.remove();
445 }
446 }
447 }
448
449 return addedSize;
450 } finally {
451 this.lock.readLock().unlock();
452 }
453 }
454
455
456
457
458
459 private static class Member {
460 final KeyValue kv;
461 final NavigableSet<KeyValue> set;
462 Member(final NavigableSet<KeyValue> s, final KeyValue kv) {
463 this.kv = kv;
464 this.set = s;
465 }
466 }
467
468
469
470
471
472
473
474
475
476 private Member memberOfPreviousRow(NavigableSet<KeyValue> set,
477 final GetClosestRowBeforeTracker state, final KeyValue firstOnRow) {
478 NavigableSet<KeyValue> head = set.headSet(firstOnRow, false);
479 if (head.isEmpty()) return null;
480 for (Iterator<KeyValue> i = head.descendingIterator(); i.hasNext();) {
481 KeyValue found = i.next();
482 if (state.isExpired(found)) {
483 i.remove();
484 continue;
485 }
486 return new Member(head, found);
487 }
488 return null;
489 }
490
491
492
493
494 List<KeyValueScanner> getScanners() {
495 this.lock.readLock().lock();
496 try {
497 return Collections.<KeyValueScanner>singletonList(
498 new MemStoreScanner());
499 } finally {
500 this.lock.readLock().unlock();
501 }
502 }
503
504
505
506
507
508
509 public boolean shouldSeek(Scan scan) {
510 return timeRangeTracker.includesTimeRange(scan.getTimeRange()) ||
511 snapshotTimeRangeTracker.includesTimeRange(scan.getTimeRange());
512 }
513
514 public TimeRangeTracker getSnapshotTimeRangeTracker() {
515 return this.snapshotTimeRangeTracker;
516 }
517
518
519
520
521
522
523
524 protected class MemStoreScanner implements KeyValueScanner {
525
526 private KeyValue kvsetNextRow = null;
527 private KeyValue snapshotNextRow = null;
528
529
530 Iterator<KeyValue> kvsetIt;
531 Iterator<KeyValue> snapshotIt;
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551 MemStoreScanner() {
552 super();
553
554
555 }
556
557 protected KeyValue getNext(Iterator<KeyValue> it) {
558 KeyValue ret = null;
559 long readPoint = ReadWriteConsistencyControl.getThreadReadPoint();
560
561
562 while (ret == null && it.hasNext()) {
563 KeyValue v = it.next();
564 if (v.getMemstoreTS() <= readPoint) {
565
566 ret = v;
567 }
568 }
569 return ret;
570 }
571
572 public synchronized boolean seek(KeyValue key) {
573 if (key == null) {
574 close();
575 return false;
576 }
577
578
579
580 SortedSet<KeyValue> kvTail = kvset.tailSet(key);
581 SortedSet<KeyValue> snapshotTail = snapshot.tailSet(key);
582
583 kvsetIt = kvTail.iterator();
584 snapshotIt = snapshotTail.iterator();
585
586 kvsetNextRow = getNext(kvsetIt);
587 snapshotNextRow = getNext(snapshotIt);
588
589
590
591
592
593
594
595
596
597 KeyValue lowest = getLowest();
598
599
600 return lowest != null;
601 }
602
603 @Override
604 public boolean reseek(KeyValue key) {
605 while (kvsetNextRow != null &&
606 comparator.compare(kvsetNextRow, key) < 0) {
607 kvsetNextRow = getNext(kvsetIt);
608 }
609
610 while (snapshotNextRow != null &&
611 comparator.compare(snapshotNextRow, key) < 0) {
612 snapshotNextRow = getNext(snapshotIt);
613 }
614 return (kvsetNextRow != null || snapshotNextRow != null);
615 }
616
617 public synchronized KeyValue peek() {
618
619 return getLowest();
620 }
621
622
623 public synchronized KeyValue next() {
624 KeyValue theNext = getLowest();
625
626 if (theNext == null) {
627 return null;
628 }
629
630
631 if (theNext == kvsetNextRow) {
632 kvsetNextRow = getNext(kvsetIt);
633 } else {
634 snapshotNextRow = getNext(snapshotIt);
635 }
636
637
638
639
640 return theNext;
641 }
642
643 protected KeyValue getLowest() {
644 return getLower(kvsetNextRow,
645 snapshotNextRow);
646 }
647
648
649
650
651
652
653 protected KeyValue getLower(KeyValue first, KeyValue second) {
654 if (first == null && second == null) {
655 return null;
656 }
657 if (first != null && second != null) {
658 int compare = comparator.compare(first, second);
659 return (compare <= 0 ? first : second);
660 }
661 return (first != null ? first : second);
662 }
663
664 public synchronized void close() {
665 this.kvsetNextRow = null;
666 this.snapshotNextRow = null;
667
668 this.kvsetIt = null;
669 this.snapshotIt = null;
670 }
671 }
672
673 public final static long FIXED_OVERHEAD = ClassSize.align(
674 ClassSize.OBJECT + (9 * ClassSize.REFERENCE));
675
676 public final static long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
677 ClassSize.REENTRANT_LOCK + ClassSize.ATOMIC_LONG +
678 ClassSize.COPYONWRITE_ARRAYSET + ClassSize.COPYONWRITE_ARRAYLIST +
679 (2 * ClassSize.CONCURRENT_SKIPLISTMAP));
680
681
682
683
684
685
686
687
688 long heapSizeChange(final KeyValue kv, final boolean notpresent) {
689 return notpresent ?
690 ClassSize.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + kv.heapSize()):
691 0;
692 }
693
694
695
696
697
698 @Override
699 public long heapSize() {
700 return size.get();
701 }
702
703
704
705
706 public long keySize() {
707 return heapSize() - DEEP_OVERHEAD;
708 }
709
710
711
712
713
714
715
716
717 public static void main(String [] args) {
718 RuntimeMXBean runtime = ManagementFactory.getRuntimeMXBean();
719 LOG.info("vmName=" + runtime.getVmName() + ", vmVendor=" +
720 runtime.getVmVendor() + ", vmVersion=" + runtime.getVmVersion());
721 LOG.info("vmInputArguments=" + runtime.getInputArguments());
722 MemStore memstore1 = new MemStore();
723
724 long size = 0;
725 final int count = 10000;
726 byte [] fam = Bytes.toBytes("col");
727 byte [] qf = Bytes.toBytes("umn");
728 byte [] empty = new byte[0];
729 for (int i = 0; i < count; i++) {
730
731 size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
732 }
733 LOG.info("memstore1 estimated size=" + size);
734 for (int i = 0; i < count; i++) {
735 size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
736 }
737 LOG.info("memstore1 estimated size (2nd loading of same data)=" + size);
738
739 MemStore memstore2 = new MemStore();
740 for (int i = 0; i < count; i++) {
741 size += memstore2.add(new KeyValue(Bytes.toBytes(i), fam, qf, i,
742 new byte[i]));
743 }
744 LOG.info("memstore2 estimated size=" + size);
745 final int seconds = 30;
746 LOG.info("Waiting " + seconds + " seconds while heap dump is taken");
747 for (int i = 0; i < seconds; i++) {
748
749 }
750 LOG.info("Exiting.");
751 }
752 }