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