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.Random;
30 import java.util.Set;
31 import java.util.SortedSet;
32 import java.util.concurrent.Callable;
33 import java.util.concurrent.CompletionService;
34 import java.util.concurrent.ConcurrentHashMap;
35 import java.util.concurrent.ExecutionException;
36 import java.util.concurrent.ExecutorCompletionService;
37 import java.util.concurrent.Future;
38 import java.util.concurrent.ThreadPoolExecutor;
39 import java.util.concurrent.atomic.AtomicLong;
40 import java.util.concurrent.locks.ReentrantReadWriteLock;
41
42 import org.apache.commons.logging.Log;
43 import org.apache.commons.logging.LogFactory;
44 import org.apache.hadoop.conf.Configuration;
45 import org.apache.hadoop.fs.FileStatus;
46 import org.apache.hadoop.fs.FileSystem;
47 import org.apache.hadoop.fs.FileUtil;
48 import org.apache.hadoop.fs.Path;
49 import org.apache.hadoop.hbase.HBaseFileSystem;
50 import org.apache.hadoop.hbase.HColumnDescriptor;
51 import org.apache.hadoop.hbase.HConstants;
52 import org.apache.hadoop.hbase.HRegionInfo;
53 import org.apache.hadoop.hbase.KeyValue;
54 import org.apache.hadoop.hbase.KeyValue.KVComparator;
55 import org.apache.hadoop.hbase.RemoteExceptionHandler;
56 import org.apache.hadoop.hbase.backup.HFileArchiver;
57 import org.apache.hadoop.hbase.client.Scan;
58 import org.apache.hadoop.hbase.fs.HFileSystem;
59 import org.apache.hadoop.hbase.io.HFileLink;
60 import org.apache.hadoop.hbase.io.HeapSize;
61 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
62 import org.apache.hadoop.hbase.io.hfile.Compression;
63 import org.apache.hadoop.hbase.io.hfile.HFile;
64 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
65 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl;
66 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
67 import org.apache.hadoop.hbase.io.hfile.InvalidHFileException;
68 import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder;
69 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
70 import org.apache.hadoop.hbase.regionserver.compactions.CompactSelection;
71 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
72 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
73 import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
74 import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
75 import org.apache.hadoop.hbase.util.*;
76 import org.apache.hadoop.util.StringUtils;
77
78 import com.google.common.base.Preconditions;
79 import com.google.common.base.Predicate;
80 import com.google.common.collect.Collections2;
81 import com.google.common.collect.ImmutableList;
82 import com.google.common.collect.Lists;
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107 public class Store extends SchemaConfigured implements HeapSize {
108 static final Log LOG = LogFactory.getLog(Store.class);
109
110 public static final String BLOCKING_STOREFILES_KEY = "hbase.hstore.blockingStoreFiles";
111 public static final int DEFAULT_BLOCKING_STOREFILE_COUNT = 7;
112
113 protected final MemStore memstore;
114
115 private final Path homedir;
116 private final HRegion region;
117 private final HColumnDescriptor family;
118 final FileSystem fs;
119 final Configuration conf;
120 final CacheConfig cacheConf;
121
122 private long ttl;
123 private final int minFilesToCompact;
124 private final int maxFilesToCompact;
125 private final long minCompactSize;
126 private final long maxCompactSize;
127 private final float minStoreFileLocalitySkipCompact;
128 private long lastCompactSize = 0;
129 volatile boolean forceMajor = false;
130
131 static int closeCheckInterval = 0;
132 private final int blockingStoreFileCount;
133 private volatile long storeSize = 0L;
134 private volatile long totalUncompressedBytes = 0L;
135 private final Object flushLock = new Object();
136 final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
137 private final boolean verifyBulkLoads;
138
139 private long blockingFileCount;
140
141
142
143
144 public static final int PRIORITY_USER = 1;
145 public static final int NO_PRIORITY = Integer.MIN_VALUE;
146
147
148
149
150
151
152
153 private volatile ImmutableList<StoreFile> storefiles = null;
154
155 List<StoreFile> filesCompacting = Lists.newArrayList();
156
157
158 private final Set<ChangedReadersObserver> changedReaderObservers =
159 Collections.newSetFromMap(new ConcurrentHashMap<ChangedReadersObserver, Boolean>());
160
161 private final int blocksize;
162 private HFileDataBlockEncoder dataBlockEncoder;
163
164
165 private ChecksumType checksumType;
166 private int bytesPerChecksum;
167
168
169 final KeyValue.KVComparator comparator;
170
171 private final Compactor compactor;
172
173 private static final int DEFAULT_FLUSH_RETRIES_NUMBER = 10;
174 private static int flush_retries_number;
175 private static int pauseTime;
176
177
178
179
180
181
182
183
184
185
186
187
188 protected Store(Path basedir, HRegion region, HColumnDescriptor family,
189 FileSystem fs, Configuration confParam)
190 throws IOException {
191 super(new CompoundConfiguration().add(confParam).add(
192 family.getValues()), region.getTableDesc().getNameAsString(),
193 Bytes.toString(family.getName()));
194 HRegionInfo info = region.getRegionInfo();
195 this.fs = fs;
196 Path p = getStoreHomedir(basedir, info.getEncodedName(), family.getName());
197 this.homedir = createStoreHomeDir(this.fs, p);
198 this.region = region;
199 this.family = family;
200
201 this.conf = new CompoundConfiguration().add(confParam).add(family.getValues());
202 this.blocksize = family.getBlocksize();
203
204 this.dataBlockEncoder =
205 new HFileDataBlockEncoderImpl(family.getDataBlockEncodingOnDisk(),
206 family.getDataBlockEncoding());
207
208 this.comparator = info.getComparator();
209
210 this.ttl = family.getTimeToLive();
211 if (ttl == HConstants.FOREVER) {
212
213 ttl = Long.MAX_VALUE;
214 } else if (ttl == -1) {
215 ttl = Long.MAX_VALUE;
216 } else {
217
218 this.ttl *= 1000;
219 }
220
221 long timeToPurgeDeletes =
222 Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);
223 LOG.info("time to purge deletes set to " + timeToPurgeDeletes +
224 "ms in store " + this);
225 scanInfo = new ScanInfo(family, ttl, timeToPurgeDeletes, this.comparator);
226 this.memstore = new MemStore(conf, this.comparator);
227
228
229 this.minFilesToCompact = Math.max(2,
230 conf.getInt("hbase.hstore.compaction.min",
231
232
233 LOG.info("hbase.hstore.compaction.min = " + this.minFilesToCompact);
234
235
236 this.cacheConf = new CacheConfig(conf, family);
237 this.blockingStoreFileCount =
238 conf.getInt("hbase.hstore.blockingStoreFiles", 7);
239
240 this.maxFilesToCompact = conf.getInt("hbase.hstore.compaction.max", 10);
241 this.minCompactSize = conf.getLong("hbase.hstore.compaction.min.size",
242 this.region.memstoreFlushSize);
243 this.maxCompactSize
244 = conf.getLong("hbase.hstore.compaction.max.size", Long.MAX_VALUE);
245 this.minStoreFileLocalitySkipCompact
246 = conf.getFloat("hbase.hstore.min.locality.to.skip.major.compact", 0f);
247
248 this.verifyBulkLoads = conf.getBoolean("hbase.hstore.bulkload.verify", false);
249
250 this.blockingFileCount =
251 conf.getInt(BLOCKING_STOREFILES_KEY, DEFAULT_BLOCKING_STOREFILE_COUNT);
252
253 if (Store.closeCheckInterval == 0) {
254 Store.closeCheckInterval = conf.getInt(
255 "hbase.hstore.close.check.interval", 10*1000*1000
256 }
257 this.storefiles = sortAndClone(loadStoreFiles());
258
259
260 this.checksumType = getChecksumType(conf);
261
262 this.bytesPerChecksum = getBytesPerChecksum(conf);
263
264 this.compactor = new Compactor(this.conf);
265 if (Store.flush_retries_number == 0) {
266 Store.flush_retries_number = conf.getInt(
267 "hbase.hstore.flush.retries.number", DEFAULT_FLUSH_RETRIES_NUMBER);
268 Store.pauseTime = conf.getInt(HConstants.HBASE_SERVER_PAUSE,
269 HConstants.DEFAULT_HBASE_SERVER_PAUSE);
270 if (Store.flush_retries_number <= 0) {
271 throw new IllegalArgumentException(
272 "hbase.hstore.flush.retries.number must be > 0, not "
273 + Store.flush_retries_number);
274 }
275 }
276 }
277
278
279
280
281
282 long getTTL(final HColumnDescriptor family) {
283
284 long ttl = family.getTimeToLive();
285 if (ttl == HConstants.FOREVER) {
286
287 ttl = Long.MAX_VALUE;
288 } else if (ttl == -1) {
289 ttl = Long.MAX_VALUE;
290 } else {
291
292 ttl *= 1000;
293 }
294 return ttl;
295 }
296
297
298
299
300
301
302
303
304 Path createStoreHomeDir(final FileSystem fs,
305 final Path homedir) throws IOException {
306 if (!fs.exists(homedir) && !HBaseFileSystem.makeDirOnFileSystem(fs, homedir)) {
307 throw new IOException("Failed create of: " + homedir.toString());
308 }
309 return homedir;
310 }
311
312 FileSystem getFileSystem() {
313 return this.fs;
314 }
315
316
317
318
319
320
321 public static int getBytesPerChecksum(Configuration conf) {
322 return conf.getInt(HConstants.BYTES_PER_CHECKSUM,
323 HFile.DEFAULT_BYTES_PER_CHECKSUM);
324 }
325
326
327
328
329
330
331 public static ChecksumType getChecksumType(Configuration conf) {
332 String checksumName = conf.get(HConstants.CHECKSUM_TYPE_NAME);
333 if (checksumName == null) {
334 return HFile.DEFAULT_CHECKSUM_TYPE;
335 } else {
336 return ChecksumType.nameToType(checksumName);
337 }
338 }
339
340 public HColumnDescriptor getFamily() {
341 return this.family;
342 }
343
344
345
346
347 long getMaxSequenceId() {
348 return StoreFile.getMaxSequenceIdInList(this.getStorefiles());
349 }
350
351
352
353
354 public long getMaxMemstoreTS() {
355 return StoreFile.getMaxMemstoreTSInList(this.getStorefiles());
356 }
357
358
359
360
361
362
363
364 public static Path getStoreHomedir(final Path tabledir,
365 final String encodedName, final byte [] family) {
366 return getStoreHomedir(tabledir, encodedName, Bytes.toString(family));
367 }
368
369 public long getFlushableSize() {
370 return this.memstore.getFlushableSize();
371 }
372
373
374
375
376
377
378
379
380 public static Path getStoreHomedir(final Path tabledir,
381 final String encodedName, final String family) {
382 return new Path(tabledir, new Path(encodedName, new Path(family)));
383 }
384
385
386
387
388
389
390 public static Path getStoreHomedir(final Path parentRegionDirectory, final byte[] family) {
391 return new Path(parentRegionDirectory, new Path(Bytes.toString(family)));
392 }
393
394
395
396
397
398 Path getHomedir() {
399 return homedir;
400 }
401
402
403
404
405 public HFileDataBlockEncoder getDataBlockEncoder() {
406 return dataBlockEncoder;
407 }
408
409
410
411
412
413 void setDataBlockEncoderInTest(HFileDataBlockEncoder blockEncoder) {
414 this.dataBlockEncoder = blockEncoder;
415 }
416
417 FileStatus [] getStoreFiles() throws IOException {
418 return FSUtils.listStatus(this.fs, this.homedir, null);
419 }
420
421
422
423
424
425
426 private List<StoreFile> loadStoreFiles() throws IOException {
427 ArrayList<StoreFile> results = new ArrayList<StoreFile>();
428 FileStatus files[] = getStoreFiles();
429
430 if (files == null || files.length == 0) {
431 return results;
432 }
433
434 ThreadPoolExecutor storeFileOpenerThreadPool =
435 this.region.getStoreFileOpenAndCloseThreadPool("StoreFileOpenerThread-" +
436 this.family.getNameAsString());
437 CompletionService<StoreFile> completionService =
438 new ExecutorCompletionService<StoreFile>(storeFileOpenerThreadPool);
439
440 int totalValidStoreFile = 0;
441 for (int i = 0; i < files.length; i++) {
442
443 if (files[i].isDir()) {
444 continue;
445 }
446 final Path p = files[i].getPath();
447
448
449
450 if (!HFileLink.isHFileLink(p) && this.fs.getFileStatus(p).getLen() <= 0) {
451 LOG.warn("Skipping " + p + " because its empty. HBASE-646 DATA LOSS?");
452 continue;
453 }
454
455
456 completionService.submit(new Callable<StoreFile>() {
457 public StoreFile call() throws IOException {
458 StoreFile storeFile = new StoreFile(fs, p, conf, cacheConf,
459 family.getBloomFilterType(), dataBlockEncoder);
460 passSchemaMetricsTo(storeFile);
461 storeFile.createReader();
462 return storeFile;
463 }
464 });
465 totalValidStoreFile++;
466 }
467
468 IOException ioe = null;
469 try {
470 for (int i = 0; i < totalValidStoreFile; i++) {
471 try {
472 Future<StoreFile> future = completionService.take();
473 StoreFile storeFile = future.get();
474 long length = storeFile.getReader().length();
475 this.storeSize += length;
476 this.totalUncompressedBytes +=
477 storeFile.getReader().getTotalUncompressedBytes();
478 if (LOG.isDebugEnabled()) {
479 LOG.debug("loaded " + storeFile.toStringDetailed());
480 }
481 results.add(storeFile);
482 } catch (InterruptedException e) {
483 if (ioe == null) ioe = new InterruptedIOException(e.getMessage());
484 } catch (ExecutionException e) {
485 if (ioe == null) ioe = new IOException(e.getCause());
486 }
487 }
488 } finally {
489 storeFileOpenerThreadPool.shutdownNow();
490 }
491 if (ioe != null) {
492
493 for (StoreFile file : results) {
494 try {
495 if (file != null) file.closeReader(true);
496 } catch (IOException e) {
497 LOG.warn(e.getMessage());
498 }
499 }
500 throw ioe;
501 }
502
503 return results;
504 }
505
506
507
508
509
510
511
512 protected long add(final KeyValue kv) {
513 lock.readLock().lock();
514 try {
515 return this.memstore.add(kv);
516 } finally {
517 lock.readLock().unlock();
518 }
519 }
520
521
522
523
524 public long timeOfOldestEdit() {
525 return memstore.timeOfOldestEdit();
526 }
527
528
529
530
531
532
533
534 protected long delete(final KeyValue kv) {
535 lock.readLock().lock();
536 try {
537 return this.memstore.delete(kv);
538 } finally {
539 lock.readLock().unlock();
540 }
541 }
542
543
544
545
546
547
548
549
550 protected void rollback(final KeyValue kv) {
551 lock.readLock().lock();
552 try {
553 this.memstore.rollback(kv);
554 } finally {
555 lock.readLock().unlock();
556 }
557 }
558
559
560
561
562 public List<StoreFile> getStorefiles() {
563 return this.storefiles;
564 }
565
566
567
568
569
570 void assertBulkLoadHFileOk(Path srcPath) throws IOException {
571 HFile.Reader reader = null;
572 try {
573 LOG.info("Validating hfile at " + srcPath + " for inclusion in "
574 + "store " + this + " region " + this.region);
575 reader = HFile.createReader(srcPath.getFileSystem(conf),
576 srcPath, cacheConf);
577 reader.loadFileInfo();
578
579 byte[] firstKey = reader.getFirstRowKey();
580 byte[] lk = reader.getLastKey();
581 byte[] lastKey =
582 (lk == null) ? null :
583 KeyValue.createKeyValueFromKey(lk).getRow();
584
585 LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey) +
586 " last=" + Bytes.toStringBinary(lastKey));
587 LOG.debug("Region bounds: first=" +
588 Bytes.toStringBinary(region.getStartKey()) +
589 " last=" + Bytes.toStringBinary(region.getEndKey()));
590
591 HRegionInfo hri = region.getRegionInfo();
592 if (!hri.containsRange(firstKey, lastKey)) {
593 throw new WrongRegionException(
594 "Bulk load file " + srcPath.toString() + " does not fit inside region "
595 + this.region);
596 }
597
598 if (verifyBulkLoads) {
599 KeyValue prevKV = null;
600 HFileScanner scanner = reader.getScanner(false, false, false);
601 scanner.seekTo();
602 do {
603 KeyValue kv = scanner.getKeyValue();
604 if (prevKV != null) {
605 if (Bytes.compareTo(prevKV.getBuffer(), prevKV.getRowOffset(),
606 prevKV.getRowLength(), kv.getBuffer(), kv.getRowOffset(),
607 kv.getRowLength()) > 0) {
608 throw new InvalidHFileException("Previous row is greater than"
609 + " current row: path=" + srcPath + " previous="
610 + Bytes.toStringBinary(prevKV.getKey()) + " current="
611 + Bytes.toStringBinary(kv.getKey()));
612 }
613 if (Bytes.compareTo(prevKV.getBuffer(), prevKV.getFamilyOffset(),
614 prevKV.getFamilyLength(), kv.getBuffer(), kv.getFamilyOffset(),
615 kv.getFamilyLength()) != 0) {
616 throw new InvalidHFileException("Previous key had different"
617 + " family compared to current key: path=" + srcPath
618 + " previous=" + Bytes.toStringBinary(prevKV.getFamily())
619 + " current=" + Bytes.toStringBinary(kv.getFamily()));
620 }
621 }
622 prevKV = kv;
623 } while (scanner.next());
624 }
625 } finally {
626 if (reader != null) reader.close();
627 }
628 }
629
630
631
632
633
634
635 public void bulkLoadHFile(String srcPathStr, long seqNum) throws IOException {
636 Path srcPath = new Path(srcPathStr);
637
638
639 FileSystem srcFs = srcPath.getFileSystem(conf);
640 FileSystem desFs = fs instanceof HFileSystem ? ((HFileSystem)fs).getBackingFs() : fs;
641
642
643
644
645 if (!FSHDFSUtils.isSameHdfs(conf, srcFs, desFs)) {
646 LOG.info("File " + srcPath + " on different filesystem than " +
647 "destination store - moving to this filesystem.");
648 Path tmpPath = getTmpPath();
649 FileUtil.copy(srcFs, srcPath, fs, tmpPath, false, conf);
650 LOG.info("Copied to temporary path on dst filesystem: " + tmpPath);
651 srcPath = tmpPath;
652 }
653
654 Path dstPath =
655 StoreFile.getRandomFilename(fs, homedir, (seqNum == -1) ? null : "_SeqId_" + seqNum + "_");
656 LOG.debug("Renaming bulk load file " + srcPath + " to " + dstPath);
657 StoreFile.rename(fs, srcPath, dstPath);
658
659 StoreFile sf = new StoreFile(fs, dstPath, this.conf, this.cacheConf,
660 this.family.getBloomFilterType(), this.dataBlockEncoder);
661 passSchemaMetricsTo(sf);
662
663 StoreFile.Reader r = sf.createReader();
664 this.storeSize += r.length();
665 this.totalUncompressedBytes += r.getTotalUncompressedBytes();
666
667 LOG.info("Moved hfile " + srcPath + " into store directory " +
668 homedir + " - updating store file list.");
669
670
671 this.lock.writeLock().lock();
672 try {
673 ArrayList<StoreFile> newFiles = new ArrayList<StoreFile>(storefiles);
674 newFiles.add(sf);
675 this.storefiles = sortAndClone(newFiles);
676 } finally {
677
678
679
680
681
682 this.lock.writeLock().unlock();
683 }
684 notifyChangedReadersObservers();
685 LOG.info("Successfully loaded store file " + srcPath
686 + " into store " + this + " (new location: " + dstPath + ")");
687 }
688
689
690
691
692
693
694 private Path getTmpPath() throws IOException {
695 return StoreFile.getRandomFilename(
696 fs, region.getTmpDir());
697 }
698
699
700
701
702
703
704
705
706
707 ImmutableList<StoreFile> close() throws IOException {
708 this.lock.writeLock().lock();
709 try {
710 ImmutableList<StoreFile> result = storefiles;
711
712
713 storefiles = ImmutableList.of();
714
715 if (!result.isEmpty()) {
716
717 ThreadPoolExecutor storeFileCloserThreadPool = this.region
718 .getStoreFileOpenAndCloseThreadPool("StoreFileCloserThread-"
719 + this.family.getNameAsString());
720
721
722 CompletionService<Void> completionService =
723 new ExecutorCompletionService<Void>(storeFileCloserThreadPool);
724 for (final StoreFile f : result) {
725 completionService.submit(new Callable<Void>() {
726 public Void call() throws IOException {
727 f.closeReader(true);
728 return null;
729 }
730 });
731 }
732
733 IOException ioe = null;
734 try {
735 for (int i = 0; i < result.size(); i++) {
736 try {
737 Future<Void> future = completionService.take();
738 future.get();
739 } catch (InterruptedException e) {
740 if (ioe == null) {
741 ioe = new InterruptedIOException();
742 ioe.initCause(e);
743 }
744 } catch (ExecutionException e) {
745 if (ioe == null) ioe = new IOException(e.getCause());
746 }
747 }
748 } finally {
749 storeFileCloserThreadPool.shutdownNow();
750 }
751 if (ioe != null) throw ioe;
752 }
753 LOG.info("Closed " + this);
754 return result;
755 } finally {
756 this.lock.writeLock().unlock();
757 }
758 }
759
760
761
762
763
764 void snapshot() {
765 this.lock.writeLock().lock();
766 try {
767 this.memstore.snapshot();
768 } finally {
769 this.lock.writeLock().unlock();
770 }
771 }
772
773
774
775
776
777
778
779
780
781
782
783
784 protected Path flushCache(final long logCacheFlushId,
785 SortedSet<KeyValue> snapshot,
786 TimeRangeTracker snapshotTimeRangeTracker,
787 AtomicLong flushedSize,
788 MonitoredTask status) throws IOException {
789
790
791
792
793
794 IOException lastException = null;
795 for (int i = 0; i < Store.flush_retries_number; i++) {
796 try {
797 Path pathName = internalFlushCache(snapshot, logCacheFlushId,
798 snapshotTimeRangeTracker, flushedSize, status);
799 try {
800
801 if (pathName != null) {
802 validateStoreFile(pathName);
803 }
804 return pathName;
805 } catch (Exception e) {
806 LOG.warn("Failed validating store file " + pathName
807 + ", retrying num=" + i, e);
808 if (e instanceof IOException) {
809 lastException = (IOException) e;
810 } else {
811 lastException = new IOException(e);
812 }
813 }
814 } catch (IOException e) {
815 LOG.warn("Failed flushing store file, retrying num=" + i, e);
816 lastException = e;
817 }
818 if (lastException != null && i < (flush_retries_number - 1)) {
819 try {
820 Thread.sleep(pauseTime);
821 } catch (InterruptedException e) {
822 IOException iie = new InterruptedIOException();
823 iie.initCause(e);
824 throw iie;
825 }
826 }
827 }
828 throw lastException;
829 }
830
831
832
833
834
835
836
837
838
839 private Path internalFlushCache(final SortedSet<KeyValue> set,
840 final long logCacheFlushId,
841 TimeRangeTracker snapshotTimeRangeTracker,
842 AtomicLong flushedSize,
843 MonitoredTask status)
844 throws IOException {
845 StoreFile.Writer writer;
846
847 long smallestReadPoint = region.getSmallestReadPoint();
848 long flushed = 0;
849 Path pathName;
850
851 if (set.size() == 0) {
852 return null;
853 }
854
855
856
857 InternalScanner scanner = null;
858 KeyValueScanner memstoreScanner = new CollectionBackedScanner(set, this.comparator);
859 if (getHRegion().getCoprocessorHost() != null) {
860 scanner = getHRegion().getCoprocessorHost().preFlushScannerOpen(this, memstoreScanner);
861 }
862 if (scanner == null) {
863 Scan scan = new Scan();
864 scan.setMaxVersions(scanInfo.getMaxVersions());
865 scanner = new StoreScanner(this, scanInfo, scan,
866 Collections.singletonList(memstoreScanner), ScanType.MINOR_COMPACT,
867 this.region.getSmallestReadPoint(), HConstants.OLDEST_TIMESTAMP);
868 }
869 if (getHRegion().getCoprocessorHost() != null) {
870 InternalScanner cpScanner =
871 getHRegion().getCoprocessorHost().preFlush(this, scanner);
872
873 if (cpScanner == null) {
874 return null;
875 }
876 scanner = cpScanner;
877 }
878 try {
879 int compactionKVMax = conf.getInt(HConstants.COMPACTION_KV_MAX, 10);
880
881
882
883 synchronized (flushLock) {
884 status.setStatus("Flushing " + this + ": creating writer");
885
886 writer = createWriterInTmp(set.size());
887 writer.setTimeRangeTracker(snapshotTimeRangeTracker);
888 pathName = writer.getPath();
889 try {
890 List<KeyValue> kvs = new ArrayList<KeyValue>();
891 boolean hasMore;
892 do {
893 hasMore = scanner.next(kvs, compactionKVMax);
894 if (!kvs.isEmpty()) {
895 for (KeyValue kv : kvs) {
896
897
898 if (kv.getMemstoreTS() <= smallestReadPoint) {
899
900
901 kv = kv.shallowCopy();
902 kv.setMemstoreTS(0);
903 }
904 writer.append(kv);
905 flushed += this.memstore.heapSizeChange(kv, true);
906 }
907 kvs.clear();
908 }
909 } while (hasMore);
910 } finally {
911
912
913 status.setStatus("Flushing " + this + ": appending metadata");
914 writer.appendMetadata(logCacheFlushId, false);
915 status.setStatus("Flushing " + this + ": closing flushed file");
916 writer.close();
917 }
918 }
919 } finally {
920 flushedSize.set(flushed);
921 scanner.close();
922 }
923 if (LOG.isInfoEnabled()) {
924 LOG.info("Flushed " +
925 ", sequenceid=" + logCacheFlushId +
926 ", memsize=" + StringUtils.humanReadableInt(flushed) +
927 ", into tmp file " + pathName);
928 }
929 return pathName;
930 }
931
932
933
934
935
936
937
938 private StoreFile commitFile(final Path path,
939 final long logCacheFlushId,
940 TimeRangeTracker snapshotTimeRangeTracker,
941 AtomicLong flushedSize,
942 MonitoredTask status)
943 throws IOException {
944
945 String fileName = path.getName();
946 Path dstPath = new Path(homedir, fileName);
947 String msg = "Renaming flushed file at " + path + " to " + dstPath;
948 LOG.debug(msg);
949 status.setStatus("Flushing " + this + ": " + msg);
950 if (!HBaseFileSystem.renameDirForFileSystem(fs, path, dstPath)) {
951 LOG.warn("Unable to rename " + path + " to " + dstPath);
952 }
953
954 status.setStatus("Flushing " + this + ": reopening flushed file");
955 StoreFile sf = new StoreFile(this.fs, dstPath, this.conf, this.cacheConf,
956 this.family.getBloomFilterType(), this.dataBlockEncoder);
957 passSchemaMetricsTo(sf);
958
959 StoreFile.Reader r = sf.createReader();
960 this.storeSize += r.length();
961 this.totalUncompressedBytes += r.getTotalUncompressedBytes();
962
963
964
965
966
967
968 getSchemaMetrics().updatePersistentStoreMetric(
969 SchemaMetrics.StoreMetricType.FLUSH_SIZE, flushedSize.longValue());
970 if (LOG.isInfoEnabled()) {
971 LOG.info("Added " + sf + ", entries=" + r.getEntries() +
972 ", sequenceid=" + logCacheFlushId +
973 ", filesize=" + StringUtils.humanReadableInt(r.length()));
974 }
975 return sf;
976 }
977
978
979
980
981
982 private StoreFile.Writer createWriterInTmp(int maxKeyCount)
983 throws IOException {
984 return createWriterInTmp(maxKeyCount, this.family.getCompression(), false, true);
985 }
986
987
988
989
990
991
992
993 public StoreFile.Writer createWriterInTmp(int maxKeyCount,
994 Compression.Algorithm compression, boolean isCompaction, boolean includeMVCCReadpoint)
995 throws IOException {
996 final CacheConfig writerCacheConf;
997 if (isCompaction) {
998
999 writerCacheConf = new CacheConfig(cacheConf);
1000 writerCacheConf.setCacheDataOnWrite(false);
1001 } else {
1002 writerCacheConf = cacheConf;
1003 }
1004 StoreFile.Writer w = new StoreFile.WriterBuilder(conf, writerCacheConf,
1005 fs, blocksize)
1006 .withOutputDir(region.getTmpDir())
1007 .withDataBlockEncoder(dataBlockEncoder)
1008 .withComparator(comparator)
1009 .withBloomType(family.getBloomFilterType())
1010 .withMaxKeyCount(maxKeyCount)
1011 .withChecksumType(checksumType)
1012 .withBytesPerChecksum(bytesPerChecksum)
1013 .withCompression(compression)
1014 .includeMVCCReadpoint(includeMVCCReadpoint)
1015 .build();
1016
1017
1018 SchemaConfigured sc = (SchemaConfigured) w.writer;
1019 SchemaConfigured.resetSchemaMetricsConf(sc);
1020 passSchemaMetricsTo(sc);
1021 return w;
1022 }
1023
1024
1025
1026
1027
1028
1029
1030
1031 private boolean updateStorefiles(final StoreFile sf,
1032 final SortedSet<KeyValue> set)
1033 throws IOException {
1034 this.lock.writeLock().lock();
1035 try {
1036 ArrayList<StoreFile> newList = new ArrayList<StoreFile>(storefiles);
1037 newList.add(sf);
1038 storefiles = sortAndClone(newList);
1039
1040 this.memstore.clearSnapshot(set);
1041 } finally {
1042
1043
1044
1045
1046
1047 this.lock.writeLock().unlock();
1048 }
1049
1050
1051 notifyChangedReadersObservers();
1052
1053 return needsCompaction();
1054 }
1055
1056
1057
1058
1059
1060 private void notifyChangedReadersObservers() throws IOException {
1061 for (ChangedReadersObserver o: this.changedReaderObservers) {
1062 o.updateReaders();
1063 }
1064 }
1065
1066
1067
1068
1069
1070
1071 protected List<KeyValueScanner> getScanners(boolean cacheBlocks,
1072 boolean usePread,
1073 boolean isCompaction,
1074 ScanQueryMatcher matcher) throws IOException {
1075 List<StoreFile> storeFiles;
1076 List<KeyValueScanner> memStoreScanners;
1077 this.lock.readLock().lock();
1078 try {
1079 storeFiles = this.getStorefiles();
1080 memStoreScanners = this.memstore.getScanners();
1081 } finally {
1082 this.lock.readLock().unlock();
1083 }
1084
1085
1086
1087
1088
1089
1090 List<StoreFileScanner> sfScanners = StoreFileScanner
1091 .getScannersForStoreFiles(storeFiles, cacheBlocks, usePread, isCompaction, matcher);
1092 List<KeyValueScanner> scanners =
1093 new ArrayList<KeyValueScanner>(sfScanners.size()+1);
1094 scanners.addAll(sfScanners);
1095
1096 scanners.addAll(memStoreScanners);
1097 return scanners;
1098 }
1099
1100
1101
1102
1103 void addChangedReaderObserver(ChangedReadersObserver o) {
1104 this.changedReaderObservers.add(o);
1105 }
1106
1107
1108
1109
1110 void deleteChangedReaderObserver(ChangedReadersObserver o) {
1111
1112 this.changedReaderObservers.remove(o);
1113 }
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140 StoreFile compact(CompactionRequest cr) throws IOException {
1141 if (cr == null || cr.getFiles().isEmpty()) return null;
1142 Preconditions.checkArgument(cr.getStore().toString().equals(this.toString()));
1143 List<StoreFile> filesToCompact = cr.getFiles();
1144 synchronized (filesCompacting) {
1145
1146
1147 Preconditions.checkArgument(filesCompacting.containsAll(filesToCompact));
1148 }
1149
1150
1151 long maxId = StoreFile.getMaxSequenceIdInList(filesToCompact);
1152
1153
1154 LOG.info("Starting compaction of " + filesToCompact.size() + " file(s) in "
1155 + this + " of "
1156 + this.region.getRegionInfo().getRegionNameAsString()
1157 + " into tmpdir=" + region.getTmpDir() + ", seqid=" + maxId + ", totalSize="
1158 + StringUtils.humanReadableInt(cr.getSize()));
1159
1160 StoreFile sf = null;
1161 try {
1162 StoreFile.Writer writer = this.compactor.compact(cr, maxId);
1163
1164 if (this.conf.getBoolean("hbase.hstore.compaction.complete", true)) {
1165 sf = completeCompaction(filesToCompact, writer);
1166 if (region.getCoprocessorHost() != null) {
1167 region.getCoprocessorHost().postCompact(this, sf, cr);
1168 }
1169 } else {
1170
1171 sf = new StoreFile(this.fs, writer.getPath(), this.conf, this.cacheConf,
1172 this.family.getBloomFilterType(), this.dataBlockEncoder);
1173 sf.createReader();
1174 }
1175 } finally {
1176 synchronized (filesCompacting) {
1177 filesCompacting.removeAll(filesToCompact);
1178 }
1179 }
1180
1181 LOG.info("Completed" + (cr.isMajor() ? " major " : " ") + "compaction of "
1182 + filesToCompact.size() + " file(s) in " + this + " of "
1183 + this.region.getRegionInfo().getRegionNameAsString()
1184 + " into " +
1185 (sf == null ? "none" : sf.getPath().getName()) +
1186 ", size=" + (sf == null ? "none" :
1187 StringUtils.humanReadableInt(sf.getReader().length()))
1188 + "; total size for store is "
1189 + StringUtils.humanReadableInt(storeSize));
1190 return sf;
1191 }
1192
1193
1194
1195
1196 public void compactRecentForTesting(int N) throws IOException {
1197 List<StoreFile> filesToCompact;
1198 long maxId;
1199 boolean isMajor;
1200
1201 this.lock.readLock().lock();
1202 try {
1203 synchronized (filesCompacting) {
1204 filesToCompact = Lists.newArrayList(storefiles);
1205 if (!filesCompacting.isEmpty()) {
1206
1207
1208 StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
1209 int idx = filesToCompact.indexOf(last);
1210 Preconditions.checkArgument(idx != -1);
1211 filesToCompact.subList(0, idx + 1).clear();
1212 }
1213 int count = filesToCompact.size();
1214 if (N > count) {
1215 throw new RuntimeException("Not enough files");
1216 }
1217
1218 filesToCompact = filesToCompact.subList(count - N, count);
1219 maxId = StoreFile.getMaxSequenceIdInList(filesToCompact);
1220 isMajor = (filesToCompact.size() == storefiles.size());
1221 filesCompacting.addAll(filesToCompact);
1222 Collections.sort(filesCompacting, StoreFile.Comparators.SEQ_ID);
1223 }
1224 } finally {
1225 this.lock.readLock().unlock();
1226 }
1227
1228 try {
1229
1230 StoreFile.Writer writer = this.compactor.compactForTesting(this, conf, filesToCompact,
1231 isMajor, maxId);
1232
1233 StoreFile sf = completeCompaction(filesToCompact, writer);
1234 if (region.getCoprocessorHost() != null) {
1235 region.getCoprocessorHost().postCompact(this, sf, null);
1236 }
1237 } finally {
1238 synchronized (filesCompacting) {
1239 filesCompacting.removeAll(filesToCompact);
1240 }
1241 }
1242 }
1243
1244 boolean hasReferences() {
1245 return hasReferences(this.storefiles);
1246 }
1247
1248
1249
1250
1251
1252 private boolean hasReferences(Collection<StoreFile> files) {
1253 if (files != null && files.size() > 0) {
1254 for (StoreFile hsf: files) {
1255 if (hsf.isReference()) {
1256 return true;
1257 }
1258 }
1259 }
1260 return false;
1261 }
1262
1263
1264
1265
1266
1267
1268
1269
1270 public static long getLowestTimestamp(final List<StoreFile> candidates)
1271 throws IOException {
1272 long minTs = Long.MAX_VALUE;
1273 for (StoreFile storeFile : candidates) {
1274 minTs = Math.min(minTs, storeFile.getModificationTimeStamp());
1275 }
1276 return minTs;
1277 }
1278
1279
1280
1281
1282 public CompactionProgress getCompactionProgress() {
1283 return this.compactor.getProgress();
1284 }
1285
1286
1287
1288
1289 boolean isMajorCompaction() throws IOException {
1290 for (StoreFile sf : this.storefiles) {
1291 if (sf.getReader() == null) {
1292 LOG.debug("StoreFile " + sf + " has null Reader");
1293 return false;
1294 }
1295 }
1296
1297 List<StoreFile> candidates = new ArrayList<StoreFile>(this.storefiles);
1298
1299
1300
1301 int pos = 0;
1302 while (pos < candidates.size() &&
1303 candidates.get(pos).getReader().length() > this.maxCompactSize &&
1304 !candidates.get(pos).isReference()) ++pos;
1305 candidates.subList(0, pos).clear();
1306
1307 return isMajorCompaction(candidates);
1308 }
1309
1310
1311
1312
1313
1314 private boolean isMajorCompaction(final List<StoreFile> filesToCompact) throws IOException {
1315 boolean result = false;
1316 long mcTime = getNextMajorCompactTime();
1317 if (filesToCompact == null || filesToCompact.isEmpty() || mcTime == 0) {
1318 return result;
1319 }
1320
1321 long lowTimestamp = getLowestTimestamp(filesToCompact);
1322 long now = EnvironmentEdgeManager.currentTimeMillis();
1323 if (lowTimestamp > 0l && lowTimestamp < (now - mcTime)) {
1324
1325 if (filesToCompact.size() == 1) {
1326
1327 StoreFile sf = filesToCompact.get(0);
1328 long oldest =
1329 (sf.getReader().timeRangeTracker == null) ?
1330 Long.MIN_VALUE :
1331 now - sf.getReader().timeRangeTracker.minimumTimestamp;
1332 if (sf.isMajorCompaction() && (this.ttl == HConstants.FOREVER || oldest < this.ttl)) {
1333
1334 float blockLocalityIndex = sf.getHDFSBlockDistribution().getBlockLocalityIndex(
1335 region.getRegionServerServices().getServerName().getHostname()
1336 );
1337 if (blockLocalityIndex < minStoreFileLocalitySkipCompact) {
1338 if (LOG.isDebugEnabled()) {
1339 LOG.debug("Major compaction triggered on only store " + this +
1340 "; to make hdfs blocks local, current locality: " + blockLocalityIndex
1341 );
1342 }
1343 result = true;
1344 } else {
1345 if (LOG.isDebugEnabled()) {
1346 LOG.debug("Skipping major compaction of " + this +
1347 " because one (major) compacted file only and oldestTime " +
1348 oldest + "ms is < ttl=" + this.ttl);
1349 }
1350 }
1351 } else if (this.ttl != HConstants.FOREVER && oldest > this.ttl) {
1352 if (LOG.isDebugEnabled()) {
1353 LOG.debug("Major compaction triggered on store " + this +
1354 ", because keyvalues outdated; time since last major compaction " +
1355 (now - lowTimestamp) + "ms");
1356 }
1357 result = true;
1358 }
1359 } else {
1360 if (LOG.isDebugEnabled()) {
1361 LOG.debug("Major compaction triggered on store " + this +
1362 "; time since last major compaction " + (now - lowTimestamp) + "ms");
1363 }
1364 result = true;
1365 }
1366 }
1367 return result;
1368 }
1369
1370 long getNextMajorCompactTime() {
1371
1372 long ret = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD, 1000*60*60*24);
1373 if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) != null) {
1374 String strCompactionTime =
1375 family.getValue(HConstants.MAJOR_COMPACTION_PERIOD);
1376 ret = (new Long(strCompactionTime)).longValue();
1377 }
1378
1379 if (ret > 0) {
1380
1381 double jitterPct = conf.getFloat("hbase.hregion.majorcompaction.jitter",
1382 0.20F);
1383 if (jitterPct > 0) {
1384 long jitter = Math.round(ret * jitterPct);
1385
1386 ImmutableList<StoreFile> snapshot = storefiles;
1387 if (snapshot != null && !snapshot.isEmpty()) {
1388 String seed = snapshot.get(0).getPath().getName();
1389 double curRand = new Random(seed.hashCode()).nextDouble();
1390 ret += jitter - Math.round(2L * jitter * curRand);
1391 } else {
1392 ret = 0;
1393 }
1394 }
1395 }
1396 return ret;
1397 }
1398
1399 public CompactionRequest requestCompaction() throws IOException {
1400 return requestCompaction(NO_PRIORITY, null);
1401 }
1402
1403 public CompactionRequest requestCompaction(int priority, CompactionRequest request)
1404 throws IOException {
1405
1406 if (!this.region.areWritesEnabled()) {
1407 return null;
1408 }
1409
1410 this.lock.readLock().lock();
1411 try {
1412 synchronized (filesCompacting) {
1413
1414 List<StoreFile> candidates = Lists.newArrayList(storefiles);
1415 if (!filesCompacting.isEmpty()) {
1416
1417
1418 StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
1419 int idx = candidates.indexOf(last);
1420 Preconditions.checkArgument(idx != -1);
1421 candidates.subList(0, idx + 1).clear();
1422 }
1423
1424 boolean override = false;
1425 if (region.getCoprocessorHost() != null) {
1426 override = region.getCoprocessorHost().preCompactSelection(this, candidates, request);
1427 }
1428 CompactSelection filesToCompact;
1429 if (override) {
1430
1431 filesToCompact = new CompactSelection(conf, candidates);
1432 } else {
1433 filesToCompact = compactSelection(candidates, priority);
1434 }
1435
1436 if (region.getCoprocessorHost() != null) {
1437 region.getCoprocessorHost().postCompactSelection(this,
1438 ImmutableList.copyOf(filesToCompact.getFilesToCompact()), request);
1439 }
1440
1441
1442 if (filesToCompact.getFilesToCompact().isEmpty()) {
1443 return null;
1444 }
1445
1446
1447 if (!Collections.disjoint(filesCompacting, filesToCompact.getFilesToCompact())) {
1448
1449 Preconditions.checkArgument(false, "%s overlaps with %s",
1450 filesToCompact, filesCompacting);
1451 }
1452 filesCompacting.addAll(filesToCompact.getFilesToCompact());
1453 Collections.sort(filesCompacting, StoreFile.Comparators.SEQ_ID);
1454
1455
1456 boolean isMajor = (filesToCompact.getFilesToCompact().size() == this.storefiles.size());
1457 if (isMajor) {
1458
1459 this.forceMajor = false;
1460 }
1461
1462
1463 int pri = getCompactPriority(priority);
1464
1465 if(request == null){
1466 request = new CompactionRequest(region, this, filesToCompact, isMajor, pri);
1467 } else {
1468
1469
1470 request.setSelection(filesToCompact);
1471 request.setIsMajor(isMajor);
1472 request.setPriority(pri);
1473 }
1474 }
1475 } finally {
1476 this.lock.readLock().unlock();
1477 }
1478 if (request != null) {
1479 CompactionRequest.preRequest(request);
1480 }
1481 return request;
1482 }
1483
1484 public void finishRequest(CompactionRequest cr) {
1485 CompactionRequest.postRequest(cr);
1486 cr.finishRequest();
1487 synchronized (filesCompacting) {
1488 filesCompacting.removeAll(cr.getFiles());
1489 }
1490 }
1491
1492
1493
1494
1495
1496
1497
1498 CompactSelection compactSelection(List<StoreFile> candidates) throws IOException {
1499 return compactSelection(candidates,NO_PRIORITY);
1500 }
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521 CompactSelection compactSelection(List<StoreFile> candidates, int priority)
1522 throws IOException {
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536 CompactSelection compactSelection = new CompactSelection(conf, candidates);
1537
1538 boolean forcemajor = this.forceMajor && filesCompacting.isEmpty();
1539 if (!forcemajor) {
1540
1541 if (conf.getBoolean("hbase.store.delete.expired.storefile", true)
1542 && (ttl != Long.MAX_VALUE) && (this.scanInfo.minVersions == 0)) {
1543 CompactSelection expiredSelection = compactSelection
1544 .selectExpiredStoreFilesToCompact(
1545 EnvironmentEdgeManager.currentTimeMillis() - this.ttl);
1546
1547
1548 if (expiredSelection != null) {
1549 return expiredSelection;
1550 }
1551 }
1552
1553
1554 int pos = 0;
1555 while (pos < compactSelection.getFilesToCompact().size() &&
1556 compactSelection.getFilesToCompact().get(pos).getReader().length()
1557 > maxCompactSize &&
1558 !compactSelection.getFilesToCompact().get(pos).isReference()) ++pos;
1559 if (pos != 0) compactSelection.clearSubList(0, pos);
1560 }
1561
1562 if (compactSelection.getFilesToCompact().isEmpty()) {
1563 LOG.debug(this.getHRegionInfo().getEncodedName() + " - " +
1564 this + ": no store files to compact");
1565 compactSelection.emptyFileList();
1566 return compactSelection;
1567 }
1568
1569
1570
1571
1572 boolean majorcompaction = (forcemajor && priority == PRIORITY_USER) ||
1573 (forcemajor || isMajorCompaction(compactSelection.getFilesToCompact())) &&
1574 (compactSelection.getFilesToCompact().size() < this.maxFilesToCompact
1575 );
1576 LOG.debug(this.getHRegionInfo().getEncodedName() + " - " +
1577 this.getColumnFamilyName() + ": Initiating " +
1578 (majorcompaction ? "major" : "minor") + "compaction");
1579
1580 if (!majorcompaction &&
1581 !hasReferences(compactSelection.getFilesToCompact())) {
1582
1583
1584 compactSelection.getFilesToCompact().removeAll(Collections2.filter(
1585 compactSelection.getFilesToCompact(),
1586 new Predicate<StoreFile>() {
1587 public boolean apply(StoreFile input) {
1588 return input.excludeFromMinorCompaction();
1589 }
1590 }));
1591
1592
1593 if (compactSelection.getFilesToCompact().size() < this.minFilesToCompact) {
1594 if(LOG.isDebugEnabled()) {
1595 LOG.debug("Not compacting files because we only have " +
1596 compactSelection.getFilesToCompact().size() +
1597 " files ready for compaction. Need " + this.minFilesToCompact + " to initiate.");
1598 }
1599 compactSelection.emptyFileList();
1600 return compactSelection;
1601 }
1602 if (conf.getBoolean("hbase.hstore.useExploringCompation", false)) {
1603 compactSelection = exploringCompactionSelection(compactSelection);
1604 } else {
1605 compactSelection = defaultCompactionSelection(compactSelection);
1606 }
1607 } else {
1608 if(majorcompaction) {
1609 if (compactSelection.getFilesToCompact().size() > this.maxFilesToCompact) {
1610 LOG.debug("Warning, compacting more than " + this.maxFilesToCompact +
1611 " files, probably because of a user-requested major compaction");
1612 if(priority != PRIORITY_USER) {
1613 LOG.error("Compacting more than max files on a non user-requested compaction");
1614 }
1615 }
1616 } else if (compactSelection.getFilesToCompact().size() > this.maxFilesToCompact) {
1617
1618 int excess = compactSelection.getFilesToCompact().size() - this.maxFilesToCompact;
1619 LOG.debug("Too many admissible files. Excluding " + excess
1620 + " files from compaction candidates");
1621 candidates.subList(this.maxFilesToCompact, candidates.size()).clear();
1622 }
1623 }
1624 return compactSelection;
1625 }
1626
1627 private CompactSelection defaultCompactionSelection(CompactSelection compactSelection) {
1628
1629 int start = 0;
1630
1631 double r = compactSelection.getCompactSelectionRatio();
1632
1633
1634 int countOfFiles = compactSelection.getFilesToCompact().size();
1635 long [] fileSizes = new long[countOfFiles];
1636 long [] sumSize = new long[countOfFiles];
1637 for (int i = countOfFiles-1; i >= 0; --i) {
1638 StoreFile file = compactSelection.getFilesToCompact().get(i);
1639 fileSizes[i] = file.getReader().length();
1640
1641 int tooFar = i + this.maxFilesToCompact - 1;
1642 sumSize[i] = fileSizes[i]
1643 + ((i+1 < countOfFiles) ? sumSize[i+1] : 0)
1644 - ((tooFar < countOfFiles) ? fileSizes[tooFar] : 0);
1645 }
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660 while(countOfFiles - start >= this.minFilesToCompact &&
1661 fileSizes[start] >
1662 Math.max(minCompactSize, (long)(sumSize[start+1] * r))) {
1663 ++start;
1664 }
1665 int end = Math.min(countOfFiles, start + this.maxFilesToCompact);
1666 long totalSize = fileSizes[start]
1667 + ((start+1 < countOfFiles) ? sumSize[start+1] : 0);
1668 compactSelection = compactSelection.getSubList(start, end);
1669
1670
1671 if (compactSelection.getFilesToCompact().size() < this.minFilesToCompact) {
1672 if (LOG.isDebugEnabled()) {
1673 LOG.debug("Skipped compaction of " + this
1674 + ". Only " + (end - start) + " file(s) of size "
1675 + StringUtils.humanReadableInt(totalSize)
1676 + " have met compaction criteria.");
1677 }
1678 compactSelection.emptyFileList();
1679 return compactSelection;
1680 }
1681 return compactSelection;
1682 }
1683
1684 private CompactSelection exploringCompactionSelection(CompactSelection compactSelection) {
1685
1686 List<StoreFile> candidates = compactSelection.getFilesToCompact();
1687 int futureFiles = filesCompacting.isEmpty() ? 0 : 1;
1688 boolean mayBeStuck = (candidates.size() - filesCompacting.size() + futureFiles)
1689 >= blockingStoreFileCount;
1690
1691 List<StoreFile> bestSelection = new ArrayList<StoreFile>(0);
1692 List<StoreFile> smallest = new ArrayList<StoreFile>(0);
1693 long bestSize = 0;
1694 long smallestSize = Long.MAX_VALUE;
1695 double r = compactSelection.getCompactSelectionRatio();
1696
1697
1698 for (int startIndex = 0; startIndex < candidates.size(); startIndex++) {
1699
1700 for (int currentEnd = startIndex + minFilesToCompact - 1;
1701 currentEnd < candidates.size(); currentEnd++) {
1702 List<StoreFile> potentialMatchFiles = candidates.subList(startIndex, currentEnd + 1);
1703
1704
1705 if (potentialMatchFiles.size() < minFilesToCompact) {
1706 continue;
1707 }
1708 if (potentialMatchFiles.size() > maxFilesToCompact) {
1709 continue;
1710 }
1711
1712
1713
1714 long size = getCompactionSize(potentialMatchFiles);
1715
1716
1717
1718 if (size < smallestSize) {
1719 smallest = potentialMatchFiles;
1720 smallestSize = size;
1721 }
1722
1723 if (size >= minCompactSize
1724 && !filesInRatio(potentialMatchFiles, r)) {
1725 continue;
1726 }
1727
1728 if (size > maxCompactSize) {
1729 continue;
1730 }
1731
1732
1733 if (potentialMatchFiles.size() > bestSelection.size()
1734 || (potentialMatchFiles.size() == bestSelection.size() && size < bestSize)) {
1735 bestSelection = potentialMatchFiles;
1736 bestSize = size;
1737 }
1738 }
1739 }
1740
1741 if (bestSelection.size() == 0 && mayBeStuck) {
1742 smallest = new ArrayList<StoreFile>(smallest);
1743 compactSelection.getFilesToCompact().clear();
1744 compactSelection.getFilesToCompact().addAll(smallest);
1745 } else {
1746 bestSelection = new ArrayList<StoreFile>(bestSelection);
1747 compactSelection.getFilesToCompact().clear();
1748 compactSelection.getFilesToCompact().addAll(bestSelection);
1749 }
1750
1751 return compactSelection;
1752
1753 }
1754
1755
1756
1757
1758
1759
1760
1761
1762 private boolean filesInRatio(final List<StoreFile> files, final double currentRatio) {
1763 if (files.size() < 2) {
1764 return true;
1765 }
1766 long totalFileSize = 0;
1767 for (int i = 0; i < files.size(); i++) {
1768 totalFileSize += files.get(i).getReader().length();
1769 }
1770 for (int i = 0; i < files.size(); i++) {
1771 long singleFileSize = files.get(i).getReader().length();
1772 long sumAllOtherFilesize = totalFileSize - singleFileSize;
1773
1774 if ((singleFileSize > sumAllOtherFilesize * currentRatio)
1775 && (sumAllOtherFilesize >= this.minCompactSize)) {
1776 return false;
1777 }
1778 }
1779 return true;
1780 }
1781
1782
1783
1784
1785
1786
1787
1788 private long getCompactionSize(final List<StoreFile> files) {
1789 long size = 0;
1790 if (files == null) {
1791 return size;
1792 }
1793 for (StoreFile f : files) {
1794 size += f.getReader().length();
1795 }
1796 return size;
1797 }
1798
1799
1800
1801
1802
1803
1804
1805 private void validateStoreFile(Path path)
1806 throws IOException {
1807 StoreFile storeFile = null;
1808 try {
1809 storeFile = new StoreFile(this.fs, path, this.conf,
1810 this.cacheConf, this.family.getBloomFilterType(),
1811 NoOpDataBlockEncoder.INSTANCE);
1812 passSchemaMetricsTo(storeFile);
1813 storeFile.createReader();
1814 } catch (IOException e) {
1815 LOG.error("Failed to open store file : " + path
1816 + ", keeping it in tmp location", e);
1817 throw e;
1818 } finally {
1819 if (storeFile != null) {
1820 storeFile.closeReader(false);
1821 }
1822 }
1823 }
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844 StoreFile completeCompaction(final Collection<StoreFile> compactedFiles,
1845 final StoreFile.Writer compactedFile)
1846 throws IOException {
1847
1848
1849 StoreFile result = null;
1850 if (compactedFile != null) {
1851 validateStoreFile(compactedFile.getPath());
1852
1853 Path origPath = compactedFile.getPath();
1854 Path destPath = new Path(homedir, origPath.getName());
1855 LOG.info("Renaming compacted file at " + origPath + " to " + destPath);
1856 if (!HBaseFileSystem.renameDirForFileSystem(fs, origPath, destPath)) {
1857 LOG.error("Failed move of compacted file " + origPath + " to " +
1858 destPath);
1859 throw new IOException("Failed move of compacted file " + origPath +
1860 " to " + destPath);
1861 }
1862 result = new StoreFile(this.fs, destPath, this.conf, this.cacheConf,
1863 this.family.getBloomFilterType(), this.dataBlockEncoder);
1864 passSchemaMetricsTo(result);
1865 result.createReader();
1866 }
1867 try {
1868 this.lock.writeLock().lock();
1869 try {
1870
1871
1872
1873
1874 ArrayList<StoreFile> newStoreFiles = Lists.newArrayList(storefiles);
1875 newStoreFiles.removeAll(compactedFiles);
1876 filesCompacting.removeAll(compactedFiles);
1877
1878
1879 if (result != null) {
1880 newStoreFiles.add(result);
1881 }
1882
1883 this.storefiles = sortAndClone(newStoreFiles);
1884 } finally {
1885
1886
1887
1888
1889
1890 this.lock.writeLock().unlock();
1891 }
1892
1893
1894 notifyChangedReadersObservers();
1895
1896
1897 LOG.debug("Removing store files after compaction...");
1898 HFileArchiver.archiveStoreFiles(this.conf, this.fs, this.region, this.family.getName(),
1899 compactedFiles);
1900
1901 } catch (IOException e) {
1902 e = RemoteExceptionHandler.checkIOException(e);
1903 LOG.error("Failed replacing compacted files in " + this +
1904 ". Compacted file is " + (result == null? "none": result.toString()) +
1905 ". Files replaced " + compactedFiles.toString() +
1906 " some of which may have been already removed", e);
1907 }
1908
1909
1910 this.storeSize = 0L;
1911 this.totalUncompressedBytes = 0L;
1912 for (StoreFile hsf : this.storefiles) {
1913 StoreFile.Reader r = hsf.getReader();
1914 if (r == null) {
1915 LOG.warn("StoreFile " + hsf + " has a null Reader");
1916 continue;
1917 }
1918 this.storeSize += r.length();
1919 this.totalUncompressedBytes += r.getTotalUncompressedBytes();
1920 }
1921 return result;
1922 }
1923
1924 public ImmutableList<StoreFile> sortAndClone(List<StoreFile> storeFiles) {
1925 Collections.sort(storeFiles, StoreFile.Comparators.SEQ_ID);
1926 ImmutableList<StoreFile> newList = ImmutableList.copyOf(storeFiles);
1927 return newList;
1928 }
1929
1930
1931
1932
1933
1934
1935
1936
1937 public int getNumberOfStoreFiles() {
1938 return this.storefiles.size();
1939 }
1940
1941
1942
1943
1944
1945 int versionsToReturn(final int wantedVersions) {
1946 if (wantedVersions <= 0) {
1947 throw new IllegalArgumentException("Number of versions must be > 0");
1948 }
1949
1950 int maxVersions = this.family.getMaxVersions();
1951 return wantedVersions > maxVersions ? maxVersions: wantedVersions;
1952 }
1953
1954 static boolean isExpired(final KeyValue key, final long oldestTimestamp) {
1955 return key.getTimestamp() < oldestTimestamp;
1956 }
1957
1958
1959
1960
1961
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972 KeyValue getRowKeyAtOrBefore(final byte[] row) throws IOException {
1973
1974
1975
1976
1977
1978
1979 long ttlToUse = scanInfo.getMinVersions() > 0 ? Long.MAX_VALUE : this.ttl;
1980
1981 KeyValue kv = new KeyValue(row, HConstants.LATEST_TIMESTAMP);
1982
1983 GetClosestRowBeforeTracker state = new GetClosestRowBeforeTracker(
1984 this.comparator, kv, ttlToUse, this.region.getRegionInfo().isMetaRegion());
1985 this.lock.readLock().lock();
1986 try {
1987
1988 this.memstore.getRowKeyAtOrBefore(state);
1989
1990
1991 for (StoreFile sf : Lists.reverse(storefiles)) {
1992
1993 rowAtOrBeforeFromStoreFile(sf, state);
1994 }
1995 return state.getCandidate();
1996 } finally {
1997 this.lock.readLock().unlock();
1998 }
1999 }
2000
2001
2002
2003
2004
2005
2006
2007 private void rowAtOrBeforeFromStoreFile(final StoreFile f,
2008 final GetClosestRowBeforeTracker state)
2009 throws IOException {
2010 StoreFile.Reader r = f.getReader();
2011 if (r == null) {
2012 LOG.warn("StoreFile " + f + " has a null Reader");
2013 return;
2014 }
2015 if (r.getEntries() == 0) {
2016 LOG.warn("StoreFile " + f + " is a empty store file");
2017 return;
2018 }
2019
2020 byte [] fk = r.getFirstKey();
2021 if (fk == null) return;
2022 KeyValue firstKV = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
2023 byte [] lk = r.getLastKey();
2024 KeyValue lastKV = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
2025 KeyValue firstOnRow = state.getTargetKey();
2026 if (this.comparator.compareRows(lastKV, firstOnRow) < 0) {
2027
2028
2029 if (!state.isTargetTable(lastKV)) return;
2030
2031
2032 firstOnRow = new KeyValue(lastKV.getRow(), HConstants.LATEST_TIMESTAMP);
2033 }
2034
2035 HFileScanner scanner = r.getScanner(true, true, false);
2036
2037 if (!seekToScanner(scanner, firstOnRow, firstKV)) return;
2038
2039
2040 if (walkForwardInSingleRow(scanner, firstOnRow, state)) return;
2041
2042 while (scanner.seekBefore(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(),
2043 firstOnRow.getKeyLength())) {
2044 KeyValue kv = scanner.getKeyValue();
2045 if (!state.isTargetTable(kv)) break;
2046 if (!state.isBetterCandidate(kv)) break;
2047
2048 firstOnRow = new KeyValue(kv.getRow(), HConstants.LATEST_TIMESTAMP);
2049
2050 if (!seekToScanner(scanner, firstOnRow, firstKV)) break;
2051
2052 if (walkForwardInSingleRow(scanner, firstOnRow, state)) break;
2053 }
2054 }
2055
2056
2057
2058
2059
2060
2061
2062
2063
2064 private boolean seekToScanner(final HFileScanner scanner,
2065 final KeyValue firstOnRow,
2066 final KeyValue firstKV)
2067 throws IOException {
2068 KeyValue kv = firstOnRow;
2069
2070 if (this.comparator.compareRows(firstKV, firstOnRow) == 0) kv = firstKV;
2071 int result = scanner.seekTo(kv.getBuffer(), kv.getKeyOffset(),
2072 kv.getKeyLength());
2073 return result >= 0;
2074 }
2075
2076
2077
2078
2079
2080
2081
2082
2083
2084
2085
2086 private boolean walkForwardInSingleRow(final HFileScanner scanner,
2087 final KeyValue firstOnRow,
2088 final GetClosestRowBeforeTracker state)
2089 throws IOException {
2090 boolean foundCandidate = false;
2091 do {
2092 KeyValue kv = scanner.getKeyValue();
2093
2094 if (this.comparator.compareRows(kv, firstOnRow) < 0) continue;
2095
2096 if (state.isTooFar(kv, firstOnRow)) break;
2097 if (state.isExpired(kv)) {
2098 continue;
2099 }
2100
2101 if (state.handle(kv)) {
2102 foundCandidate = true;
2103 break;
2104 }
2105 } while(scanner.next());
2106 return foundCandidate;
2107 }
2108
2109 public boolean canSplit() {
2110 this.lock.readLock().lock();
2111 try {
2112
2113 for (StoreFile sf : storefiles) {
2114 if (sf.isReference()) {
2115 if (LOG.isDebugEnabled()) {
2116 LOG.debug(sf + " is not splittable");
2117 }
2118 return false;
2119 }
2120 }
2121
2122 return true;
2123 } finally {
2124 this.lock.readLock().unlock();
2125 }
2126 }
2127
2128
2129
2130
2131 public byte[] getSplitPoint() {
2132 this.lock.readLock().lock();
2133 try {
2134
2135 if (this.storefiles.isEmpty()) {
2136 return null;
2137 }
2138
2139 assert !this.region.getRegionInfo().isMetaRegion();
2140
2141
2142 long maxSize = 0L;
2143 StoreFile largestSf = null;
2144 for (StoreFile sf : storefiles) {
2145 if (sf.isReference()) {
2146
2147 return null;
2148 }
2149
2150 StoreFile.Reader r = sf.getReader();
2151 if (r == null) {
2152 LOG.warn("Storefile " + sf + " Reader is null");
2153 continue;
2154 }
2155
2156 long size = r.length();
2157 if (size > maxSize) {
2158
2159 maxSize = size;
2160 largestSf = sf;
2161 }
2162 }
2163
2164 StoreFile.Reader r = largestSf.getReader();
2165 if (r == null) {
2166 LOG.warn("Storefile " + largestSf + " Reader is null");
2167 return null;
2168 }
2169
2170
2171
2172 byte [] midkey = r.midkey();
2173 if (midkey != null) {
2174 KeyValue mk = KeyValue.createKeyValueFromKey(midkey, 0, midkey.length);
2175 byte [] fk = r.getFirstKey();
2176 KeyValue firstKey = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
2177 byte [] lk = r.getLastKey();
2178 KeyValue lastKey = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
2179
2180
2181 if (this.comparator.compareRows(mk, firstKey) == 0 ||
2182 this.comparator.compareRows(mk, lastKey) == 0) {
2183 if (LOG.isDebugEnabled()) {
2184 LOG.debug("cannot split because midkey is the same as first or " +
2185 "last row");
2186 }
2187 return null;
2188 }
2189 return mk.getRow();
2190 }
2191 } catch(IOException e) {
2192 LOG.warn("Failed getting store size for " + this, e);
2193 } finally {
2194 this.lock.readLock().unlock();
2195 }
2196 return null;
2197 }
2198
2199
2200 public long getLastCompactSize() {
2201 return this.lastCompactSize;
2202 }
2203
2204
2205 public long getSize() {
2206 return storeSize;
2207 }
2208
2209 public void triggerMajorCompaction() {
2210 this.forceMajor = true;
2211 }
2212
2213 boolean getForceMajorCompaction() {
2214 return this.forceMajor;
2215 }
2216
2217
2218
2219
2220
2221
2222
2223
2224
2225
2226 public KeyValueScanner getScanner(Scan scan,
2227 final NavigableSet<byte []> targetCols) throws IOException {
2228 lock.readLock().lock();
2229 try {
2230 KeyValueScanner scanner = null;
2231 if (getHRegion().getCoprocessorHost() != null) {
2232 scanner = getHRegion().getCoprocessorHost().preStoreScannerOpen(this, scan, targetCols);
2233 }
2234 if (scanner == null) {
2235 scanner = new StoreScanner(this, getScanInfo(), scan, targetCols);
2236 }
2237 return scanner;
2238 } finally {
2239 lock.readLock().unlock();
2240 }
2241 }
2242
2243 @Override
2244 public String toString() {
2245 return getColumnFamilyName();
2246 }
2247
2248
2249
2250
2251 int getStorefilesCount() {
2252 return this.storefiles.size();
2253 }
2254
2255
2256
2257
2258 long getStoreSizeUncompressed() {
2259 return this.totalUncompressedBytes;
2260 }
2261
2262
2263
2264
2265 long getStorefilesSize() {
2266 long size = 0;
2267 for (StoreFile s: storefiles) {
2268 StoreFile.Reader r = s.getReader();
2269 if (r == null) {
2270 LOG.warn("StoreFile " + s + " has a null Reader");
2271 continue;
2272 }
2273 size += r.length();
2274 }
2275 return size;
2276 }
2277
2278
2279
2280
2281 long getStorefilesIndexSize() {
2282 long size = 0;
2283 for (StoreFile s: storefiles) {
2284 StoreFile.Reader r = s.getReader();
2285 if (r == null) {
2286 LOG.warn("StoreFile " + s + " has a null Reader");
2287 continue;
2288 }
2289 size += r.indexSize();
2290 }
2291 return size;
2292 }
2293
2294
2295
2296
2297
2298
2299
2300
2301 long getTotalStaticIndexSize() {
2302 long size = 0;
2303 for (StoreFile s : storefiles) {
2304 size += s.getReader().getUncompressedDataIndexSize();
2305 }
2306 return size;
2307 }
2308
2309
2310
2311
2312
2313
2314
2315
2316 long getTotalStaticBloomSize() {
2317 long size = 0;
2318 for (StoreFile s : storefiles) {
2319 StoreFile.Reader r = s.getReader();
2320 size += r.getTotalBloomSize();
2321 }
2322 return size;
2323 }
2324
2325
2326
2327
2328 long getMemStoreSize() {
2329 return this.memstore.heapSize();
2330 }
2331
2332 public int getCompactPriority() {
2333 return getCompactPriority(NO_PRIORITY);
2334 }
2335
2336
2337
2338
2339
2340 public int getCompactPriority(int priority) {
2341
2342 if(priority == PRIORITY_USER) {
2343 return PRIORITY_USER;
2344 } else {
2345 return this.blockingStoreFileCount - this.storefiles.size();
2346 }
2347 }
2348
2349 boolean throttleCompaction(long compactionSize) {
2350 long throttlePoint = conf.getLong(
2351 "hbase.regionserver.thread.compaction.throttle",
2352 2 * this.minFilesToCompact * this.region.memstoreFlushSize);
2353 return compactionSize > throttlePoint;
2354 }
2355
2356 public HRegion getHRegion() {
2357 return this.region;
2358 }
2359
2360 HRegionInfo getHRegionInfo() {
2361 return this.region.getRegionInfo();
2362 }
2363
2364
2365
2366
2367
2368
2369
2370
2371
2372
2373
2374
2375
2376
2377
2378 public long updateColumnValue(byte [] row, byte [] f,
2379 byte [] qualifier, long newValue)
2380 throws IOException {
2381
2382 this.lock.readLock().lock();
2383 try {
2384 long now = EnvironmentEdgeManager.currentTimeMillis();
2385
2386 return this.memstore.updateColumnValue(row,
2387 f,
2388 qualifier,
2389 newValue,
2390 now);
2391
2392 } finally {
2393 this.lock.readLock().unlock();
2394 }
2395 }
2396
2397
2398
2399
2400
2401
2402
2403
2404
2405
2406
2407
2408
2409
2410 public long upsert(List<KeyValue> kvs)
2411 throws IOException {
2412 this.lock.readLock().lock();
2413 try {
2414
2415 return this.memstore.upsert(kvs);
2416 } finally {
2417 this.lock.readLock().unlock();
2418 }
2419 }
2420
2421 public StoreFlusher getStoreFlusher(long cacheFlushId) {
2422 return new StoreFlusherImpl(cacheFlushId);
2423 }
2424
2425 private class StoreFlusherImpl implements StoreFlusher {
2426
2427 private long cacheFlushId;
2428 private SortedSet<KeyValue> snapshot;
2429 private StoreFile storeFile;
2430 private Path storeFilePath;
2431 private TimeRangeTracker snapshotTimeRangeTracker;
2432 private AtomicLong flushedSize;
2433
2434 private StoreFlusherImpl(long cacheFlushId) {
2435 this.cacheFlushId = cacheFlushId;
2436 this.flushedSize = new AtomicLong();
2437 }
2438
2439 @Override
2440 public void prepare() {
2441 memstore.snapshot();
2442 this.snapshot = memstore.getSnapshot();
2443 this.snapshotTimeRangeTracker = memstore.getSnapshotTimeRangeTracker();
2444 }
2445
2446 @Override
2447 public void flushCache(MonitoredTask status) throws IOException {
2448 storeFilePath = Store.this.flushCache(
2449 cacheFlushId, snapshot, snapshotTimeRangeTracker, flushedSize, status);
2450 }
2451
2452 @Override
2453 public boolean commit(MonitoredTask status) throws IOException {
2454 if (storeFilePath == null) {
2455 return false;
2456 }
2457 storeFile = Store.this.commitFile(storeFilePath, cacheFlushId,
2458 snapshotTimeRangeTracker, flushedSize, status);
2459 if (Store.this.getHRegion().getCoprocessorHost() != null) {
2460 Store.this.getHRegion()
2461 .getCoprocessorHost()
2462 .postFlush(Store.this, storeFile);
2463 }
2464
2465
2466
2467 return Store.this.updateStorefiles(storeFile, snapshot);
2468 }
2469 }
2470
2471
2472
2473
2474
2475
2476 public boolean needsCompaction() {
2477 return (storefiles.size() - filesCompacting.size()) > minFilesToCompact;
2478 }
2479
2480
2481
2482
2483 public CacheConfig getCacheConfig() {
2484 return this.cacheConf;
2485 }
2486
2487 public static final long FIXED_OVERHEAD =
2488 ClassSize.align(SchemaConfigured.SCHEMA_CONFIGURED_UNALIGNED_HEAP_SIZE +
2489 + (17 * ClassSize.REFERENCE) + (7 * Bytes.SIZEOF_LONG)
2490 + (5 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_BOOLEAN + Bytes.SIZEOF_FLOAT);
2491
2492 public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD
2493 + ClassSize.OBJECT + ClassSize.REENTRANT_LOCK
2494 + ClassSize.CONCURRENT_SKIPLISTMAP
2495 + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + ClassSize.OBJECT
2496 + ScanInfo.FIXED_OVERHEAD);
2497
2498 @Override
2499 public long heapSize() {
2500 return DEEP_OVERHEAD + this.memstore.heapSize();
2501 }
2502
2503 public KeyValue.KVComparator getComparator() {
2504 return comparator;
2505 }
2506
2507 public ScanInfo getScanInfo() {
2508 return scanInfo;
2509 }
2510
2511 public boolean hasTooManyStoreFiles() {
2512 return getStorefilesCount() > this.blockingFileCount;
2513 }
2514
2515
2516
2517
2518 public static class ScanInfo {
2519 private byte[] family;
2520 private int minVersions;
2521 private int maxVersions;
2522 private long ttl;
2523 private boolean keepDeletedCells;
2524 private long timeToPurgeDeletes;
2525 private KVComparator comparator;
2526
2527 public static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
2528 + (2 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_INT)
2529 + Bytes.SIZEOF_LONG + Bytes.SIZEOF_BOOLEAN);
2530
2531
2532
2533
2534
2535
2536
2537
2538 public ScanInfo(HColumnDescriptor family, long ttl, long timeToPurgeDeletes, KVComparator comparator) {
2539 this(family.getName(), family.getMinVersions(), family.getMaxVersions(), ttl, family
2540 .getKeepDeletedCells(), timeToPurgeDeletes, comparator);
2541 }
2542
2543
2544
2545
2546
2547
2548
2549
2550
2551
2552 public ScanInfo(byte[] family, int minVersions, int maxVersions, long ttl,
2553 boolean keepDeletedCells, long timeToPurgeDeletes,
2554 KVComparator comparator) {
2555
2556 this.family = family;
2557 this.minVersions = minVersions;
2558 this.maxVersions = maxVersions;
2559 this.ttl = ttl;
2560 this.keepDeletedCells = keepDeletedCells;
2561 this.timeToPurgeDeletes = timeToPurgeDeletes;
2562 this.comparator = comparator;
2563 }
2564
2565 public byte[] getFamily() {
2566 return family;
2567 }
2568
2569 public int getMinVersions() {
2570 return minVersions;
2571 }
2572
2573 public int getMaxVersions() {
2574 return maxVersions;
2575 }
2576
2577 public long getTtl() {
2578 return ttl;
2579 }
2580
2581 public boolean getKeepDeletedCells() {
2582 return keepDeletedCells;
2583 }
2584
2585 public long getTimeToPurgeDeletes() {
2586 return timeToPurgeDeletes;
2587 }
2588
2589 public KVComparator getComparator() {
2590 return comparator;
2591 }
2592 }
2593
2594 }