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