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