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