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