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.io.IOException;
23 import java.io.InterruptedIOException;
24 import java.util.ArrayList;
25 import java.util.Collection;
26 import java.util.Collections;
27 import java.util.List;
28 import java.util.NavigableSet;
29 import java.util.SortedSet;
30 import java.util.concurrent.CopyOnWriteArraySet;
31 import java.util.concurrent.locks.ReentrantReadWriteLock;
32
33 import org.apache.commons.logging.Log;
34 import org.apache.commons.logging.LogFactory;
35 import org.apache.hadoop.conf.Configuration;
36 import org.apache.hadoop.fs.FileStatus;
37 import org.apache.hadoop.fs.FileSystem;
38 import org.apache.hadoop.fs.FileUtil;
39 import org.apache.hadoop.fs.Path;
40 import org.apache.hadoop.hbase.HColumnDescriptor;
41 import org.apache.hadoop.hbase.HConstants;
42 import org.apache.hadoop.hbase.HRegionInfo;
43 import org.apache.hadoop.hbase.KeyValue;
44 import org.apache.hadoop.hbase.RemoteExceptionHandler;
45 import org.apache.hadoop.hbase.client.Scan;
46 import org.apache.hadoop.hbase.io.HeapSize;
47 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
48 import org.apache.hadoop.hbase.io.hfile.Compression;
49 import org.apache.hadoop.hbase.io.hfile.HFile;
50 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
51 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
52 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
53 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
54 import org.apache.hadoop.hbase.util.Bytes;
55 import org.apache.hadoop.hbase.util.ClassSize;
56 import org.apache.hadoop.hbase.util.CollectionBackedScanner;
57 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
58 import org.apache.hadoop.hbase.util.FSUtils;
59 import org.apache.hadoop.util.StringUtils;
60
61 import com.google.common.base.Preconditions;
62 import com.google.common.collect.ImmutableList;
63 import com.google.common.collect.Lists;
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88 public class Store implements HeapSize {
89 static final Log LOG = LogFactory.getLog(Store.class);
90 protected final MemStore memstore;
91
92 private final Path homedir;
93 private final HRegion region;
94 private final HColumnDescriptor family;
95 final FileSystem fs;
96 final Configuration conf;
97 final CacheConfig cacheConf;
98
99 protected long ttl;
100 protected int minVersions;
101 protected int maxVersions;
102 long majorCompactionTime;
103 private final int minFilesToCompact;
104 private final int maxFilesToCompact;
105 private final long minCompactSize;
106 private final long maxCompactSize;
107
108
109 private double compactRatio;
110 private long lastCompactSize = 0;
111 volatile boolean forceMajor = false;
112
113 static int closeCheckInterval = 0;
114 private final int blockingStoreFileCount;
115 private volatile long storeSize = 0L;
116 private volatile long totalUncompressedBytes = 0L;
117 private final Object flushLock = new Object();
118 final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
119 private final String storeNameStr;
120 private CompactionProgress progress;
121 private final int compactionKVMax;
122
123
124
125
126
127 private ImmutableList<StoreFile> storefiles = null;
128
129 List<StoreFile> filesCompacting = Lists.newArrayList();
130
131
132 private final CopyOnWriteArraySet<ChangedReadersObserver> changedReaderObservers =
133 new CopyOnWriteArraySet<ChangedReadersObserver>();
134
135 private final int blocksize;
136
137 private final Compression.Algorithm compression;
138
139 private final Compression.Algorithm compactionCompression;
140
141
142 final KeyValue.KVComparator comparator;
143
144
145
146
147
148
149
150
151
152
153
154
155 protected Store(Path basedir, HRegion region, HColumnDescriptor family,
156 FileSystem fs, Configuration conf)
157 throws IOException {
158 HRegionInfo info = region.regionInfo;
159 this.fs = fs;
160 this.homedir = getStoreHomedir(basedir, info.getEncodedName(), family.getName());
161 if (!this.fs.exists(this.homedir)) {
162 if (!this.fs.mkdirs(this.homedir))
163 throw new IOException("Failed create of: " + this.homedir.toString());
164 }
165 this.region = region;
166 this.family = family;
167 this.conf = conf;
168 this.blocksize = family.getBlocksize();
169 this.compression = family.getCompression();
170
171
172 this.compactionCompression =
173 (family.getCompactionCompression() != Compression.Algorithm.NONE) ?
174 family.getCompactionCompression() : this.compression;
175 this.comparator = info.getComparator();
176
177 this.ttl = family.getTimeToLive();
178 if (ttl == HConstants.FOREVER) {
179
180 ttl = Long.MAX_VALUE;
181 } else if (ttl == -1) {
182 ttl = Long.MAX_VALUE;
183 } else {
184
185 this.ttl *= 1000;
186 }
187 this.minVersions = family.getMinVersions();
188 this.maxVersions = family.getMaxVersions();
189 this.memstore = new MemStore(conf, this.comparator);
190 this.storeNameStr = Bytes.toString(this.family.getName());
191
192
193 this.minFilesToCompact = Math.max(2,
194 conf.getInt("hbase.hstore.compaction.min",
195
196
197
198 this.cacheConf = new CacheConfig(conf, family);
199 this.blockingStoreFileCount =
200 conf.getInt("hbase.hstore.blockingStoreFiles", 7);
201
202 this.majorCompactionTime = getNextMajorCompactTime();
203
204 this.maxFilesToCompact = conf.getInt("hbase.hstore.compaction.max", 10);
205 this.minCompactSize = conf.getLong("hbase.hstore.compaction.min.size",
206 this.region.memstoreFlushSize);
207 this.maxCompactSize
208 = conf.getLong("hbase.hstore.compaction.max.size", Long.MAX_VALUE);
209 this.compactRatio = conf.getFloat("hbase.hstore.compaction.ratio", 1.2F);
210 this.compactionKVMax = conf.getInt("hbase.hstore.compaction.kv.max", 10);
211
212 if (Store.closeCheckInterval == 0) {
213 Store.closeCheckInterval = conf.getInt(
214 "hbase.hstore.close.check.interval", 10*1000*1000
215 }
216 this.storefiles = sortAndClone(loadStoreFiles());
217 }
218
219 public HColumnDescriptor getFamily() {
220 return this.family;
221 }
222
223
224
225
226 long getMaxSequenceId() {
227 return StoreFile.getMaxSequenceIdInList(this.getStorefiles());
228 }
229
230
231
232
233 public long getMaxMemstoreTS() {
234 return StoreFile.getMaxMemstoreTSInList(this.getStorefiles());
235 }
236
237
238
239
240
241
242
243 public static Path getStoreHomedir(final Path tabledir,
244 final String encodedName, final byte [] family) {
245 return new Path(tabledir, new Path(encodedName,
246 new Path(Bytes.toString(family))));
247 }
248
249
250
251
252
253 public Path getHomedir() {
254 return homedir;
255 }
256
257
258
259
260
261 private List<StoreFile> loadStoreFiles()
262 throws IOException {
263 ArrayList<StoreFile> results = new ArrayList<StoreFile>();
264 FileStatus files[] = FSUtils.listStatus(this.fs, this.homedir, null);
265 for (int i = 0; files != null && i < files.length; i++) {
266
267 if (files[i].isDir()) {
268 continue;
269 }
270 Path p = files[i].getPath();
271
272
273 if (this.fs.getFileStatus(p).getLen() <= 0) {
274 LOG.warn("Skipping " + p + " because its empty. HBASE-646 DATA LOSS?");
275 continue;
276 }
277 StoreFile curfile = new StoreFile(fs, p, this.conf, this.cacheConf,
278 this.family.getBloomFilterType());
279 curfile.createReader();
280 long length = curfile.getReader().length();
281 this.storeSize += length;
282 this.totalUncompressedBytes += curfile.getReader().getTotalUncompressedBytes();
283 if (LOG.isDebugEnabled()) {
284 LOG.debug("loaded " + curfile.toStringDetailed());
285 }
286 results.add(curfile);
287 }
288 return results;
289 }
290
291
292
293
294
295
296
297 protected long add(final KeyValue kv) {
298 lock.readLock().lock();
299 try {
300 return this.memstore.add(kv);
301 } finally {
302 lock.readLock().unlock();
303 }
304 }
305
306
307
308
309
310
311
312 protected long delete(final KeyValue kv) {
313 lock.readLock().lock();
314 try {
315 return this.memstore.delete(kv);
316 } finally {
317 lock.readLock().unlock();
318 }
319 }
320
321
322
323
324 List<StoreFile> getStorefiles() {
325 return this.storefiles;
326 }
327
328
329
330
331
332
333 void assertBulkLoadHFileOk(Path srcPath) throws IOException {
334 HFile.Reader reader = null;
335 try {
336 LOG.info("Validating hfile at " + srcPath + " for inclusion in "
337 + "store " + this + " region " + this.region);
338 reader = HFile.createReader(srcPath.getFileSystem(conf),
339 srcPath, cacheConf);
340 reader.loadFileInfo();
341
342 byte[] firstKey = reader.getFirstRowKey();
343 byte[] lk = reader.getLastKey();
344 byte[] lastKey =
345 (lk == null) ? null :
346 KeyValue.createKeyValueFromKey(lk).getRow();
347
348 LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey) +
349 " last=" + Bytes.toStringBinary(lastKey));
350 LOG.debug("Region bounds: first=" +
351 Bytes.toStringBinary(region.getStartKey()) +
352 " last=" + Bytes.toStringBinary(region.getEndKey()));
353
354 HRegionInfo hri = region.getRegionInfo();
355 if (!hri.containsRange(firstKey, lastKey)) {
356 throw new WrongRegionException(
357 "Bulk load file " + srcPath.toString() + " does not fit inside region "
358 + this.region);
359 }
360 } finally {
361 if (reader != null) reader.close();
362 }
363 }
364
365
366
367
368
369
370 void bulkLoadHFile(String srcPathStr) throws IOException {
371 Path srcPath = new Path(srcPathStr);
372
373
374 FileSystem srcFs = srcPath.getFileSystem(conf);
375 if (!srcFs.equals(fs)) {
376 LOG.info("File " + srcPath + " on different filesystem than " +
377 "destination store - moving to this filesystem.");
378 Path tmpPath = getTmpPath();
379 FileUtil.copy(srcFs, srcPath, fs, tmpPath, false, conf);
380 LOG.info("Copied to temporary path on dst filesystem: " + tmpPath);
381 srcPath = tmpPath;
382 }
383
384 Path dstPath = StoreFile.getRandomFilename(fs, homedir);
385 LOG.debug("Renaming bulk load file " + srcPath + " to " + dstPath);
386 StoreFile.rename(fs, srcPath, dstPath);
387
388 StoreFile sf = new StoreFile(fs, dstPath, this.conf, this.cacheConf,
389 this.family.getBloomFilterType());
390 sf.createReader();
391
392 LOG.info("Moved hfile " + srcPath + " into store directory " +
393 homedir + " - updating store file list.");
394
395
396 this.lock.writeLock().lock();
397 try {
398 ArrayList<StoreFile> newFiles = new ArrayList<StoreFile>(storefiles);
399 newFiles.add(sf);
400 this.storefiles = sortAndClone(newFiles);
401 } finally {
402
403
404
405
406
407 this.lock.writeLock().unlock();
408 }
409 notifyChangedReadersObservers();
410 LOG.info("Successfully loaded store file " + srcPath
411 + " into store " + this + " (new location: " + dstPath + ")");
412 }
413
414
415
416
417
418
419 private Path getTmpPath() throws IOException {
420 return StoreFile.getRandomFilename(
421 fs, region.getTmpDir());
422 }
423
424
425
426
427
428
429
430
431
432 ImmutableList<StoreFile> close() throws IOException {
433 this.lock.writeLock().lock();
434 try {
435 ImmutableList<StoreFile> result = storefiles;
436
437
438 storefiles = ImmutableList.of();
439
440 for (StoreFile f: result) {
441 f.closeReader(true);
442 }
443 LOG.debug("closed " + this.storeNameStr);
444 return result;
445 } finally {
446 this.lock.writeLock().unlock();
447 }
448 }
449
450
451
452
453
454 void snapshot() {
455 this.memstore.snapshot();
456 }
457
458
459
460
461
462
463
464
465
466
467 private StoreFile flushCache(final long logCacheFlushId,
468 SortedSet<KeyValue> snapshot,
469 TimeRangeTracker snapshotTimeRangeTracker,
470 MonitoredTask status) throws IOException {
471
472
473
474 return internalFlushCache(
475 snapshot, logCacheFlushId, snapshotTimeRangeTracker, status);
476 }
477
478
479
480
481
482
483
484 private StoreFile internalFlushCache(final SortedSet<KeyValue> set,
485 final long logCacheFlushId,
486 TimeRangeTracker snapshotTimeRangeTracker,
487 MonitoredTask status)
488 throws IOException {
489 StoreFile.Writer writer;
490 String fileName;
491
492 long smallestReadPoint = region.getSmallestReadPoint();
493 long flushed = 0;
494
495 if (set.size() == 0) {
496 return null;
497 }
498 Scan scan = new Scan();
499 scan.setMaxVersions(maxVersions);
500
501
502
503 InternalScanner scanner = new StoreScanner(this, scan,
504 Collections.singletonList(new CollectionBackedScanner(set,
505 this.comparator)), true, this.region.getSmallestReadPoint());
506 try {
507
508
509
510 synchronized (flushLock) {
511 status.setStatus("Flushing " + this + ": creating writer");
512
513 writer = createWriterInTmp(set.size());
514 writer.setTimeRangeTracker(snapshotTimeRangeTracker);
515 fileName = writer.getPath().getName();
516 try {
517 List<KeyValue> kvs = new ArrayList<KeyValue>();
518 boolean hasMore;
519 do {
520 hasMore = scanner.next(kvs);
521 if (!kvs.isEmpty()) {
522 for (KeyValue kv : kvs) {
523
524
525 if (kv.getMemstoreTS() <= smallestReadPoint) {
526
527
528 kv = kv.shallowCopy();
529 kv.setMemstoreTS(0);
530 }
531 writer.append(kv);
532 flushed += this.memstore.heapSizeChange(kv, true);
533 }
534 kvs.clear();
535 }
536 } while (hasMore);
537 } finally {
538
539
540 status.setStatus("Flushing " + this + ": appending metadata");
541 writer.appendMetadata(logCacheFlushId, false);
542 status.setStatus("Flushing " + this + ": closing flushed file");
543 writer.close();
544 }
545 }
546 } finally {
547 scanner.close();
548 }
549
550
551 Path dstPath = new Path(homedir, fileName);
552 validateStoreFile(writer.getPath());
553 String msg = "Renaming flushed file at " + writer.getPath() + " to " + dstPath;
554 LOG.debug(msg);
555 status.setStatus("Flushing " + this + ": " + msg);
556 if (!fs.rename(writer.getPath(), dstPath)) {
557 LOG.warn("Unable to rename " + writer.getPath() + " to " + dstPath);
558 }
559
560 status.setStatus("Flushing " + this + ": reopening flushed file");
561 StoreFile sf = new StoreFile(this.fs, dstPath, this.conf, this.cacheConf,
562 this.family.getBloomFilterType());
563 StoreFile.Reader r = sf.createReader();
564 this.storeSize += r.length();
565 this.totalUncompressedBytes += r.getTotalUncompressedBytes();
566 if (LOG.isInfoEnabled()) {
567 LOG.info("Added " + sf + ", entries=" + r.getEntries() +
568 ", sequenceid=" + logCacheFlushId +
569 ", memsize=" + StringUtils.humanReadableInt(flushed) +
570 ", filesize=" + StringUtils.humanReadableInt(r.length()));
571 }
572 return sf;
573 }
574
575
576
577
578
579 private StoreFile.Writer createWriterInTmp(int maxKeyCount)
580 throws IOException {
581 return createWriterInTmp(maxKeyCount, this.compression);
582 }
583
584
585
586
587
588
589 private StoreFile.Writer createWriterInTmp(int maxKeyCount,
590 Compression.Algorithm compression)
591 throws IOException {
592 return StoreFile.createWriter(this.fs, region.getTmpDir(), this.blocksize,
593 compression, this.comparator, this.conf, this.cacheConf,
594 this.family.getBloomFilterType(), maxKeyCount);
595 }
596
597
598
599
600
601
602
603
604 private boolean updateStorefiles(final StoreFile sf,
605 final SortedSet<KeyValue> set)
606 throws IOException {
607 this.lock.writeLock().lock();
608 try {
609 ArrayList<StoreFile> newList = new ArrayList<StoreFile>(storefiles);
610 newList.add(sf);
611 storefiles = sortAndClone(newList);
612
613 this.memstore.clearSnapshot(set);
614 } finally {
615
616
617
618
619
620 this.lock.writeLock().unlock();
621 }
622
623
624 notifyChangedReadersObservers();
625
626 return needsCompaction();
627 }
628
629
630
631
632
633 private void notifyChangedReadersObservers() throws IOException {
634 for (ChangedReadersObserver o: this.changedReaderObservers) {
635 o.updateReaders();
636 }
637 }
638
639 protected List<KeyValueScanner> getScanners(boolean cacheBlocks,
640 boolean isGet,
641 boolean isCompaction) throws IOException {
642 List<StoreFile> storeFiles;
643 List<KeyValueScanner> memStoreScanners;
644 this.lock.readLock().lock();
645 try {
646 storeFiles = this.getStorefiles();
647 memStoreScanners = this.memstore.getScanners();
648 } finally {
649 this.lock.readLock().unlock();
650 }
651
652
653
654
655
656
657 List<StoreFileScanner> sfScanners = StoreFileScanner
658 .getScannersForStoreFiles(storeFiles, cacheBlocks, isGet, isCompaction);
659 List<KeyValueScanner> scanners =
660 new ArrayList<KeyValueScanner>(sfScanners.size()+1);
661 scanners.addAll(sfScanners);
662
663 scanners.addAll(memStoreScanners);
664 return scanners;
665 }
666
667
668
669
670 void addChangedReaderObserver(ChangedReadersObserver o) {
671 this.changedReaderObservers.add(o);
672 }
673
674
675
676
677 void deleteChangedReaderObserver(ChangedReadersObserver o) {
678
679 this.changedReaderObservers.remove(o);
680 }
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706 void compact(CompactionRequest cr) throws IOException {
707 if (cr == null || cr.getFiles().isEmpty()) {
708 return;
709 }
710 Preconditions.checkArgument(cr.getStore().toString()
711 .equals(this.toString()));
712
713 List<StoreFile> filesToCompact = cr.getFiles();
714
715 synchronized (filesCompacting) {
716
717
718 Preconditions.checkArgument(filesCompacting.containsAll(filesToCompact));
719 }
720
721
722 long maxId = StoreFile.getMaxSequenceIdInList(filesToCompact);
723
724
725 LOG.info("Starting compaction of " + filesToCompact.size() + " file(s) in "
726 + this.storeNameStr + " of "
727 + this.region.getRegionInfo().getRegionNameAsString()
728 + " into tmpdir=" + region.getTmpDir() + ", seqid=" + maxId + ", totalSize="
729 + StringUtils.humanReadableInt(cr.getSize()));
730
731 StoreFile sf = null;
732 try {
733 StoreFile.Writer writer = compactStore(filesToCompact, cr.isMajor(),
734 maxId);
735
736 sf = completeCompaction(filesToCompact, writer);
737 if (region.getCoprocessorHost() != null) {
738 region.getCoprocessorHost().postCompact(this, sf);
739 }
740 } finally {
741 synchronized (filesCompacting) {
742 filesCompacting.removeAll(filesToCompact);
743 }
744 }
745
746 LOG.info("Completed" + (cr.isMajor() ? " major " : " ") + "compaction of "
747 + filesToCompact.size() + " file(s) in " + this.storeNameStr + " of "
748 + this.region.getRegionInfo().getRegionNameAsString()
749 + " into " +
750 (sf == null ? "none" : sf.getPath().getName()) +
751 ", size=" + (sf == null ? "none" :
752 StringUtils.humanReadableInt(sf.getReader().length()))
753 + "; total size for store is "
754 + StringUtils.humanReadableInt(storeSize));
755 }
756
757
758
759
760 protected void compactRecent(int N) throws IOException {
761 List<StoreFile> filesToCompact;
762 long maxId;
763 boolean isMajor;
764
765 this.lock.readLock().lock();
766 try {
767 synchronized (filesCompacting) {
768 filesToCompact = Lists.newArrayList(storefiles);
769 if (!filesCompacting.isEmpty()) {
770
771
772 StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
773 int idx = filesToCompact.indexOf(last);
774 Preconditions.checkArgument(idx != -1);
775 filesToCompact.subList(0, idx + 1).clear();
776 }
777 int count = filesToCompact.size();
778 if (N > count) {
779 throw new RuntimeException("Not enough files");
780 }
781
782 filesToCompact = filesToCompact.subList(count - N, count);
783 maxId = StoreFile.getMaxSequenceIdInList(filesToCompact);
784 isMajor = (filesToCompact.size() == storefiles.size());
785 filesCompacting.addAll(filesToCompact);
786 Collections.sort(filesCompacting, StoreFile.Comparators.FLUSH_TIME);
787 }
788 } finally {
789 this.lock.readLock().unlock();
790 }
791
792 try {
793
794 StoreFile.Writer writer = compactStore(filesToCompact, isMajor, maxId);
795
796 StoreFile sf = completeCompaction(filesToCompact, writer);
797 if (region.getCoprocessorHost() != null) {
798 region.getCoprocessorHost().postCompact(this, sf);
799 }
800 } finally {
801 synchronized (filesCompacting) {
802 filesCompacting.removeAll(filesToCompact);
803 }
804 }
805 }
806
807 boolean hasReferences() {
808 return hasReferences(this.storefiles);
809 }
810
811
812
813
814
815 private boolean hasReferences(Collection<StoreFile> files) {
816 if (files != null && files.size() > 0) {
817 for (StoreFile hsf: files) {
818 if (hsf.isReference()) {
819 return true;
820 }
821 }
822 }
823 return false;
824 }
825
826
827
828
829
830
831
832
833 public static long getLowestTimestamp(final List<StoreFile> candidates)
834 throws IOException {
835 long minTs = Long.MAX_VALUE;
836 for (StoreFile storeFile : candidates) {
837 minTs = Math.min(minTs, storeFile.getModificationTimeStamp());
838 }
839 return minTs;
840 }
841
842
843
844
845 public CompactionProgress getCompactionProgress() {
846 return this.progress;
847 }
848
849
850
851
852 boolean isMajorCompaction() throws IOException {
853 for (StoreFile sf : this.storefiles) {
854 if (sf.getReader() == null) {
855 LOG.debug("StoreFile " + sf + " has null Reader");
856 return false;
857 }
858 }
859
860 List<StoreFile> candidates = new ArrayList<StoreFile>(this.storefiles);
861
862
863
864 int pos = 0;
865 while (pos < candidates.size() &&
866 candidates.get(pos).getReader().length() > this.maxCompactSize &&
867 !candidates.get(pos).isReference()) ++pos;
868 candidates.subList(0, pos).clear();
869
870 return isMajorCompaction(candidates);
871 }
872
873
874
875
876
877 private boolean isMajorCompaction(final List<StoreFile> filesToCompact) throws IOException {
878 boolean result = false;
879 if (filesToCompact == null || filesToCompact.isEmpty() ||
880 majorCompactionTime == 0) {
881 return result;
882 }
883
884 long lowTimestamp = getLowestTimestamp(filesToCompact);
885 long now = System.currentTimeMillis();
886 if (lowTimestamp > 0l && lowTimestamp < (now - this.majorCompactionTime)) {
887
888 if (filesToCompact.size() == 1) {
889
890 StoreFile sf = filesToCompact.get(0);
891 long oldest =
892 (sf.getReader().timeRangeTracker == null) ?
893 Long.MIN_VALUE :
894 now - sf.getReader().timeRangeTracker.minimumTimestamp;
895 if (sf.isMajorCompaction() &&
896 (this.ttl == HConstants.FOREVER || oldest < this.ttl)) {
897 if (LOG.isDebugEnabled()) {
898 LOG.debug("Skipping major compaction of " + this.storeNameStr +
899 " because one (major) compacted file only and oldestTime " +
900 oldest + "ms is < ttl=" + this.ttl);
901 }
902 } else if (this.ttl != HConstants.FOREVER && oldest > this.ttl) {
903 LOG.debug("Major compaction triggered on store " + this.storeNameStr +
904 ", because keyvalues outdated; time since last major compaction " +
905 (now - lowTimestamp) + "ms");
906 result = true;
907 }
908 } else {
909 if (LOG.isDebugEnabled()) {
910 LOG.debug("Major compaction triggered on store " + this.storeNameStr +
911 "; time since last major compaction " + (now - lowTimestamp) + "ms");
912 }
913 result = true;
914 }
915 }
916 return result;
917 }
918
919 long getNextMajorCompactTime() {
920
921 long ret = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD, 1000*60*60*24);
922 if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) != null) {
923 String strCompactionTime =
924 family.getValue(HConstants.MAJOR_COMPACTION_PERIOD);
925 ret = (new Long(strCompactionTime)).longValue();
926 }
927
928 if (ret > 0) {
929
930 double jitterPct = conf.getFloat("hbase.hregion.majorcompaction.jitter",
931 0.20F);
932 if (jitterPct > 0) {
933 long jitter = Math.round(ret * jitterPct);
934 ret += jitter - Math.round(2L * jitter * Math.random());
935 }
936 }
937 return ret;
938 }
939
940 public CompactionRequest requestCompaction() {
941
942 if (!this.region.areWritesEnabled()) {
943 return null;
944 }
945
946 CompactionRequest ret = null;
947 this.lock.readLock().lock();
948 try {
949 synchronized (filesCompacting) {
950
951 List<StoreFile> candidates = Lists.newArrayList(storefiles);
952 if (!filesCompacting.isEmpty()) {
953
954
955 StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
956 int idx = candidates.indexOf(last);
957 Preconditions.checkArgument(idx != -1);
958 candidates.subList(0, idx + 1).clear();
959 }
960
961 boolean override = false;
962 if (region.getCoprocessorHost() != null) {
963 override = region.getCoprocessorHost().preCompactSelection(
964 this, candidates);
965 }
966 List<StoreFile> filesToCompact;
967 if (override) {
968
969 filesToCompact = candidates;
970 } else {
971 filesToCompact = compactSelection(candidates);
972 }
973
974 if (region.getCoprocessorHost() != null) {
975 region.getCoprocessorHost().postCompactSelection(this,
976 ImmutableList.copyOf(filesToCompact));
977 }
978
979
980 if (filesToCompact.isEmpty()) {
981 return null;
982 }
983
984
985 if (!Collections.disjoint(filesCompacting, filesToCompact)) {
986
987 Preconditions.checkArgument(false, "%s overlaps with %s",
988 filesToCompact, filesCompacting);
989 }
990 filesCompacting.addAll(filesToCompact);
991 Collections.sort(filesCompacting, StoreFile.Comparators.FLUSH_TIME);
992
993
994 boolean isMajor = (filesToCompact.size() == this.storefiles.size());
995 if (isMajor) {
996
997 this.forceMajor = false;
998 this.majorCompactionTime = getNextMajorCompactTime();
999 }
1000
1001
1002 int pri = getCompactPriority();
1003 ret = new CompactionRequest(region, this, filesToCompact, isMajor, pri);
1004 }
1005 } catch (IOException ex) {
1006 LOG.error("Compaction Request failed for region " + region + ", store "
1007 + this, RemoteExceptionHandler.checkIOException(ex));
1008 } finally {
1009 this.lock.readLock().unlock();
1010 }
1011 return ret;
1012 }
1013
1014 public void finishRequest(CompactionRequest cr) {
1015 synchronized (filesCompacting) {
1016 filesCompacting.removeAll(cr.getFiles());
1017 }
1018 }
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039 List<StoreFile> compactSelection(List<StoreFile> candidates)
1040 throws IOException {
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054 List<StoreFile> filesToCompact = new ArrayList<StoreFile>(candidates);
1055
1056 boolean forcemajor = this.forceMajor && filesCompacting.isEmpty();
1057 if (!forcemajor) {
1058
1059
1060 int pos = 0;
1061 while (pos < filesToCompact.size() &&
1062 filesToCompact.get(pos).getReader().length() > maxCompactSize &&
1063 !filesToCompact.get(pos).isReference()) ++pos;
1064 filesToCompact.subList(0, pos).clear();
1065 }
1066
1067 if (filesToCompact.isEmpty()) {
1068 LOG.debug(this.getHRegionInfo().getEncodedName() + " - " +
1069 this.storeNameStr + ": no store files to compact");
1070 return filesToCompact;
1071 }
1072
1073
1074 boolean majorcompaction = filesToCompact.size() < this.maxFilesToCompact
1075 && (forcemajor || isMajorCompaction(filesToCompact));
1076
1077 if (!majorcompaction && !hasReferences(filesToCompact)) {
1078
1079 int start = 0;
1080 double r = this.compactRatio;
1081
1082
1083 if (filesToCompact.size() < this.minFilesToCompact) {
1084 return Collections.emptyList();
1085 }
1086
1087
1088
1089
1090
1091
1092
1093 int countOfFiles = filesToCompact.size();
1094 long [] fileSizes = new long[countOfFiles];
1095 long [] sumSize = new long[countOfFiles];
1096 for (int i = countOfFiles-1; i >= 0; --i) {
1097 StoreFile file = filesToCompact.get(i);
1098 fileSizes[i] = file.getReader().length();
1099
1100 int tooFar = i + this.maxFilesToCompact - 1;
1101 sumSize[i] = fileSizes[i]
1102 + ((i+1 < countOfFiles) ? sumSize[i+1] : 0)
1103 - ((tooFar < countOfFiles) ? fileSizes[tooFar] : 0);
1104 }
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119 while(countOfFiles - start >= this.minFilesToCompact &&
1120 fileSizes[start] >
1121 Math.max(minCompactSize, (long)(sumSize[start+1] * r))) {
1122 ++start;
1123 }
1124 int end = Math.min(countOfFiles, start + this.maxFilesToCompact);
1125 long totalSize = fileSizes[start]
1126 + ((start+1 < countOfFiles) ? sumSize[start+1] : 0);
1127 filesToCompact = filesToCompact.subList(start, end);
1128
1129
1130 if (filesToCompact.size() < this.minFilesToCompact) {
1131 if (LOG.isDebugEnabled()) {
1132 LOG.debug("Skipped compaction of " + this.storeNameStr
1133 + ". Only " + (end - start) + " file(s) of size "
1134 + StringUtils.humanReadableInt(totalSize)
1135 + " have met compaction criteria.");
1136 }
1137 return Collections.emptyList();
1138 }
1139 } else {
1140
1141 if (filesToCompact.size() > this.maxFilesToCompact) {
1142 int pastMax = filesToCompact.size() - this.maxFilesToCompact;
1143 filesToCompact.subList(0, pastMax).clear();
1144 }
1145 }
1146 return filesToCompact;
1147 }
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160 StoreFile.Writer compactStore(final Collection<StoreFile> filesToCompact,
1161 final boolean majorCompaction, final long maxId)
1162 throws IOException {
1163
1164 int maxKeyCount = 0;
1165 for (StoreFile file : filesToCompact) {
1166 StoreFile.Reader r = file.getReader();
1167 if (r != null) {
1168
1169
1170 long keyCount = (r.getBloomFilterType() == family.getBloomFilterType())
1171 ? r.getFilterEntries() : r.getEntries();
1172 maxKeyCount += keyCount;
1173 if (LOG.isDebugEnabled()) {
1174 LOG.debug("Compacting " + file +
1175 ", keycount=" + keyCount +
1176 ", bloomtype=" + r.getBloomFilterType().toString() +
1177 ", size=" + StringUtils.humanReadableInt(r.length()) );
1178 }
1179 }
1180 }
1181
1182
1183 progress = new CompactionProgress(maxKeyCount);
1184
1185
1186 List<StoreFileScanner> scanners = StoreFileScanner
1187 .getScannersForStoreFiles(filesToCompact, false, false, true);
1188
1189
1190
1191 StoreFile.Writer writer = null;
1192
1193 long smallestReadPoint = region.getSmallestReadPoint();
1194 MultiVersionConsistencyControl.setThreadReadPoint(smallestReadPoint);
1195 try {
1196 InternalScanner scanner = null;
1197 try {
1198 Scan scan = new Scan();
1199 scan.setMaxVersions(family.getMaxVersions());
1200
1201 scanner = new StoreScanner(this, scan, scanners, !majorCompaction, smallestReadPoint);
1202 if (region.getCoprocessorHost() != null) {
1203 InternalScanner cpScanner = region.getCoprocessorHost().preCompact(
1204 this, scanner);
1205
1206 if (cpScanner == null) {
1207 return null;
1208 }
1209
1210 scanner = cpScanner;
1211 }
1212
1213 int bytesWritten = 0;
1214
1215
1216 ArrayList<KeyValue> kvs = new ArrayList<KeyValue>();
1217
1218 boolean hasMore;
1219 do {
1220 hasMore = scanner.next(kvs, this.compactionKVMax);
1221 if (writer == null && !kvs.isEmpty()) {
1222 writer = createWriterInTmp(maxKeyCount,
1223 this.compactionCompression);
1224 }
1225 if (writer != null) {
1226
1227 for (KeyValue kv : kvs) {
1228 if (kv.getMemstoreTS() <= smallestReadPoint) {
1229 kv.setMemstoreTS(0);
1230 }
1231 writer.append(kv);
1232
1233 ++progress.currentCompactedKVs;
1234
1235
1236 if (Store.closeCheckInterval > 0) {
1237 bytesWritten += kv.getLength();
1238 if (bytesWritten > Store.closeCheckInterval) {
1239 bytesWritten = 0;
1240 if (!this.region.areWritesEnabled()) {
1241 writer.close();
1242 fs.delete(writer.getPath(), false);
1243 throw new InterruptedIOException(
1244 "Aborting compaction of store " + this +
1245 " in region " + this.region +
1246 " because user requested stop.");
1247 }
1248 }
1249 }
1250 }
1251 }
1252 kvs.clear();
1253 } while (hasMore);
1254 } finally {
1255 if (scanner != null) {
1256 scanner.close();
1257 }
1258 }
1259 } finally {
1260 if (writer != null) {
1261 writer.appendMetadata(maxId, majorCompaction);
1262 writer.close();
1263 }
1264 }
1265 return writer;
1266 }
1267
1268
1269
1270
1271
1272
1273
1274 private void validateStoreFile(Path path)
1275 throws IOException {
1276 StoreFile storeFile = null;
1277 try {
1278 storeFile = new StoreFile(this.fs, path, this.conf,
1279 this.cacheConf, this.family.getBloomFilterType());
1280 storeFile.createReader();
1281 } catch (IOException e) {
1282 LOG.error("Failed to open store file : " + path
1283 + ", keeping it in tmp location", e);
1284 throw e;
1285 } finally {
1286 if (storeFile != null) {
1287 storeFile.closeReader(false);
1288 }
1289 }
1290 }
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311 StoreFile completeCompaction(final Collection<StoreFile> compactedFiles,
1312 final StoreFile.Writer compactedFile)
1313 throws IOException {
1314
1315
1316 StoreFile result = null;
1317 if (compactedFile != null) {
1318 validateStoreFile(compactedFile.getPath());
1319
1320 Path origPath = compactedFile.getPath();
1321 Path destPath = new Path(homedir, origPath.getName());
1322 LOG.info("Renaming compacted file at " + origPath + " to " + destPath);
1323 if (!fs.rename(origPath, destPath)) {
1324 LOG.error("Failed move of compacted file " + origPath + " to " +
1325 destPath);
1326 throw new IOException("Failed move of compacted file " + origPath +
1327 " to " + destPath);
1328 }
1329 result = new StoreFile(this.fs, destPath, this.conf, this.cacheConf,
1330 this.family.getBloomFilterType());
1331 result.createReader();
1332 }
1333 try {
1334 this.lock.writeLock().lock();
1335 try {
1336
1337
1338
1339
1340 ArrayList<StoreFile> newStoreFiles = Lists.newArrayList(storefiles);
1341 newStoreFiles.removeAll(compactedFiles);
1342 filesCompacting.removeAll(compactedFiles);
1343
1344
1345 if (result != null) {
1346 newStoreFiles.add(result);
1347 }
1348
1349 this.storefiles = sortAndClone(newStoreFiles);
1350 } finally {
1351
1352
1353
1354
1355
1356 this.lock.writeLock().unlock();
1357 }
1358
1359
1360 notifyChangedReadersObservers();
1361
1362 for (StoreFile hsf: compactedFiles) {
1363 hsf.deleteReader();
1364 }
1365 } catch (IOException e) {
1366 e = RemoteExceptionHandler.checkIOException(e);
1367 LOG.error("Failed replacing compacted files in " + this.storeNameStr +
1368 ". Compacted file is " + (result == null? "none": result.toString()) +
1369 ". Files replaced " + compactedFiles.toString() +
1370 " some of which may have been already removed", e);
1371 }
1372
1373
1374 this.storeSize = 0L;
1375 this.totalUncompressedBytes = 0L;
1376 for (StoreFile hsf : this.storefiles) {
1377 StoreFile.Reader r = hsf.getReader();
1378 if (r == null) {
1379 LOG.warn("StoreFile " + hsf + " has a null Reader");
1380 continue;
1381 }
1382 this.storeSize += r.length();
1383 this.totalUncompressedBytes += r.getTotalUncompressedBytes();
1384 }
1385 return result;
1386 }
1387
1388 public ImmutableList<StoreFile> sortAndClone(List<StoreFile> storeFiles) {
1389 Collections.sort(storeFiles, StoreFile.Comparators.FLUSH_TIME);
1390 ImmutableList<StoreFile> newList = ImmutableList.copyOf(storeFiles);
1391 return newList;
1392 }
1393
1394
1395
1396
1397
1398
1399
1400
1401 public int getNumberOfstorefiles() {
1402 return this.storefiles.size();
1403 }
1404
1405
1406
1407
1408
1409 int versionsToReturn(final int wantedVersions) {
1410 if (wantedVersions <= 0) {
1411 throw new IllegalArgumentException("Number of versions must be > 0");
1412 }
1413
1414 int maxVersions = this.family.getMaxVersions();
1415 return wantedVersions > maxVersions ? maxVersions: wantedVersions;
1416 }
1417
1418 static boolean isExpired(final KeyValue key, final long oldestTimestamp) {
1419 return key.getTimestamp() < oldestTimestamp;
1420 }
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436 KeyValue getRowKeyAtOrBefore(final byte[] row) throws IOException {
1437
1438
1439
1440
1441
1442
1443 long ttlToUse = this.minVersions > 0 ? Long.MAX_VALUE : this.ttl;
1444
1445 KeyValue kv = new KeyValue(row, HConstants.LATEST_TIMESTAMP);
1446
1447 GetClosestRowBeforeTracker state = new GetClosestRowBeforeTracker(
1448 this.comparator, kv, ttlToUse, this.region.getRegionInfo().isMetaRegion());
1449 this.lock.readLock().lock();
1450 try {
1451
1452 this.memstore.getRowKeyAtOrBefore(state);
1453
1454
1455 for (StoreFile sf : Lists.reverse(storefiles)) {
1456
1457 rowAtOrBeforeFromStoreFile(sf, state);
1458 }
1459 return state.getCandidate();
1460 } finally {
1461 this.lock.readLock().unlock();
1462 }
1463 }
1464
1465
1466
1467
1468
1469
1470
1471 private void rowAtOrBeforeFromStoreFile(final StoreFile f,
1472 final GetClosestRowBeforeTracker state)
1473 throws IOException {
1474 StoreFile.Reader r = f.getReader();
1475 if (r == null) {
1476 LOG.warn("StoreFile " + f + " has a null Reader");
1477 return;
1478 }
1479
1480 byte [] fk = r.getFirstKey();
1481 KeyValue firstKV = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
1482 byte [] lk = r.getLastKey();
1483 KeyValue lastKV = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
1484 KeyValue firstOnRow = state.getTargetKey();
1485 if (this.comparator.compareRows(lastKV, firstOnRow) < 0) {
1486
1487
1488 if (!state.isTargetTable(lastKV)) return;
1489
1490
1491 firstOnRow = new KeyValue(lastKV.getRow(), HConstants.LATEST_TIMESTAMP);
1492 }
1493
1494 HFileScanner scanner = r.getHFileReader().getScanner(true, true, false);
1495
1496 if (!seekToScanner(scanner, firstOnRow, firstKV)) return;
1497
1498
1499 if (walkForwardInSingleRow(scanner, firstOnRow, state)) return;
1500
1501 while (scanner.seekBefore(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(),
1502 firstOnRow.getKeyLength())) {
1503 KeyValue kv = scanner.getKeyValue();
1504 if (!state.isTargetTable(kv)) break;
1505 if (!state.isBetterCandidate(kv)) break;
1506
1507 firstOnRow = new KeyValue(kv.getRow(), HConstants.LATEST_TIMESTAMP);
1508
1509 if (!seekToScanner(scanner, firstOnRow, firstKV)) break;
1510
1511 if (walkForwardInSingleRow(scanner, firstOnRow, state)) break;
1512 }
1513 }
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523 private boolean seekToScanner(final HFileScanner scanner,
1524 final KeyValue firstOnRow,
1525 final KeyValue firstKV)
1526 throws IOException {
1527 KeyValue kv = firstOnRow;
1528
1529 if (this.comparator.compareRows(firstKV, firstOnRow) == 0) kv = firstKV;
1530 int result = scanner.seekTo(kv.getBuffer(), kv.getKeyOffset(),
1531 kv.getKeyLength());
1532 return result >= 0;
1533 }
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545 private boolean walkForwardInSingleRow(final HFileScanner scanner,
1546 final KeyValue firstOnRow,
1547 final GetClosestRowBeforeTracker state)
1548 throws IOException {
1549 boolean foundCandidate = false;
1550 do {
1551 KeyValue kv = scanner.getKeyValue();
1552
1553 if (this.comparator.compareRows(kv, firstOnRow) < 0) continue;
1554
1555 if (state.isTooFar(kv, firstOnRow)) break;
1556 if (state.isExpired(kv)) {
1557 continue;
1558 }
1559
1560 if (state.handle(kv)) {
1561 foundCandidate = true;
1562 break;
1563 }
1564 } while(scanner.next());
1565 return foundCandidate;
1566 }
1567
1568 public boolean canSplit() {
1569 this.lock.readLock().lock();
1570 try {
1571
1572 for (StoreFile sf : storefiles) {
1573 if (sf.isReference()) {
1574 if (LOG.isDebugEnabled()) {
1575 LOG.debug(sf + " is not splittable");
1576 }
1577 return false;
1578 }
1579 }
1580
1581 return true;
1582 } finally {
1583 this.lock.readLock().unlock();
1584 }
1585 }
1586
1587
1588
1589
1590 public byte[] getSplitPoint() {
1591 this.lock.readLock().lock();
1592 try {
1593
1594 if (this.storefiles.isEmpty()) {
1595 return null;
1596 }
1597
1598 assert !this.region.getRegionInfo().isMetaRegion();
1599
1600
1601 long maxSize = 0L;
1602 StoreFile largestSf = null;
1603 for (StoreFile sf : storefiles) {
1604 if (sf.isReference()) {
1605
1606 assert false : "getSplitPoint() called on a region that can't split!";
1607 return null;
1608 }
1609
1610 StoreFile.Reader r = sf.getReader();
1611 if (r == null) {
1612 LOG.warn("Storefile " + sf + " Reader is null");
1613 continue;
1614 }
1615
1616 long size = r.length();
1617 if (size > maxSize) {
1618
1619 maxSize = size;
1620 largestSf = sf;
1621 }
1622 }
1623
1624 StoreFile.Reader r = largestSf.getReader();
1625 if (r == null) {
1626 LOG.warn("Storefile " + largestSf + " Reader is null");
1627 return null;
1628 }
1629
1630
1631
1632 byte [] midkey = r.midkey();
1633 if (midkey != null) {
1634 KeyValue mk = KeyValue.createKeyValueFromKey(midkey, 0, midkey.length);
1635 byte [] fk = r.getFirstKey();
1636 KeyValue firstKey = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
1637 byte [] lk = r.getLastKey();
1638 KeyValue lastKey = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
1639
1640
1641 if (this.comparator.compareRows(mk, firstKey) == 0 &&
1642 this.comparator.compareRows(mk, lastKey) == 0) {
1643 if (LOG.isDebugEnabled()) {
1644 LOG.debug("cannot split because midkey is the same as first or " +
1645 "last row");
1646 }
1647 return null;
1648 }
1649 return mk.getRow();
1650 }
1651 } catch(IOException e) {
1652 LOG.warn("Failed getting store size for " + this.storeNameStr, e);
1653 } finally {
1654 this.lock.readLock().unlock();
1655 }
1656 return null;
1657 }
1658
1659
1660 public long getLastCompactSize() {
1661 return this.lastCompactSize;
1662 }
1663
1664
1665 public long getSize() {
1666 return storeSize;
1667 }
1668
1669 public void triggerMajorCompaction() {
1670 this.forceMajor = true;
1671 }
1672
1673 boolean getForceMajorCompaction() {
1674 return this.forceMajor;
1675 }
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685 public StoreScanner getScanner(Scan scan,
1686 final NavigableSet<byte []> targetCols) throws IOException {
1687 lock.readLock().lock();
1688 try {
1689 return new StoreScanner(this, scan, targetCols);
1690 } finally {
1691 lock.readLock().unlock();
1692 }
1693 }
1694
1695 @Override
1696 public String toString() {
1697 return this.storeNameStr;
1698 }
1699
1700
1701
1702
1703 int getStorefilesCount() {
1704 return this.storefiles.size();
1705 }
1706
1707
1708
1709
1710 long getStoreSizeUncompressed() {
1711 return this.totalUncompressedBytes;
1712 }
1713
1714
1715
1716
1717 long getStorefilesSize() {
1718 long size = 0;
1719 for (StoreFile s: storefiles) {
1720 StoreFile.Reader r = s.getReader();
1721 if (r == null) {
1722 LOG.warn("StoreFile " + s + " has a null Reader");
1723 continue;
1724 }
1725 size += r.length();
1726 }
1727 return size;
1728 }
1729
1730
1731
1732
1733 long getStorefilesIndexSize() {
1734 long size = 0;
1735 for (StoreFile s: storefiles) {
1736 StoreFile.Reader r = s.getReader();
1737 if (r == null) {
1738 LOG.warn("StoreFile " + s + " has a null Reader");
1739 continue;
1740 }
1741 size += r.indexSize();
1742 }
1743 return size;
1744 }
1745
1746
1747
1748
1749
1750
1751
1752
1753 long getTotalStaticIndexSize() {
1754 long size = 0;
1755 for (StoreFile s : storefiles) {
1756 size += s.getReader().getUncompressedDataIndexSize();
1757 }
1758 return size;
1759 }
1760
1761
1762
1763
1764
1765
1766
1767
1768 long getTotalStaticBloomSize() {
1769 long size = 0;
1770 for (StoreFile s : storefiles) {
1771 StoreFile.Reader r = s.getReader();
1772 size += r.getTotalBloomSize();
1773 }
1774 return size;
1775 }
1776
1777
1778
1779
1780 public int getCompactPriority() {
1781 return this.blockingStoreFileCount - this.storefiles.size();
1782 }
1783
1784 HRegion getHRegion() {
1785 return this.region;
1786 }
1787
1788 HRegionInfo getHRegionInfo() {
1789 return this.region.regionInfo;
1790 }
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806 public long updateColumnValue(byte [] row, byte [] f,
1807 byte [] qualifier, long newValue)
1808 throws IOException {
1809
1810 this.lock.readLock().lock();
1811 try {
1812 long now = EnvironmentEdgeManager.currentTimeMillis();
1813
1814 return this.memstore.updateColumnValue(row,
1815 f,
1816 qualifier,
1817 newValue,
1818 now);
1819
1820 } finally {
1821 this.lock.readLock().unlock();
1822 }
1823 }
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838 public long upsert(List<KeyValue> kvs)
1839 throws IOException {
1840 this.lock.readLock().lock();
1841 try {
1842
1843 return this.memstore.upsert(kvs);
1844 } finally {
1845 this.lock.readLock().unlock();
1846 }
1847 }
1848
1849 public StoreFlusher getStoreFlusher(long cacheFlushId) {
1850 return new StoreFlusherImpl(cacheFlushId);
1851 }
1852
1853 private class StoreFlusherImpl implements StoreFlusher {
1854
1855 private long cacheFlushId;
1856 private SortedSet<KeyValue> snapshot;
1857 private StoreFile storeFile;
1858 private TimeRangeTracker snapshotTimeRangeTracker;
1859
1860 private StoreFlusherImpl(long cacheFlushId) {
1861 this.cacheFlushId = cacheFlushId;
1862 }
1863
1864 @Override
1865 public void prepare() {
1866 memstore.snapshot();
1867 this.snapshot = memstore.getSnapshot();
1868 this.snapshotTimeRangeTracker = memstore.getSnapshotTimeRangeTracker();
1869 }
1870
1871 @Override
1872 public void flushCache(MonitoredTask status) throws IOException {
1873 storeFile = Store.this.flushCache(
1874 cacheFlushId, snapshot, snapshotTimeRangeTracker, status);
1875 }
1876
1877 @Override
1878 public boolean commit() throws IOException {
1879 if (storeFile == null) {
1880 return false;
1881 }
1882
1883
1884 return Store.this.updateStorefiles(storeFile, snapshot);
1885 }
1886 }
1887
1888
1889
1890
1891
1892
1893 public boolean needsCompaction() {
1894 return (storefiles.size() - filesCompacting.size()) > minFilesToCompact;
1895 }
1896
1897
1898
1899
1900 public CacheConfig getCacheConfig() {
1901 return this.cacheConf;
1902 }
1903
1904 public static final long FIXED_OVERHEAD = ClassSize.align(
1905 ClassSize.OBJECT + (17 * ClassSize.REFERENCE) +
1906 (7 * Bytes.SIZEOF_LONG) + (1 * Bytes.SIZEOF_DOUBLE) +
1907 (7 * Bytes.SIZEOF_INT) + (1 * Bytes.SIZEOF_BOOLEAN));
1908
1909 public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
1910 ClassSize.OBJECT + ClassSize.REENTRANT_LOCK +
1911 ClassSize.CONCURRENT_SKIPLISTMAP +
1912 ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + ClassSize.OBJECT);
1913
1914 @Override
1915 public long heapSize() {
1916 return DEEP_OVERHEAD + this.memstore.heapSize();
1917 }
1918
1919 public KeyValue.KVComparator getComparator() {
1920 return comparator;
1921 }
1922
1923 }