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