1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.regionserver;
21
22 import java.lang.management.ManagementFactory;
23 import java.lang.management.RuntimeMXBean;
24 import java.rmi.UnexpectedException;
25 import java.util.ArrayList;
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.classification.InterfaceAudience;
37 import org.apache.hadoop.conf.Configuration;
38 import org.apache.hadoop.hbase.Cell;
39 import org.apache.hadoop.hbase.HBaseConfiguration;
40 import org.apache.hadoop.hbase.HConstants;
41 import org.apache.hadoop.hbase.KeyValue;
42 import org.apache.hadoop.hbase.KeyValueUtil;
43 import org.apache.hadoop.hbase.client.Scan;
44 import org.apache.hadoop.hbase.io.HeapSize;
45 import org.apache.hadoop.hbase.regionserver.MemStoreLAB.Allocation;
46 import org.apache.hadoop.hbase.util.Bytes;
47 import org.apache.hadoop.hbase.util.ClassSize;
48
49
50
51
52
53
54
55
56
57
58
59
60 @InterfaceAudience.Private
61 public class MemStore implements HeapSize {
62 private static final Log LOG = LogFactory.getLog(MemStore.class);
63
64 static final String USEMSLAB_KEY =
65 "hbase.hregion.memstore.mslab.enabled";
66 private static final boolean USEMSLAB_DEFAULT = true;
67
68 private Configuration conf;
69
70
71
72
73
74
75 volatile KeyValueSkipListSet kvset;
76
77
78 volatile KeyValueSkipListSet snapshot;
79
80 final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
81
82 final KeyValue.KVComparator comparator;
83
84
85 final KeyValue.KVComparator comparatorIgnoreType;
86
87
88 final KeyValue.KVComparator comparatorIgnoreTimestamp;
89
90
91 final AtomicLong size;
92
93 TimeRangeTracker timeRangeTracker;
94 TimeRangeTracker snapshotTimeRangeTracker;
95
96 MemStoreChunkPool chunkPool;
97 volatile MemStoreLAB allocator;
98 volatile MemStoreLAB snapshotAllocator;
99
100
101
102
103
104
105 public MemStore() {
106 this(HBaseConfiguration.create(), KeyValue.COMPARATOR);
107 }
108
109
110
111
112
113 public MemStore(final Configuration conf,
114 final KeyValue.KVComparator c) {
115 this.conf = conf;
116 this.comparator = c;
117 this.comparatorIgnoreTimestamp =
118 this.comparator.getComparatorIgnoringTimestamps();
119 this.comparatorIgnoreType = this.comparator.getComparatorIgnoringType();
120 this.kvset = new KeyValueSkipListSet(c);
121 this.snapshot = new KeyValueSkipListSet(c);
122 timeRangeTracker = new TimeRangeTracker();
123 snapshotTimeRangeTracker = new TimeRangeTracker();
124 this.size = new AtomicLong(DEEP_OVERHEAD);
125 if (conf.getBoolean(USEMSLAB_KEY, USEMSLAB_DEFAULT)) {
126 this.chunkPool = MemStoreChunkPool.getPool(conf);
127 this.allocator = new MemStoreLAB(conf, chunkPool);
128 } else {
129 this.allocator = null;
130 this.chunkPool = null;
131 }
132 }
133
134 void dump() {
135 for (KeyValue kv: this.kvset) {
136 LOG.info(kv);
137 }
138 for (KeyValue kv: this.snapshot) {
139 LOG.info(kv);
140 }
141 }
142
143
144
145
146
147
148 void snapshot() {
149 this.lock.writeLock().lock();
150 try {
151
152
153 if (!this.snapshot.isEmpty()) {
154 LOG.warn("Snapshot called again without clearing previous. " +
155 "Doing nothing. Another ongoing flush or did we fail last attempt?");
156 } else {
157 if (!this.kvset.isEmpty()) {
158 this.snapshot = this.kvset;
159 this.kvset = new KeyValueSkipListSet(this.comparator);
160 this.snapshotTimeRangeTracker = this.timeRangeTracker;
161 this.timeRangeTracker = new TimeRangeTracker();
162
163 this.size.set(DEEP_OVERHEAD);
164 this.snapshotAllocator = this.allocator;
165
166 if (allocator != null) {
167 this.allocator = new MemStoreLAB(conf, chunkPool);
168 }
169 }
170 }
171 } finally {
172 this.lock.writeLock().unlock();
173 }
174 }
175
176
177
178
179
180
181
182
183
184 KeyValueSkipListSet getSnapshot() {
185 return this.snapshot;
186 }
187
188
189
190
191
192
193
194 void clearSnapshot(final SortedSet<KeyValue> ss)
195 throws UnexpectedException {
196 MemStoreLAB tmpAllocator = null;
197 this.lock.writeLock().lock();
198 try {
199 if (this.snapshot != ss) {
200 throw new UnexpectedException("Current snapshot is " +
201 this.snapshot + ", was passed " + ss);
202 }
203
204
205 if (!ss.isEmpty()) {
206 this.snapshot = new KeyValueSkipListSet(this.comparator);
207 this.snapshotTimeRangeTracker = new TimeRangeTracker();
208 }
209 if (this.snapshotAllocator != null) {
210 tmpAllocator = this.snapshotAllocator;
211 this.snapshotAllocator = null;
212 }
213 } finally {
214 this.lock.writeLock().unlock();
215 }
216 if (tmpAllocator != null) {
217 tmpAllocator.close();
218 }
219 }
220
221
222
223
224
225
226 long add(final KeyValue kv) {
227 this.lock.readLock().lock();
228 try {
229 KeyValue toAdd = maybeCloneWithAllocator(kv);
230 return internalAdd(toAdd);
231 } finally {
232 this.lock.readLock().unlock();
233 }
234 }
235
236
237
238
239
240
241
242 private long internalAdd(final KeyValue toAdd) {
243 long s = heapSizeChange(toAdd, this.kvset.add(toAdd));
244 timeRangeTracker.includeTimestamp(toAdd);
245 this.size.addAndGet(s);
246 return s;
247 }
248
249 private KeyValue maybeCloneWithAllocator(KeyValue kv) {
250 if (allocator == null) {
251 return kv;
252 }
253
254 int len = kv.getLength();
255 Allocation alloc = allocator.allocateBytes(len);
256 if (alloc == null) {
257
258
259 return kv;
260 }
261 assert alloc != null && alloc.getData() != null;
262 System.arraycopy(kv.getBuffer(), kv.getOffset(), alloc.getData(), alloc.getOffset(), len);
263 KeyValue newKv = new KeyValue(alloc.getData(), alloc.getOffset(), len);
264 newKv.setMemstoreTS(kv.getMemstoreTS());
265 return newKv;
266 }
267
268
269
270
271
272
273
274
275
276 void rollback(final KeyValue kv) {
277 this.lock.readLock().lock();
278 try {
279
280
281
282
283
284 KeyValue found = this.snapshot.get(kv);
285 if (found != null && found.getMemstoreTS() == kv.getMemstoreTS()) {
286 this.snapshot.remove(kv);
287 }
288
289 found = this.kvset.get(kv);
290 if (found != null && found.getMemstoreTS() == kv.getMemstoreTS()) {
291 this.kvset.remove(kv);
292 long s = heapSizeChange(kv, true);
293 this.size.addAndGet(-s);
294 }
295 } finally {
296 this.lock.readLock().unlock();
297 }
298 }
299
300
301
302
303
304
305 long delete(final KeyValue delete) {
306 long s = 0;
307 this.lock.readLock().lock();
308 try {
309 KeyValue toAdd = maybeCloneWithAllocator(delete);
310 s += heapSizeChange(toAdd, this.kvset.add(toAdd));
311 timeRangeTracker.includeTimestamp(toAdd);
312 } finally {
313 this.lock.readLock().unlock();
314 }
315 this.size.addAndGet(s);
316 return s;
317 }
318
319
320
321
322
323
324 KeyValue getNextRow(final KeyValue kv) {
325 this.lock.readLock().lock();
326 try {
327 return getLowest(getNextRow(kv, this.kvset), getNextRow(kv, this.snapshot));
328 } finally {
329 this.lock.readLock().unlock();
330 }
331 }
332
333
334
335
336
337
338 private KeyValue getLowest(final KeyValue a, final KeyValue b) {
339 if (a == null) {
340 return b;
341 }
342 if (b == null) {
343 return a;
344 }
345 return comparator.compareRows(a, b) <= 0? a: b;
346 }
347
348
349
350
351
352
353
354 private KeyValue getNextRow(final KeyValue key,
355 final NavigableSet<KeyValue> set) {
356 KeyValue result = null;
357 SortedSet<KeyValue> tail = key == null? set: set.tailSet(key);
358
359 for (KeyValue kv: tail) {
360 if (comparator.compareRows(kv, key) <= 0)
361 continue;
362
363
364 result = kv;
365 break;
366 }
367 return result;
368 }
369
370
371
372
373 void getRowKeyAtOrBefore(final GetClosestRowBeforeTracker state) {
374 this.lock.readLock().lock();
375 try {
376 getRowKeyAtOrBefore(kvset, state);
377 getRowKeyAtOrBefore(snapshot, state);
378 } finally {
379 this.lock.readLock().unlock();
380 }
381 }
382
383
384
385
386
387 private void getRowKeyAtOrBefore(final NavigableSet<KeyValue> set,
388 final GetClosestRowBeforeTracker state) {
389 if (set.isEmpty()) {
390 return;
391 }
392 if (!walkForwardInSingleRow(set, state.getTargetKey(), state)) {
393
394 getRowKeyBefore(set, state);
395 }
396 }
397
398
399
400
401
402
403
404
405
406
407
408 private boolean walkForwardInSingleRow(final SortedSet<KeyValue> set,
409 final KeyValue firstOnRow, final GetClosestRowBeforeTracker state) {
410 boolean foundCandidate = false;
411 SortedSet<KeyValue> tail = set.tailSet(firstOnRow);
412 if (tail.isEmpty()) return foundCandidate;
413 for (Iterator<KeyValue> i = tail.iterator(); i.hasNext();) {
414 KeyValue kv = i.next();
415
416 if (state.isTooFar(kv, firstOnRow)) break;
417 if (state.isExpired(kv)) {
418 i.remove();
419 continue;
420 }
421
422 if (state.handle(kv)) {
423 foundCandidate = true;
424 break;
425 }
426 }
427 return foundCandidate;
428 }
429
430
431
432
433
434
435
436 private void getRowKeyBefore(NavigableSet<KeyValue> set,
437 final GetClosestRowBeforeTracker state) {
438 KeyValue firstOnRow = state.getTargetKey();
439 for (Member p = memberOfPreviousRow(set, state, firstOnRow);
440 p != null; p = memberOfPreviousRow(p.set, state, firstOnRow)) {
441
442 if (!state.isTargetTable(p.kv)) break;
443
444 if (!state.isBetterCandidate(p.kv)) break;
445
446 firstOnRow = new KeyValue(p.kv.getRow(), HConstants.LATEST_TIMESTAMP);
447
448 if (walkForwardInSingleRow(p.set, firstOnRow, state)) break;
449 }
450 }
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469 long updateColumnValue(byte[] row,
470 byte[] family,
471 byte[] qualifier,
472 long newValue,
473 long now) {
474 this.lock.readLock().lock();
475 try {
476 KeyValue firstKv = KeyValue.createFirstOnRow(
477 row, family, qualifier);
478
479 SortedSet<KeyValue> snSs = snapshot.tailSet(firstKv);
480 if (!snSs.isEmpty()) {
481 KeyValue snKv = snSs.first();
482
483 if (snKv.matchingRow(firstKv) && snKv.matchingQualifier(firstKv)) {
484 if (snKv.getTimestamp() == now) {
485
486 now += 1;
487 }
488 }
489 }
490
491
492
493
494
495
496
497 SortedSet<KeyValue> ss = kvset.tailSet(firstKv);
498 Iterator<KeyValue> it = ss.iterator();
499 while ( it.hasNext() ) {
500 KeyValue kv = it.next();
501
502
503 if (!kv.matchingColumn(family,qualifier) || !kv.matchingRow(firstKv) ) {
504 break;
505 }
506
507
508 if (kv.getType() == KeyValue.Type.Put.getCode() &&
509 kv.getTimestamp() > now && firstKv.matchingQualifier(kv)) {
510 now = kv.getTimestamp();
511 }
512 }
513
514
515
516 List<Cell> cells = new ArrayList<Cell>(1);
517 cells.add(new KeyValue(row, family, qualifier, now, Bytes.toBytes(newValue)));
518 return upsert(cells, 1L);
519 } finally {
520 this.lock.readLock().unlock();
521 }
522 }
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542 public long upsert(Iterable<? extends Cell> cells, long readpoint) {
543 this.lock.readLock().lock();
544 try {
545 long size = 0;
546 for (Cell cell : cells) {
547 size += upsert(cell, readpoint);
548 }
549 return size;
550 } finally {
551 this.lock.readLock().unlock();
552 }
553 }
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569 private long upsert(Cell cell, long readpoint) {
570
571
572
573
574
575
576 KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
577 long addedSize = internalAdd(kv);
578
579
580
581 KeyValue firstKv = KeyValue.createFirstOnRow(
582 kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(),
583 kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(),
584 kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
585 SortedSet<KeyValue> ss = kvset.tailSet(firstKv);
586 Iterator<KeyValue> it = ss.iterator();
587
588 int versionsVisible = 0;
589 while ( it.hasNext() ) {
590 KeyValue cur = it.next();
591
592 if (kv == cur) {
593
594 continue;
595 }
596
597 if (kv.matchingRow(cur) && kv.matchingQualifier(cur)) {
598
599 if (cur.getType() == KeyValue.Type.Put.getCode() && cur.getMemstoreTS() <= readpoint) {
600 if (versionsVisible > 1) {
601
602
603
604
605 long delta = heapSizeChange(cur, true);
606 addedSize -= delta;
607 this.size.addAndGet(-delta);
608 it.remove();
609 } else {
610 versionsVisible++;
611 }
612 }
613 } else {
614
615 break;
616 }
617 }
618 return addedSize;
619 }
620
621
622
623
624
625 private static class Member {
626 final KeyValue kv;
627 final NavigableSet<KeyValue> set;
628 Member(final NavigableSet<KeyValue> s, final KeyValue kv) {
629 this.kv = kv;
630 this.set = s;
631 }
632 }
633
634
635
636
637
638
639
640
641
642 private Member memberOfPreviousRow(NavigableSet<KeyValue> set,
643 final GetClosestRowBeforeTracker state, final KeyValue firstOnRow) {
644 NavigableSet<KeyValue> head = set.headSet(firstOnRow, false);
645 if (head.isEmpty()) return null;
646 for (Iterator<KeyValue> i = head.descendingIterator(); i.hasNext();) {
647 KeyValue found = i.next();
648 if (state.isExpired(found)) {
649 i.remove();
650 continue;
651 }
652 return new Member(head, found);
653 }
654 return null;
655 }
656
657
658
659
660 List<KeyValueScanner> getScanners() {
661 this.lock.readLock().lock();
662 try {
663 return Collections.<KeyValueScanner>singletonList(
664 new MemStoreScanner());
665 } finally {
666 this.lock.readLock().unlock();
667 }
668 }
669
670
671
672
673
674
675 public boolean shouldSeek(Scan scan, long oldestUnexpiredTS) {
676 return (timeRangeTracker.includesTimeRange(scan.getTimeRange()) ||
677 snapshotTimeRangeTracker.includesTimeRange(scan.getTimeRange()))
678 && (Math.max(timeRangeTracker.getMaximumTimestamp(),
679 snapshotTimeRangeTracker.getMaximumTimestamp()) >=
680 oldestUnexpiredTS);
681 }
682
683 public TimeRangeTracker getSnapshotTimeRangeTracker() {
684 return this.snapshotTimeRangeTracker;
685 }
686
687
688
689
690
691
692
693 protected class MemStoreScanner extends NonLazyKeyValueScanner {
694
695 private KeyValue kvsetNextRow = null;
696 private KeyValue snapshotNextRow = null;
697
698
699 private KeyValue kvsetItRow = null;
700 private KeyValue snapshotItRow = null;
701
702
703 private Iterator<KeyValue> kvsetIt;
704 private Iterator<KeyValue> snapshotIt;
705
706
707 private KeyValueSkipListSet kvsetAtCreation;
708 private KeyValueSkipListSet snapshotAtCreation;
709
710
711 private KeyValue theNext;
712
713
714 volatile MemStoreLAB allocatorAtCreation;
715 volatile MemStoreLAB snapshotAllocatorAtCreation;
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738 MemStoreScanner() {
739 super();
740
741 kvsetAtCreation = kvset;
742 snapshotAtCreation = snapshot;
743 if (allocator != null) {
744 this.allocatorAtCreation = allocator;
745 this.allocatorAtCreation.incScannerCount();
746 }
747 if (snapshotAllocator != null) {
748 this.snapshotAllocatorAtCreation = snapshotAllocator;
749 this.snapshotAllocatorAtCreation.incScannerCount();
750 }
751 }
752
753 private KeyValue getNext(Iterator<KeyValue> it) {
754 long readPoint = MultiVersionConsistencyControl.getThreadReadPoint();
755
756 KeyValue v = null;
757 try {
758 while (it.hasNext()) {
759 v = it.next();
760 if (v.getMemstoreTS() <= readPoint) {
761 return v;
762 }
763 }
764
765 return null;
766 } finally {
767 if (v != null) {
768
769 if (it == snapshotIt) {
770 snapshotItRow = v;
771 } else {
772 kvsetItRow = v;
773 }
774 }
775 }
776 }
777
778
779
780
781
782
783
784
785 @Override
786 public synchronized boolean seek(KeyValue key) {
787 if (key == null) {
788 close();
789 return false;
790 }
791
792
793
794 kvsetIt = kvsetAtCreation.tailSet(key).iterator();
795 snapshotIt = snapshotAtCreation.tailSet(key).iterator();
796 kvsetItRow = null;
797 snapshotItRow = null;
798
799 return seekInSubLists(key);
800 }
801
802
803
804
805
806 private synchronized boolean seekInSubLists(KeyValue key){
807 kvsetNextRow = getNext(kvsetIt);
808 snapshotNextRow = getNext(snapshotIt);
809
810
811 theNext = getLowest(kvsetNextRow, snapshotNextRow);
812
813
814 return (theNext != null);
815 }
816
817
818
819
820
821
822
823 @Override
824 public synchronized boolean reseek(KeyValue key) {
825
826
827
828
829
830
831
832
833
834
835
836
837
838 kvsetIt = kvsetAtCreation.tailSet(getHighest(key, kvsetItRow)).iterator();
839 snapshotIt = snapshotAtCreation.tailSet(getHighest(key, snapshotItRow)).iterator();
840
841 return seekInSubLists(key);
842 }
843
844
845 @Override
846 public synchronized KeyValue peek() {
847
848 return theNext;
849 }
850
851 @Override
852 public synchronized KeyValue next() {
853 if (theNext == null) {
854 return null;
855 }
856
857 final KeyValue ret = theNext;
858
859
860 if (theNext == kvsetNextRow) {
861 kvsetNextRow = getNext(kvsetIt);
862 } else {
863 snapshotNextRow = getNext(snapshotIt);
864 }
865
866
867 theNext = getLowest(kvsetNextRow, snapshotNextRow);
868
869
870
871
872 return ret;
873 }
874
875
876
877
878
879
880 private KeyValue getLowest(KeyValue first, KeyValue second) {
881 if (first == null && second == null) {
882 return null;
883 }
884 if (first != null && second != null) {
885 int compare = comparator.compare(first, second);
886 return (compare <= 0 ? first : second);
887 }
888 return (first != null ? first : second);
889 }
890
891
892
893
894
895
896 private KeyValue getHighest(KeyValue first, KeyValue second) {
897 if (first == null && second == null) {
898 return null;
899 }
900 if (first != null && second != null) {
901 int compare = comparator.compare(first, second);
902 return (compare > 0 ? first : second);
903 }
904 return (first != null ? first : second);
905 }
906
907 public synchronized void close() {
908 this.kvsetNextRow = null;
909 this.snapshotNextRow = null;
910
911 this.kvsetIt = null;
912 this.snapshotIt = null;
913
914 if (allocatorAtCreation != null) {
915 this.allocatorAtCreation.decScannerCount();
916 this.allocatorAtCreation = null;
917 }
918 if (snapshotAllocatorAtCreation != null) {
919 this.snapshotAllocatorAtCreation.decScannerCount();
920 this.snapshotAllocatorAtCreation = null;
921 }
922
923 this.kvsetItRow = null;
924 this.snapshotItRow = null;
925 }
926
927
928
929
930
931 @Override
932 public long getSequenceID() {
933 return Long.MAX_VALUE;
934 }
935
936 @Override
937 public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns,
938 long oldestUnexpiredTS) {
939 return shouldSeek(scan, oldestUnexpiredTS);
940 }
941 }
942
943 public final static long FIXED_OVERHEAD = ClassSize.align(
944 ClassSize.OBJECT + (13 * ClassSize.REFERENCE));
945
946 public final static long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
947 ClassSize.REENTRANT_LOCK + ClassSize.ATOMIC_LONG +
948 ClassSize.COPYONWRITE_ARRAYSET + ClassSize.COPYONWRITE_ARRAYLIST +
949 (2 * ClassSize.CONCURRENT_SKIPLISTMAP));
950
951
952 public static final boolean NO_PERSISTENT_TS = false;
953
954
955
956
957
958
959
960
961 long heapSizeChange(final KeyValue kv, final boolean notpresent) {
962 return notpresent ?
963 ClassSize.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + kv.heapSize()):
964 0;
965 }
966
967
968
969
970
971 @Override
972 public long heapSize() {
973 return size.get();
974 }
975
976
977
978
979 public long keySize() {
980 return heapSize() - DEEP_OVERHEAD;
981 }
982
983
984
985
986
987
988
989
990 public static void main(String [] args) {
991 RuntimeMXBean runtime = ManagementFactory.getRuntimeMXBean();
992 LOG.info("vmName=" + runtime.getVmName() + ", vmVendor=" +
993 runtime.getVmVendor() + ", vmVersion=" + runtime.getVmVersion());
994 LOG.info("vmInputArguments=" + runtime.getInputArguments());
995 MemStore memstore1 = new MemStore();
996
997 long size = 0;
998 final int count = 10000;
999 byte [] fam = Bytes.toBytes("col");
1000 byte [] qf = Bytes.toBytes("umn");
1001 byte [] empty = new byte[0];
1002 for (int i = 0; i < count; i++) {
1003
1004 size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
1005 }
1006 LOG.info("memstore1 estimated size=" + size);
1007 for (int i = 0; i < count; i++) {
1008 size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
1009 }
1010 LOG.info("memstore1 estimated size (2nd loading of same data)=" + size);
1011
1012 MemStore memstore2 = new MemStore();
1013 for (int i = 0; i < count; i++) {
1014 size += memstore2.add(new KeyValue(Bytes.toBytes(i), fam, qf, i,
1015 new byte[i]));
1016 }
1017 LOG.info("memstore2 estimated size=" + size);
1018 final int seconds = 30;
1019 LOG.info("Waiting " + seconds + " seconds while heap dump is taken");
1020 for (int i = 0; i < seconds; i++) {
1021
1022 }
1023 LOG.info("Exiting.");
1024 }
1025 }