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