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