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.util.ArrayList;
24 import java.util.Collection;
25 import java.util.Collections;
26 import java.util.List;
27 import java.util.NavigableSet;
28 import java.util.SortedSet;
29 import java.util.concurrent.CopyOnWriteArraySet;
30 import java.util.concurrent.locks.ReentrantReadWriteLock;
31
32 import org.apache.commons.logging.Log;
33 import org.apache.commons.logging.LogFactory;
34 import org.apache.hadoop.conf.Configuration;
35 import org.apache.hadoop.fs.FileStatus;
36 import org.apache.hadoop.fs.FileSystem;
37 import org.apache.hadoop.fs.FileUtil;
38 import org.apache.hadoop.fs.Path;
39 import org.apache.hadoop.hbase.HColumnDescriptor;
40 import org.apache.hadoop.hbase.HConstants;
41 import org.apache.hadoop.hbase.HRegionInfo;
42 import org.apache.hadoop.hbase.KeyValue;
43 import org.apache.hadoop.hbase.RemoteExceptionHandler;
44 import org.apache.hadoop.hbase.client.Scan;
45 import org.apache.hadoop.hbase.io.HeapSize;
46 import org.apache.hadoop.hbase.io.hfile.Compression;
47 import org.apache.hadoop.hbase.io.hfile.HFile;
48 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
49 import org.apache.hadoop.hbase.util.Bytes;
50 import org.apache.hadoop.hbase.util.ClassSize;
51 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
52 import org.apache.hadoop.util.StringUtils;
53
54 import com.google.common.collect.ImmutableList;
55 import com.google.common.collect.Iterables;
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80 public class Store implements HeapSize {
81 static final Log LOG = LogFactory.getLog(Store.class);
82 protected final MemStore memstore;
83
84 private final Path homedir;
85 private final HRegion region;
86 private final HColumnDescriptor family;
87 final FileSystem fs;
88 final Configuration conf;
89
90 protected long ttl;
91 private long majorCompactionTime;
92 private int maxFilesToCompact;
93 private final long desiredMaxFileSize;
94 private volatile long storeSize = 0L;
95 private final Object flushLock = new Object();
96 final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
97 private final String storeNameStr;
98 private final boolean inMemory;
99
100
101
102
103
104 private ImmutableList<StoreFile> storefiles = null;
105
106
107
108 private final CopyOnWriteArraySet<ChangedReadersObserver> changedReaderObservers =
109 new CopyOnWriteArraySet<ChangedReadersObserver>();
110
111 private final Object compactLock = new Object();
112 private final int compactionThreshold;
113 private final int blocksize;
114 private final boolean blockcache;
115 private final Compression.Algorithm compression;
116
117
118 final KeyValue.KVComparator comparator;
119
120
121
122
123
124
125
126
127
128
129
130
131 protected Store(Path basedir, HRegion region, HColumnDescriptor family,
132 FileSystem fs, Configuration conf)
133 throws IOException {
134 HRegionInfo info = region.regionInfo;
135 this.fs = fs;
136 this.homedir = getStoreHomedir(basedir, info.getEncodedName(), family.getName());
137 if (!this.fs.exists(this.homedir)) {
138 if (!this.fs.mkdirs(this.homedir))
139 throw new IOException("Failed create of: " + this.homedir.toString());
140 }
141 this.region = region;
142 this.family = family;
143 this.conf = conf;
144 this.blockcache = family.isBlockCacheEnabled();
145 this.blocksize = family.getBlocksize();
146 this.compression = family.getCompression();
147 this.comparator = info.getComparator();
148
149 this.ttl = family.getTimeToLive();
150 if (ttl == HConstants.FOREVER) {
151
152 ttl = Long.MAX_VALUE;
153 } else if (ttl == -1) {
154 ttl = Long.MAX_VALUE;
155 } else {
156
157 this.ttl *= 1000;
158 }
159 this.memstore = new MemStore(this.comparator);
160 this.storeNameStr = Bytes.toString(this.family.getName());
161
162
163
164 this.compactionThreshold =
165 conf.getInt("hbase.hstore.compactionThreshold", 3);
166
167
168 this.inMemory = family.isInMemory();
169
170
171 long maxFileSize = info.getTableDesc().getMaxFileSize();
172 if (maxFileSize == HConstants.DEFAULT_MAX_FILE_SIZE) {
173 maxFileSize = conf.getLong("hbase.hregion.max.filesize",
174 HConstants.DEFAULT_MAX_FILE_SIZE);
175 }
176 this.desiredMaxFileSize = maxFileSize;
177
178 this.majorCompactionTime =
179 conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD, 86400000);
180 if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) != null) {
181 String strCompactionTime =
182 family.getValue(HConstants.MAJOR_COMPACTION_PERIOD);
183 this.majorCompactionTime = (new Long(strCompactionTime)).longValue();
184 }
185
186 this.maxFilesToCompact = conf.getInt("hbase.hstore.compaction.max", 10);
187 this.storefiles = ImmutableList.copyOf(loadStoreFiles());
188 }
189
190 public HColumnDescriptor getFamily() {
191 return this.family;
192 }
193
194
195
196
197 long getMaxSequenceId() {
198 return StoreFile.getMaxSequenceIdInList(this.getStorefiles());
199 }
200
201
202
203
204
205
206
207 public static Path getStoreHomedir(final Path tabledir,
208 final String encodedName, final byte [] family) {
209 return new Path(tabledir, new Path(encodedName,
210 new Path(Bytes.toString(family))));
211 }
212
213
214
215
216
217 public Path getHomedir() {
218 return homedir;
219 }
220
221
222
223
224
225 private List<StoreFile> loadStoreFiles()
226 throws IOException {
227 ArrayList<StoreFile> results = new ArrayList<StoreFile>();
228 FileStatus files[] = this.fs.listStatus(this.homedir);
229 for (int i = 0; files != null && i < files.length; i++) {
230
231 if (files[i].isDir()) {
232 continue;
233 }
234 Path p = files[i].getPath();
235
236
237 if (this.fs.getFileStatus(p).getLen() <= 0) {
238 LOG.warn("Skipping " + p + " because its empty. HBASE-646 DATA LOSS?");
239 continue;
240 }
241 StoreFile curfile = null;
242 try {
243 curfile = new StoreFile(fs, p, blockcache, this.conf,
244 this.family.getBloomFilterType(), this.inMemory);
245 curfile.createReader();
246 } catch (IOException ioe) {
247 LOG.warn("Failed open of " + p + "; presumption is that file was " +
248 "corrupted at flush and lost edits picked up by commit log replay. " +
249 "Verify!", ioe);
250 continue;
251 }
252 long length = curfile.getReader().length();
253 this.storeSize += length;
254 if (LOG.isDebugEnabled()) {
255 LOG.debug("loaded " + curfile.toStringDetailed());
256 }
257 results.add(curfile);
258 }
259 Collections.sort(results, StoreFile.Comparators.FLUSH_TIME);
260 return results;
261 }
262
263
264
265
266
267
268
269 protected long add(final KeyValue kv) {
270 lock.readLock().lock();
271 try {
272 return this.memstore.add(kv);
273 } finally {
274 lock.readLock().unlock();
275 }
276 }
277
278
279
280
281
282
283
284 protected long delete(final KeyValue kv) {
285 lock.readLock().lock();
286 try {
287 return this.memstore.delete(kv);
288 } finally {
289 lock.readLock().unlock();
290 }
291 }
292
293
294
295
296 List<StoreFile> getStorefiles() {
297 return this.storefiles;
298 }
299
300 public void bulkLoadHFile(String srcPathStr) throws IOException {
301 Path srcPath = new Path(srcPathStr);
302
303 HFile.Reader reader = null;
304 try {
305 LOG.info("Validating hfile at " + srcPath + " for inclusion in "
306 + "store " + this + " region " + this.region);
307 reader = new HFile.Reader(srcPath.getFileSystem(conf),
308 srcPath, null, false);
309 reader.loadFileInfo();
310
311 byte[] firstKey = reader.getFirstRowKey();
312 byte[] lk = reader.getLastKey();
313 byte[] lastKey =
314 (lk == null) ? null :
315 KeyValue.createKeyValueFromKey(lk).getRow();
316
317 LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey) +
318 " last=" + Bytes.toStringBinary(lastKey));
319 LOG.debug("Region bounds: first=" +
320 Bytes.toStringBinary(region.getStartKey()) +
321 " last=" + Bytes.toStringBinary(region.getEndKey()));
322
323 HRegionInfo hri = region.getRegionInfo();
324 if (!hri.containsRange(firstKey, lastKey)) {
325 throw new WrongRegionException(
326 "Bulk load file " + srcPathStr + " does not fit inside region "
327 + this.region);
328 }
329 } finally {
330 if (reader != null) reader.close();
331 }
332
333
334 FileSystem srcFs = srcPath.getFileSystem(conf);
335 if (!srcFs.equals(fs)) {
336 LOG.info("File " + srcPath + " on different filesystem than " +
337 "destination store - moving to this filesystem.");
338 Path tmpPath = getTmpPath();
339 FileUtil.copy(srcFs, srcPath, fs, tmpPath, false, conf);
340 LOG.info("Copied to temporary path on dst filesystem: " + tmpPath);
341 srcPath = tmpPath;
342 }
343
344 Path dstPath = StoreFile.getRandomFilename(fs, homedir);
345 LOG.info("Renaming bulk load file " + srcPath + " to " + dstPath);
346 StoreFile.rename(fs, srcPath, dstPath);
347
348 StoreFile sf = new StoreFile(fs, dstPath, blockcache,
349 this.conf, this.family.getBloomFilterType(), this.inMemory);
350 sf.createReader();
351
352 LOG.info("Moved hfile " + srcPath + " into store directory " +
353 homedir + " - updating store file list.");
354
355
356 this.lock.writeLock().lock();
357 try {
358 ArrayList<StoreFile> newFiles = new ArrayList<StoreFile>(storefiles);
359 newFiles.add(sf);
360 this.storefiles = ImmutableList.copyOf(newFiles);
361 notifyChangedReadersObservers();
362 } finally {
363 this.lock.writeLock().unlock();
364 }
365 LOG.info("Successfully loaded store file " + srcPath
366 + " into store " + this + " (new location: " + dstPath + ")");
367 }
368
369
370
371
372
373
374 private Path getTmpPath() throws IOException {
375 return StoreFile.getRandomFilename(
376 fs, region.getTmpDir());
377 }
378
379
380
381
382
383
384
385
386
387 ImmutableList<StoreFile> close() throws IOException {
388 this.lock.writeLock().lock();
389 try {
390 ImmutableList<StoreFile> result = storefiles;
391
392
393 storefiles = ImmutableList.of();
394
395 for (StoreFile f: result) {
396 f.closeReader();
397 }
398 LOG.debug("closed " + this.storeNameStr);
399 return result;
400 } finally {
401 this.lock.writeLock().unlock();
402 }
403 }
404
405
406
407
408
409 void snapshot() {
410 this.memstore.snapshot();
411 }
412
413
414
415
416
417
418
419
420
421 private StoreFile flushCache(final long logCacheFlushId,
422 SortedSet<KeyValue> snapshot,
423 TimeRangeTracker snapshotTimeRangeTracker) throws IOException {
424
425
426
427 return internalFlushCache(snapshot, logCacheFlushId, snapshotTimeRangeTracker);
428 }
429
430
431
432
433
434
435
436 private StoreFile internalFlushCache(final SortedSet<KeyValue> set,
437 final long logCacheFlushId,
438 TimeRangeTracker snapshotTimeRangeTracker)
439 throws IOException {
440 StoreFile.Writer writer = null;
441 long flushed = 0;
442
443 if (set.size() == 0) {
444 return null;
445 }
446 long oldestTimestamp = System.currentTimeMillis() - ttl;
447
448
449
450 synchronized (flushLock) {
451
452 writer = createWriterInTmp(set.size());
453 writer.setTimeRangeTracker(snapshotTimeRangeTracker);
454 int entries = 0;
455 try {
456 for (KeyValue kv: set) {
457 if (!isExpired(kv, oldestTimestamp)) {
458 writer.append(kv);
459 entries++;
460 flushed += this.memstore.heapSizeChange(kv, true);
461 }
462 }
463 } finally {
464
465
466 writer.appendMetadata(logCacheFlushId, false);
467 writer.close();
468 }
469 }
470
471
472 Path dstPath = StoreFile.getUniqueFile(fs, homedir);
473 LOG.info("Renaming flushed file at " + writer.getPath() + " to " + dstPath);
474 fs.rename(writer.getPath(), dstPath);
475
476 StoreFile sf = new StoreFile(this.fs, dstPath, blockcache,
477 this.conf, this.family.getBloomFilterType(), this.inMemory);
478 StoreFile.Reader r = sf.createReader();
479 this.storeSize += r.length();
480 if(LOG.isInfoEnabled()) {
481 LOG.info("Added " + sf + ", entries=" + r.getEntries() +
482 ", sequenceid=" + logCacheFlushId +
483 ", memsize=" + StringUtils.humanReadableInt(flushed) +
484 ", filesize=" + StringUtils.humanReadableInt(r.length()) +
485 " to " + this.region.regionInfo.getRegionNameAsString());
486 }
487 return sf;
488 }
489
490
491
492
493 private StoreFile.Writer createWriterInTmp(int maxKeyCount)
494 throws IOException {
495 return StoreFile.createWriter(this.fs, region.getTmpDir(), this.blocksize,
496 this.compression, this.comparator, this.conf,
497 this.family.getBloomFilterType(), maxKeyCount);
498 }
499
500
501
502
503
504
505
506
507 private boolean updateStorefiles(final StoreFile sf,
508 final SortedSet<KeyValue> set)
509 throws IOException {
510 this.lock.writeLock().lock();
511 try {
512 ArrayList<StoreFile> newList = new ArrayList<StoreFile>(storefiles);
513 newList.add(sf);
514 storefiles = ImmutableList.copyOf(newList);
515 this.memstore.clearSnapshot(set);
516
517
518 notifyChangedReadersObservers();
519
520 return this.storefiles.size() >= this.compactionThreshold;
521 } finally {
522 this.lock.writeLock().unlock();
523 }
524 }
525
526
527
528
529
530 private void notifyChangedReadersObservers() throws IOException {
531 for (ChangedReadersObserver o: this.changedReaderObservers) {
532 o.updateReaders();
533 }
534 }
535
536
537
538
539 void addChangedReaderObserver(ChangedReadersObserver o) {
540 this.changedReaderObservers.add(o);
541 }
542
543
544
545
546 void deleteChangedReaderObserver(ChangedReadersObserver o) {
547 if (!this.changedReaderObservers.remove(o)) {
548 LOG.warn("Not in set" + o);
549 }
550 }
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576 StoreSize compact(final boolean mc) throws IOException {
577 boolean forceSplit = this.region.shouldSplit(false);
578 boolean majorcompaction = mc;
579 synchronized (compactLock) {
580
581 List<StoreFile> filesToCompact = this.storefiles;
582 if (filesToCompact.isEmpty()) {
583 LOG.debug(this.storeNameStr + ": no store files to compact");
584 return null;
585 }
586
587
588 long maxId = StoreFile.getMaxSequenceIdInList(storefiles);
589
590
591
592
593 if (!majorcompaction) {
594 majorcompaction = isMajorCompaction(filesToCompact);
595 }
596
597 boolean references = hasReferences(filesToCompact);
598 if (!majorcompaction && !references &&
599 (forceSplit || (filesToCompact.size() < compactionThreshold))) {
600 return checkSplit(forceSplit);
601 }
602
603
604
605 int countOfFiles = filesToCompact.size();
606 long totalSize = 0;
607 long [] fileSizes = new long[countOfFiles];
608 long skipped = 0;
609 int point = 0;
610 for (int i = 0; i < countOfFiles; i++) {
611 StoreFile file = filesToCompact.get(i);
612 Path path = file.getPath();
613 if (path == null) {
614 LOG.warn("Path is null for " + file);
615 return null;
616 }
617 StoreFile.Reader r = file.getReader();
618 if (r == null) {
619 LOG.warn("StoreFile " + file + " has a null Reader");
620 return null;
621 }
622 long len = file.getReader().length();
623 fileSizes[i] = len;
624 totalSize += len;
625 }
626
627 if (!majorcompaction && !references) {
628
629
630
631
632
633
634
635
636
637
638 int tail = Math.min(countOfFiles, 4);
639 for (point = 0; point < (countOfFiles - tail); point++) {
640 if (((fileSizes[point] < fileSizes[point + 1] * 2) &&
641 (countOfFiles - point) <= maxFilesToCompact)) {
642 break;
643 }
644 skipped += fileSizes[point];
645 }
646 filesToCompact = new ArrayList<StoreFile>(filesToCompact.subList(point,
647 countOfFiles));
648 if (filesToCompact.size() <= 1) {
649 if (LOG.isDebugEnabled()) {
650 LOG.debug("Skipped compaction of 1 file; compaction size of " +
651 this.storeNameStr + ": " +
652 StringUtils.humanReadableInt(totalSize) + "; Skipped " + point +
653 " files, size: " + skipped);
654 }
655 return checkSplit(forceSplit);
656 }
657 if (LOG.isDebugEnabled()) {
658 LOG.debug("Compaction size of " + this.storeNameStr + ": " +
659 StringUtils.humanReadableInt(totalSize) + "; Skipped " + point +
660 " file(s), size: " + skipped);
661 }
662 }
663
664
665 LOG.info("Started compaction of " + filesToCompact.size() + " file(s) in " +
666 this.storeNameStr + " of " + this.region.getRegionInfo().getRegionNameAsString() +
667 (references? ", hasReferences=true,": " ") + " into " +
668 region.getTmpDir() + ", sequenceid=" + maxId);
669 StoreFile.Writer writer = compact(filesToCompact, majorcompaction, maxId);
670
671 StoreFile sf = completeCompaction(filesToCompact, writer);
672 if (LOG.isInfoEnabled()) {
673 LOG.info("Completed" + (majorcompaction? " major ": " ") +
674 "compaction of " + filesToCompact.size() + " file(s) in " +
675 this.storeNameStr + " of " + this.region.getRegionInfo().getRegionNameAsString() +
676 "; new storefile is " + (sf == null? "none": sf.toString()) +
677 "; store size is " + StringUtils.humanReadableInt(storeSize));
678 }
679 }
680 return checkSplit(forceSplit);
681 }
682
683
684
685
686
687 private boolean hasReferences(Collection<StoreFile> files) {
688 if (files != null && files.size() > 0) {
689 for (StoreFile hsf: files) {
690 if (hsf.isReference()) {
691 return true;
692 }
693 }
694 }
695 return false;
696 }
697
698
699
700
701
702
703
704
705 private static long getLowestTimestamp(FileSystem fs, Path dir) throws IOException {
706 FileStatus[] stats = fs.listStatus(dir);
707 if (stats == null || stats.length == 0) {
708 return 0l;
709 }
710 long lowTimestamp = Long.MAX_VALUE;
711 for (int i = 0; i < stats.length; i++) {
712 long timestamp = stats[i].getModificationTime();
713 if (timestamp < lowTimestamp){
714 lowTimestamp = timestamp;
715 }
716 }
717 return lowTimestamp;
718 }
719
720
721
722
723 boolean isMajorCompaction() throws IOException {
724 return isMajorCompaction(storefiles);
725 }
726
727
728
729
730
731 private boolean isMajorCompaction(final List<StoreFile> filesToCompact) throws IOException {
732 boolean result = false;
733 if (filesToCompact == null || filesToCompact.isEmpty() ||
734 majorCompactionTime == 0) {
735 return result;
736 }
737 long lowTimestamp = getLowestTimestamp(fs,
738 filesToCompact.get(0).getPath().getParent());
739 long now = System.currentTimeMillis();
740 if (lowTimestamp > 0l && lowTimestamp < (now - this.majorCompactionTime)) {
741
742 long elapsedTime = now - lowTimestamp;
743 if (filesToCompact.size() == 1 &&
744 filesToCompact.get(0).isMajorCompaction() &&
745 (this.ttl == HConstants.FOREVER || elapsedTime < this.ttl)) {
746 if (LOG.isDebugEnabled()) {
747 LOG.debug("Skipping major compaction of " + this.storeNameStr +
748 " because one (major) compacted file only and elapsedTime " +
749 elapsedTime + "ms is < ttl=" + this.ttl);
750 }
751 } else {
752 if (LOG.isDebugEnabled()) {
753 LOG.debug("Major compaction triggered on store " + this.storeNameStr +
754 "; time since last major compaction " + (now - lowTimestamp) + "ms");
755 }
756 result = true;
757 }
758 }
759 return result;
760 }
761
762
763
764
765
766
767
768
769
770
771
772 private StoreFile.Writer compact(final List<StoreFile> filesToCompact,
773 final boolean majorCompaction, final long maxId)
774 throws IOException {
775
776 int maxKeyCount = 0;
777 for (StoreFile file : filesToCompact) {
778 StoreFile.Reader r = file.getReader();
779 if (r != null) {
780
781
782 maxKeyCount += (r.getBloomFilterType() == family.getBloomFilterType())
783 ? r.getFilterEntries() : r.getEntries();
784 }
785 }
786
787
788 List<StoreFileScanner> scanners = StoreFileScanner
789 .getScannersForStoreFiles(filesToCompact, false, false);
790
791
792
793 StoreFile.Writer writer = null;
794 try {
795 if (majorCompaction) {
796 InternalScanner scanner = null;
797 try {
798 Scan scan = new Scan();
799 scan.setMaxVersions(family.getMaxVersions());
800 scanner = new StoreScanner(this, scan, scanners);
801
802
803 ArrayList<KeyValue> kvs = new ArrayList<KeyValue>();
804 while (scanner.next(kvs)) {
805
806 for (KeyValue kv : kvs) {
807 if (writer == null) {
808 writer = createWriterInTmp(maxKeyCount);
809 }
810 writer.append(kv);
811 }
812 kvs.clear();
813 }
814 } finally {
815 if (scanner != null) {
816 scanner.close();
817 }
818 }
819 } else {
820 MinorCompactingStoreScanner scanner = null;
821 try {
822 scanner = new MinorCompactingStoreScanner(this, scanners);
823 writer = createWriterInTmp(maxKeyCount);
824 while (scanner.next(writer)) {
825
826 }
827 } finally {
828 if (scanner != null)
829 scanner.close();
830 }
831 }
832 } finally {
833 if (writer != null) {
834 writer.appendMetadata(maxId, majorCompaction);
835 writer.close();
836 }
837 }
838 return writer;
839 }
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863 private StoreFile completeCompaction(final List<StoreFile> compactedFiles,
864 final StoreFile.Writer compactedFile)
865 throws IOException {
866
867
868 StoreFile result = null;
869 if (compactedFile != null) {
870 Path p = null;
871 try {
872 p = StoreFile.rename(this.fs, compactedFile.getPath(),
873 StoreFile.getRandomFilename(fs, this.homedir));
874 } catch (IOException e) {
875 LOG.error("Failed move of compacted file " + compactedFile.getPath(), e);
876 return null;
877 }
878 result = new StoreFile(this.fs, p, blockcache, this.conf,
879 this.family.getBloomFilterType(), this.inMemory);
880 result.createReader();
881 }
882 this.lock.writeLock().lock();
883 try {
884 try {
885
886
887
888
889
890
891 ArrayList<StoreFile> newStoreFiles = new ArrayList<StoreFile>();
892 for (StoreFile sf : storefiles) {
893 if (!compactedFiles.contains(sf)) {
894 newStoreFiles.add(sf);
895 }
896 }
897
898
899 if (result != null) {
900 newStoreFiles.add(result);
901 }
902
903 this.storefiles = ImmutableList.copyOf(newStoreFiles);
904
905
906 notifyChangedReadersObservers();
907
908 for (StoreFile hsf: compactedFiles) {
909 hsf.deleteReader();
910 }
911 } catch (IOException e) {
912 e = RemoteExceptionHandler.checkIOException(e);
913 LOG.error("Failed replacing compacted files in " + this.storeNameStr +
914 ". Compacted file is " + (result == null? "none": result.toString()) +
915 ". Files replaced " + compactedFiles.toString() +
916 " some of which may have been already removed", e);
917 }
918
919 this.storeSize = 0L;
920 for (StoreFile hsf : this.storefiles) {
921 StoreFile.Reader r = hsf.getReader();
922 if (r == null) {
923 LOG.warn("StoreFile " + hsf + " has a null Reader");
924 continue;
925 }
926 this.storeSize += r.length();
927 }
928 } finally {
929 this.lock.writeLock().unlock();
930 }
931 return result;
932 }
933
934
935
936
937
938
939
940
941 public int getNumberOfstorefiles() {
942 return this.storefiles.size();
943 }
944
945
946
947
948
949 int versionsToReturn(final int wantedVersions) {
950 if (wantedVersions <= 0) {
951 throw new IllegalArgumentException("Number of versions must be > 0");
952 }
953
954 int maxVersions = this.family.getMaxVersions();
955 return wantedVersions > maxVersions ? maxVersions: wantedVersions;
956 }
957
958 static boolean isExpired(final KeyValue key, final long oldestTimestamp) {
959 return key.getTimestamp() < oldestTimestamp;
960 }
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977 KeyValue getRowKeyAtOrBefore(final KeyValue kv) throws IOException {
978 GetClosestRowBeforeTracker state = new GetClosestRowBeforeTracker(
979 this.comparator, kv, this.ttl, this.region.getRegionInfo().isMetaRegion());
980 this.lock.readLock().lock();
981 try {
982
983 this.memstore.getRowKeyAtOrBefore(state);
984
985
986 for (StoreFile sf : Iterables.reverse(storefiles)) {
987
988 rowAtOrBeforeFromStoreFile(sf, state);
989 }
990 return state.getCandidate();
991 } finally {
992 this.lock.readLock().unlock();
993 }
994 }
995
996
997
998
999
1000
1001
1002 private void rowAtOrBeforeFromStoreFile(final StoreFile f,
1003 final GetClosestRowBeforeTracker state)
1004 throws IOException {
1005 StoreFile.Reader r = f.getReader();
1006 if (r == null) {
1007 LOG.warn("StoreFile " + f + " has a null Reader");
1008 return;
1009 }
1010
1011 byte [] fk = r.getFirstKey();
1012 KeyValue firstKV = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
1013 byte [] lk = r.getLastKey();
1014 KeyValue lastKV = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
1015 KeyValue firstOnRow = state.getTargetKey();
1016 if (this.comparator.compareRows(lastKV, firstOnRow) < 0) {
1017
1018
1019 if (!state.isTargetTable(lastKV)) return;
1020
1021
1022 firstOnRow = new KeyValue(lastKV.getRow(), HConstants.LATEST_TIMESTAMP);
1023 }
1024
1025 HFileScanner scanner = r.getScanner(true, true);
1026
1027 if (!seekToScanner(scanner, firstOnRow, firstKV)) return;
1028
1029
1030 if (walkForwardInSingleRow(scanner, firstOnRow, state)) return;
1031
1032 while (scanner.seekBefore(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(),
1033 firstOnRow.getKeyLength())) {
1034 KeyValue kv = scanner.getKeyValue();
1035 if (!state.isTargetTable(kv)) break;
1036 if (!state.isBetterCandidate(kv)) break;
1037
1038 firstOnRow = new KeyValue(kv.getRow(), HConstants.LATEST_TIMESTAMP);
1039
1040 if (!seekToScanner(scanner, firstOnRow, firstKV)) break;
1041
1042 if (walkForwardInSingleRow(scanner, firstOnRow, state)) break;
1043 }
1044 }
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054 private boolean seekToScanner(final HFileScanner scanner,
1055 final KeyValue firstOnRow,
1056 final KeyValue firstKV)
1057 throws IOException {
1058 KeyValue kv = firstOnRow;
1059
1060 if (this.comparator.compareRows(firstKV, firstOnRow) == 0) kv = firstKV;
1061 int result = scanner.seekTo(kv.getBuffer(), kv.getKeyOffset(),
1062 kv.getKeyLength());
1063 return result >= 0;
1064 }
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076 private boolean walkForwardInSingleRow(final HFileScanner scanner,
1077 final KeyValue firstOnRow,
1078 final GetClosestRowBeforeTracker state)
1079 throws IOException {
1080 boolean foundCandidate = false;
1081 do {
1082 KeyValue kv = scanner.getKeyValue();
1083
1084 if (this.comparator.compareRows(kv, firstOnRow) < 0) continue;
1085
1086 if (state.isTooFar(kv, firstOnRow)) break;
1087 if (state.isExpired(kv)) {
1088 continue;
1089 }
1090
1091 if (state.handle(kv)) {
1092 foundCandidate = true;
1093 break;
1094 }
1095 } while(scanner.next());
1096 return foundCandidate;
1097 }
1098
1099
1100
1101
1102
1103
1104 StoreSize checkSplit(final boolean force) {
1105 this.lock.readLock().lock();
1106 try {
1107
1108 if (this.storefiles.isEmpty()) {
1109 return null;
1110 }
1111 if (!force && (storeSize < this.desiredMaxFileSize)) {
1112 return null;
1113 }
1114
1115 if (this.region.getRegionInfo().isMetaRegion()) {
1116 if (force) {
1117 LOG.warn("Cannot split meta regions in HBase 0.20");
1118 }
1119 return null;
1120 }
1121
1122
1123 boolean splitable = true;
1124 long maxSize = 0L;
1125 StoreFile largestSf = null;
1126 for (StoreFile sf : storefiles) {
1127 if (splitable) {
1128 splitable = !sf.isReference();
1129 if (!splitable) {
1130
1131 if (LOG.isDebugEnabled()) {
1132 LOG.debug(sf + " is not splittable");
1133 }
1134 return null;
1135 }
1136 }
1137 StoreFile.Reader r = sf.getReader();
1138 if (r == null) {
1139 LOG.warn("Storefile " + sf + " Reader is null");
1140 continue;
1141 }
1142 long size = r.length();
1143 if (size > maxSize) {
1144
1145 maxSize = size;
1146 largestSf = sf;
1147 }
1148 }
1149 StoreFile.Reader r = largestSf.getReader();
1150 if (r == null) {
1151 LOG.warn("Storefile " + largestSf + " Reader is null");
1152 return null;
1153 }
1154
1155
1156
1157 byte [] midkey = r.midkey();
1158 if (midkey != null) {
1159 KeyValue mk = KeyValue.createKeyValueFromKey(midkey, 0, midkey.length);
1160 byte [] fk = r.getFirstKey();
1161 KeyValue firstKey = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
1162 byte [] lk = r.getLastKey();
1163 KeyValue lastKey = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
1164
1165
1166 if (this.comparator.compareRows(mk, firstKey) == 0 &&
1167 this.comparator.compareRows(mk, lastKey) == 0) {
1168 if (LOG.isDebugEnabled()) {
1169 LOG.debug("cannot split because midkey is the same as first or " +
1170 "last row");
1171 }
1172 return null;
1173 }
1174 return new StoreSize(maxSize, mk.getRow());
1175 }
1176 } catch(IOException e) {
1177 LOG.warn("Failed getting store size for " + this.storeNameStr, e);
1178 } finally {
1179 this.lock.readLock().unlock();
1180 }
1181 return null;
1182 }
1183
1184
1185 public long getSize() {
1186 return storeSize;
1187 }
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197 public KeyValueScanner getScanner(Scan scan,
1198 final NavigableSet<byte []> targetCols) throws IOException {
1199 lock.readLock().lock();
1200 try {
1201 return new StoreScanner(this, scan, targetCols);
1202 } finally {
1203 lock.readLock().unlock();
1204 }
1205 }
1206
1207 @Override
1208 public String toString() {
1209 return this.storeNameStr;
1210 }
1211
1212
1213
1214
1215 int getStorefilesCount() {
1216 return this.storefiles.size();
1217 }
1218
1219
1220
1221
1222 long getStorefilesSize() {
1223 long size = 0;
1224 for (StoreFile s: storefiles) {
1225 StoreFile.Reader r = s.getReader();
1226 if (r == null) {
1227 LOG.warn("StoreFile " + s + " has a null Reader");
1228 continue;
1229 }
1230 size += r.length();
1231 }
1232 return size;
1233 }
1234
1235
1236
1237
1238 long getStorefilesIndexSize() {
1239 long size = 0;
1240 for (StoreFile s: storefiles) {
1241 StoreFile.Reader r = s.getReader();
1242 if (r == null) {
1243 LOG.warn("StoreFile " + s + " has a null Reader");
1244 continue;
1245 }
1246 size += r.indexSize();
1247 }
1248 return size;
1249 }
1250
1251
1252
1253
1254
1255 static class StoreSize {
1256 private final long size;
1257 private final byte [] row;
1258
1259 StoreSize(long size, byte [] row) {
1260 this.size = size;
1261 this.row = row;
1262 }
1263
1264 long getSize() {
1265 return size;
1266 }
1267
1268 byte [] getSplitRow() {
1269 return this.row;
1270 }
1271 }
1272
1273 HRegion getHRegion() {
1274 return this.region;
1275 }
1276
1277 HRegionInfo getHRegionInfo() {
1278 return this.region.regionInfo;
1279 }
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295 public long updateColumnValue(byte [] row, byte [] f,
1296 byte [] qualifier, long newValue)
1297 throws IOException {
1298
1299 this.lock.readLock().lock();
1300 try {
1301 long now = EnvironmentEdgeManager.currentTimeMillis();
1302
1303 return this.memstore.updateColumnValue(row,
1304 f,
1305 qualifier,
1306 newValue,
1307 now);
1308
1309 } finally {
1310 this.lock.readLock().unlock();
1311 }
1312 }
1313
1314 public StoreFlusher getStoreFlusher(long cacheFlushId) {
1315 return new StoreFlusherImpl(cacheFlushId);
1316 }
1317
1318 private class StoreFlusherImpl implements StoreFlusher {
1319
1320 private long cacheFlushId;
1321 private SortedSet<KeyValue> snapshot;
1322 private StoreFile storeFile;
1323 private TimeRangeTracker snapshotTimeRangeTracker;
1324
1325 private StoreFlusherImpl(long cacheFlushId) {
1326 this.cacheFlushId = cacheFlushId;
1327 }
1328
1329 @Override
1330 public void prepare() {
1331 memstore.snapshot();
1332 this.snapshot = memstore.getSnapshot();
1333 this.snapshotTimeRangeTracker = memstore.getSnapshotTimeRangeTracker();
1334 }
1335
1336 @Override
1337 public void flushCache() throws IOException {
1338 storeFile = Store.this.flushCache(cacheFlushId, snapshot, snapshotTimeRangeTracker);
1339 }
1340
1341 @Override
1342 public boolean commit() throws IOException {
1343 if (storeFile == null) {
1344 return false;
1345 }
1346
1347
1348 return Store.this.updateStorefiles(storeFile, snapshot);
1349 }
1350 }
1351
1352
1353
1354
1355
1356
1357 public boolean hasTooManyStoreFiles() {
1358 return this.storefiles.size() > this.compactionThreshold;
1359 }
1360
1361 public static final long FIXED_OVERHEAD = ClassSize.align(
1362 ClassSize.OBJECT + (14 * ClassSize.REFERENCE) +
1363 (4 * Bytes.SIZEOF_LONG) + (3 * Bytes.SIZEOF_INT) + (Bytes.SIZEOF_BOOLEAN * 2));
1364
1365 public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
1366 ClassSize.OBJECT + ClassSize.REENTRANT_LOCK +
1367 ClassSize.CONCURRENT_SKIPLISTMAP +
1368 ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + ClassSize.OBJECT);
1369
1370 @Override
1371 public long heapSize() {
1372 return DEEP_OVERHEAD + this.memstore.heapSize();
1373 }
1374 }