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
639 int tail = Math.min(countOfFiles, 2);
640 for (point = 0; point < (countOfFiles - tail); point++) {
641 if (((fileSizes[point] < fileSizes[point + 1] * 2) &&
642 (countOfFiles - point) <= maxFilesToCompact)) {
643 break;
644 }
645 skipped += fileSizes[point];
646 }
647 filesToCompact = new ArrayList<StoreFile>(filesToCompact.subList(point,
648 countOfFiles));
649 if (filesToCompact.size() <= 1) {
650 if (LOG.isDebugEnabled()) {
651 LOG.debug("Skipped compaction of 1 file; compaction size of " +
652 this.storeNameStr + ": " +
653 StringUtils.humanReadableInt(totalSize) + "; Skipped " + point +
654 " files, size: " + skipped);
655 }
656 return checkSplit(forceSplit);
657 }
658 if (LOG.isDebugEnabled()) {
659 LOG.debug("Compaction size of " + this.storeNameStr + ": " +
660 StringUtils.humanReadableInt(totalSize) + "; Skipped " + point +
661 " file(s), size: " + skipped);
662 }
663 }
664
665
666 LOG.info("Started compaction of " + filesToCompact.size() + " file(s) in " +
667 this.storeNameStr + " of " + this.region.getRegionInfo().getRegionNameAsString() +
668 (references? ", hasReferences=true,": " ") + " into " +
669 region.getTmpDir() + ", sequenceid=" + maxId);
670 StoreFile.Writer writer = compact(filesToCompact, majorcompaction, maxId);
671
672 StoreFile sf = completeCompaction(filesToCompact, writer);
673 if (LOG.isInfoEnabled()) {
674 LOG.info("Completed" + (majorcompaction? " major ": " ") +
675 "compaction of " + filesToCompact.size() + " file(s) in " +
676 this.storeNameStr + " of " + this.region.getRegionInfo().getRegionNameAsString() +
677 "; new storefile is " + (sf == null? "none": sf.toString()) +
678 "; store size is " + StringUtils.humanReadableInt(storeSize));
679 }
680 }
681 return checkSplit(forceSplit);
682 }
683
684
685
686
687
688 private boolean hasReferences(Collection<StoreFile> files) {
689 if (files != null && files.size() > 0) {
690 for (StoreFile hsf: files) {
691 if (hsf.isReference()) {
692 return true;
693 }
694 }
695 }
696 return false;
697 }
698
699
700
701
702
703
704
705
706 private static long getLowestTimestamp(FileSystem fs, Path dir) throws IOException {
707 FileStatus[] stats = fs.listStatus(dir);
708 if (stats == null || stats.length == 0) {
709 return 0l;
710 }
711 long lowTimestamp = Long.MAX_VALUE;
712 for (int i = 0; i < stats.length; i++) {
713 long timestamp = stats[i].getModificationTime();
714 if (timestamp < lowTimestamp){
715 lowTimestamp = timestamp;
716 }
717 }
718 return lowTimestamp;
719 }
720
721
722
723
724 boolean isMajorCompaction() throws IOException {
725 return isMajorCompaction(storefiles);
726 }
727
728
729
730
731
732 private boolean isMajorCompaction(final List<StoreFile> filesToCompact) throws IOException {
733 boolean result = false;
734 if (filesToCompact == null || filesToCompact.isEmpty() ||
735 majorCompactionTime == 0) {
736 return result;
737 }
738 long lowTimestamp = getLowestTimestamp(fs,
739 filesToCompact.get(0).getPath().getParent());
740 long now = System.currentTimeMillis();
741 if (lowTimestamp > 0l && lowTimestamp < (now - this.majorCompactionTime)) {
742
743 long elapsedTime = now - lowTimestamp;
744 if (filesToCompact.size() == 1 &&
745 filesToCompact.get(0).isMajorCompaction() &&
746 (this.ttl == HConstants.FOREVER || elapsedTime < this.ttl)) {
747 if (LOG.isDebugEnabled()) {
748 LOG.debug("Skipping major compaction of " + this.storeNameStr +
749 " because one (major) compacted file only and elapsedTime " +
750 elapsedTime + "ms is < ttl=" + this.ttl);
751 }
752 } else {
753 if (LOG.isDebugEnabled()) {
754 LOG.debug("Major compaction triggered on store " + this.storeNameStr +
755 "; time since last major compaction " + (now - lowTimestamp) + "ms");
756 }
757 result = true;
758 }
759 }
760 return result;
761 }
762
763
764
765
766
767
768
769
770
771
772
773 private StoreFile.Writer compact(final List<StoreFile> filesToCompact,
774 final boolean majorCompaction, final long maxId)
775 throws IOException {
776
777 int maxKeyCount = 0;
778 for (StoreFile file : filesToCompact) {
779 StoreFile.Reader r = file.getReader();
780 if (r != null) {
781
782
783 maxKeyCount += (r.getBloomFilterType() == family.getBloomFilterType())
784 ? r.getFilterEntries() : r.getEntries();
785 }
786 }
787
788
789 List<StoreFileScanner> scanners = StoreFileScanner
790 .getScannersForStoreFiles(filesToCompact, false, false);
791
792
793
794 StoreFile.Writer writer = null;
795 try {
796 if (majorCompaction) {
797 InternalScanner scanner = null;
798 try {
799 Scan scan = new Scan();
800 scan.setMaxVersions(family.getMaxVersions());
801 scanner = new StoreScanner(this, scan, scanners);
802
803
804 ArrayList<KeyValue> kvs = new ArrayList<KeyValue>();
805 while (scanner.next(kvs)) {
806
807 for (KeyValue kv : kvs) {
808 if (writer == null) {
809 writer = createWriterInTmp(maxKeyCount);
810 }
811 writer.append(kv);
812 }
813 kvs.clear();
814 }
815 } finally {
816 if (scanner != null) {
817 scanner.close();
818 }
819 }
820 } else {
821 MinorCompactingStoreScanner scanner = null;
822 try {
823 scanner = new MinorCompactingStoreScanner(this, scanners);
824 writer = createWriterInTmp(maxKeyCount);
825 while (scanner.next(writer)) {
826
827 }
828 } finally {
829 if (scanner != null)
830 scanner.close();
831 }
832 }
833 } finally {
834 if (writer != null) {
835 writer.appendMetadata(maxId, majorCompaction);
836 writer.close();
837 }
838 }
839 return writer;
840 }
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864 private StoreFile completeCompaction(final List<StoreFile> compactedFiles,
865 final StoreFile.Writer compactedFile)
866 throws IOException {
867
868
869 StoreFile result = null;
870 if (compactedFile != null) {
871 Path p = null;
872 try {
873 p = StoreFile.rename(this.fs, compactedFile.getPath(),
874 StoreFile.getRandomFilename(fs, this.homedir));
875 } catch (IOException e) {
876 LOG.error("Failed move of compacted file " + compactedFile.getPath(), e);
877 return null;
878 }
879 result = new StoreFile(this.fs, p, blockcache, this.conf,
880 this.family.getBloomFilterType(), this.inMemory);
881 result.createReader();
882 }
883 this.lock.writeLock().lock();
884 try {
885 try {
886
887
888
889
890
891
892 ArrayList<StoreFile> newStoreFiles = new ArrayList<StoreFile>();
893 for (StoreFile sf : storefiles) {
894 if (!compactedFiles.contains(sf)) {
895 newStoreFiles.add(sf);
896 }
897 }
898
899
900 if (result != null) {
901 newStoreFiles.add(result);
902 }
903
904 this.storefiles = ImmutableList.copyOf(newStoreFiles);
905
906
907 notifyChangedReadersObservers();
908
909 for (StoreFile hsf: compactedFiles) {
910 hsf.deleteReader();
911 }
912 } catch (IOException e) {
913 e = RemoteExceptionHandler.checkIOException(e);
914 LOG.error("Failed replacing compacted files in " + this.storeNameStr +
915 ". Compacted file is " + (result == null? "none": result.toString()) +
916 ". Files replaced " + compactedFiles.toString() +
917 " some of which may have been already removed", e);
918 }
919
920 this.storeSize = 0L;
921 for (StoreFile hsf : this.storefiles) {
922 StoreFile.Reader r = hsf.getReader();
923 if (r == null) {
924 LOG.warn("StoreFile " + hsf + " has a null Reader");
925 continue;
926 }
927 this.storeSize += r.length();
928 }
929 } finally {
930 this.lock.writeLock().unlock();
931 }
932 return result;
933 }
934
935
936
937
938
939
940
941
942 public int getNumberOfstorefiles() {
943 return this.storefiles.size();
944 }
945
946
947
948
949
950 int versionsToReturn(final int wantedVersions) {
951 if (wantedVersions <= 0) {
952 throw new IllegalArgumentException("Number of versions must be > 0");
953 }
954
955 int maxVersions = this.family.getMaxVersions();
956 return wantedVersions > maxVersions ? maxVersions: wantedVersions;
957 }
958
959 static boolean isExpired(final KeyValue key, final long oldestTimestamp) {
960 return key.getTimestamp() < oldestTimestamp;
961 }
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978 KeyValue getRowKeyAtOrBefore(final KeyValue kv) throws IOException {
979 GetClosestRowBeforeTracker state = new GetClosestRowBeforeTracker(
980 this.comparator, kv, this.ttl, this.region.getRegionInfo().isMetaRegion());
981 this.lock.readLock().lock();
982 try {
983
984 this.memstore.getRowKeyAtOrBefore(state);
985
986
987 for (StoreFile sf : Iterables.reverse(storefiles)) {
988
989 rowAtOrBeforeFromStoreFile(sf, state);
990 }
991 return state.getCandidate();
992 } finally {
993 this.lock.readLock().unlock();
994 }
995 }
996
997
998
999
1000
1001
1002
1003 private void rowAtOrBeforeFromStoreFile(final StoreFile f,
1004 final GetClosestRowBeforeTracker state)
1005 throws IOException {
1006 StoreFile.Reader r = f.getReader();
1007 if (r == null) {
1008 LOG.warn("StoreFile " + f + " has a null Reader");
1009 return;
1010 }
1011
1012 byte [] fk = r.getFirstKey();
1013 KeyValue firstKV = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
1014 byte [] lk = r.getLastKey();
1015 KeyValue lastKV = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
1016 KeyValue firstOnRow = state.getTargetKey();
1017 if (this.comparator.compareRows(lastKV, firstOnRow) < 0) {
1018
1019
1020 if (!state.isTargetTable(lastKV)) return;
1021
1022
1023 firstOnRow = new KeyValue(lastKV.getRow(), HConstants.LATEST_TIMESTAMP);
1024 }
1025
1026 HFileScanner scanner = r.getScanner(true, true);
1027
1028 if (!seekToScanner(scanner, firstOnRow, firstKV)) return;
1029
1030
1031 if (walkForwardInSingleRow(scanner, firstOnRow, state)) return;
1032
1033 while (scanner.seekBefore(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(),
1034 firstOnRow.getKeyLength())) {
1035 KeyValue kv = scanner.getKeyValue();
1036 if (!state.isTargetTable(kv)) break;
1037 if (!state.isBetterCandidate(kv)) break;
1038
1039 firstOnRow = new KeyValue(kv.getRow(), HConstants.LATEST_TIMESTAMP);
1040
1041 if (!seekToScanner(scanner, firstOnRow, firstKV)) break;
1042
1043 if (walkForwardInSingleRow(scanner, firstOnRow, state)) break;
1044 }
1045 }
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055 private boolean seekToScanner(final HFileScanner scanner,
1056 final KeyValue firstOnRow,
1057 final KeyValue firstKV)
1058 throws IOException {
1059 KeyValue kv = firstOnRow;
1060
1061 if (this.comparator.compareRows(firstKV, firstOnRow) == 0) kv = firstKV;
1062 int result = scanner.seekTo(kv.getBuffer(), kv.getKeyOffset(),
1063 kv.getKeyLength());
1064 return result >= 0;
1065 }
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077 private boolean walkForwardInSingleRow(final HFileScanner scanner,
1078 final KeyValue firstOnRow,
1079 final GetClosestRowBeforeTracker state)
1080 throws IOException {
1081 boolean foundCandidate = false;
1082 do {
1083 KeyValue kv = scanner.getKeyValue();
1084
1085 if (this.comparator.compareRows(kv, firstOnRow) < 0) continue;
1086
1087 if (state.isTooFar(kv, firstOnRow)) break;
1088 if (state.isExpired(kv)) {
1089 continue;
1090 }
1091
1092 if (state.handle(kv)) {
1093 foundCandidate = true;
1094 break;
1095 }
1096 } while(scanner.next());
1097 return foundCandidate;
1098 }
1099
1100
1101
1102
1103
1104
1105 StoreSize checkSplit(final boolean force) {
1106 this.lock.readLock().lock();
1107 try {
1108
1109 if (this.storefiles.isEmpty()) {
1110 return null;
1111 }
1112 if (!force && (storeSize < this.desiredMaxFileSize)) {
1113 return null;
1114 }
1115
1116 if (this.region.getRegionInfo().isMetaRegion()) {
1117 if (force) {
1118 LOG.warn("Cannot split meta regions in HBase 0.20");
1119 }
1120 return null;
1121 }
1122
1123
1124 boolean splitable = true;
1125 long maxSize = 0L;
1126 StoreFile largestSf = null;
1127 for (StoreFile sf : storefiles) {
1128 if (splitable) {
1129 splitable = !sf.isReference();
1130 if (!splitable) {
1131
1132 if (LOG.isDebugEnabled()) {
1133 LOG.debug(sf + " is not splittable");
1134 }
1135 return null;
1136 }
1137 }
1138 StoreFile.Reader r = sf.getReader();
1139 if (r == null) {
1140 LOG.warn("Storefile " + sf + " Reader is null");
1141 continue;
1142 }
1143 long size = r.length();
1144 if (size > maxSize) {
1145
1146 maxSize = size;
1147 largestSf = sf;
1148 }
1149 }
1150 StoreFile.Reader r = largestSf.getReader();
1151 if (r == null) {
1152 LOG.warn("Storefile " + largestSf + " Reader is null");
1153 return null;
1154 }
1155
1156
1157
1158 byte [] midkey = r.midkey();
1159 if (midkey != null) {
1160 KeyValue mk = KeyValue.createKeyValueFromKey(midkey, 0, midkey.length);
1161 byte [] fk = r.getFirstKey();
1162 KeyValue firstKey = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
1163 byte [] lk = r.getLastKey();
1164 KeyValue lastKey = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
1165
1166
1167 if (this.comparator.compareRows(mk, firstKey) == 0 &&
1168 this.comparator.compareRows(mk, lastKey) == 0) {
1169 if (LOG.isDebugEnabled()) {
1170 LOG.debug("cannot split because midkey is the same as first or " +
1171 "last row");
1172 }
1173 return null;
1174 }
1175 return new StoreSize(maxSize, mk.getRow());
1176 }
1177 } catch(IOException e) {
1178 LOG.warn("Failed getting store size for " + this.storeNameStr, e);
1179 } finally {
1180 this.lock.readLock().unlock();
1181 }
1182 return null;
1183 }
1184
1185
1186 public long getSize() {
1187 return storeSize;
1188 }
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198 public KeyValueScanner getScanner(Scan scan,
1199 final NavigableSet<byte []> targetCols) throws IOException {
1200 lock.readLock().lock();
1201 try {
1202 return new StoreScanner(this, scan, targetCols);
1203 } finally {
1204 lock.readLock().unlock();
1205 }
1206 }
1207
1208 @Override
1209 public String toString() {
1210 return this.storeNameStr;
1211 }
1212
1213
1214
1215
1216 int getStorefilesCount() {
1217 return this.storefiles.size();
1218 }
1219
1220
1221
1222
1223 long getStorefilesSize() {
1224 long size = 0;
1225 for (StoreFile s: storefiles) {
1226 StoreFile.Reader r = s.getReader();
1227 if (r == null) {
1228 LOG.warn("StoreFile " + s + " has a null Reader");
1229 continue;
1230 }
1231 size += r.length();
1232 }
1233 return size;
1234 }
1235
1236
1237
1238
1239 long getStorefilesIndexSize() {
1240 long size = 0;
1241 for (StoreFile s: storefiles) {
1242 StoreFile.Reader r = s.getReader();
1243 if (r == null) {
1244 LOG.warn("StoreFile " + s + " has a null Reader");
1245 continue;
1246 }
1247 size += r.indexSize();
1248 }
1249 return size;
1250 }
1251
1252
1253
1254
1255
1256 static class StoreSize {
1257 private final long size;
1258 private final byte [] row;
1259
1260 StoreSize(long size, byte [] row) {
1261 this.size = size;
1262 this.row = row;
1263 }
1264
1265 long getSize() {
1266 return size;
1267 }
1268
1269 byte [] getSplitRow() {
1270 return this.row;
1271 }
1272 }
1273
1274 HRegion getHRegion() {
1275 return this.region;
1276 }
1277
1278 HRegionInfo getHRegionInfo() {
1279 return this.region.regionInfo;
1280 }
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296 public long updateColumnValue(byte [] row, byte [] f,
1297 byte [] qualifier, long newValue)
1298 throws IOException {
1299
1300 this.lock.readLock().lock();
1301 try {
1302 long now = EnvironmentEdgeManager.currentTimeMillis();
1303
1304 return this.memstore.updateColumnValue(row,
1305 f,
1306 qualifier,
1307 newValue,
1308 now);
1309
1310 } finally {
1311 this.lock.readLock().unlock();
1312 }
1313 }
1314
1315 public StoreFlusher getStoreFlusher(long cacheFlushId) {
1316 return new StoreFlusherImpl(cacheFlushId);
1317 }
1318
1319 private class StoreFlusherImpl implements StoreFlusher {
1320
1321 private long cacheFlushId;
1322 private SortedSet<KeyValue> snapshot;
1323 private StoreFile storeFile;
1324 private TimeRangeTracker snapshotTimeRangeTracker;
1325
1326 private StoreFlusherImpl(long cacheFlushId) {
1327 this.cacheFlushId = cacheFlushId;
1328 }
1329
1330 @Override
1331 public void prepare() {
1332 memstore.snapshot();
1333 this.snapshot = memstore.getSnapshot();
1334 this.snapshotTimeRangeTracker = memstore.getSnapshotTimeRangeTracker();
1335 }
1336
1337 @Override
1338 public void flushCache() throws IOException {
1339 storeFile = Store.this.flushCache(cacheFlushId, snapshot, snapshotTimeRangeTracker);
1340 }
1341
1342 @Override
1343 public boolean commit() throws IOException {
1344 if (storeFile == null) {
1345 return false;
1346 }
1347
1348
1349 return Store.this.updateStorefiles(storeFile, snapshot);
1350 }
1351 }
1352
1353
1354
1355
1356
1357
1358 public boolean hasTooManyStoreFiles() {
1359 return this.storefiles.size() > this.compactionThreshold;
1360 }
1361
1362 public static final long FIXED_OVERHEAD = ClassSize.align(
1363 ClassSize.OBJECT + (14 * ClassSize.REFERENCE) +
1364 (4 * Bytes.SIZEOF_LONG) + (3 * Bytes.SIZEOF_INT) + (Bytes.SIZEOF_BOOLEAN * 2));
1365
1366 public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
1367 ClassSize.OBJECT + ClassSize.REENTRANT_LOCK +
1368 ClassSize.CONCURRENT_SKIPLISTMAP +
1369 ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + ClassSize.OBJECT);
1370
1371 @Override
1372 public long heapSize() {
1373 return DEEP_OVERHEAD + this.memstore.heapSize();
1374 }
1375 }