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