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