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