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