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 com.google.common.base.Preconditions;
22 import com.google.common.collect.ImmutableCollection;
23 import com.google.common.collect.ImmutableList;
24 import com.google.common.collect.Lists;
25 import com.google.common.collect.Sets;
26 import java.io.IOException;
27 import java.io.InterruptedIOException;
28 import java.net.InetSocketAddress;
29 import java.security.Key;
30 import java.security.KeyException;
31 import java.util.ArrayList;
32 import java.util.Collection;
33 import java.util.Collections;
34 import java.util.HashMap;
35 import java.util.HashSet;
36 import java.util.Iterator;
37 import java.util.List;
38 import java.util.NavigableSet;
39 import java.util.Set;
40 import java.util.concurrent.Callable;
41 import java.util.concurrent.CompletionService;
42 import java.util.concurrent.ConcurrentHashMap;
43 import java.util.concurrent.ExecutionException;
44 import java.util.concurrent.ExecutorCompletionService;
45 import java.util.concurrent.Future;
46 import java.util.concurrent.ThreadPoolExecutor;
47 import java.util.concurrent.atomic.AtomicBoolean;
48 import java.util.concurrent.atomic.AtomicLong;
49 import java.util.concurrent.locks.ReentrantLock;
50 import java.util.concurrent.locks.ReentrantReadWriteLock;
51 import org.apache.commons.logging.Log;
52 import org.apache.commons.logging.LogFactory;
53 import org.apache.hadoop.conf.Configuration;
54 import org.apache.hadoop.fs.FileSystem;
55 import org.apache.hadoop.fs.Path;
56 import org.apache.hadoop.hbase.Cell;
57 import org.apache.hadoop.hbase.CellComparator;
58 import org.apache.hadoop.hbase.CellUtil;
59 import org.apache.hadoop.hbase.CompoundConfiguration;
60 import org.apache.hadoop.hbase.HColumnDescriptor;
61 import org.apache.hadoop.hbase.HConstants;
62 import org.apache.hadoop.hbase.HRegionInfo;
63 import org.apache.hadoop.hbase.KeyValue;
64 import org.apache.hadoop.hbase.TableName;
65 import org.apache.hadoop.hbase.Tag;
66 import org.apache.hadoop.hbase.TagType;
67 import org.apache.hadoop.hbase.backup.FailedArchiveException;
68 import org.apache.hadoop.hbase.classification.InterfaceAudience;
69 import org.apache.hadoop.hbase.client.Scan;
70 import org.apache.hadoop.hbase.conf.ConfigurationManager;
71 import org.apache.hadoop.hbase.io.compress.Compression;
72 import org.apache.hadoop.hbase.io.crypto.Cipher;
73 import org.apache.hadoop.hbase.io.crypto.Encryption;
74 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
75 import org.apache.hadoop.hbase.io.hfile.HFile;
76 import org.apache.hadoop.hbase.io.hfile.HFileContext;
77 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
78 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
79 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl;
80 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
81 import org.apache.hadoop.hbase.io.hfile.InvalidHFileException;
82 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
83 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
84 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
85 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
86 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
87 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
88 import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
89 import org.apache.hadoop.hbase.regionserver.compactions.OffPeakHours;
90 import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
91 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
92 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
93 import org.apache.hadoop.hbase.security.EncryptionUtil;
94 import org.apache.hadoop.hbase.security.User;
95 import org.apache.hadoop.hbase.util.Bytes;
96 import org.apache.hadoop.hbase.util.ChecksumType;
97 import org.apache.hadoop.hbase.util.ClassSize;
98 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
99 import org.apache.hadoop.hbase.util.Pair;
100 import org.apache.hadoop.hbase.util.ReflectionUtils;
101 import org.apache.hadoop.util.StringUtils;
102 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117 @InterfaceAudience.Private
118 public class HStore implements Store {
119 private static final String MEMSTORE_CLASS_NAME = "hbase.regionserver.memstore.class";
120 public static final String COMPACTCHECKER_INTERVAL_MULTIPLIER_KEY =
121 "hbase.server.compactchecker.interval.multiplier";
122 public static final String BLOCKING_STOREFILES_KEY = "hbase.hstore.blockingStoreFiles";
123 public static final String BLOCK_STORAGE_POLICY_KEY = "hbase.hstore.block.storage.policy";
124
125 public static final String DEFAULT_BLOCK_STORAGE_POLICY = "HOT";
126 public static final int DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER = 1000;
127 public static final int DEFAULT_BLOCKING_STOREFILE_COUNT = 7;
128
129 private static final Log LOG = LogFactory.getLog(HStore.class);
130
131
132
133
134
135 private static final int SPLIT_REGION_COMPACTION_PRIORITY = Integer.MIN_VALUE + 1000;
136
137 protected final MemStore memstore;
138
139 private final HRegion region;
140 private final HColumnDescriptor family;
141 private final HRegionFileSystem fs;
142 private Configuration conf;
143 private final CacheConfig cacheConf;
144 private long lastCompactSize = 0;
145 volatile boolean forceMajor = false;
146
147 static int closeCheckInterval = 0;
148 private AtomicLong storeSize = new AtomicLong();
149 private AtomicLong totalUncompressedBytes = new AtomicLong();
150
151 private boolean cacheOnWriteLogged;
152
153
154
155
156
157
158
159
160
161
162 final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
163
164
165
166
167
168
169
170
171
172
173
174 final ReentrantLock archiveLock = new ReentrantLock();
175
176 private final boolean verifyBulkLoads;
177
178 private ScanInfo scanInfo;
179
180
181
182 private final List<StoreFile> filesCompacting = Lists.newArrayList();
183
184
185 private final Set<ChangedReadersObserver> changedReaderObservers =
186 Collections.newSetFromMap(new ConcurrentHashMap<ChangedReadersObserver, Boolean>());
187
188 private final int blocksize;
189 private HFileDataBlockEncoder dataBlockEncoder;
190
191
192 private ChecksumType checksumType;
193 private int bytesPerChecksum;
194
195
196 private final KeyValue.KVComparator comparator;
197
198 final StoreEngine<?, ?, ?, ?> storeEngine;
199
200 private static final AtomicBoolean offPeakCompactionTracker = new AtomicBoolean();
201 private volatile OffPeakHours offPeakHours;
202
203 private static final int DEFAULT_FLUSH_RETRIES_NUMBER = 10;
204 private int flushRetriesNumber;
205 private int pauseTime;
206
207 private long blockingFileCount;
208 private int compactionCheckMultiplier;
209
210 private Encryption.Context cryptoContext = Encryption.Context.NONE;
211
212 private AtomicLong flushedCellsCount = new AtomicLong();
213 private AtomicLong compactedCellsCount = new AtomicLong();
214 private AtomicLong majorCompactedCellsCount = new AtomicLong();
215 private AtomicLong flushedCellsSize = new AtomicLong();
216 private AtomicLong flushedOutputFileSize = new AtomicLong();
217 private AtomicLong compactedCellsSize = new AtomicLong();
218 private AtomicLong majorCompactedCellsSize = new AtomicLong();
219
220
221
222
223
224
225
226
227
228 protected HStore(final HRegion region, final HColumnDescriptor family,
229 final Configuration confParam) throws IOException {
230
231 HRegionInfo info = region.getRegionInfo();
232 this.fs = region.getRegionFileSystem();
233
234
235 fs.createStoreDir(family.getNameAsString());
236 this.region = region;
237 this.family = family;
238
239
240
241 this.conf = new CompoundConfiguration()
242 .add(confParam)
243 .addStringMap(region.getTableDesc().getConfiguration())
244 .addStringMap(family.getConfiguration())
245 .addWritableMap(family.getValues());
246 this.blocksize = family.getBlocksize();
247
248
249 String policyName = family.getStoragePolicy();
250 if (null == policyName) {
251 policyName = this.conf.get(BLOCK_STORAGE_POLICY_KEY, DEFAULT_BLOCK_STORAGE_POLICY);
252 }
253 this.fs.setStoragePolicy(family.getNameAsString(), policyName.trim());
254
255 this.dataBlockEncoder =
256 new HFileDataBlockEncoderImpl(family.getDataBlockEncoding());
257
258 this.comparator = info.getComparator();
259
260 long timeToPurgeDeletes =
261 Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);
262 LOG.trace("Time to purge deletes set to " + timeToPurgeDeletes +
263 "ms in store " + this);
264
265 long ttl = determineTTLFromFamily(family);
266
267
268 scanInfo = new ScanInfo(conf, family, ttl, timeToPurgeDeletes, this.comparator);
269 String className = conf.get(MEMSTORE_CLASS_NAME, DefaultMemStore.class.getName());
270 this.memstore = ReflectionUtils.instantiateWithCustomCtor(className, new Class[] {
271 Configuration.class, KeyValue.KVComparator.class }, new Object[] { conf, this.comparator });
272 this.offPeakHours = OffPeakHours.getInstance(conf);
273
274
275 this.cacheConf = new CacheConfig(conf, family);
276
277 this.verifyBulkLoads = conf.getBoolean("hbase.hstore.bulkload.verify", false);
278
279 this.blockingFileCount =
280 conf.getInt(BLOCKING_STOREFILES_KEY, DEFAULT_BLOCKING_STOREFILE_COUNT);
281 this.compactionCheckMultiplier = conf.getInt(
282 COMPACTCHECKER_INTERVAL_MULTIPLIER_KEY, DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER);
283 if (this.compactionCheckMultiplier <= 0) {
284 LOG.error("Compaction check period multiplier must be positive, setting default: "
285 + DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER);
286 this.compactionCheckMultiplier = DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER;
287 }
288
289 if (HStore.closeCheckInterval == 0) {
290 HStore.closeCheckInterval = conf.getInt(
291 "hbase.hstore.close.check.interval", 10*1000*1000
292 }
293
294 this.storeEngine = StoreEngine.create(this, this.conf, this.comparator);
295 List<StoreFile> storeFiles = loadStoreFiles();
296
297
298
299
300 this.storeSize.addAndGet(getStorefilesSize(storeFiles));
301 this.totalUncompressedBytes.addAndGet(getTotalUmcompressedBytes(storeFiles));
302 this.storeEngine.getStoreFileManager().loadFiles(storeFiles);
303
304
305 this.checksumType = getChecksumType(conf);
306
307 this.bytesPerChecksum = getBytesPerChecksum(conf);
308 flushRetriesNumber = conf.getInt(
309 "hbase.hstore.flush.retries.number", DEFAULT_FLUSH_RETRIES_NUMBER);
310 pauseTime = conf.getInt(HConstants.HBASE_SERVER_PAUSE, HConstants.DEFAULT_HBASE_SERVER_PAUSE);
311 if (flushRetriesNumber <= 0) {
312 throw new IllegalArgumentException(
313 "hbase.hstore.flush.retries.number must be > 0, not "
314 + flushRetriesNumber);
315 }
316
317
318 String cipherName = family.getEncryptionType();
319 if (cipherName != null) {
320 Cipher cipher;
321 Key key;
322 byte[] keyBytes = family.getEncryptionKey();
323 if (keyBytes != null) {
324
325 String masterKeyName = conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY,
326 User.getCurrent().getShortName());
327 try {
328
329 key = EncryptionUtil.unwrapKey(conf, masterKeyName, keyBytes);
330 } catch (KeyException e) {
331
332
333 if (LOG.isDebugEnabled()) {
334 LOG.debug("Unable to unwrap key with current master key '" + masterKeyName + "'");
335 }
336 String alternateKeyName =
337 conf.get(HConstants.CRYPTO_MASTERKEY_ALTERNATE_NAME_CONF_KEY);
338 if (alternateKeyName != null) {
339 try {
340 key = EncryptionUtil.unwrapKey(conf, alternateKeyName, keyBytes);
341 } catch (KeyException ex) {
342 throw new IOException(ex);
343 }
344 } else {
345 throw new IOException(e);
346 }
347 }
348
349 cipher = Encryption.getCipher(conf, key.getAlgorithm());
350 if (cipher == null) {
351 throw new RuntimeException("Cipher '" + key.getAlgorithm() + "' is not available");
352 }
353
354
355
356 if (!cipher.getName().equalsIgnoreCase(cipherName)) {
357 throw new RuntimeException("Encryption for family '" + family.getNameAsString() +
358 "' configured with type '" + cipherName +
359 "' but key specifies algorithm '" + cipher.getName() + "'");
360 }
361 } else {
362
363 cipher = Encryption.getCipher(conf, cipherName);
364 if (cipher == null) {
365 throw new RuntimeException("Cipher '" + cipherName + "' is not available");
366 }
367 key = cipher.getRandomKey();
368 }
369 cryptoContext = Encryption.newContext(conf);
370 cryptoContext.setCipher(cipher);
371 cryptoContext.setKey(key);
372 }
373
374 LOG.info("Store=" + getColumnFamilyName() +
375 ", memstore type=" + this.memstore.getClass().getSimpleName() +
376 ", storagePolicy=" + policyName + ", verifyBulkLoads=" + verifyBulkLoads +
377 ", encoding=" + family.getDataBlockEncoding() +
378 ", compression=" + family.getCompressionType());
379 cacheOnWriteLogged = false;
380 }
381
382
383
384
385
386 private static long determineTTLFromFamily(final HColumnDescriptor family) {
387
388 long ttl = family.getTimeToLive();
389 if (ttl == HConstants.FOREVER) {
390
391 ttl = Long.MAX_VALUE;
392 } else if (ttl == -1) {
393 ttl = Long.MAX_VALUE;
394 } else {
395
396 ttl *= 1000;
397 }
398 return ttl;
399 }
400
401 @Override
402 public String getColumnFamilyName() {
403 return this.family.getNameAsString();
404 }
405
406 @Override
407 public TableName getTableName() {
408 return this.getRegionInfo().getTable();
409 }
410
411 @Override
412 public FileSystem getFileSystem() {
413 return this.fs.getFileSystem();
414 }
415
416 public HRegionFileSystem getRegionFileSystem() {
417 return this.fs;
418 }
419
420
421 @Override
422 public long getStoreFileTtl() {
423
424 return (this.scanInfo.getMinVersions() == 0) ? this.scanInfo.getTtl() : Long.MAX_VALUE;
425 }
426
427 @Override
428 public long getMemstoreFlushSize() {
429
430 return this.region.memstoreFlushSize;
431 }
432
433 @Override
434 public long getFlushableSize() {
435 return this.memstore.getFlushableSize();
436 }
437
438 @Override
439 public long getSnapshotSize() {
440 return this.memstore.getSnapshotSize();
441 }
442
443 @Override
444 public long getCompactionCheckMultiplier() {
445 return this.compactionCheckMultiplier;
446 }
447
448 @Override
449 public long getBlockingFileCount() {
450 return blockingFileCount;
451 }
452
453
454
455
456
457
458
459 public static int getBytesPerChecksum(Configuration conf) {
460 return conf.getInt(HConstants.BYTES_PER_CHECKSUM,
461 HFile.DEFAULT_BYTES_PER_CHECKSUM);
462 }
463
464
465
466
467
468
469 public static ChecksumType getChecksumType(Configuration conf) {
470 String checksumName = conf.get(HConstants.CHECKSUM_TYPE_NAME);
471 if (checksumName == null) {
472 return ChecksumType.getDefaultChecksumType();
473 } else {
474 return ChecksumType.nameToType(checksumName);
475 }
476 }
477
478
479
480
481 public static int getCloseCheckInterval() {
482 return closeCheckInterval;
483 }
484
485 @Override
486 public HColumnDescriptor getFamily() {
487 return this.family;
488 }
489
490
491
492
493 @Override
494 public long getMaxSequenceId() {
495 return StoreFile.getMaxSequenceIdInList(this.getStorefiles());
496 }
497
498 @Override
499 public long getMaxMemstoreTS() {
500 return StoreFile.getMaxMemstoreTSInList(this.getStorefiles());
501 }
502
503
504
505
506
507
508
509 @Deprecated
510 public static Path getStoreHomedir(final Path tabledir,
511 final HRegionInfo hri, final byte[] family) {
512 return getStoreHomedir(tabledir, hri.getEncodedName(), family);
513 }
514
515
516
517
518
519
520
521 @Deprecated
522 public static Path getStoreHomedir(final Path tabledir,
523 final String encodedName, final byte[] family) {
524 return new Path(tabledir, new Path(encodedName, Bytes.toString(family)));
525 }
526
527 @Override
528 public HFileDataBlockEncoder getDataBlockEncoder() {
529 return dataBlockEncoder;
530 }
531
532
533
534
535
536 void setDataBlockEncoderInTest(HFileDataBlockEncoder blockEncoder) {
537 this.dataBlockEncoder = blockEncoder;
538 }
539
540
541
542
543
544
545 private List<StoreFile> loadStoreFiles() throws IOException {
546 Collection<StoreFileInfo> files = fs.getStoreFiles(getColumnFamilyName());
547 return openStoreFiles(files);
548 }
549
550 private List<StoreFile> openStoreFiles(Collection<StoreFileInfo> files) throws IOException {
551 if (files == null || files.size() == 0) {
552 return new ArrayList<StoreFile>();
553 }
554
555 ThreadPoolExecutor storeFileOpenerThreadPool =
556 this.region.getStoreFileOpenAndCloseThreadPool("StoreFileOpenerThread-" +
557 this.getColumnFamilyName());
558 CompletionService<StoreFile> completionService =
559 new ExecutorCompletionService<StoreFile>(storeFileOpenerThreadPool);
560
561 int totalValidStoreFile = 0;
562 for (final StoreFileInfo storeFileInfo: files) {
563
564 completionService.submit(new Callable<StoreFile>() {
565 @Override
566 public StoreFile call() throws IOException {
567 StoreFile storeFile = createStoreFileAndReader(storeFileInfo);
568 return storeFile;
569 }
570 });
571 totalValidStoreFile++;
572 }
573
574 Set<String> compactedStoreFiles = new HashSet<>();
575 ArrayList<StoreFile> results = new ArrayList<StoreFile>(files.size());
576 IOException ioe = null;
577 try {
578 for (int i = 0; i < totalValidStoreFile; i++) {
579 try {
580 Future<StoreFile> future = completionService.take();
581 StoreFile storeFile = future.get();
582 if (storeFile != null) {
583 if (LOG.isDebugEnabled()) {
584 LOG.debug("loaded " + storeFile.toStringDetailed());
585 }
586 results.add(storeFile);
587 compactedStoreFiles.addAll(storeFile.getCompactedStoreFiles());
588 }
589 } catch (InterruptedException e) {
590 if (ioe == null) ioe = new InterruptedIOException(e.getMessage());
591 } catch (ExecutionException e) {
592 if (ioe == null) ioe = new IOException(e.getCause());
593 }
594 }
595 } finally {
596 storeFileOpenerThreadPool.shutdownNow();
597 }
598 if (ioe != null) {
599
600 boolean evictOnClose =
601 cacheConf != null? cacheConf.shouldEvictOnClose(): true;
602 for (StoreFile file : results) {
603 try {
604 if (file != null) file.closeReader(evictOnClose);
605 } catch (IOException e) {
606 LOG.warn("Could not close store file " + file, e);
607 }
608 }
609 throw ioe;
610 }
611
612
613 List<StoreFile> filesToRemove = new ArrayList<>(compactedStoreFiles.size());
614 for (StoreFile storeFile : results) {
615 if (compactedStoreFiles.contains(storeFile.getPath().getName())) {
616 LOG.warn("Clearing the compacted storefile " + storeFile + " from this store");
617 storeFile.getReader().close(true);
618 filesToRemove.add(storeFile);
619 }
620 }
621 results.removeAll(filesToRemove);
622 if (!filesToRemove.isEmpty() && this.isPrimaryReplicaStore()) {
623 LOG.debug("Moving the files " + filesToRemove + " to archive");
624 this.fs.removeStoreFiles(this.getColumnFamilyName(), filesToRemove);
625 }
626
627 return results;
628 }
629
630
631
632
633
634
635
636
637 @Override
638 public void refreshStoreFiles() throws IOException {
639 Collection<StoreFileInfo> newFiles = fs.getStoreFiles(getColumnFamilyName());
640 refreshStoreFilesInternal(newFiles);
641 }
642
643 @Override
644 public void refreshStoreFiles(Collection<String> newFiles) throws IOException {
645 List<StoreFileInfo> storeFiles = new ArrayList<StoreFileInfo>(newFiles.size());
646 for (String file : newFiles) {
647 storeFiles.add(fs.getStoreFileInfo(getColumnFamilyName(), file));
648 }
649 refreshStoreFilesInternal(storeFiles);
650 }
651
652
653
654
655
656
657
658
659 private void refreshStoreFilesInternal(Collection<StoreFileInfo> newFiles) throws IOException {
660 StoreFileManager sfm = storeEngine.getStoreFileManager();
661 Collection<StoreFile> currentFiles = sfm.getStorefiles();
662 Collection<StoreFile> compactedFiles = sfm.getCompactedfiles();
663 if (currentFiles == null) currentFiles = Collections.emptySet();
664 if (newFiles == null) newFiles = Collections.emptySet();
665 if (compactedFiles == null) compactedFiles = Collections.emptySet();
666
667 HashMap<StoreFileInfo, StoreFile> currentFilesSet =
668 new HashMap<StoreFileInfo, StoreFile>(currentFiles.size());
669 for (StoreFile sf : currentFiles) {
670 currentFilesSet.put(sf.getFileInfo(), sf);
671 }
672 HashMap<StoreFileInfo, StoreFile> compactedFilesSet =
673 new HashMap<StoreFileInfo, StoreFile>(compactedFiles.size());
674 for (StoreFile sf : compactedFiles) {
675 compactedFilesSet.put(sf.getFileInfo(), sf);
676 }
677
678 Set<StoreFileInfo> newFilesSet = new HashSet<StoreFileInfo>(newFiles);
679
680 newFilesSet = Sets.difference(newFilesSet, compactedFilesSet.keySet());
681 Set<StoreFileInfo> toBeAddedFiles = Sets.difference(newFilesSet, currentFilesSet.keySet());
682 Set<StoreFileInfo> toBeRemovedFiles = Sets.difference(currentFilesSet.keySet(), newFilesSet);
683
684 if (toBeAddedFiles.isEmpty() && toBeRemovedFiles.isEmpty()) {
685 return;
686 }
687
688 LOG.info("Refreshing store files for region " + this.getRegionInfo().getRegionNameAsString()
689 + " files to add: " + toBeAddedFiles + " files to remove: " + toBeRemovedFiles);
690
691 Set<StoreFile> toBeRemovedStoreFiles = new HashSet<StoreFile>(toBeRemovedFiles.size());
692 for (StoreFileInfo sfi : toBeRemovedFiles) {
693 toBeRemovedStoreFiles.add(currentFilesSet.get(sfi));
694 }
695
696
697 List<StoreFile> openedFiles = openStoreFiles(toBeAddedFiles);
698
699
700 replaceStoreFiles(toBeRemovedStoreFiles, openedFiles);
701
702
703
704
705 if (!toBeAddedFiles.isEmpty()) {
706 region.getMVCC().advanceTo(this.getMaxSequenceId());
707 }
708
709 completeCompaction(toBeRemovedStoreFiles);
710 }
711
712 private StoreFile createStoreFileAndReader(final Path p) throws IOException {
713 StoreFileInfo info = new StoreFileInfo(conf, this.getFileSystem(), p);
714 return createStoreFileAndReader(info);
715 }
716
717 private StoreFile createStoreFileAndReader(final StoreFileInfo info)
718 throws IOException {
719 info.setRegionCoprocessorHost(this.region.getCoprocessorHost());
720 StoreFile storeFile = new StoreFile(this.getFileSystem(), info, this.conf, this.cacheConf,
721 this.family.getBloomFilterType());
722 StoreFile.Reader r = storeFile.createReader();
723 r.setReplicaStoreFile(isPrimaryReplicaStore());
724 return storeFile;
725 }
726
727 @Override
728 public long add(final Cell cell) {
729 lock.readLock().lock();
730 try {
731 return this.memstore.add(cell);
732 } finally {
733 lock.readLock().unlock();
734 }
735 }
736
737 @Override
738 public long add(Iterable<Cell> cells) {
739 lock.readLock().lock();
740 try {
741 return this.memstore.add(cells);
742 } finally {
743 lock.readLock().unlock();
744 }
745 }
746
747 @Override
748 public long timeOfOldestEdit() {
749 return memstore.timeOfOldestEdit();
750 }
751
752
753
754
755
756
757
758 protected long delete(final KeyValue kv) {
759 lock.readLock().lock();
760 try {
761 return this.memstore.delete(kv);
762 } finally {
763 lock.readLock().unlock();
764 }
765 }
766
767 @Override
768 public void rollback(final Cell cell) {
769 lock.readLock().lock();
770 try {
771 this.memstore.rollback(cell);
772 } finally {
773 lock.readLock().unlock();
774 }
775 }
776
777
778
779
780 @Override
781 public Collection<StoreFile> getStorefiles() {
782 return this.storeEngine.getStoreFileManager().getStorefiles();
783 }
784
785 public Collection<StoreFile> getCompactedfiles() {
786 Collection<StoreFile> compactedFiles =
787 this.storeEngine.getStoreFileManager().getCompactedfiles();
788 return compactedFiles == null ? new ArrayList<StoreFile>() : compactedFiles;
789 }
790
791 @Override
792 public void assertBulkLoadHFileOk(Path srcPath) throws IOException {
793 HFile.Reader reader = null;
794 try {
795 LOG.info("Validating hfile at " + srcPath + " for inclusion in "
796 + "store " + this + " region " + this.getRegionInfo().getRegionNameAsString());
797 reader = HFile.createReader(srcPath.getFileSystem(conf),
798 srcPath, cacheConf, conf);
799 reader.loadFileInfo();
800
801 byte[] firstKey = reader.getFirstRowKey();
802 Preconditions.checkState(firstKey != null, "First key can not be null");
803 byte[] lk = reader.getLastKey();
804 Preconditions.checkState(lk != null, "Last key can not be null");
805 byte[] lastKey = KeyValue.createKeyValueFromKey(lk).getRow();
806
807 LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey) +
808 " last=" + Bytes.toStringBinary(lastKey));
809 LOG.debug("Region bounds: first=" +
810 Bytes.toStringBinary(getRegionInfo().getStartKey()) +
811 " last=" + Bytes.toStringBinary(getRegionInfo().getEndKey()));
812
813 if (!this.getRegionInfo().containsRange(firstKey, lastKey)) {
814 throw new WrongRegionException(
815 "Bulk load file " + srcPath.toString() + " does not fit inside region "
816 + this.getRegionInfo().getRegionNameAsString());
817 }
818
819 if(reader.length() > conf.getLong(HConstants.HREGION_MAX_FILESIZE,
820 HConstants.DEFAULT_MAX_FILE_SIZE)) {
821 LOG.warn("Trying to bulk load hfile " + srcPath.toString() + " with size: " +
822 reader.length() + " bytes can be problematic as it may lead to oversplitting.");
823 }
824
825 if (verifyBulkLoads) {
826 long verificationStartTime = EnvironmentEdgeManager.currentTime();
827 LOG.info("Full verification started for bulk load hfile: " + srcPath.toString());
828 Cell prevCell = null;
829 HFileScanner scanner = reader.getScanner(false, false, false);
830 scanner.seekTo();
831 do {
832 Cell cell = scanner.getKeyValue();
833 if (prevCell != null) {
834 if (CellComparator.compareRows(prevCell, cell) > 0) {
835 throw new InvalidHFileException("Previous row is greater than"
836 + " current row: path=" + srcPath + " previous="
837 + CellUtil.getCellKeyAsString(prevCell) + " current="
838 + CellUtil.getCellKeyAsString(cell));
839 }
840 if (CellComparator.compareFamilies(prevCell, cell) != 0) {
841 throw new InvalidHFileException("Previous key had different"
842 + " family compared to current key: path=" + srcPath
843 + " previous="
844 + Bytes.toStringBinary(prevCell.getFamilyArray(), prevCell.getFamilyOffset(),
845 prevCell.getFamilyLength())
846 + " current="
847 + Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
848 cell.getFamilyLength()));
849 }
850 }
851 prevCell = cell;
852 } while (scanner.next());
853 LOG.info("Full verification complete for bulk load hfile: " + srcPath.toString()
854 + " took " + (EnvironmentEdgeManager.currentTime() - verificationStartTime)
855 + " ms");
856 }
857 } finally {
858 if (reader != null) reader.close();
859 }
860 }
861
862 public Pair<Path, Path> preBulkLoadHFile(String srcPathStr, long seqNum) throws IOException {
863 Path srcPath = new Path(srcPathStr);
864 return fs.bulkLoadStoreFile(getColumnFamilyName(), srcPath, seqNum);
865 }
866
867 @Override
868 public Path bulkLoadHFile(byte[] family, String srcPathStr, Path dstPath) throws IOException {
869 Path srcPath = new Path(srcPathStr);
870 try {
871 fs.commitStoreFile(srcPath, dstPath);
872 } finally {
873 if (this.getCoprocessorHost() != null) {
874 this.getCoprocessorHost().postCommitStoreFile(family, srcPath, dstPath);
875 }
876 }
877
878 LOG.info("Loaded HFile " + srcPath + " into store '" + getColumnFamilyName() + "' as "
879 + dstPath + " - updating store file list.");
880
881 StoreFile sf = createStoreFileAndReader(dstPath);
882 bulkLoadHFile(sf);
883
884 LOG.info("Successfully loaded store file " + srcPath + " into store " + this
885 + " (new location: " + dstPath + ")");
886
887 return dstPath;
888 }
889
890 @Override
891 public void bulkLoadHFile(StoreFileInfo fileInfo) throws IOException {
892 StoreFile sf = createStoreFileAndReader(fileInfo);
893 bulkLoadHFile(sf);
894 }
895
896 private void bulkLoadHFile(StoreFile sf) throws IOException {
897 StoreFile.Reader r = sf.getReader();
898 this.storeSize.addAndGet(r.length());
899 this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes());
900
901
902 this.lock.writeLock().lock();
903 try {
904 this.storeEngine.getStoreFileManager().insertNewFiles(Lists.newArrayList(sf));
905 } finally {
906
907
908
909
910
911 this.lock.writeLock().unlock();
912 }
913 LOG.info("Loaded HFile " + sf.getFileInfo() + " into store '" + getColumnFamilyName());
914 if (LOG.isTraceEnabled()) {
915 String traceMessage = "BULK LOAD time,size,store size,store files ["
916 + EnvironmentEdgeManager.currentTime() + "," + r.length() + "," + storeSize
917 + "," + storeEngine.getStoreFileManager().getStorefileCount() + "]";
918 LOG.trace(traceMessage);
919 }
920 }
921
922 @Override
923 public ImmutableCollection<StoreFile> close() throws IOException {
924 this.archiveLock.lock();
925 this.lock.writeLock().lock();
926 try {
927
928 ImmutableCollection<StoreFile> result = storeEngine.getStoreFileManager().clearFiles();
929 Collection<StoreFile> compactedfiles =
930 storeEngine.getStoreFileManager().clearCompactedFiles();
931
932 if (compactedfiles != null && !compactedfiles.isEmpty()) {
933 removeCompactedfiles(compactedfiles);
934 }
935 if (!result.isEmpty()) {
936
937 ThreadPoolExecutor storeFileCloserThreadPool = this.region
938 .getStoreFileOpenAndCloseThreadPool("StoreFileCloserThread-"
939 + this.getColumnFamilyName());
940
941
942 CompletionService<Void> completionService =
943 new ExecutorCompletionService<Void>(storeFileCloserThreadPool);
944 for (final StoreFile f : result) {
945 completionService.submit(new Callable<Void>() {
946 @Override
947 public Void call() throws IOException {
948 boolean evictOnClose =
949 cacheConf != null? cacheConf.shouldEvictOnClose(): true;
950 f.closeReader(evictOnClose);
951 return null;
952 }
953 });
954 }
955
956 IOException ioe = null;
957 try {
958 for (int i = 0; i < result.size(); i++) {
959 try {
960 Future<Void> future = completionService.take();
961 future.get();
962 } catch (InterruptedException e) {
963 if (ioe == null) {
964 ioe = new InterruptedIOException();
965 ioe.initCause(e);
966 }
967 } catch (ExecutionException e) {
968 if (ioe == null) ioe = new IOException(e.getCause());
969 }
970 }
971 } finally {
972 storeFileCloserThreadPool.shutdownNow();
973 }
974 if (ioe != null) throw ioe;
975 }
976 LOG.info("Closed " + this);
977 return result;
978 } finally {
979 this.lock.writeLock().unlock();
980 this.archiveLock.unlock();
981 }
982 }
983
984
985
986
987
988
989 void snapshot() {
990 this.lock.writeLock().lock();
991 try {
992 this.memstore.snapshot();
993 } finally {
994 this.lock.writeLock().unlock();
995 }
996 }
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007 protected List<Path> flushCache(final long logCacheFlushId, MemStoreSnapshot snapshot,
1008 MonitoredTask status, ThroughputController throughputController) throws IOException {
1009
1010
1011
1012
1013
1014 StoreFlusher flusher = storeEngine.getStoreFlusher();
1015 IOException lastException = null;
1016 for (int i = 0; i < flushRetriesNumber; i++) {
1017 try {
1018 List<Path> pathNames =
1019 flusher.flushSnapshot(snapshot, logCacheFlushId, status, throughputController);
1020 Path lastPathName = null;
1021 try {
1022 for (Path pathName : pathNames) {
1023 lastPathName = pathName;
1024 validateStoreFile(pathName);
1025 }
1026 return pathNames;
1027 } catch (Exception e) {
1028 LOG.warn("Failed validating store file " + lastPathName + ", retrying num=" + i, e);
1029 if (e instanceof IOException) {
1030 lastException = (IOException) e;
1031 } else {
1032 lastException = new IOException(e);
1033 }
1034 }
1035 } catch (IOException e) {
1036 LOG.warn("Failed flushing store file, retrying num=" + i, e);
1037 lastException = e;
1038 }
1039 if (lastException != null && i < (flushRetriesNumber - 1)) {
1040 try {
1041 Thread.sleep(pauseTime);
1042 } catch (InterruptedException e) {
1043 IOException iie = new InterruptedIOException();
1044 iie.initCause(e);
1045 throw iie;
1046 }
1047 }
1048 }
1049 throw lastException;
1050 }
1051
1052
1053
1054
1055
1056
1057
1058
1059 private StoreFile commitFile(final Path path, final long logCacheFlushId, MonitoredTask status)
1060 throws IOException {
1061
1062 Path dstPath = fs.commitStoreFile(getColumnFamilyName(), path);
1063
1064 status.setStatus("Flushing " + this + ": reopening flushed file");
1065 StoreFile sf = createStoreFileAndReader(dstPath);
1066
1067 StoreFile.Reader r = sf.getReader();
1068 this.storeSize.addAndGet(r.length());
1069 this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes());
1070
1071 if (LOG.isInfoEnabled()) {
1072 LOG.info("Added " + sf + ", entries=" + r.getEntries() +
1073 ", sequenceid=" + logCacheFlushId +
1074 ", filesize=" + TraditionalBinaryPrefix.long2String(r.length(), "", 1));
1075 }
1076 return sf;
1077 }
1078
1079 @Override
1080 public StoreFile.Writer createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
1081 boolean isCompaction, boolean includeMVCCReadpoint,
1082 boolean includesTag)
1083 throws IOException {
1084 return createWriterInTmp(maxKeyCount, compression, isCompaction, includeMVCCReadpoint,
1085 includesTag, false, -1);
1086 }
1087
1088 @Override
1089 public StoreFile.Writer createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
1090 boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTags,
1091 boolean shouldDropBehind) throws IOException {
1092 return createWriterInTmp(maxKeyCount, compression, isCompaction, includeMVCCReadpoint,
1093 includesTags, shouldDropBehind, -1);
1094 }
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104 @Override
1105 public StoreFile.Writer createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
1106 boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag,
1107 boolean shouldDropBehind, long totalCompactedFilesSize) throws IOException {
1108 return createWriterInTmp(maxKeyCount, compression, isCompaction, includeMVCCReadpoint,
1109 includesTag, shouldDropBehind, null, totalCompactedFilesSize);
1110 }
1111
1112 @Override
1113 public StoreFile.Writer createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
1114 boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTags,
1115 boolean shouldDropBehind, TimeRangeTracker trt) throws IOException {
1116 return createWriterInTmp(maxKeyCount, compression, isCompaction, includeMVCCReadpoint,
1117 includesTags, shouldDropBehind, trt, -1);
1118 }
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133 @Override
1134 public StoreFile.Writer createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
1135 boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag,
1136 boolean shouldDropBehind, final TimeRangeTracker trt, long totalCompactedFilesSize)
1137 throws IOException {
1138
1139 final CacheConfig writerCacheConf = new CacheConfig(cacheConf);
1140 if (isCompaction) {
1141
1142
1143 final boolean cacheCompactedBlocksOnWrite =
1144 cacheConf.shouldCacheCompactedBlocksOnWrite();
1145
1146
1147
1148 if (cacheCompactedBlocksOnWrite && totalCompactedFilesSize <= cacheConf
1149 .getCacheCompactedBlocksOnWriteThreshold()) {
1150 writerCacheConf.enableCacheOnWrite();
1151 if (!cacheOnWriteLogged) {
1152 LOG.info("For Store " + getColumnFamilyName() +
1153 " , cacheCompactedBlocksOnWrite is true, hence enabled " +
1154 "cacheOnWrite for Data blocks, Index blocks and Bloom filter blocks");
1155 cacheOnWriteLogged = true;
1156 }
1157 } else {
1158 writerCacheConf.setCacheDataOnWrite(false);
1159 if (totalCompactedFilesSize > cacheConf.getCacheCompactedBlocksOnWriteThreshold()) {
1160
1161 LOG.debug("For Store " + getColumnFamilyName()
1162 + ", setting cacheCompactedBlocksOnWrite as false as total size of compacted "
1163 + "files - " + totalCompactedFilesSize
1164 + ", is greater than cacheCompactedBlocksOnWriteThreshold - "
1165 + cacheConf.getCacheCompactedBlocksOnWriteThreshold());
1166 }
1167 }
1168 } else {
1169 final boolean shouldCacheDataOnWrite = cacheConf.shouldCacheDataOnWrite();
1170 if (shouldCacheDataOnWrite) {
1171 writerCacheConf.enableCacheOnWrite();
1172 if (!cacheOnWriteLogged) {
1173 LOG.info("For Store " + getColumnFamilyName() +
1174 " , cacheDataOnWrite is true, hence enabled cacheOnWrite for " +
1175 "Index blocks and Bloom filter blocks");
1176 cacheOnWriteLogged = true;
1177 }
1178 }
1179 }
1180 InetSocketAddress[] favoredNodes = null;
1181 if (region.getRegionServerServices() != null) {
1182 favoredNodes = region.getRegionServerServices().getFavoredNodesForRegion(
1183 region.getRegionInfo().getEncodedName());
1184 }
1185 Path familyTempDir = new Path(fs.getTempDir(), family.getNameAsString());
1186 HFileContext hFileContext = createFileContext(compression, includeMVCCReadpoint, includesTag,
1187 cryptoContext);
1188 StoreFile.WriterBuilder builder = new StoreFile.WriterBuilder(conf, writerCacheConf,
1189 this.getFileSystem())
1190 .withOutputDir(familyTempDir)
1191 .withComparator(comparator)
1192 .withBloomType(family.getBloomFilterType())
1193 .withMaxKeyCount(maxKeyCount)
1194 .withFavoredNodes(favoredNodes)
1195 .withFileContext(hFileContext)
1196 .withShouldDropCacheBehind(shouldDropBehind)
1197 .withCompactedFiles(this.getCompactedfiles());
1198 if (trt != null) {
1199 builder.withTimeRangeTracker(trt);
1200 }
1201 return builder.build();
1202 }
1203
1204 private HFileContext createFileContext(Compression.Algorithm compression,
1205 boolean includeMVCCReadpoint, boolean includesTag, Encryption.Context cryptoContext) {
1206 if (compression == null) {
1207 compression = HFile.DEFAULT_COMPRESSION_ALGORITHM;
1208 }
1209 HFileContext hFileContext = new HFileContextBuilder()
1210 .withIncludesMvcc(includeMVCCReadpoint)
1211 .withIncludesTags(includesTag)
1212 .withCompression(compression)
1213 .withCompressTags(family.isCompressTags())
1214 .withChecksumType(checksumType)
1215 .withBytesPerCheckSum(bytesPerChecksum)
1216 .withBlockSize(blocksize)
1217 .withHBaseCheckSum(true)
1218 .withDataBlockEncoding(family.getDataBlockEncoding())
1219 .withEncryptionContext(cryptoContext)
1220 .withCreateTime(EnvironmentEdgeManager.currentTime())
1221 .withColumnFamily(family.getName())
1222 .withTableName(region.getTableDesc().
1223 getTableName().getName())
1224 .build();
1225 return hFileContext;
1226 }
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236 private boolean updateStorefiles(final List<StoreFile> sfs, final long snapshotId)
1237 throws IOException {
1238 this.lock.writeLock().lock();
1239 try {
1240 this.storeEngine.getStoreFileManager().insertNewFiles(sfs);
1241 if (snapshotId > 0) {
1242 this.memstore.clearSnapshot(snapshotId);
1243 }
1244 } finally {
1245
1246
1247
1248
1249
1250 this.lock.writeLock().unlock();
1251 }
1252
1253 notifyChangedReadersObservers(sfs);
1254 if (LOG.isTraceEnabled()) {
1255 long totalSize = 0;
1256 for (StoreFile sf : sfs) {
1257 totalSize += sf.getReader().length();
1258 }
1259 String traceMessage = "FLUSH time,count,size,store size,store files ["
1260 + EnvironmentEdgeManager.currentTime() + "," + sfs.size() + "," + totalSize
1261 + "," + storeSize + "," + storeEngine.getStoreFileManager().getStorefileCount() + "]";
1262 LOG.trace(traceMessage);
1263 }
1264 return needsCompaction();
1265 }
1266
1267
1268
1269
1270
1271 private void notifyChangedReadersObservers(List<StoreFile> sfs) throws IOException {
1272 for (ChangedReadersObserver o : this.changedReaderObservers) {
1273 List<KeyValueScanner> memStoreScanners;
1274 this.lock.readLock().lock();
1275 try {
1276 memStoreScanners = this.memstore.getScanners(o.getReadPoint());
1277 } finally {
1278 this.lock.readLock().unlock();
1279 }
1280 o.updateReaders(sfs, memStoreScanners);
1281 }
1282 }
1283
1284
1285
1286
1287
1288
1289 @Override
1290 public List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean isGet,
1291 boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow,
1292 byte[] stopRow, long readPt) throws IOException {
1293 Collection<StoreFile> storeFilesToScan;
1294 List<KeyValueScanner> memStoreScanners;
1295 this.lock.readLock().lock();
1296 try {
1297
1298
1299
1300
1301
1302 storeFilesToScan = this.storeEngine.getStoreFileManager().getFilesForScanOrGet(startRow, true,
1303 stopRow, true);
1304 memStoreScanners = this.memstore.getScanners(readPt);
1305 } finally {
1306 this.lock.readLock().unlock();
1307 }
1308
1309
1310
1311
1312
1313
1314 List<StoreFileScanner> sfScanners = StoreFileScanner.getScannersForStoreFiles(storeFilesToScan,
1315 cacheBlocks, usePread, isCompaction, false, matcher, readPt, isPrimaryReplicaStore());
1316 List<KeyValueScanner> scanners =
1317 new ArrayList<KeyValueScanner>(sfScanners.size()+1);
1318 scanners.addAll(sfScanners);
1319
1320 if (memStoreScanners != null) {
1321 scanners.addAll(memStoreScanners);
1322 }
1323 return scanners;
1324 }
1325
1326 @Override
1327 public List<KeyValueScanner> getScanners(List<StoreFile> files, boolean cacheBlocks,
1328 boolean isGet, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher,
1329 byte[] startRow, byte[] stopRow, long readPt, boolean includeMemstoreScanner) throws IOException {
1330 List<KeyValueScanner> memStoreScanners = null;
1331 if (includeMemstoreScanner) {
1332 this.lock.readLock().lock();
1333 try {
1334 memStoreScanners = this.memstore.getScanners(readPt);
1335 } finally {
1336 this.lock.readLock().unlock();
1337 }
1338 }
1339 List<StoreFileScanner> sfScanners = StoreFileScanner.getScannersForStoreFiles(files,
1340 cacheBlocks, usePread, isCompaction, false, matcher, readPt, isPrimaryReplicaStore());
1341 List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>(sfScanners.size() + 1);
1342 scanners.addAll(sfScanners);
1343
1344 if (memStoreScanners != null) {
1345 scanners.addAll(memStoreScanners);
1346 }
1347 return scanners;
1348 }
1349
1350 @Override
1351 public void addChangedReaderObserver(ChangedReadersObserver o) {
1352 this.changedReaderObservers.add(o);
1353 }
1354
1355 @Override
1356 public void deleteChangedReaderObserver(ChangedReadersObserver o) {
1357
1358 this.changedReaderObservers.remove(o);
1359 }
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408 @Override
1409 public List<StoreFile> compact(CompactionContext compaction,
1410 ThroughputController throughputController) throws IOException {
1411 return compact(compaction, throughputController, null);
1412 }
1413
1414 @Override
1415 public List<StoreFile> compact(CompactionContext compaction,
1416 ThroughputController throughputController, User user) throws IOException {
1417 assert compaction != null;
1418 List<StoreFile> sfs = null;
1419 CompactionRequest cr = compaction.getRequest();
1420 try {
1421
1422
1423
1424 long compactionStartTime = EnvironmentEdgeManager.currentTime();
1425 assert compaction.hasSelection();
1426 Collection<StoreFile> filesToCompact = cr.getFiles();
1427 assert !filesToCompact.isEmpty();
1428 synchronized (filesCompacting) {
1429
1430
1431 Preconditions.checkArgument(filesCompacting.containsAll(filesToCompact));
1432 }
1433
1434
1435 LOG.info("Starting compaction of " + filesToCompact.size() + " file(s) in "
1436 + this + " of " + this.getRegionInfo().getRegionNameAsString()
1437 + " into tmpdir=" + fs.getTempDir() + ", totalSize="
1438 + TraditionalBinaryPrefix.long2String(cr.getSize(), "", 1));
1439
1440
1441 List<Path> newFiles = compaction.compact(throughputController, user);
1442
1443 long outputBytes = 0L;
1444
1445 if (!this.conf.getBoolean("hbase.hstore.compaction.complete", true)) {
1446 LOG.warn("hbase.hstore.compaction.complete is set to false");
1447 sfs = new ArrayList<StoreFile>(newFiles.size());
1448 final boolean evictOnClose =
1449 cacheConf != null? cacheConf.shouldEvictOnClose(): true;
1450 for (Path newFile : newFiles) {
1451
1452 StoreFile sf = createStoreFileAndReader(newFile);
1453 sf.closeReader(evictOnClose);
1454 sfs.add(sf);
1455 }
1456 return sfs;
1457 }
1458
1459 sfs = moveCompatedFilesIntoPlace(cr, newFiles, user);
1460 writeCompactionWalRecord(filesToCompact, sfs);
1461 replaceStoreFiles(filesToCompact, sfs);
1462 if (cr.isMajor()) {
1463 majorCompactedCellsCount.addAndGet(getCompactionProgress().totalCompactingKVs);
1464 majorCompactedCellsSize.addAndGet(getCompactionProgress().totalCompactedSize);
1465 } else {
1466 compactedCellsCount.addAndGet(getCompactionProgress().totalCompactingKVs);
1467 compactedCellsSize.addAndGet(getCompactionProgress().totalCompactedSize);
1468 }
1469
1470 for (StoreFile sf : sfs) {
1471 outputBytes += sf.getReader().length();
1472 }
1473
1474
1475 completeCompaction(filesToCompact);
1476
1477 long now = EnvironmentEdgeManager.currentTime();
1478 if (region.getRegionServerServices() != null
1479 && region.getRegionServerServices().getMetrics() != null) {
1480 region.getRegionServerServices().getMetrics().updateCompaction(cr.isMajor(),
1481 now - compactionStartTime, cr.getFiles().size(), newFiles.size(), cr.getSize(),
1482 outputBytes);
1483 }
1484
1485 logCompactionEndMessage(cr, sfs, now, compactionStartTime);
1486 return sfs;
1487 } finally {
1488 finishCompactionRequest(cr);
1489 }
1490 }
1491
1492 private List<StoreFile> moveCompatedFilesIntoPlace(
1493 final CompactionRequest cr, List<Path> newFiles, User user) throws IOException {
1494 List<StoreFile> sfs = new ArrayList<StoreFile>(newFiles.size());
1495 for (Path newFile : newFiles) {
1496 assert newFile != null;
1497 final StoreFile sf = moveFileIntoPlace(newFile);
1498 if (this.getCoprocessorHost() != null) {
1499 final Store thisStore = this;
1500 getCoprocessorHost().postCompact(thisStore, sf, cr, user);
1501 }
1502 assert sf != null;
1503 sfs.add(sf);
1504 }
1505 return sfs;
1506 }
1507
1508
1509 StoreFile moveFileIntoPlace(final Path newFile) throws IOException {
1510 validateStoreFile(newFile);
1511
1512 Path destPath = fs.commitStoreFile(getColumnFamilyName(), newFile);
1513 return createStoreFileAndReader(destPath);
1514 }
1515
1516
1517
1518
1519
1520
1521 private void writeCompactionWalRecord(Collection<StoreFile> filesCompacted,
1522 Collection<StoreFile> newFiles) throws IOException {
1523 if (region.getWAL() == null) return;
1524 List<Path> inputPaths = new ArrayList<Path>(filesCompacted.size());
1525 for (StoreFile f : filesCompacted) {
1526 inputPaths.add(f.getPath());
1527 }
1528 List<Path> outputPaths = new ArrayList<Path>(newFiles.size());
1529 for (StoreFile f : newFiles) {
1530 outputPaths.add(f.getPath());
1531 }
1532 HRegionInfo info = this.region.getRegionInfo();
1533 CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(info,
1534 family.getName(), inputPaths, outputPaths, fs.getStoreDir(getFamily().getNameAsString()));
1535 WALUtil.writeCompactionMarker(region.getWAL(), this.region.getTableDesc(),
1536 this.region.getRegionInfo(), compactionDescriptor, region.getMVCC());
1537 }
1538
1539 void replaceStoreFiles(final Collection<StoreFile> compactedFiles,
1540 final Collection<StoreFile> result) throws IOException {
1541 this.lock.writeLock().lock();
1542 try {
1543 this.storeEngine.getStoreFileManager().addCompactionResults(compactedFiles, result);
1544 synchronized (filesCompacting) {
1545 filesCompacting.removeAll(compactedFiles);
1546 }
1547 } finally {
1548 this.lock.writeLock().unlock();
1549 }
1550 }
1551
1552
1553
1554
1555
1556
1557
1558 private void logCompactionEndMessage(
1559 CompactionRequest cr, List<StoreFile> sfs, long now, long compactionStartTime) {
1560 StringBuilder message = new StringBuilder(
1561 "Completed" + (cr.isMajor() ? " major" : "") + " compaction of "
1562 + cr.getFiles().size() + (cr.isAllFiles() ? " (all)" : "") + " file(s) in "
1563 + this + " of " + this.getRegionInfo().getRegionNameAsString() + " into ");
1564 if (sfs.isEmpty()) {
1565 message.append("none, ");
1566 } else {
1567 for (StoreFile sf: sfs) {
1568 message.append(sf.getPath().getName());
1569 message.append("(size=");
1570 message.append(TraditionalBinaryPrefix.long2String(sf.getReader().length(), "", 1));
1571 message.append("), ");
1572 }
1573 }
1574 message.append("total size for store is ")
1575 .append(StringUtils.TraditionalBinaryPrefix.long2String(storeSize.get(), "", 1))
1576 .append(". This selection was in queue for ")
1577 .append(StringUtils.formatTimeDiff(compactionStartTime, cr.getSelectionTime()))
1578 .append(", and took ").append(StringUtils.formatTimeDiff(now, compactionStartTime))
1579 .append(" to execute.");
1580 LOG.info(message.toString());
1581 if (LOG.isTraceEnabled()) {
1582 int fileCount = storeEngine.getStoreFileManager().getStorefileCount();
1583 long resultSize = 0;
1584 for (StoreFile sf : sfs) {
1585 resultSize += sf.getReader().length();
1586 }
1587 String traceMessage = "COMPACTION start,end,size out,files in,files out,store size,"
1588 + "store files [" + compactionStartTime + "," + now + "," + resultSize + ","
1589 + cr.getFiles().size() + "," + sfs.size() + "," + storeSize + "," + fileCount + "]";
1590 LOG.trace(traceMessage);
1591 }
1592 }
1593
1594
1595
1596
1597
1598
1599
1600 @Override
1601 public void replayCompactionMarker(CompactionDescriptor compaction,
1602 boolean pickCompactionFiles, boolean removeFiles)
1603 throws IOException {
1604 LOG.debug("Completing compaction from the WAL marker");
1605 List<String> compactionInputs = compaction.getCompactionInputList();
1606 List<String> compactionOutputs = Lists.newArrayList(compaction.getCompactionOutputList());
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622 String familyName = this.getColumnFamilyName();
1623 List<String> inputFiles = new ArrayList<String>(compactionInputs.size());
1624 for (String compactionInput : compactionInputs) {
1625 Path inputPath = fs.getStoreFilePath(familyName, compactionInput);
1626 inputFiles.add(inputPath.getName());
1627 }
1628
1629
1630 List<StoreFile> inputStoreFiles = new ArrayList<StoreFile>(compactionInputs.size());
1631 for (StoreFile sf : this.getStorefiles()) {
1632 if (inputFiles.contains(sf.getPath().getName())) {
1633 inputStoreFiles.add(sf);
1634 }
1635 }
1636
1637
1638 List<StoreFile> outputStoreFiles = new ArrayList<StoreFile>(compactionOutputs.size());
1639
1640 if (pickCompactionFiles) {
1641 for (StoreFile sf : this.getStorefiles()) {
1642 compactionOutputs.remove(sf.getPath().getName());
1643 }
1644 for (String compactionOutput : compactionOutputs) {
1645 StoreFileInfo storeFileInfo = fs.getStoreFileInfo(getColumnFamilyName(), compactionOutput);
1646 StoreFile storeFile = createStoreFileAndReader(storeFileInfo);
1647 outputStoreFiles.add(storeFile);
1648 }
1649 }
1650
1651 if (!inputStoreFiles.isEmpty() || !outputStoreFiles.isEmpty()) {
1652 LOG.info("Replaying compaction marker, replacing input files: " +
1653 inputStoreFiles + " with output files : " + outputStoreFiles);
1654 this.replaceStoreFiles(inputStoreFiles, outputStoreFiles);
1655 this.completeCompaction(inputStoreFiles);
1656 }
1657 }
1658
1659
1660
1661
1662
1663
1664
1665
1666 public void compactRecentForTestingAssumingDefaultPolicy(int N) throws IOException {
1667 List<StoreFile> filesToCompact;
1668 boolean isMajor;
1669
1670 this.lock.readLock().lock();
1671 try {
1672 synchronized (filesCompacting) {
1673 filesToCompact = Lists.newArrayList(storeEngine.getStoreFileManager().getStorefiles());
1674 if (!filesCompacting.isEmpty()) {
1675
1676
1677 StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
1678 int idx = filesToCompact.indexOf(last);
1679 Preconditions.checkArgument(idx != -1);
1680 filesToCompact.subList(0, idx + 1).clear();
1681 }
1682 int count = filesToCompact.size();
1683 if (N > count) {
1684 throw new RuntimeException("Not enough files");
1685 }
1686
1687 filesToCompact = filesToCompact.subList(count - N, count);
1688 isMajor = (filesToCompact.size() == storeEngine.getStoreFileManager().getStorefileCount());
1689 filesCompacting.addAll(filesToCompact);
1690 Collections.sort(filesCompacting, storeEngine.getStoreFileManager()
1691 .getStoreFileComparator());
1692 }
1693 } finally {
1694 this.lock.readLock().unlock();
1695 }
1696
1697 try {
1698
1699 List<Path> newFiles = ((DefaultCompactor)this.storeEngine.getCompactor())
1700 .compactForTesting(filesToCompact, isMajor);
1701 for (Path newFile: newFiles) {
1702
1703 StoreFile sf = moveFileIntoPlace(newFile);
1704 if (this.getCoprocessorHost() != null) {
1705 this.getCoprocessorHost().postCompact(this, sf, null, null);
1706 }
1707 replaceStoreFiles(filesToCompact, Lists.newArrayList(sf));
1708 completeCompaction(filesToCompact);
1709 }
1710 } finally {
1711 synchronized (filesCompacting) {
1712 filesCompacting.removeAll(filesToCompact);
1713 }
1714 }
1715 }
1716
1717 @Override
1718 public boolean hasReferences() {
1719
1720
1721 this.lock.readLock().lock();
1722 try {
1723
1724 Collection<StoreFile> allStoreFiles = new ArrayList<>(getStorefiles());
1725 allStoreFiles.addAll(getCompactedfiles());
1726 return StoreUtils.hasReferences(allStoreFiles);
1727 } finally {
1728 this.lock.readLock().unlock();
1729 }
1730 }
1731
1732 @Override
1733 public CompactionProgress getCompactionProgress() {
1734 return this.storeEngine.getCompactor().getProgress();
1735 }
1736
1737 @Override
1738 public boolean isMajorCompaction() throws IOException {
1739 for (StoreFile sf : this.storeEngine.getStoreFileManager().getStorefiles()) {
1740
1741 if (sf.getReader() == null) {
1742 LOG.debug("StoreFile " + sf + " has null Reader");
1743 return false;
1744 }
1745 }
1746 return storeEngine.getCompactionPolicy().shouldPerformMajorCompaction(
1747 this.storeEngine.getStoreFileManager().getStorefiles());
1748 }
1749
1750 @Override
1751 public CompactionContext requestCompaction() throws IOException {
1752 return requestCompaction(Store.NO_PRIORITY, null);
1753 }
1754
1755 @Override
1756 public CompactionContext requestCompaction(int priority, CompactionRequest baseRequest)
1757 throws IOException {
1758 return requestCompaction(priority, baseRequest, null);
1759 }
1760 @Override
1761 public CompactionContext requestCompaction(int priority, final CompactionRequest baseRequest,
1762 User user) throws IOException {
1763
1764 if (!this.areWritesEnabled()) {
1765 return null;
1766 }
1767
1768
1769 removeUnneededFiles();
1770
1771 final CompactionContext compaction = storeEngine.createCompaction();
1772 CompactionRequest request = null;
1773 this.lock.readLock().lock();
1774 try {
1775 synchronized (filesCompacting) {
1776
1777 if (this.getCoprocessorHost() != null) {
1778 final List<StoreFile> candidatesForCoproc = compaction.preSelect(this.filesCompacting);
1779 boolean override = false;
1780 override = getCoprocessorHost().preCompactSelection(this, candidatesForCoproc,
1781 baseRequest, user);
1782 if (override) {
1783
1784 compaction.forceSelect(new CompactionRequest(candidatesForCoproc));
1785 }
1786 }
1787
1788
1789 if (!compaction.hasSelection()) {
1790 boolean isUserCompaction = priority == Store.PRIORITY_USER;
1791 boolean mayUseOffPeak = offPeakHours.isOffPeakHour() &&
1792 offPeakCompactionTracker.compareAndSet(false, true);
1793 try {
1794 compaction.select(this.filesCompacting, isUserCompaction,
1795 mayUseOffPeak, forceMajor && filesCompacting.isEmpty());
1796 } catch (IOException e) {
1797 if (mayUseOffPeak) {
1798 offPeakCompactionTracker.set(false);
1799 }
1800 throw e;
1801 }
1802 assert compaction.hasSelection();
1803 if (mayUseOffPeak && !compaction.getRequest().isOffPeak()) {
1804
1805 offPeakCompactionTracker.set(false);
1806 }
1807 }
1808 if (this.getCoprocessorHost() != null) {
1809 this.getCoprocessorHost().postCompactSelection(
1810 this, ImmutableList.copyOf(compaction.getRequest().getFiles()), baseRequest, user);
1811 }
1812
1813
1814 if (baseRequest != null) {
1815
1816
1817 compaction.forceSelect(
1818 baseRequest.combineWith(compaction.getRequest()));
1819 }
1820
1821 request = compaction.getRequest();
1822 final Collection<StoreFile> selectedFiles = request.getFiles();
1823 if (selectedFiles.isEmpty()) {
1824 return null;
1825 }
1826
1827 addToCompactingFiles(selectedFiles);
1828
1829
1830 this.forceMajor = this.forceMajor && !request.isMajor();
1831
1832
1833
1834 final int compactionPriority =
1835 (priority != Store.NO_PRIORITY) ? priority : getCompactPriority();
1836 request.setPriority(compactionPriority);
1837
1838 if (request.isAfterSplit()) {
1839
1840
1841
1842
1843 final int splitHousekeepingPriority =
1844 Math.min(compactionPriority, SPLIT_REGION_COMPACTION_PRIORITY);
1845 request.setPriority(splitHousekeepingPriority);
1846 LOG.info("Keeping/Overriding Compaction request priority to " + splitHousekeepingPriority
1847 + " for CF " + this.getColumnFamilyName() + " since it"
1848 + " belongs to recently split daughter region " + getRegionInfo()
1849 .getRegionNameAsString());
1850 }
1851 request.setDescription(getRegionInfo().getRegionNameAsString(), getColumnFamilyName());
1852 }
1853 } finally {
1854 this.lock.readLock().unlock();
1855 }
1856
1857 LOG.debug(getRegionInfo().getEncodedName() + " - " + getColumnFamilyName()
1858 + ": Initiating " + (request.isMajor() ? "major" : "minor") + " compaction"
1859 + (request.isAllFiles() ? " (all files)" : ""));
1860 this.region.reportCompactionRequestStart(request.isMajor());
1861 return compaction;
1862 }
1863
1864
1865 private void addToCompactingFiles(final Collection<StoreFile> filesToAdd) {
1866 if (filesToAdd == null) return;
1867
1868 if (!Collections.disjoint(filesCompacting, filesToAdd)) {
1869 Preconditions.checkArgument(false, "%s overlaps with %s", filesToAdd, filesCompacting);
1870 }
1871 filesCompacting.addAll(filesToAdd);
1872 Collections.sort(filesCompacting, storeEngine.getStoreFileManager().getStoreFileComparator());
1873 }
1874
1875 private void removeUnneededFiles() throws IOException {
1876 if (!conf.getBoolean("hbase.store.delete.expired.storefile", true)) return;
1877 if (getFamily().getMinVersions() > 0) {
1878 LOG.debug("Skipping expired store file removal due to min version being " +
1879 getFamily().getMinVersions());
1880 return;
1881 }
1882 this.lock.readLock().lock();
1883 Collection<StoreFile> delSfs = null;
1884 try {
1885 synchronized (filesCompacting) {
1886 long cfTtl = getStoreFileTtl();
1887 if (cfTtl != Long.MAX_VALUE) {
1888 delSfs = storeEngine.getStoreFileManager().getUnneededFiles(
1889 EnvironmentEdgeManager.currentTime() - cfTtl, filesCompacting);
1890 addToCompactingFiles(delSfs);
1891 }
1892 }
1893 } finally {
1894 this.lock.readLock().unlock();
1895 }
1896 if (delSfs == null || delSfs.isEmpty()) return;
1897
1898 Collection<StoreFile> newFiles = new ArrayList<StoreFile>();
1899 writeCompactionWalRecord(delSfs, newFiles);
1900 replaceStoreFiles(delSfs, newFiles);
1901 completeCompaction(delSfs);
1902 LOG.info("Completed removal of " + delSfs.size() + " unnecessary (expired) file(s) in "
1903 + this + " of " + this.getRegionInfo().getRegionNameAsString()
1904 + "; total size for store is " + TraditionalBinaryPrefix.long2String(storeSize.get(), "", 1));
1905 }
1906
1907 @Override
1908 public void cancelRequestedCompaction(CompactionContext compaction) {
1909 finishCompactionRequest(compaction.getRequest());
1910 }
1911
1912 private void finishCompactionRequest(CompactionRequest cr) {
1913 this.region.reportCompactionRequestEnd(cr.isMajor(), cr.getFiles().size(), cr.getSize());
1914 if (cr.isOffPeak()) {
1915 offPeakCompactionTracker.set(false);
1916 cr.setOffPeak(false);
1917 }
1918 synchronized (filesCompacting) {
1919 filesCompacting.removeAll(cr.getFiles());
1920 }
1921 }
1922
1923
1924
1925
1926
1927
1928
1929 private void validateStoreFile(Path path)
1930 throws IOException {
1931 StoreFile storeFile = null;
1932 try {
1933 storeFile = createStoreFileAndReader(path);
1934 } catch (IOException e) {
1935 LOG.error("Failed to open store file : " + path
1936 + ", keeping it in tmp location", e);
1937 throw e;
1938 } finally {
1939 if (storeFile != null) {
1940 storeFile.closeReader(false);
1941 }
1942 }
1943 }
1944
1945
1946
1947
1948
1949
1950
1951
1952
1953
1954
1955
1956
1957
1958
1959 protected void completeCompaction(final Collection<StoreFile> compactedFiles)
1960 throws IOException {
1961 completeCompaction(compactedFiles, true);
1962 }
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973
1974
1975
1976
1977
1978
1979 protected void completeCompaction(final Collection<StoreFile> compactedFiles, boolean removeFiles)
1980 throws IOException {
1981 LOG.debug("Completing compaction...");
1982 this.storeSize.set(0L);
1983 this.totalUncompressedBytes.set(0L);
1984 for (StoreFile hsf : this.storeEngine.getStoreFileManager().getStorefiles()) {
1985 StoreFile.Reader r = hsf.getReader();
1986 if (r == null) {
1987 LOG.warn("StoreFile " + hsf + " has a null Reader");
1988 continue;
1989 }
1990 this.storeSize.addAndGet(r.length());
1991 this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes());
1992 }
1993 }
1994
1995
1996
1997
1998
1999 int versionsToReturn(final int wantedVersions) {
2000 if (wantedVersions <= 0) {
2001 throw new IllegalArgumentException("Number of versions must be > 0");
2002 }
2003
2004 int maxVersions = this.family.getMaxVersions();
2005 return wantedVersions > maxVersions ? maxVersions: wantedVersions;
2006 }
2007
2008
2009
2010
2011
2012
2013 public static boolean isCellTTLExpired(final Cell cell, final long oldestTimestamp, final long now) {
2014
2015
2016 if (cell.getTagsLength() > 0) {
2017
2018
2019
2020 Iterator<Tag> i = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
2021 cell.getTagsLength());
2022 while (i.hasNext()) {
2023 Tag t = i.next();
2024 if (TagType.TTL_TAG_TYPE == t.getType()) {
2025
2026
2027 long ts = cell.getTimestamp();
2028 assert t.getTagLength() == Bytes.SIZEOF_LONG;
2029 long ttl = Bytes.toLong(t.getBuffer(), t.getTagOffset(), t.getTagLength());
2030 if (ts + ttl < now) {
2031 return true;
2032 }
2033
2034
2035 break;
2036 }
2037 }
2038 }
2039 return false;
2040 }
2041
2042 @Override
2043 public Cell getRowKeyAtOrBefore(final byte[] row) throws IOException {
2044
2045
2046
2047
2048
2049
2050 long ttlToUse = scanInfo.getMinVersions() > 0 ? Long.MAX_VALUE : this.scanInfo.getTtl();
2051
2052 KeyValue kv = new KeyValue(row, HConstants.LATEST_TIMESTAMP);
2053
2054 GetClosestRowBeforeTracker state = new GetClosestRowBeforeTracker(
2055 this.comparator, kv, ttlToUse, this.getRegionInfo().isMetaRegion());
2056 this.lock.readLock().lock();
2057 try {
2058
2059 this.memstore.getRowKeyAtOrBefore(state);
2060
2061
2062 Iterator<StoreFile> sfIterator = this.storeEngine.getStoreFileManager()
2063 .getCandidateFilesForRowKeyBefore(state.getTargetKey());
2064 while (sfIterator.hasNext()) {
2065 StoreFile sf = sfIterator.next();
2066 sfIterator.remove();
2067 boolean haveNewCandidate = rowAtOrBeforeFromStoreFile(sf, state);
2068 Cell candidate = state.getCandidate();
2069
2070 if (candidate != null && CellUtil.matchingRow(candidate, row)) {
2071 return candidate;
2072 }
2073 if (haveNewCandidate) {
2074 sfIterator = this.storeEngine.getStoreFileManager().updateCandidateFilesForRowKeyBefore(
2075 sfIterator, state.getTargetKey(), candidate);
2076 }
2077 }
2078 return state.getCandidate();
2079 } finally {
2080 this.lock.readLock().unlock();
2081 }
2082 }
2083
2084
2085
2086
2087
2088
2089
2090
2091 private boolean rowAtOrBeforeFromStoreFile(final StoreFile f,
2092 final GetClosestRowBeforeTracker state)
2093 throws IOException {
2094 StoreFile.Reader r = f.getReader();
2095 if (r == null) {
2096 LOG.warn("StoreFile " + f + " has a null Reader");
2097 return false;
2098 }
2099 if (r.getEntries() == 0) {
2100 LOG.warn("StoreFile " + f + " is a empty store file");
2101 return false;
2102 }
2103
2104 byte [] fk = r.getFirstKey();
2105 if (fk == null) return false;
2106 KeyValue firstKV = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
2107 byte [] lk = r.getLastKey();
2108 KeyValue lastKV = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
2109 KeyValue firstOnRow = state.getTargetKey();
2110 if (this.comparator.compareRows(lastKV, firstOnRow) < 0) {
2111
2112
2113 if (!state.isTargetTable(lastKV)) return false;
2114
2115
2116 firstOnRow = new KeyValue(lastKV.getRow(), HConstants.LATEST_TIMESTAMP);
2117 }
2118
2119 HFileScanner scanner = r.getScanner(true, true, false);
2120
2121 if (!seekToScanner(scanner, firstOnRow, firstKV)) return false;
2122
2123
2124 if (walkForwardInSingleRow(scanner, firstOnRow, state)) return true;
2125
2126 while (scanner.seekBefore(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(),
2127 firstOnRow.getKeyLength())) {
2128 Cell kv = scanner.getKeyValue();
2129 if (!state.isTargetTable(kv)) break;
2130 if (!state.isBetterCandidate(kv)) break;
2131
2132 firstOnRow = new KeyValue(kv.getRow(), HConstants.LATEST_TIMESTAMP);
2133
2134 if (!seekToScanner(scanner, firstOnRow, firstKV)) return false;
2135
2136 if (walkForwardInSingleRow(scanner, firstOnRow, state)) return true;
2137 }
2138 return false;
2139 }
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149 private boolean seekToScanner(final HFileScanner scanner,
2150 final KeyValue firstOnRow,
2151 final KeyValue firstKV)
2152 throws IOException {
2153 KeyValue kv = firstOnRow;
2154
2155 if (this.comparator.compareRows(firstKV, firstOnRow) == 0) kv = firstKV;
2156 int result = scanner.seekTo(kv);
2157 return result != -1;
2158 }
2159
2160
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170 private boolean walkForwardInSingleRow(final HFileScanner scanner,
2171 final KeyValue firstOnRow,
2172 final GetClosestRowBeforeTracker state)
2173 throws IOException {
2174 boolean foundCandidate = false;
2175 do {
2176 Cell kv = scanner.getKeyValue();
2177
2178 if (this.comparator.compareRows(kv, firstOnRow) < 0) continue;
2179
2180 if (state.isTooFar(kv, firstOnRow)) break;
2181 if (state.isExpired(kv)) {
2182 continue;
2183 }
2184
2185 if (state.handle(kv)) {
2186 foundCandidate = true;
2187 break;
2188 }
2189 } while(scanner.next());
2190 return foundCandidate;
2191 }
2192
2193 @Override
2194 public boolean canSplit() {
2195
2196 boolean result = !hasReferences();
2197 if (!result && LOG.isTraceEnabled()) {
2198 LOG.trace(String.format("Not splittable; has references: %s", this));
2199 }
2200 return result;
2201 }
2202
2203 @Override
2204 public byte[] getSplitPoint() {
2205 this.lock.readLock().lock();
2206 try {
2207
2208 assert !this.getRegionInfo().isMetaRegion();
2209
2210 if (hasReferences()) {
2211 if (LOG.isTraceEnabled()) {
2212 LOG.trace("Not splittable; has references: " + this);
2213 }
2214 return null;
2215 }
2216 return this.storeEngine.getStoreFileManager().getSplitPoint();
2217 } catch(IOException e) {
2218 LOG.warn("Failed getting store size for " + this, e);
2219 } finally {
2220 this.lock.readLock().unlock();
2221 }
2222 return null;
2223 }
2224
2225 @Override
2226 public long getLastCompactSize() {
2227 return this.lastCompactSize;
2228 }
2229
2230 @Override
2231 public long getSize() {
2232 return storeSize.get();
2233 }
2234
2235 @Override
2236 public void triggerMajorCompaction() {
2237 this.forceMajor = true;
2238 }
2239
2240
2241
2242
2243
2244
2245 @Override
2246 public KeyValueScanner getScanner(Scan scan,
2247 final NavigableSet<byte []> targetCols, long readPt) throws IOException {
2248 lock.readLock().lock();
2249 try {
2250 KeyValueScanner scanner = null;
2251 if (this.getCoprocessorHost() != null) {
2252 scanner = this.getCoprocessorHost().preStoreScannerOpen(this, scan, targetCols);
2253 }
2254 if (scanner == null) {
2255 scanner = scan.isReversed() ? new ReversedStoreScanner(this,
2256 getScanInfo(), scan, targetCols, readPt) : new StoreScanner(this,
2257 getScanInfo(), scan, targetCols, readPt);
2258 }
2259 return scanner;
2260 } finally {
2261 lock.readLock().unlock();
2262 }
2263 }
2264
2265 @Override
2266 public String toString() {
2267 return this.getColumnFamilyName();
2268 }
2269
2270 @Override
2271 public int getStorefilesCount() {
2272 return this.storeEngine.getStoreFileManager().getStorefileCount();
2273 }
2274
2275 @Override
2276 public long getMaxStoreFileAge() {
2277 long earliestTS = Long.MAX_VALUE;
2278 for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) {
2279 StoreFile.Reader r = s.getReader();
2280 if (r == null) {
2281 LOG.warn("StoreFile " + s + " has a null Reader");
2282 continue;
2283 }
2284 if (!s.isHFile()) {
2285 continue;
2286 }
2287 long createdTS = s.getFileInfo().getCreatedTimestamp();
2288 earliestTS = (createdTS < earliestTS) ? createdTS : earliestTS;
2289 }
2290 long now = EnvironmentEdgeManager.currentTime();
2291 return now - earliestTS;
2292 }
2293
2294 @Override
2295 public long getMinStoreFileAge() {
2296 long latestTS = 0;
2297 for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) {
2298 StoreFile.Reader r = s.getReader();
2299 if (r == null) {
2300 LOG.warn("StoreFile " + s + " has a null Reader");
2301 continue;
2302 }
2303 if (!s.isHFile()) {
2304 continue;
2305 }
2306 long createdTS = s.getFileInfo().getCreatedTimestamp();
2307 latestTS = (createdTS > latestTS) ? createdTS : latestTS;
2308 }
2309 long now = EnvironmentEdgeManager.currentTime();
2310 return now - latestTS;
2311 }
2312
2313 @Override
2314 public long getAvgStoreFileAge() {
2315 long sum = 0, count = 0;
2316 for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) {
2317 StoreFile.Reader r = s.getReader();
2318 if (r == null) {
2319 LOG.warn("StoreFile " + s + " has a null Reader");
2320 continue;
2321 }
2322 if (!s.isHFile()) {
2323 continue;
2324 }
2325 sum += s.getFileInfo().getCreatedTimestamp();
2326 count++;
2327 }
2328 if (count == 0) {
2329 return 0;
2330 }
2331 long avgTS = sum / count;
2332 long now = EnvironmentEdgeManager.currentTime();
2333 return now - avgTS;
2334 }
2335
2336 @Override
2337 public long getNumReferenceFiles() {
2338 long numRefFiles = 0;
2339 for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) {
2340 if (s.isReference()) {
2341 numRefFiles++;
2342 }
2343 }
2344 return numRefFiles;
2345 }
2346
2347 @Override
2348 public long getNumHFiles() {
2349 long numHFiles = 0;
2350 for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) {
2351 if (s.isHFile()) {
2352 numHFiles++;
2353 }
2354 }
2355 return numHFiles;
2356 }
2357
2358 @Override
2359 public long getStoreSizeUncompressed() {
2360 return this.totalUncompressedBytes.get();
2361 }
2362
2363 private long getTotalUmcompressedBytes(Collection<StoreFile> files) {
2364 long size = 0;
2365 for (StoreFile sf : files) {
2366 if (sf != null && sf.getReader() != null) {
2367 size += sf.getReader().getTotalUncompressedBytes();
2368 }
2369 }
2370 return size;
2371 }
2372
2373 private long getStorefilesSize(Collection<StoreFile> files) {
2374 long size = 0;
2375 for (StoreFile sf : files) {
2376 if (sf != null) {
2377 if (sf.getReader() == null) {
2378 LOG.warn("StoreFile " + sf + " has a null Reader");
2379 continue;
2380 }
2381 size += sf.getReader().length();
2382 }
2383 }
2384 return size;
2385 }
2386
2387 @Override
2388 public long getStorefilesSize() {
2389 return getStorefilesSize(storeEngine.getStoreFileManager().getStorefiles());
2390 }
2391
2392 @Override
2393 public long getStorefilesIndexSize() {
2394 long size = 0;
2395 for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) {
2396 StoreFile.Reader r = s.getReader();
2397 if (r == null) {
2398 LOG.warn("StoreFile " + s + " has a null Reader");
2399 continue;
2400 }
2401 size += r.indexSize();
2402 }
2403 return size;
2404 }
2405
2406 @Override
2407 public long getTotalStaticIndexSize() {
2408 long size = 0;
2409 for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) {
2410 StoreFile.Reader r = s.getReader();
2411 if (r == null) {
2412 continue;
2413 }
2414 size += r.getUncompressedDataIndexSize();
2415 }
2416 return size;
2417 }
2418
2419 @Override
2420 public long getTotalStaticBloomSize() {
2421 long size = 0;
2422 for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) {
2423 StoreFile.Reader r = s.getReader();
2424 if (r == null) {
2425 continue;
2426 }
2427 size += r.getTotalBloomSize();
2428 }
2429 return size;
2430 }
2431
2432 @Override
2433 public long getMemStoreSize() {
2434 return this.memstore.size();
2435 }
2436
2437 @Override
2438 public int getCompactPriority() {
2439 int priority = this.storeEngine.getStoreFileManager().getStoreCompactionPriority();
2440 if (priority == PRIORITY_USER) {
2441 LOG.warn("Compaction priority is USER despite there being no user compaction");
2442 }
2443 return priority;
2444 }
2445
2446 @Override
2447 public boolean throttleCompaction(long compactionSize) {
2448 return storeEngine.getCompactionPolicy().throttleCompaction(compactionSize);
2449 }
2450
2451 public HRegion getHRegion() {
2452 return this.region;
2453 }
2454
2455 @Override
2456 public RegionCoprocessorHost getCoprocessorHost() {
2457 return this.region.getCoprocessorHost();
2458 }
2459
2460 @Override
2461 public HRegionInfo getRegionInfo() {
2462 return this.fs.getRegionInfo();
2463 }
2464
2465 @Override
2466 public boolean areWritesEnabled() {
2467 return this.region.areWritesEnabled();
2468 }
2469
2470 @Override
2471 public long getSmallestReadPoint() {
2472 return this.region.getSmallestReadPoint();
2473 }
2474
2475
2476
2477
2478
2479
2480
2481
2482
2483
2484
2485
2486
2487
2488 public long updateColumnValue(byte [] row, byte [] f,
2489 byte [] qualifier, long newValue)
2490 throws IOException {
2491
2492 this.lock.readLock().lock();
2493 try {
2494 long now = EnvironmentEdgeManager.currentTime();
2495
2496 return this.memstore.updateColumnValue(row,
2497 f,
2498 qualifier,
2499 newValue,
2500 now);
2501
2502 } finally {
2503 this.lock.readLock().unlock();
2504 }
2505 }
2506
2507 @Override
2508 public long upsert(Iterable<Cell> cells, long readpoint,
2509 List<Cell> removedCells) throws IOException {
2510 this.lock.readLock().lock();
2511 try {
2512 return this.memstore.upsert(cells, readpoint, removedCells);
2513 } finally {
2514 this.lock.readLock().unlock();
2515 }
2516 }
2517
2518 @Override
2519 public StoreFlushContext createFlushContext(long cacheFlushId) {
2520 return new StoreFlusherImpl(cacheFlushId);
2521 }
2522
2523 private final class StoreFlusherImpl implements StoreFlushContext {
2524
2525 private long cacheFlushSeqNum;
2526 private MemStoreSnapshot snapshot;
2527 private List<Path> tempFiles;
2528 private List<Path> committedFiles;
2529 private long cacheFlushCount;
2530 private long cacheFlushSize;
2531 private long outputFileSize;
2532
2533 private StoreFlusherImpl(long cacheFlushSeqNum) {
2534 this.cacheFlushSeqNum = cacheFlushSeqNum;
2535 }
2536
2537
2538
2539
2540
2541 @Override
2542 public void prepare() {
2543 this.snapshot = memstore.snapshot();
2544 this.cacheFlushCount = snapshot.getCellsCount();
2545 this.cacheFlushSize = snapshot.getSize();
2546 committedFiles = new ArrayList<Path>(1);
2547 }
2548
2549 @Override
2550 public void flushCache(MonitoredTask status) throws IOException {
2551 RegionServerServices rsService = region.getRegionServerServices();
2552 ThroughputController throughputController =
2553 rsService == null ? null : rsService.getFlushThroughputController();
2554 tempFiles = HStore.this.flushCache(cacheFlushSeqNum, snapshot, status, throughputController);
2555 }
2556
2557 @Override
2558 public boolean commit(MonitoredTask status) throws IOException {
2559 if (this.tempFiles == null || this.tempFiles.isEmpty()) {
2560 return false;
2561 }
2562 List<StoreFile> storeFiles = new ArrayList<StoreFile>(this.tempFiles.size());
2563 for (Path storeFilePath : tempFiles) {
2564 try {
2565 StoreFile sf = HStore.this.commitFile(storeFilePath, cacheFlushSeqNum, status);
2566 outputFileSize += sf.getReader().length();
2567 storeFiles.add(sf);
2568 } catch (IOException ex) {
2569 LOG.error("Failed to commit store file " + storeFilePath, ex);
2570
2571 for (StoreFile sf : storeFiles) {
2572 Path pathToDelete = sf.getPath();
2573 try {
2574 sf.deleteReader();
2575 } catch (IOException deleteEx) {
2576 LOG.fatal("Failed to delete store file we committed, halting " + pathToDelete, ex);
2577 Runtime.getRuntime().halt(1);
2578 }
2579 }
2580 throw new IOException("Failed to commit the flush", ex);
2581 }
2582 }
2583
2584 for (StoreFile sf : storeFiles) {
2585 if (HStore.this.getCoprocessorHost() != null) {
2586 HStore.this.getCoprocessorHost().postFlush(HStore.this, sf);
2587 }
2588 committedFiles.add(sf.getPath());
2589 }
2590
2591 HStore.this.flushedCellsCount.addAndGet(cacheFlushCount);
2592 HStore.this.flushedCellsSize.addAndGet(cacheFlushSize);
2593 HStore.this.flushedOutputFileSize.addAndGet(outputFileSize);
2594
2595
2596 return HStore.this.updateStorefiles(storeFiles, snapshot.getId());
2597 }
2598
2599 @Override
2600 public long getOutputFileSize() {
2601 return outputFileSize;
2602 }
2603
2604 @Override
2605 public List<Path> getCommittedFiles() {
2606 return committedFiles;
2607 }
2608
2609
2610
2611
2612
2613
2614
2615
2616
2617 @Override
2618 public void replayFlush(List<String> fileNames, boolean dropMemstoreSnapshot)
2619 throws IOException {
2620 List<StoreFile> storeFiles = new ArrayList<StoreFile>(fileNames.size());
2621 for (String file : fileNames) {
2622
2623 StoreFileInfo storeFileInfo = fs.getStoreFileInfo(getColumnFamilyName(), file);
2624 StoreFile storeFile = createStoreFileAndReader(storeFileInfo);
2625 storeFiles.add(storeFile);
2626 HStore.this.storeSize.addAndGet(storeFile.getReader().length());
2627 HStore.this.totalUncompressedBytes.addAndGet(
2628 storeFile.getReader().getTotalUncompressedBytes());
2629 if (LOG.isInfoEnabled()) {
2630 LOG.info("Region: " + HStore.this.getRegionInfo().getEncodedName() +
2631 " added " + storeFile + ", entries=" + storeFile.getReader().getEntries() +
2632 ", sequenceid=" + storeFile.getReader().getSequenceID() +
2633 ", filesize=" + StringUtils.humanReadableInt(storeFile.getReader().length()));
2634 }
2635 }
2636
2637 long snapshotId = -1;
2638 if (dropMemstoreSnapshot && snapshot != null) {
2639 snapshotId = snapshot.getId();
2640 }
2641 HStore.this.updateStorefiles(storeFiles, snapshotId);
2642 }
2643
2644
2645
2646
2647
2648 @Override
2649 public void abort() throws IOException {
2650 if (snapshot == null) {
2651 return;
2652 }
2653 HStore.this.updateStorefiles(new ArrayList<StoreFile>(0), snapshot.getId());
2654 }
2655 }
2656
2657 @Override
2658 public boolean needsCompaction() {
2659 List<StoreFile> filesCompactingClone = null;
2660 synchronized (filesCompacting) {
2661 filesCompactingClone = Lists.newArrayList(filesCompacting);
2662 }
2663 return this.storeEngine.needsCompaction(filesCompactingClone);
2664 }
2665
2666 @Override
2667 public CacheConfig getCacheConfig() {
2668 return this.cacheConf;
2669 }
2670
2671 public static final long FIXED_OVERHEAD =
2672 ClassSize.align(ClassSize.OBJECT + (26 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG)
2673 + (5 * Bytes.SIZEOF_INT) + (2 * Bytes.SIZEOF_BOOLEAN));
2674
2675 public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD
2676 + ClassSize.OBJECT + ClassSize.REENTRANT_LOCK
2677 + ClassSize.CONCURRENT_SKIPLISTMAP
2678 + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + ClassSize.OBJECT
2679 + ScanInfo.FIXED_OVERHEAD);
2680
2681 @Override
2682 public long heapSize() {
2683 return DEEP_OVERHEAD + this.memstore.heapSize();
2684 }
2685
2686 @Override
2687 public KeyValue.KVComparator getComparator() {
2688 return comparator;
2689 }
2690
2691 @Override
2692 public ScanInfo getScanInfo() {
2693 return scanInfo;
2694 }
2695
2696
2697
2698
2699
2700 void setScanInfo(ScanInfo scanInfo) {
2701 this.scanInfo = scanInfo;
2702 }
2703
2704 @Override
2705 public boolean hasTooManyStoreFiles() {
2706 return getStorefilesCount() > this.blockingFileCount;
2707 }
2708
2709 @Override
2710 public long getFlushedCellsCount() {
2711 return flushedCellsCount.get();
2712 }
2713
2714 @Override
2715 public long getFlushedCellsSize() {
2716 return flushedCellsSize.get();
2717 }
2718
2719 @Override
2720 public long getFlushedOutputFileSize() {
2721 return flushedOutputFileSize.get();
2722 }
2723
2724 @Override
2725 public long getCompactedCellsCount() {
2726 return compactedCellsCount.get();
2727 }
2728
2729 @Override
2730 public long getCompactedCellsSize() {
2731 return compactedCellsSize.get();
2732 }
2733
2734 @Override
2735 public long getMajorCompactedCellsCount() {
2736 return majorCompactedCellsCount.get();
2737 }
2738
2739 @Override
2740 public long getMajorCompactedCellsSize() {
2741 return majorCompactedCellsSize.get();
2742 }
2743
2744
2745
2746
2747
2748 public StoreEngine<?, ?, ?, ?> getStoreEngine() {
2749 return this.storeEngine;
2750 }
2751
2752 protected OffPeakHours getOffPeakHours() {
2753 return this.offPeakHours;
2754 }
2755
2756
2757
2758
2759 @Override
2760 public void onConfigurationChange(Configuration conf) {
2761 this.conf = new CompoundConfiguration()
2762 .add(conf)
2763 .addWritableMap(family.getValues());
2764 this.storeEngine.compactionPolicy.setConf(conf);
2765 this.offPeakHours = OffPeakHours.getInstance(conf);
2766 }
2767
2768
2769
2770
2771 @Override
2772 public void registerChildren(ConfigurationManager manager) {
2773
2774 }
2775
2776
2777
2778
2779 @Override
2780 public void deregisterChildren(ConfigurationManager manager) {
2781
2782 }
2783
2784 @Override
2785 public double getCompactionPressure() {
2786 return storeEngine.getStoreFileManager().getCompactionPressure();
2787 }
2788
2789 @Override
2790 public boolean isPrimaryReplicaStore() {
2791 return getRegionInfo().getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID;
2792 }
2793
2794
2795
2796
2797
2798 public void preSnapshotOperation() {
2799 archiveLock.lock();
2800 }
2801
2802
2803
2804
2805
2806 public void postSnapshotOperation() {
2807 archiveLock.unlock();
2808 }
2809
2810 @Override
2811 public synchronized void closeAndArchiveCompactedFiles() throws IOException {
2812
2813 archiveLock.lock();
2814 try {
2815 lock.readLock().lock();
2816 Collection<StoreFile> copyCompactedfiles = null;
2817 try {
2818 Collection<StoreFile> compactedfiles = getCompactedfiles();
2819 if (compactedfiles != null && compactedfiles.size() != 0) {
2820
2821 copyCompactedfiles = new ArrayList<StoreFile>(compactedfiles);
2822 } else {
2823 if (LOG.isTraceEnabled()) {
2824 LOG.trace("No compacted files to archive");
2825 return;
2826 }
2827 }
2828 } finally {
2829 lock.readLock().unlock();
2830 }
2831 if (copyCompactedfiles != null && !copyCompactedfiles.isEmpty()) {
2832 removeCompactedfiles(copyCompactedfiles);
2833 }
2834 } finally {
2835 archiveLock.unlock();
2836 }
2837 }
2838
2839
2840
2841
2842
2843
2844 private void removeCompactedfiles(Collection<StoreFile> compactedfiles)
2845 throws IOException {
2846 final List<StoreFile> filesToRemove = new ArrayList<StoreFile>(compactedfiles.size());
2847 for (final StoreFile file : compactedfiles) {
2848 synchronized (file) {
2849 try {
2850 StoreFile.Reader r = file.getReader();
2851
2852 if (r == null) {
2853 if (LOG.isDebugEnabled()) {
2854 LOG.debug("The file " + file + " was closed but still not archived.");
2855 }
2856 filesToRemove.add(file);
2857 }
2858
2859 if (r != null && r.isCompactedAway() && !r.isReferencedInReads()) {
2860
2861
2862 if (LOG.isTraceEnabled()) {
2863 LOG.trace("Closing and archiving the file " + file.getPath());
2864 }
2865 r.close(true);
2866
2867 filesToRemove.add(file);
2868 } else {
2869 if (r != null) {
2870 LOG.info("Can't archive compacted file " + file.getPath()
2871 + " because of either isCompactedAway=" + r.isCompactedAway()
2872 + " or file has reference, isReferencedInReads=" + r.isReferencedInReads()
2873 + ", refCount=" + r.getRefCount() + ", skipping for now.");
2874 } else {
2875 LOG.info("Can't archive compacted file " + file.getPath() + ", skipping for now.");
2876 }
2877 }
2878 } catch (Exception e) {
2879 LOG.error(
2880 "Exception while trying to close the compacted store file " + file.getPath().getName());
2881 }
2882 }
2883 }
2884 if (this.isPrimaryReplicaStore()) {
2885
2886
2887
2888 if (!filesToRemove.isEmpty()) {
2889 if (LOG.isDebugEnabled()) {
2890 LOG.debug("Moving the files " + filesToRemove + " to archive");
2891 }
2892
2893 try {
2894 this.fs.removeStoreFiles(this.getFamily().getNameAsString(), filesToRemove);
2895 } catch (FailedArchiveException fae) {
2896
2897
2898
2899 Collection<Path> failedFiles = fae.getFailedFiles();
2900 Iterator<StoreFile> iter = filesToRemove.iterator();
2901 while (iter.hasNext()) {
2902 if (failedFiles.contains(iter.next().getPath())) {
2903 iter.remove();
2904 }
2905 }
2906 if (!filesToRemove.isEmpty()) {
2907 clearCompactedfiles(filesToRemove);
2908 }
2909 throw fae;
2910 }
2911 }
2912 }
2913 if (!filesToRemove.isEmpty()) {
2914
2915 clearCompactedfiles(filesToRemove);
2916 }
2917 }
2918
2919 private void clearCompactedfiles(final List<StoreFile> filesToRemove) throws IOException {
2920 if (LOG.isTraceEnabled()) {
2921 LOG.trace("Clearing the compacted file " + filesToRemove + " from this store");
2922 }
2923 try {
2924 lock.writeLock().lock();
2925 this.getStoreEngine().getStoreFileManager().removeCompactedFiles(filesToRemove);
2926 } finally {
2927 lock.writeLock().unlock();
2928 }
2929 }
2930
2931 public int getStoreRefCount() {
2932 int refCount = 0;
2933 for (StoreFile store: storeEngine.getStoreFileManager().getStorefiles()) {
2934 StoreFile.Reader r = store.getReader();
2935 if (r != null) {
2936 refCount += r.getRefCount();
2937 }
2938 }
2939 return refCount;
2940 }
2941
2942
2943
2944
2945
2946 public int getMaxCompactedStoreFileRefCount() {
2947 int maxCompactedStoreFileRefCount = 0;
2948 Collection<StoreFile> compactedFiles = storeEngine.getStoreFileManager().getCompactedfiles();
2949 if (compactedFiles != null) {
2950 for (StoreFile store : compactedFiles) {
2951 if (store.isHFile()) {
2952 StoreFile.Reader storeReader = store.getReader();
2953 if (storeReader != null) {
2954 maxCompactedStoreFileRefCount = Math.max(maxCompactedStoreFileRefCount,
2955 storeReader.getRefCount());
2956 }
2957 }
2958 }
2959 }
2960 return maxCompactedStoreFileRefCount;
2961 }
2962
2963 }