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