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