1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.regionserver;
20
21 import java.io.IOException;
22 import java.io.InterruptedIOException;
23 import java.util.ArrayList;
24 import java.util.Collection;
25 import java.util.Collections;
26 import java.util.Iterator;
27 import java.util.List;
28 import java.util.NavigableSet;
29 import java.util.SortedSet;
30 import java.util.concurrent.Callable;
31 import java.util.concurrent.CompletionService;
32 import java.util.concurrent.CopyOnWriteArraySet;
33 import java.util.concurrent.ExecutionException;
34 import java.util.concurrent.ExecutorCompletionService;
35 import java.util.concurrent.Future;
36 import java.util.concurrent.ThreadPoolExecutor;
37 import java.util.concurrent.atomic.AtomicLong;
38 import java.util.concurrent.locks.ReentrantReadWriteLock;
39
40 import org.apache.commons.logging.Log;
41 import org.apache.commons.logging.LogFactory;
42 import org.apache.hadoop.classification.InterfaceAudience;
43 import org.apache.hadoop.conf.Configuration;
44 import org.apache.hadoop.fs.FileSystem;
45 import org.apache.hadoop.fs.Path;
46 import org.apache.hadoop.hbase.Cell;
47 import org.apache.hadoop.hbase.CompoundConfiguration;
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.RemoteExceptionHandler;
53 import org.apache.hadoop.hbase.client.Scan;
54 import org.apache.hadoop.hbase.exceptions.WrongRegionException;
55 import org.apache.hadoop.hbase.fs.HFileSystem;
56 import org.apache.hadoop.hbase.io.compress.Compression;
57 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
58 import org.apache.hadoop.hbase.io.hfile.HFile;
59 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
60 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl;
61 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
62 import org.apache.hadoop.hbase.exceptions.InvalidHFileException;
63 import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder;
64 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
65 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
66 import org.apache.hadoop.hbase.regionserver.compactions.CompactionPolicy;
67 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
68 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
69 import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
70 import org.apache.hadoop.hbase.regionserver.compactions.OffPeakCompactions;
71 import org.apache.hadoop.hbase.util.Bytes;
72 import org.apache.hadoop.hbase.util.ChecksumType;
73 import org.apache.hadoop.hbase.util.ClassSize;
74 import org.apache.hadoop.hbase.util.CollectionBackedScanner;
75 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
76 import org.apache.hadoop.util.StringUtils;
77
78 import com.google.common.base.Preconditions;
79 import com.google.common.collect.ImmutableCollection;
80 import com.google.common.collect.ImmutableList;
81 import com.google.common.collect.Lists;
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106 @InterfaceAudience.Private
107 public class HStore implements Store {
108 public static final String BLOCKING_STOREFILES_KEY = "hbase.hstore.blockingStoreFiles";
109 public static final int DEFAULT_BLOCKING_STOREFILE_COUNT = 7;
110
111 static final Log LOG = LogFactory.getLog(HStore.class);
112
113 protected final MemStore memstore;
114 private final HRegion region;
115 private final HColumnDescriptor family;
116 private final HRegionFileSystem fs;
117 private final Configuration conf;
118 private final CacheConfig cacheConf;
119 private long lastCompactSize = 0;
120 volatile boolean forceMajor = false;
121
122 static int closeCheckInterval = 0;
123 private volatile long storeSize = 0L;
124 private volatile long totalUncompressedBytes = 0L;
125 private final Object flushLock = new Object();
126 final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
127 private final boolean verifyBulkLoads;
128
129 private ScanInfo scanInfo;
130
131 final List<StoreFile> filesCompacting = Lists.newArrayList();
132
133
134 private final CopyOnWriteArraySet<ChangedReadersObserver> changedReaderObservers =
135 new CopyOnWriteArraySet<ChangedReadersObserver>();
136
137 private final int blocksize;
138 private HFileDataBlockEncoder dataBlockEncoder;
139
140
141 private ChecksumType checksumType;
142 private int bytesPerChecksum;
143
144
145 private final KeyValue.KVComparator comparator;
146
147 final StoreEngine<?, ?, ?> storeEngine;
148
149 private OffPeakCompactions offPeakCompactions;
150
151 private static final int DEFAULT_FLUSH_RETRIES_NUMBER = 10;
152 private static int flush_retries_number;
153 private static int pauseTime;
154
155 private long blockingFileCount;
156
157
158
159
160
161
162
163
164
165 protected HStore(final HRegion region, final HColumnDescriptor family,
166 final Configuration confParam) throws IOException {
167
168 HRegionInfo info = region.getRegionInfo();
169 this.fs = region.getRegionFileSystem();
170
171
172 fs.createStoreDir(family.getNameAsString());
173 this.region = region;
174 this.family = family;
175
176
177
178 this.conf = new CompoundConfiguration()
179 .add(confParam)
180 .addStringMap(region.getTableDesc().getConfiguration())
181 .addStringMap(family.getConfiguration())
182 .addWritableMap(family.getValues());
183 this.blocksize = family.getBlocksize();
184
185 this.dataBlockEncoder =
186 new HFileDataBlockEncoderImpl(family.getDataBlockEncodingOnDisk(),
187 family.getDataBlockEncoding());
188
189 this.comparator = info.getComparator();
190
191 long timeToPurgeDeletes =
192 Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);
193 LOG.trace("Time to purge deletes set to " + timeToPurgeDeletes +
194 "ms in store " + this);
195
196 long ttl = determineTTLFromFamily(family);
197
198
199 scanInfo = new ScanInfo(family, ttl, timeToPurgeDeletes, this.comparator);
200 this.memstore = new MemStore(conf, this.comparator);
201 this.offPeakCompactions = new OffPeakCompactions(conf);
202
203
204 this.cacheConf = new CacheConfig(conf, family);
205
206 this.verifyBulkLoads = conf.getBoolean("hbase.hstore.bulkload.verify", false);
207
208 this.blockingFileCount =
209 conf.getInt(BLOCKING_STOREFILES_KEY, DEFAULT_BLOCKING_STOREFILE_COUNT);
210
211 if (HStore.closeCheckInterval == 0) {
212 HStore.closeCheckInterval = conf.getInt(
213 "hbase.hstore.close.check.interval", 10*1000*1000
214 }
215
216 this.storeEngine = StoreEngine.create(this, this.conf, this.comparator);
217 this.storeEngine.getStoreFileManager().loadFiles(loadStoreFiles());
218
219
220 this.checksumType = getChecksumType(conf);
221
222 this.bytesPerChecksum = getBytesPerChecksum(conf);
223
224 if (HStore.flush_retries_number == 0) {
225 HStore.flush_retries_number = conf.getInt(
226 "hbase.hstore.flush.retries.number", DEFAULT_FLUSH_RETRIES_NUMBER);
227 HStore.pauseTime = conf.getInt(HConstants.HBASE_SERVER_PAUSE,
228 HConstants.DEFAULT_HBASE_SERVER_PAUSE);
229 if (HStore.flush_retries_number <= 0) {
230 throw new IllegalArgumentException(
231 "hbase.hstore.flush.retries.number must be > 0, not "
232 + HStore.flush_retries_number);
233 }
234 }
235 }
236
237
238
239
240
241 private static long determineTTLFromFamily(final HColumnDescriptor family) {
242
243 long ttl = family.getTimeToLive();
244 if (ttl == HConstants.FOREVER) {
245
246 ttl = Long.MAX_VALUE;
247 } else if (ttl == -1) {
248 ttl = Long.MAX_VALUE;
249 } else {
250
251 ttl *= 1000;
252 }
253 return ttl;
254 }
255
256 public String getColumnFamilyName() {
257 return this.family.getNameAsString();
258 }
259
260 @Override
261 public String getTableName() {
262 return this.getRegionInfo().getTableNameAsString();
263 }
264
265 @Override
266 public FileSystem getFileSystem() {
267 return this.fs.getFileSystem();
268 }
269
270 public HRegionFileSystem getRegionFileSystem() {
271 return this.fs;
272 }
273
274
275 @Override
276 public long getStoreFileTtl() {
277
278 return (this.scanInfo.getMinVersions() == 0) ? this.scanInfo.getTtl() : Long.MAX_VALUE;
279 }
280
281 @Override
282 public long getMemstoreFlushSize() {
283 return this.region.memstoreFlushSize;
284 }
285
286
287
288
289
290
291
292 public static int getBytesPerChecksum(Configuration conf) {
293 return conf.getInt(HConstants.BYTES_PER_CHECKSUM,
294 HFile.DEFAULT_BYTES_PER_CHECKSUM);
295 }
296
297
298
299
300
301
302 public static ChecksumType getChecksumType(Configuration conf) {
303 String checksumName = conf.get(HConstants.CHECKSUM_TYPE_NAME);
304 if (checksumName == null) {
305 return HFile.DEFAULT_CHECKSUM_TYPE;
306 } else {
307 return ChecksumType.nameToType(checksumName);
308 }
309 }
310
311
312
313
314 public static int getCloseCheckInterval() {
315 return closeCheckInterval;
316 }
317
318 public HColumnDescriptor getFamily() {
319 return this.family;
320 }
321
322
323
324
325 long getMaxSequenceId(boolean includeBulkFiles) {
326 return StoreFile.getMaxSequenceIdInList(this.getStorefiles(), includeBulkFiles);
327 }
328
329 @Override
330 public long getMaxMemstoreTS() {
331 return StoreFile.getMaxMemstoreTSInList(this.getStorefiles());
332 }
333
334
335
336
337
338
339
340 @Deprecated
341 public static Path getStoreHomedir(final Path tabledir,
342 final HRegionInfo hri, final byte[] family) {
343 return getStoreHomedir(tabledir, hri.getEncodedName(), family);
344 }
345
346
347
348
349
350
351
352 @Deprecated
353 public static Path getStoreHomedir(final Path tabledir,
354 final String encodedName, final byte[] family) {
355 return new Path(tabledir, new Path(encodedName, Bytes.toString(family)));
356 }
357
358 @Override
359 public HFileDataBlockEncoder getDataBlockEncoder() {
360 return dataBlockEncoder;
361 }
362
363
364
365
366
367 void setDataBlockEncoderInTest(HFileDataBlockEncoder blockEncoder) {
368 this.dataBlockEncoder = blockEncoder;
369 }
370
371
372
373
374
375
376 private List<StoreFile> loadStoreFiles() throws IOException {
377 Collection<StoreFileInfo> files = fs.getStoreFiles(getColumnFamilyName());
378 if (files == null || files.size() == 0) {
379 return new ArrayList<StoreFile>();
380 }
381
382
383 ThreadPoolExecutor storeFileOpenerThreadPool =
384 this.region.getStoreFileOpenAndCloseThreadPool("StoreFileOpenerThread-" +
385 this.getColumnFamilyName());
386 CompletionService<StoreFile> completionService =
387 new ExecutorCompletionService<StoreFile>(storeFileOpenerThreadPool);
388
389 int totalValidStoreFile = 0;
390 final FileSystem fs = this.getFileSystem();
391 for (final StoreFileInfo storeFileInfo: files) {
392
393 completionService.submit(new Callable<StoreFile>() {
394 public StoreFile call() throws IOException {
395 StoreFile storeFile = new StoreFile(fs, storeFileInfo.getPath(), conf, cacheConf,
396 family.getBloomFilterType(), dataBlockEncoder);
397 storeFile.createReader();
398 return storeFile;
399 }
400 });
401 totalValidStoreFile++;
402 }
403
404 ArrayList<StoreFile> results = new ArrayList<StoreFile>(files.size());
405 IOException ioe = null;
406 try {
407 for (int i = 0; i < totalValidStoreFile; i++) {
408 try {
409 Future<StoreFile> future = completionService.take();
410 StoreFile storeFile = future.get();
411 long length = storeFile.getReader().length();
412 this.storeSize += length;
413 this.totalUncompressedBytes +=
414 storeFile.getReader().getTotalUncompressedBytes();
415 if (LOG.isDebugEnabled()) {
416 LOG.debug("loaded " + storeFile.toStringDetailed());
417 }
418 results.add(storeFile);
419 } catch (InterruptedException e) {
420 if (ioe == null) ioe = new InterruptedIOException(e.getMessage());
421 } catch (ExecutionException e) {
422 if (ioe == null) ioe = new IOException(e.getCause());
423 }
424 }
425 } finally {
426 storeFileOpenerThreadPool.shutdownNow();
427 }
428 if (ioe != null) {
429
430 try {
431 for (StoreFile file : results) {
432 if (file != null) file.closeReader(true);
433 }
434 } catch (IOException e) { }
435 throw ioe;
436 }
437
438 return results;
439 }
440
441 @Override
442 public long add(final KeyValue kv) {
443 lock.readLock().lock();
444 try {
445 return this.memstore.add(kv);
446 } finally {
447 lock.readLock().unlock();
448 }
449 }
450
451
452
453
454
455
456
457 protected long delete(final KeyValue kv) {
458 lock.readLock().lock();
459 try {
460 return this.memstore.delete(kv);
461 } finally {
462 lock.readLock().unlock();
463 }
464 }
465
466 @Override
467 public void rollback(final KeyValue kv) {
468 lock.readLock().lock();
469 try {
470 this.memstore.rollback(kv);
471 } finally {
472 lock.readLock().unlock();
473 }
474 }
475
476
477
478
479 @Override
480 public Collection<StoreFile> getStorefiles() {
481 return this.storeEngine.getStoreFileManager().getStorefiles();
482 }
483
484 @Override
485 public void assertBulkLoadHFileOk(Path srcPath) throws IOException {
486 HFile.Reader reader = null;
487 try {
488 LOG.info("Validating hfile at " + srcPath + " for inclusion in "
489 + "store " + this + " region " + this.getRegionInfo().getRegionNameAsString());
490 reader = HFile.createReader(srcPath.getFileSystem(conf),
491 srcPath, cacheConf);
492 reader.loadFileInfo();
493
494 byte[] firstKey = reader.getFirstRowKey();
495 Preconditions.checkState(firstKey != null, "First key can not be null");
496 byte[] lk = reader.getLastKey();
497 Preconditions.checkState(lk != null, "Last key can not be null");
498 byte[] lastKey = KeyValue.createKeyValueFromKey(lk).getRow();
499
500 LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey) +
501 " last=" + Bytes.toStringBinary(lastKey));
502 LOG.debug("Region bounds: first=" +
503 Bytes.toStringBinary(getRegionInfo().getStartKey()) +
504 " last=" + Bytes.toStringBinary(getRegionInfo().getEndKey()));
505
506 if (!this.getRegionInfo().containsRange(firstKey, lastKey)) {
507 throw new WrongRegionException(
508 "Bulk load file " + srcPath.toString() + " does not fit inside region "
509 + this.getRegionInfo().getRegionNameAsString());
510 }
511
512 if (verifyBulkLoads) {
513 KeyValue prevKV = null;
514 HFileScanner scanner = reader.getScanner(false, false, false);
515 scanner.seekTo();
516 do {
517 KeyValue kv = scanner.getKeyValue();
518 if (prevKV != null) {
519 if (Bytes.compareTo(prevKV.getBuffer(), prevKV.getRowOffset(),
520 prevKV.getRowLength(), kv.getBuffer(), kv.getRowOffset(),
521 kv.getRowLength()) > 0) {
522 throw new InvalidHFileException("Previous row is greater than"
523 + " current row: path=" + srcPath + " previous="
524 + Bytes.toStringBinary(prevKV.getKey()) + " current="
525 + Bytes.toStringBinary(kv.getKey()));
526 }
527 if (Bytes.compareTo(prevKV.getBuffer(), prevKV.getFamilyOffset(),
528 prevKV.getFamilyLength(), kv.getBuffer(), kv.getFamilyOffset(),
529 kv.getFamilyLength()) != 0) {
530 throw new InvalidHFileException("Previous key had different"
531 + " family compared to current key: path=" + srcPath
532 + " previous=" + Bytes.toStringBinary(prevKV.getFamily())
533 + " current=" + Bytes.toStringBinary(kv.getFamily()));
534 }
535 }
536 prevKV = kv;
537 } while (scanner.next());
538 }
539 } finally {
540 if (reader != null) reader.close();
541 }
542 }
543
544 @Override
545 public void bulkLoadHFile(String srcPathStr, long seqNum) throws IOException {
546 Path srcPath = new Path(srcPathStr);
547 Path dstPath = fs.bulkLoadStoreFile(getColumnFamilyName(), srcPath, seqNum);
548
549 StoreFile sf = new StoreFile(this.getFileSystem(), dstPath, this.conf, this.cacheConf,
550 this.family.getBloomFilterType(), this.dataBlockEncoder);
551
552 StoreFile.Reader r = sf.createReader();
553 this.storeSize += r.length();
554 this.totalUncompressedBytes += r.getTotalUncompressedBytes();
555
556 LOG.info("Loaded HFile " + srcPath + " into store '" + getColumnFamilyName() +
557 "' as " + dstPath + " - updating store file list.");
558
559
560 this.lock.writeLock().lock();
561 try {
562 this.storeEngine.getStoreFileManager().insertNewFile(sf);
563 } finally {
564
565
566
567
568
569 this.lock.writeLock().unlock();
570 }
571 notifyChangedReadersObservers();
572 LOG.info("Successfully loaded store file " + srcPath
573 + " into store " + this + " (new location: " + dstPath + ")");
574 }
575
576 @Override
577 public ImmutableCollection<StoreFile> close() throws IOException {
578 this.lock.writeLock().lock();
579 try {
580
581 ImmutableCollection<StoreFile> result = storeEngine.getStoreFileManager().clearFiles();
582
583 if (!result.isEmpty()) {
584
585 ThreadPoolExecutor storeFileCloserThreadPool = this.region
586 .getStoreFileOpenAndCloseThreadPool("StoreFileCloserThread-"
587 + this.getColumnFamilyName());
588
589
590 CompletionService<Void> completionService =
591 new ExecutorCompletionService<Void>(storeFileCloserThreadPool);
592 for (final StoreFile f : result) {
593 completionService.submit(new Callable<Void>() {
594 public Void call() throws IOException {
595 f.closeReader(true);
596 return null;
597 }
598 });
599 }
600
601 IOException ioe = null;
602 try {
603 for (int i = 0; i < result.size(); i++) {
604 try {
605 Future<Void> future = completionService.take();
606 future.get();
607 } catch (InterruptedException e) {
608 if (ioe == null) {
609 ioe = new InterruptedIOException();
610 ioe.initCause(e);
611 }
612 } catch (ExecutionException e) {
613 if (ioe == null) ioe = new IOException(e.getCause());
614 }
615 }
616 } finally {
617 storeFileCloserThreadPool.shutdownNow();
618 }
619 if (ioe != null) throw ioe;
620 }
621 LOG.info("Closed " + this);
622 return result;
623 } finally {
624 this.lock.writeLock().unlock();
625 }
626 }
627
628
629
630
631
632
633 void snapshot() {
634 this.memstore.snapshot();
635 }
636
637
638
639
640
641
642
643
644
645
646
647
648 protected Path flushCache(final long logCacheFlushId,
649 SortedSet<KeyValue> snapshot,
650 TimeRangeTracker snapshotTimeRangeTracker,
651 AtomicLong flushedSize,
652 MonitoredTask status) throws IOException {
653
654
655
656
657
658 IOException lastException = null;
659 for (int i = 0; i < HStore.flush_retries_number; i++) {
660 try {
661 Path pathName = internalFlushCache(snapshot, logCacheFlushId,
662 snapshotTimeRangeTracker, flushedSize, status);
663 try {
664
665 if (pathName != null) {
666 validateStoreFile(pathName);
667 }
668 return pathName;
669 } catch (Exception e) {
670 LOG.warn("Failed validating store file " + pathName
671 + ", retring num=" + i, e);
672 if (e instanceof IOException) {
673 lastException = (IOException) e;
674 } else {
675 lastException = new IOException(e);
676 }
677 }
678 } catch (IOException e) {
679 LOG.warn("Failed flushing store file, retring num=" + i, e);
680 lastException = e;
681 }
682 if (lastException != null) {
683 try {
684 Thread.sleep(pauseTime);
685 } catch (InterruptedException e) {
686 IOException iie = new InterruptedIOException();
687 iie.initCause(e);
688 throw iie;
689 }
690 }
691 }
692 throw lastException;
693 }
694
695
696
697
698
699
700
701
702
703 private Path internalFlushCache(final SortedSet<KeyValue> set,
704 final long logCacheFlushId,
705 TimeRangeTracker snapshotTimeRangeTracker,
706 AtomicLong flushedSize,
707 MonitoredTask status)
708 throws IOException {
709 StoreFile.Writer writer;
710
711 long smallestReadPoint = region.getSmallestReadPoint();
712 long flushed = 0;
713 Path pathName;
714
715 if (set.size() == 0) {
716 return null;
717 }
718
719
720
721 InternalScanner scanner = null;
722 KeyValueScanner memstoreScanner = new CollectionBackedScanner(set, this.comparator);
723 if (this.getCoprocessorHost() != null) {
724 scanner = this.getCoprocessorHost().preFlushScannerOpen(this, memstoreScanner);
725 }
726 if (scanner == null) {
727 Scan scan = new Scan();
728 scan.setMaxVersions(scanInfo.getMaxVersions());
729 scanner = new StoreScanner(this, scanInfo, scan,
730 Collections.singletonList(memstoreScanner), ScanType.COMPACT_RETAIN_DELETES,
731 smallestReadPoint, HConstants.OLDEST_TIMESTAMP);
732 }
733 if (this.getCoprocessorHost() != null) {
734 InternalScanner cpScanner =
735 this.getCoprocessorHost().preFlush(this, scanner);
736
737 if (cpScanner == null) {
738 return null;
739 }
740 scanner = cpScanner;
741 }
742 try {
743 int compactionKVMax = conf.getInt(HConstants.COMPACTION_KV_MAX, 10);
744
745
746
747 synchronized (flushLock) {
748 status.setStatus("Flushing " + this + ": creating writer");
749
750 writer = createWriterInTmp(set.size());
751 writer.setTimeRangeTracker(snapshotTimeRangeTracker);
752 pathName = writer.getPath();
753 try {
754 List<KeyValue> kvs = new ArrayList<KeyValue>();
755 boolean hasMore;
756 do {
757 hasMore = scanner.next(kvs, compactionKVMax);
758 if (!kvs.isEmpty()) {
759 for (KeyValue kv : kvs) {
760
761
762
763 if (kv.getMemstoreTS() <= smallestReadPoint) {
764
765
766 kv = kv.shallowCopy();
767 kv.setMemstoreTS(0);
768 }
769 writer.append(kv);
770 flushed += this.memstore.heapSizeChange(kv, true);
771 }
772 kvs.clear();
773 }
774 } while (hasMore);
775 } finally {
776
777
778
779 status.setStatus("Flushing " + this + ": appending metadata");
780 writer.appendMetadata(logCacheFlushId, false);
781 status.setStatus("Flushing " + this + ": closing flushed file");
782 writer.close();
783 }
784 }
785 } finally {
786 flushedSize.set(flushed);
787 scanner.close();
788 }
789 if (LOG.isInfoEnabled()) {
790 LOG.info("Flushed " +
791 ", sequenceid=" + logCacheFlushId +
792 ", memsize=" + StringUtils.humanReadableInt(flushed) +
793 ", into tmp file " + pathName);
794 }
795 return pathName;
796 }
797
798
799
800
801
802
803
804 private StoreFile commitFile(final Path path,
805 final long logCacheFlushId,
806 TimeRangeTracker snapshotTimeRangeTracker,
807 AtomicLong flushedSize,
808 MonitoredTask status)
809 throws IOException {
810
811 Path dstPath = fs.commitStoreFile(getColumnFamilyName(), path);
812
813 status.setStatus("Flushing " + this + ": reopening flushed file");
814 StoreFile sf = new StoreFile(this.getFileSystem(), dstPath, this.conf, this.cacheConf,
815 this.family.getBloomFilterType(), this.dataBlockEncoder);
816
817 StoreFile.Reader r = sf.createReader();
818 this.storeSize += r.length();
819 this.totalUncompressedBytes += r.getTotalUncompressedBytes();
820
821 if (LOG.isInfoEnabled()) {
822 LOG.info("Added " + sf + ", entries=" + r.getEntries() +
823 ", sequenceid=" + logCacheFlushId +
824 ", filesize=" + StringUtils.humanReadableInt(r.length()));
825 }
826 return sf;
827 }
828
829
830
831
832
833 private StoreFile.Writer createWriterInTmp(long maxKeyCount)
834 throws IOException {
835 return createWriterInTmp(maxKeyCount, this.family.getCompression(), false, true);
836 }
837
838
839
840
841
842
843
844 public StoreFile.Writer createWriterInTmp(long maxKeyCount,
845 Compression.Algorithm compression, boolean isCompaction, boolean includeMVCCReadpoint)
846 throws IOException {
847 final CacheConfig writerCacheConf;
848 if (isCompaction) {
849
850 writerCacheConf = new CacheConfig(cacheConf);
851 writerCacheConf.setCacheDataOnWrite(false);
852 } else {
853 writerCacheConf = cacheConf;
854 }
855 StoreFile.Writer w = new StoreFile.WriterBuilder(conf, writerCacheConf,
856 this.getFileSystem(), blocksize)
857 .withFilePath(fs.createTempName())
858 .withDataBlockEncoder(dataBlockEncoder)
859 .withComparator(comparator)
860 .withBloomType(family.getBloomFilterType())
861 .withMaxKeyCount(maxKeyCount)
862 .withChecksumType(checksumType)
863 .withBytesPerChecksum(bytesPerChecksum)
864 .withCompression(compression)
865 .includeMVCCReadpoint(includeMVCCReadpoint)
866 .build();
867 return w;
868 }
869
870
871
872
873
874
875
876
877 private boolean updateStorefiles(final StoreFile sf,
878 final SortedSet<KeyValue> set)
879 throws IOException {
880 this.lock.writeLock().lock();
881 try {
882 this.storeEngine.getStoreFileManager().insertNewFile(sf);
883 this.memstore.clearSnapshot(set);
884 } finally {
885
886
887
888
889
890 this.lock.writeLock().unlock();
891 }
892
893
894 notifyChangedReadersObservers();
895
896 return needsCompaction();
897 }
898
899
900
901
902
903 private void notifyChangedReadersObservers() throws IOException {
904 for (ChangedReadersObserver o: this.changedReaderObservers) {
905 o.updateReaders();
906 }
907 }
908
909
910
911
912
913
914 @Override
915 public List<KeyValueScanner> getScanners(boolean cacheBlocks,
916 boolean isGet, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow,
917 byte[] stopRow) throws IOException {
918 Collection<StoreFile> storeFilesToScan;
919 List<KeyValueScanner> memStoreScanners;
920 this.lock.readLock().lock();
921 try {
922 storeFilesToScan =
923 this.storeEngine.getStoreFileManager().getFilesForScanOrGet(isGet, startRow, stopRow);
924 memStoreScanners = this.memstore.getScanners();
925 } finally {
926 this.lock.readLock().unlock();
927 }
928
929
930
931
932
933
934 List<StoreFileScanner> sfScanners = StoreFileScanner
935 .getScannersForStoreFiles(storeFilesToScan, cacheBlocks, isGet, isCompaction, matcher);
936 List<KeyValueScanner> scanners =
937 new ArrayList<KeyValueScanner>(sfScanners.size()+1);
938 scanners.addAll(sfScanners);
939
940 scanners.addAll(memStoreScanners);
941 return scanners;
942 }
943
944 @Override
945 public void addChangedReaderObserver(ChangedReadersObserver o) {
946 this.changedReaderObservers.add(o);
947 }
948
949 @Override
950 public void deleteChangedReaderObserver(ChangedReadersObserver o) {
951
952 this.changedReaderObservers.remove(o);
953 }
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979 public List<StoreFile> compact(CompactionContext compaction) throws IOException {
980 assert compaction != null && compaction.hasSelection();
981 CompactionRequest cr = compaction.getRequest();
982 Collection<StoreFile> filesToCompact = cr.getFiles();
983 assert !filesToCompact.isEmpty();
984 synchronized (filesCompacting) {
985
986
987 Preconditions.checkArgument(filesCompacting.containsAll(filesToCompact));
988 }
989
990
991 LOG.info("Starting compaction of " + filesToCompact.size() + " file(s) in "
992 + this + " of " + this.getRegionInfo().getRegionNameAsString()
993 + " into tmpdir=" + fs.getTempDir() + ", totalSize="
994 + StringUtils.humanReadableInt(cr.getSize()));
995
996 List<StoreFile> sfs = new ArrayList<StoreFile>();
997 long compactionStartTime = EnvironmentEdgeManager.currentTimeMillis();
998 try {
999
1000 List<Path> newFiles = compaction.compact();
1001
1002 if (this.conf.getBoolean("hbase.hstore.compaction.complete", true)) {
1003 for (Path newFile: newFiles) {
1004 assert newFile != null;
1005 StoreFile sf = moveFileIntoPlace(newFile);
1006 if (this.getCoprocessorHost() != null) {
1007 this.getCoprocessorHost().postCompact(this, sf, cr);
1008 }
1009 assert sf != null;
1010 sfs.add(sf);
1011 }
1012 completeCompaction(filesToCompact, sfs);
1013 } else {
1014 for (Path newFile: newFiles) {
1015
1016 StoreFile sf = new StoreFile(this.getFileSystem(), newFile, this.conf, this.cacheConf,
1017 this.family.getBloomFilterType(), this.dataBlockEncoder);
1018 sf.createReader();
1019 sfs.add(sf);
1020 }
1021 }
1022 } finally {
1023 finishCompactionRequest(cr);
1024 }
1025 logCompactionEndMessage(cr, sfs, compactionStartTime);
1026 return sfs;
1027 }
1028
1029
1030
1031
1032
1033
1034
1035 private void logCompactionEndMessage(
1036 CompactionRequest cr, List<StoreFile> sfs, long compactionStartTime) {
1037 long now = EnvironmentEdgeManager.currentTimeMillis();
1038 StringBuilder message = new StringBuilder(
1039 "Completed" + (cr.isMajor() ? " major " : " ") + "compaction of "
1040 + cr.getFiles().size() + " file(s) in " + this + " of "
1041 + this.getRegionInfo().getRegionNameAsString()
1042 + " into ");
1043 if (sfs.isEmpty()) {
1044 message.append("none, ");
1045 } else {
1046 for (StoreFile sf: sfs) {
1047 message.append(sf.getPath().getName());
1048 message.append("(size=");
1049 message.append(StringUtils.humanReadableInt(sf.getReader().length()));
1050 message.append("), ");
1051 }
1052 }
1053 message.append("total size for store is ")
1054 .append(StringUtils.humanReadableInt(storeSize))
1055 .append(". This selection was in queue for ")
1056 .append(StringUtils.formatTimeDiff(compactionStartTime, cr.getSelectionTime()))
1057 .append(", and took ").append(StringUtils.formatTimeDiff(now, compactionStartTime))
1058 .append(" to execute.");
1059 LOG.info(message.toString());
1060 }
1061
1062
1063 StoreFile moveFileIntoPlace(final Path newFile) throws IOException {
1064 validateStoreFile(newFile);
1065
1066 Path destPath = fs.commitStoreFile(getColumnFamilyName(), newFile);
1067 StoreFile result = new StoreFile(this.getFileSystem(), destPath, this.conf, this.cacheConf,
1068 this.family.getBloomFilterType(), this.dataBlockEncoder);
1069 result.createReader();
1070 return result;
1071 }
1072
1073
1074
1075
1076
1077
1078
1079
1080 public void compactRecentForTestingAssumingDefaultPolicy(int N) throws IOException {
1081 List<StoreFile> filesToCompact;
1082 boolean isMajor;
1083
1084 this.lock.readLock().lock();
1085 try {
1086 synchronized (filesCompacting) {
1087 filesToCompact = Lists.newArrayList(storeEngine.getStoreFileManager().getStorefiles());
1088 if (!filesCompacting.isEmpty()) {
1089
1090
1091 StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
1092 int idx = filesToCompact.indexOf(last);
1093 Preconditions.checkArgument(idx != -1);
1094 filesToCompact.subList(0, idx + 1).clear();
1095 }
1096 int count = filesToCompact.size();
1097 if (N > count) {
1098 throw new RuntimeException("Not enough files");
1099 }
1100
1101 filesToCompact = filesToCompact.subList(count - N, count);
1102 isMajor = (filesToCompact.size() == storeEngine.getStoreFileManager().getStorefileCount());
1103 filesCompacting.addAll(filesToCompact);
1104 Collections.sort(filesCompacting, StoreFile.Comparators.SEQ_ID);
1105 }
1106 } finally {
1107 this.lock.readLock().unlock();
1108 }
1109
1110 try {
1111
1112 List<Path> newFiles =
1113 this.storeEngine.getCompactor().compactForTesting(filesToCompact, isMajor);
1114 for (Path newFile: newFiles) {
1115
1116 StoreFile sf = moveFileIntoPlace(newFile);
1117 if (this.getCoprocessorHost() != null) {
1118 this.getCoprocessorHost().postCompact(this, sf, null);
1119 }
1120 ArrayList<StoreFile> tmp = new ArrayList<StoreFile>();
1121 tmp.add(sf);
1122 completeCompaction(filesToCompact, tmp);
1123 }
1124 } finally {
1125 synchronized (filesCompacting) {
1126 filesCompacting.removeAll(filesToCompact);
1127 }
1128 }
1129 }
1130
1131 @Override
1132 public boolean hasReferences() {
1133 return StoreUtils.hasReferences(this.storeEngine.getStoreFileManager().getStorefiles());
1134 }
1135
1136 @Override
1137 public CompactionProgress getCompactionProgress() {
1138 return this.storeEngine.getCompactor().getProgress();
1139 }
1140
1141 @Override
1142 public boolean isMajorCompaction() throws IOException {
1143 for (StoreFile sf : this.storeEngine.getStoreFileManager().getStorefiles()) {
1144
1145 if (sf.getReader() == null) {
1146 LOG.debug("StoreFile " + sf + " has null Reader");
1147 return false;
1148 }
1149 }
1150 return storeEngine.getCompactionPolicy().isMajorCompaction(
1151 this.storeEngine.getStoreFileManager().getStorefiles());
1152 }
1153
1154 @Override
1155 public CompactionContext requestCompaction() throws IOException {
1156 return requestCompaction(Store.NO_PRIORITY, null);
1157 }
1158
1159 @Override
1160 public CompactionContext requestCompaction(int priority, CompactionRequest baseRequest)
1161 throws IOException {
1162
1163 if (!this.areWritesEnabled()) {
1164 return null;
1165 }
1166
1167 CompactionContext compaction = storeEngine.createCompaction();
1168 this.lock.readLock().lock();
1169 try {
1170 synchronized (filesCompacting) {
1171
1172 if (this.getCoprocessorHost() != null) {
1173 List<StoreFile> candidatesForCoproc = compaction.preSelect(this.filesCompacting);
1174 boolean override = this.getCoprocessorHost().preCompactSelection(
1175 this, candidatesForCoproc, baseRequest);
1176 if (override) {
1177
1178 compaction.forceSelect(new CompactionRequest(candidatesForCoproc));
1179 }
1180 }
1181
1182
1183 if (!compaction.hasSelection()) {
1184 boolean isUserCompaction = priority == Store.PRIORITY_USER;
1185 boolean mayUseOffPeak = this.offPeakCompactions.tryStartOffPeakRequest();
1186 compaction.select(this.filesCompacting, isUserCompaction,
1187 mayUseOffPeak, forceMajor && filesCompacting.isEmpty());
1188 assert compaction.hasSelection();
1189 if (mayUseOffPeak && !compaction.getRequest().isOffPeak()) {
1190
1191 this.offPeakCompactions.endOffPeakRequest();
1192 }
1193 }
1194 if (this.getCoprocessorHost() != null) {
1195 this.getCoprocessorHost().postCompactSelection(
1196 this, ImmutableList.copyOf(compaction.getRequest().getFiles()), baseRequest);
1197 }
1198
1199
1200 if (baseRequest != null) {
1201
1202
1203 compaction.forceSelect(
1204 baseRequest.combineWith(compaction.getRequest()));
1205 }
1206
1207
1208 final Collection<StoreFile> selectedFiles = compaction.getRequest().getFiles();
1209 if (selectedFiles.isEmpty()) {
1210 return null;
1211 }
1212
1213
1214 if (!Collections.disjoint(filesCompacting, selectedFiles)) {
1215
1216 Preconditions.checkArgument(false, "%s overlaps with %s",
1217 selectedFiles, filesCompacting);
1218 }
1219 filesCompacting.addAll(selectedFiles);
1220 Collections.sort(filesCompacting, StoreFile.Comparators.SEQ_ID);
1221
1222
1223 boolean isMajor = selectedFiles.size() == this.getStorefilesCount();
1224 this.forceMajor = this.forceMajor && !isMajor;
1225
1226
1227
1228 compaction.getRequest().setPriority(
1229 (priority != Store.NO_PRIORITY) ? priority : getCompactPriority());
1230 compaction.getRequest().setIsMajor(isMajor);
1231 compaction.getRequest().setDescription(
1232 getRegionInfo().getRegionNameAsString(), getColumnFamilyName());
1233 }
1234 } finally {
1235 this.lock.readLock().unlock();
1236 }
1237
1238 LOG.debug(getRegionInfo().getEncodedName() + " - " + getColumnFamilyName() + ": Initiating "
1239 + (compaction.getRequest().isMajor() ? "major" : "minor") + " compaction");
1240 this.region.reportCompactionRequestStart(compaction.getRequest().isMajor());
1241 return compaction;
1242 }
1243
1244 public void cancelRequestedCompaction(CompactionContext compaction) {
1245 finishCompactionRequest(compaction.getRequest());
1246 }
1247
1248 private void finishCompactionRequest(CompactionRequest cr) {
1249 this.region.reportCompactionRequestEnd(cr.isMajor());
1250 if (cr.isOffPeak()) {
1251 this.offPeakCompactions.endOffPeakRequest();
1252 cr.setOffPeak(false);
1253 }
1254 synchronized (filesCompacting) {
1255 filesCompacting.removeAll(cr.getFiles());
1256 }
1257 }
1258
1259
1260
1261
1262
1263
1264
1265 private void validateStoreFile(Path path)
1266 throws IOException {
1267 StoreFile storeFile = null;
1268 try {
1269 storeFile = new StoreFile(this.getFileSystem(), path, this.conf,
1270 this.cacheConf, this.family.getBloomFilterType(),
1271 NoOpDataBlockEncoder.INSTANCE);
1272 storeFile.createReader();
1273 } catch (IOException e) {
1274 LOG.error("Failed to open store file : " + path
1275 + ", keeping it in tmp location", e);
1276 throw e;
1277 } finally {
1278 if (storeFile != null) {
1279 storeFile.closeReader(false);
1280 }
1281 }
1282 }
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303 private void completeCompaction(final Collection<StoreFile> compactedFiles,
1304 final Collection<StoreFile> result) throws IOException {
1305 try {
1306 this.lock.writeLock().lock();
1307 try {
1308
1309
1310
1311
1312 this.storeEngine.getStoreFileManager().addCompactionResults(compactedFiles, result);
1313 filesCompacting.removeAll(compactedFiles);
1314 } finally {
1315
1316
1317
1318
1319
1320 this.lock.writeLock().unlock();
1321 }
1322
1323
1324 notifyChangedReadersObservers();
1325
1326
1327 LOG.debug("Removing store files after compaction...");
1328 this.fs.removeStoreFiles(this.getColumnFamilyName(), compactedFiles);
1329
1330 } catch (IOException e) {
1331 e = RemoteExceptionHandler.checkIOException(e);
1332 LOG.error("Failed replacing compacted files in " + this +
1333 ". Compacted files are " + (result == null? "none": result.toString()) +
1334 ". Files replaced " + compactedFiles.toString() +
1335 " some of which may have been already removed", e);
1336 }
1337
1338
1339 this.storeSize = 0L;
1340 this.totalUncompressedBytes = 0L;
1341 for (StoreFile hsf : this.storeEngine.getStoreFileManager().getStorefiles()) {
1342 StoreFile.Reader r = hsf.getReader();
1343 if (r == null) {
1344 LOG.warn("StoreFile " + hsf + " has a null Reader");
1345 continue;
1346 }
1347 this.storeSize += r.length();
1348 this.totalUncompressedBytes += r.getTotalUncompressedBytes();
1349 }
1350 }
1351
1352
1353
1354
1355
1356 int versionsToReturn(final int wantedVersions) {
1357 if (wantedVersions <= 0) {
1358 throw new IllegalArgumentException("Number of versions must be > 0");
1359 }
1360
1361 int maxVersions = this.family.getMaxVersions();
1362 return wantedVersions > maxVersions ? maxVersions: wantedVersions;
1363 }
1364
1365 static boolean isExpired(final KeyValue key, final long oldestTimestamp) {
1366 return key.getTimestamp() < oldestTimestamp;
1367 }
1368
1369 @Override
1370 public KeyValue getRowKeyAtOrBefore(final byte[] row) throws IOException {
1371
1372
1373
1374
1375
1376
1377 long ttlToUse = scanInfo.getMinVersions() > 0 ? Long.MAX_VALUE : this.scanInfo.getTtl();
1378
1379 KeyValue kv = new KeyValue(row, HConstants.LATEST_TIMESTAMP);
1380
1381 GetClosestRowBeforeTracker state = new GetClosestRowBeforeTracker(
1382 this.comparator, kv, ttlToUse, this.getRegionInfo().isMetaRegion());
1383 this.lock.readLock().lock();
1384 try {
1385
1386 this.memstore.getRowKeyAtOrBefore(state);
1387
1388
1389 Iterator<StoreFile> sfIterator = this.storeEngine.getStoreFileManager()
1390 .getCandidateFilesForRowKeyBefore(state.getTargetKey());
1391 while (sfIterator.hasNext()) {
1392 StoreFile sf = sfIterator.next();
1393 sfIterator.remove();
1394 boolean haveNewCandidate = rowAtOrBeforeFromStoreFile(sf, state);
1395 if (haveNewCandidate) {
1396
1397 sfIterator = this.storeEngine.getStoreFileManager().updateCandidateFilesForRowKeyBefore(
1398 sfIterator, state.getTargetKey(), state.getCandidate());
1399 }
1400 }
1401 return state.getCandidate();
1402 } finally {
1403 this.lock.readLock().unlock();
1404 }
1405 }
1406
1407
1408
1409
1410
1411
1412
1413
1414 private boolean rowAtOrBeforeFromStoreFile(final StoreFile f,
1415 final GetClosestRowBeforeTracker state)
1416 throws IOException {
1417 StoreFile.Reader r = f.getReader();
1418 if (r == null) {
1419 LOG.warn("StoreFile " + f + " has a null Reader");
1420 return false;
1421 }
1422 if (r.getEntries() == 0) {
1423 LOG.warn("StoreFile " + f + " is a empty store file");
1424 return false;
1425 }
1426
1427 byte [] fk = r.getFirstKey();
1428 if (fk == null) return false;
1429 KeyValue firstKV = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
1430 byte [] lk = r.getLastKey();
1431 KeyValue lastKV = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
1432 KeyValue firstOnRow = state.getTargetKey();
1433 if (this.comparator.compareRows(lastKV, firstOnRow) < 0) {
1434
1435
1436 if (!state.isTargetTable(lastKV)) return false;
1437
1438
1439 firstOnRow = new KeyValue(lastKV.getRow(), HConstants.LATEST_TIMESTAMP);
1440 }
1441
1442 HFileScanner scanner = r.getScanner(true, true, false);
1443
1444 if (!seekToScanner(scanner, firstOnRow, firstKV)) return false;
1445
1446
1447 if (walkForwardInSingleRow(scanner, firstOnRow, state)) return true;
1448
1449 while (scanner.seekBefore(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(),
1450 firstOnRow.getKeyLength())) {
1451 KeyValue kv = scanner.getKeyValue();
1452 if (!state.isTargetTable(kv)) break;
1453 if (!state.isBetterCandidate(kv)) break;
1454
1455 firstOnRow = new KeyValue(kv.getRow(), HConstants.LATEST_TIMESTAMP);
1456
1457 if (!seekToScanner(scanner, firstOnRow, firstKV)) return false;
1458
1459 if (walkForwardInSingleRow(scanner, firstOnRow, state)) return true;
1460 }
1461 return false;
1462 }
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472 private boolean seekToScanner(final HFileScanner scanner,
1473 final KeyValue firstOnRow,
1474 final KeyValue firstKV)
1475 throws IOException {
1476 KeyValue kv = firstOnRow;
1477
1478 if (this.comparator.compareRows(firstKV, firstOnRow) == 0) kv = firstKV;
1479 int result = scanner.seekTo(kv.getBuffer(), kv.getKeyOffset(),
1480 kv.getKeyLength());
1481 return result >= 0;
1482 }
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494 private boolean walkForwardInSingleRow(final HFileScanner scanner,
1495 final KeyValue firstOnRow,
1496 final GetClosestRowBeforeTracker state)
1497 throws IOException {
1498 boolean foundCandidate = false;
1499 do {
1500 KeyValue kv = scanner.getKeyValue();
1501
1502 if (this.comparator.compareRows(kv, firstOnRow) < 0) continue;
1503
1504 if (state.isTooFar(kv, firstOnRow)) break;
1505 if (state.isExpired(kv)) {
1506 continue;
1507 }
1508
1509 if (state.handle(kv)) {
1510 foundCandidate = true;
1511 break;
1512 }
1513 } while(scanner.next());
1514 return foundCandidate;
1515 }
1516
1517 public boolean canSplit() {
1518 this.lock.readLock().lock();
1519 try {
1520
1521 boolean result = !hasReferences();
1522 if (!result && LOG.isDebugEnabled()) {
1523 LOG.debug("Cannot split region due to reference files being there");
1524 }
1525 return result;
1526 } finally {
1527 this.lock.readLock().unlock();
1528 }
1529 }
1530
1531 @Override
1532 public byte[] getSplitPoint() {
1533 this.lock.readLock().lock();
1534 try {
1535
1536 assert !this.getRegionInfo().isMetaRegion();
1537
1538 if (hasReferences()) {
1539 assert false : "getSplitPoint() called on a region that can't split!";
1540 return null;
1541 }
1542 return this.storeEngine.getStoreFileManager().getSplitPoint();
1543 } catch(IOException e) {
1544 LOG.warn("Failed getting store size for " + this, e);
1545 } finally {
1546 this.lock.readLock().unlock();
1547 }
1548 return null;
1549 }
1550
1551 @Override
1552 public long getLastCompactSize() {
1553 return this.lastCompactSize;
1554 }
1555
1556 @Override
1557 public long getSize() {
1558 return storeSize;
1559 }
1560
1561 public void triggerMajorCompaction() {
1562 this.forceMajor = true;
1563 }
1564
1565 boolean getForceMajorCompaction() {
1566 return this.forceMajor;
1567 }
1568
1569
1570
1571
1572
1573 @Override
1574 public KeyValueScanner getScanner(Scan scan,
1575 final NavigableSet<byte []> targetCols) throws IOException {
1576 lock.readLock().lock();
1577 try {
1578 KeyValueScanner scanner = null;
1579 if (this.getCoprocessorHost() != null) {
1580 scanner = this.getCoprocessorHost().preStoreScannerOpen(this, scan, targetCols);
1581 }
1582 if (scanner == null) {
1583 scanner = new StoreScanner(this, getScanInfo(), scan, targetCols);
1584 }
1585 return scanner;
1586 } finally {
1587 lock.readLock().unlock();
1588 }
1589 }
1590
1591 @Override
1592 public String toString() {
1593 return this.getColumnFamilyName();
1594 }
1595
1596 @Override
1597
1598 public int getStorefilesCount() {
1599 return this.storeEngine.getStoreFileManager().getStorefileCount();
1600 }
1601
1602 @Override
1603 public long getStoreSizeUncompressed() {
1604 return this.totalUncompressedBytes;
1605 }
1606
1607 @Override
1608 public long getStorefilesSize() {
1609 long size = 0;
1610 for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) {
1611 StoreFile.Reader r = s.getReader();
1612 if (r == null) {
1613 LOG.warn("StoreFile " + s + " has a null Reader");
1614 continue;
1615 }
1616 size += r.length();
1617 }
1618 return size;
1619 }
1620
1621 @Override
1622 public long getStorefilesIndexSize() {
1623 long size = 0;
1624 for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) {
1625 StoreFile.Reader r = s.getReader();
1626 if (r == null) {
1627 LOG.warn("StoreFile " + s + " has a null Reader");
1628 continue;
1629 }
1630 size += r.indexSize();
1631 }
1632 return size;
1633 }
1634
1635 @Override
1636 public long getTotalStaticIndexSize() {
1637 long size = 0;
1638 for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) {
1639 size += s.getReader().getUncompressedDataIndexSize();
1640 }
1641 return size;
1642 }
1643
1644 @Override
1645 public long getTotalStaticBloomSize() {
1646 long size = 0;
1647 for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) {
1648 StoreFile.Reader r = s.getReader();
1649 size += r.getTotalBloomSize();
1650 }
1651 return size;
1652 }
1653
1654 @Override
1655 public long getMemStoreSize() {
1656 return this.memstore.heapSize();
1657 }
1658
1659 @Override
1660 public int getCompactPriority() {
1661 return this.storeEngine.getStoreFileManager().getStoreCompactionPriority();
1662 }
1663
1664 @Override
1665 public boolean throttleCompaction(long compactionSize) {
1666 return storeEngine.getCompactionPolicy().throttleCompaction(compactionSize);
1667 }
1668
1669 public HRegion getHRegion() {
1670 return this.region;
1671 }
1672
1673 @Override
1674 public RegionCoprocessorHost getCoprocessorHost() {
1675 return this.region.getCoprocessorHost();
1676 }
1677
1678 @Override
1679 public HRegionInfo getRegionInfo() {
1680 return this.fs.getRegionInfo();
1681 }
1682
1683 @Override
1684 public boolean areWritesEnabled() {
1685 return this.region.areWritesEnabled();
1686 }
1687
1688 @Override
1689 public long getSmallestReadPoint() {
1690 return this.region.getSmallestReadPoint();
1691 }
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706 public long updateColumnValue(byte [] row, byte [] f,
1707 byte [] qualifier, long newValue)
1708 throws IOException {
1709
1710 this.lock.readLock().lock();
1711 try {
1712 long now = EnvironmentEdgeManager.currentTimeMillis();
1713
1714 return this.memstore.updateColumnValue(row,
1715 f,
1716 qualifier,
1717 newValue,
1718 now);
1719
1720 } finally {
1721 this.lock.readLock().unlock();
1722 }
1723 }
1724
1725 @Override
1726 public long upsert(Iterable<? extends Cell> cells, long readpoint) throws IOException {
1727 this.lock.readLock().lock();
1728 try {
1729 return this.memstore.upsert(cells, readpoint);
1730 } finally {
1731 this.lock.readLock().unlock();
1732 }
1733 }
1734
1735 public StoreFlusher getStoreFlusher(long cacheFlushId) {
1736 return new StoreFlusherImpl(cacheFlushId);
1737 }
1738
1739 private class StoreFlusherImpl implements StoreFlusher {
1740
1741 private long cacheFlushId;
1742 private SortedSet<KeyValue> snapshot;
1743 private StoreFile storeFile;
1744 private Path storeFilePath;
1745 private TimeRangeTracker snapshotTimeRangeTracker;
1746 private AtomicLong flushedSize;
1747
1748 private StoreFlusherImpl(long cacheFlushId) {
1749 this.cacheFlushId = cacheFlushId;
1750 this.flushedSize = new AtomicLong();
1751 }
1752
1753 @Override
1754 public void prepare() {
1755 memstore.snapshot();
1756 this.snapshot = memstore.getSnapshot();
1757 this.snapshotTimeRangeTracker = memstore.getSnapshotTimeRangeTracker();
1758 }
1759
1760 @Override
1761 public void flushCache(MonitoredTask status) throws IOException {
1762 storeFilePath = HStore.this.flushCache(
1763 cacheFlushId, snapshot, snapshotTimeRangeTracker, flushedSize, status);
1764 }
1765
1766 @Override
1767 public boolean commit(MonitoredTask status) throws IOException {
1768 if (storeFilePath == null) {
1769 return false;
1770 }
1771 storeFile = HStore.this.commitFile(storeFilePath, cacheFlushId,
1772 snapshotTimeRangeTracker, flushedSize, status);
1773 if (HStore.this.getCoprocessorHost() != null) {
1774 HStore.this.getCoprocessorHost().postFlush(HStore.this, storeFile);
1775 }
1776
1777
1778
1779 return HStore.this.updateStorefiles(storeFile, snapshot);
1780 }
1781 }
1782
1783 @Override
1784 public boolean needsCompaction() {
1785 return storeEngine.getCompactionPolicy().needsCompaction(
1786 this.storeEngine.getStoreFileManager().getStorefiles(), filesCompacting);
1787 }
1788
1789 @Override
1790 public CacheConfig getCacheConfig() {
1791 return this.cacheConf;
1792 }
1793
1794 public static final long FIXED_OVERHEAD =
1795 ClassSize.align((17 * ClassSize.REFERENCE) + (5 * Bytes.SIZEOF_LONG)
1796 + (2 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_BOOLEAN);
1797
1798 public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD
1799 + ClassSize.OBJECT + ClassSize.REENTRANT_LOCK
1800 + ClassSize.CONCURRENT_SKIPLISTMAP
1801 + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + ClassSize.OBJECT
1802 + ScanInfo.FIXED_OVERHEAD);
1803
1804 @Override
1805 public long heapSize() {
1806 return DEEP_OVERHEAD + this.memstore.heapSize();
1807 }
1808
1809 public KeyValue.KVComparator getComparator() {
1810 return comparator;
1811 }
1812
1813 @Override
1814 public ScanInfo getScanInfo() {
1815 return scanInfo;
1816 }
1817
1818
1819
1820
1821
1822 void setScanInfo(ScanInfo scanInfo) {
1823 this.scanInfo = scanInfo;
1824 }
1825
1826 @Override
1827 public boolean hasTooManyStoreFiles() {
1828 return getStorefilesCount() > this.blockingFileCount;
1829 }
1830 }