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