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