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