-
Notifications
You must be signed in to change notification settings - Fork 3.3k
/
HStore.java
2809 lines (2552 loc) · 109 KB
/
HStore.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.NavigableSet;
import java.util.Optional;
import java.util.OptionalDouble;
import java.util.OptionalInt;
import java.util.OptionalLong;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.CompletionService;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.Future;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.LongAdder;
import java.util.concurrent.locks.ReentrantLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Predicate;
import java.util.function.ToLongFunction;
import java.util.stream.Collectors;
import java.util.stream.LongStream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MemoryCompactionPolicy;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.FailedArchiveException;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.conf.ConfigurationManager;
import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
import org.apache.hadoop.hbase.coprocessor.ReadOnlyConfiguration;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.crypto.Encryption;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.io.hfile.InvalidHFileException;
import org.apache.hadoop.hbase.log.HBaseMarkers;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.quotas.RegionSizeStore;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
import org.apache.hadoop.hbase.regionserver.compactions.OffPeakHours;
import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
import org.apache.hadoop.hbase.security.EncryptionUtil;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableCollection;
import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.IterableUtils;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
/**
* A Store holds a column family in a Region. Its a memstore and a set of zero
* or more StoreFiles, which stretch backwards over time.
*
* <p>There's no reason to consider append-logging at this level; all logging
* and locking is handled at the HRegion level. Store just provides
* services to manage sets of StoreFiles. One of the most important of those
* services is compaction services where files are aggregated once they pass
* a configurable threshold.
*
* <p>Locking and transactions are handled at a higher level. This API should
* not be called directly but by an HRegion manager.
*/
@InterfaceAudience.Private
public class HStore implements Store, HeapSize, StoreConfigInformation,
PropagatingConfigurationObserver {
public static final String MEMSTORE_CLASS_NAME = "hbase.regionserver.memstore.class";
public static final String COMPACTCHECKER_INTERVAL_MULTIPLIER_KEY =
"hbase.server.compactchecker.interval.multiplier";
public static final String BLOCKING_STOREFILES_KEY = "hbase.hstore.blockingStoreFiles";
public static final String BLOCK_STORAGE_POLICY_KEY = "hbase.hstore.block.storage.policy";
// "NONE" is not a valid storage policy and means we defer the policy to HDFS
public static final String DEFAULT_BLOCK_STORAGE_POLICY = "NONE";
public static final int DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER = 1000;
public static final int DEFAULT_BLOCKING_STOREFILE_COUNT = 16;
// HBASE-24428 : Update compaction priority for recently split daughter regions
// so as to prioritize their compaction.
// Any compaction candidate with higher priority than compaction of newly split daugher regions
// should have priority value < (Integer.MIN_VALUE + 1000)
private static final int SPLIT_REGION_COMPACTION_PRIORITY = Integer.MIN_VALUE + 1000;
private static final Logger LOG = LoggerFactory.getLogger(HStore.class);
protected final MemStore memstore;
// This stores directory in the filesystem.
private final HRegion region;
protected Configuration conf;
private long lastCompactSize = 0;
volatile boolean forceMajor = false;
private AtomicLong storeSize = new AtomicLong();
private AtomicLong totalUncompressedBytes = new AtomicLong();
private LongAdder memstoreOnlyRowReadsCount = new LongAdder();
// rows that has cells from both memstore and files (or only files)
private LongAdder mixedRowReadsCount = new LongAdder();
private boolean cacheOnWriteLogged;
/**
* RWLock for store operations.
* Locked in shared mode when the list of component stores is looked at:
* - all reads/writes to table data
* - checking for split
* Locked in exclusive mode when the list of component stores is modified:
* - closing
* - completing a compaction
*/
final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
/**
* Lock specific to archiving compacted store files. This avoids races around
* the combination of retrieving the list of compacted files and moving them to
* the archive directory. Since this is usually a background process (other than
* on close), we don't want to handle this with the store write lock, which would
* block readers and degrade performance.
*
* Locked by:
* - CompactedHFilesDispatchHandler via closeAndArchiveCompactedFiles()
* - close()
*/
final ReentrantLock archiveLock = new ReentrantLock();
private final boolean verifyBulkLoads;
/**
* Use this counter to track concurrent puts. If TRACE-log is enabled, if we are over the
* threshold set by hbase.region.store.parallel.put.print.threshold (Default is 50) we will
* log a message that identifies the Store experience this high-level of concurrency.
*/
private final AtomicInteger currentParallelPutCount = new AtomicInteger(0);
private final int parallelPutCountPrintThreshold;
private ScanInfo scanInfo;
// All access must be synchronized.
// TODO: ideally, this should be part of storeFileManager, as we keep passing this to it.
private final List<HStoreFile> filesCompacting = Lists.newArrayList();
// All access must be synchronized.
private final Set<ChangedReadersObserver> changedReaderObservers =
Collections.newSetFromMap(new ConcurrentHashMap<ChangedReadersObserver, Boolean>());
private HFileDataBlockEncoder dataBlockEncoder;
final StoreEngine<?, ?, ?, ?> storeEngine;
private static final AtomicBoolean offPeakCompactionTracker = new AtomicBoolean();
private volatile OffPeakHours offPeakHours;
private static final int DEFAULT_FLUSH_RETRIES_NUMBER = 10;
private int flushRetriesNumber;
private int pauseTime;
private long blockingFileCount;
private int compactionCheckMultiplier;
private AtomicLong flushedCellsCount = new AtomicLong();
private AtomicLong compactedCellsCount = new AtomicLong();
private AtomicLong majorCompactedCellsCount = new AtomicLong();
private AtomicLong flushedCellsSize = new AtomicLong();
private AtomicLong flushedOutputFileSize = new AtomicLong();
private AtomicLong compactedCellsSize = new AtomicLong();
private AtomicLong majorCompactedCellsSize = new AtomicLong();
private final StoreContext storeContext;
/**
* Constructor
* @param family HColumnDescriptor for this column
* @param confParam configuration object failed. Can be null.
*/
protected HStore(final HRegion region, final ColumnFamilyDescriptor family,
final Configuration confParam, boolean warmup) throws IOException {
this.conf = StoreUtils.createStoreConfiguration(confParam, region.getTableDescriptor(), family);
this.region = region;
this.storeContext = initializeStoreContext(family);
// Assemble the store's home directory and Ensure it exists.
region.getRegionFileSystem().createStoreDir(family.getNameAsString());
// set block storage policy for store directory
String policyName = family.getStoragePolicy();
if (null == policyName) {
policyName = this.conf.get(BLOCK_STORAGE_POLICY_KEY, DEFAULT_BLOCK_STORAGE_POLICY);
}
region.getRegionFileSystem().setStoragePolicy(family.getNameAsString(), policyName.trim());
this.dataBlockEncoder = new HFileDataBlockEncoderImpl(family.getDataBlockEncoding());
// used by ScanQueryMatcher
long timeToPurgeDeletes =
Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);
LOG.trace("Time to purge deletes set to {}ms in {}", timeToPurgeDeletes, this);
// Get TTL
long ttl = determineTTLFromFamily(family);
// Why not just pass a HColumnDescriptor in here altogether? Even if have
// to clone it?
scanInfo = new ScanInfo(conf, family, ttl, timeToPurgeDeletes, region.getCellComparator());
this.memstore = getMemstore();
this.offPeakHours = OffPeakHours.getInstance(conf);
this.verifyBulkLoads = conf.getBoolean("hbase.hstore.bulkload.verify", false);
this.blockingFileCount =
conf.getInt(BLOCKING_STOREFILES_KEY, DEFAULT_BLOCKING_STOREFILE_COUNT);
this.compactionCheckMultiplier = conf.getInt(
COMPACTCHECKER_INTERVAL_MULTIPLIER_KEY, DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER);
if (this.compactionCheckMultiplier <= 0) {
LOG.error("Compaction check period multiplier must be positive, setting default: {}",
DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER);
this.compactionCheckMultiplier = DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER;
}
this.storeEngine = createStoreEngine(this, this.conf, region.getCellComparator());
List<HStoreFile> hStoreFiles = loadStoreFiles(warmup);
// Move the storeSize calculation out of loadStoreFiles() method, because the secondary read
// replica's refreshStoreFiles() will also use loadStoreFiles() to refresh its store files and
// update the storeSize in the refreshStoreSizeAndTotalBytes() finally (just like compaction) , so
// no need calculate the storeSize twice.
this.storeSize.addAndGet(getStorefilesSize(hStoreFiles, sf -> true));
this.totalUncompressedBytes.addAndGet(getTotalUncompressedBytes(hStoreFiles));
this.storeEngine.getStoreFileManager().loadFiles(hStoreFiles);
flushRetriesNumber = conf.getInt(
"hbase.hstore.flush.retries.number", DEFAULT_FLUSH_RETRIES_NUMBER);
pauseTime = conf.getInt(HConstants.HBASE_SERVER_PAUSE, HConstants.DEFAULT_HBASE_SERVER_PAUSE);
if (flushRetriesNumber <= 0) {
throw new IllegalArgumentException(
"hbase.hstore.flush.retries.number must be > 0, not "
+ flushRetriesNumber);
}
int confPrintThreshold =
this.conf.getInt("hbase.region.store.parallel.put.print.threshold", 50);
if (confPrintThreshold < 10) {
confPrintThreshold = 10;
}
this.parallelPutCountPrintThreshold = confPrintThreshold;
LOG.info("Store={}, memstore type={}, storagePolicy={}, verifyBulkLoads={}, "
+ "parallelPutCountPrintThreshold={}, encoding={}, compression={}",
this, memstore.getClass().getSimpleName(), policyName, verifyBulkLoads,
parallelPutCountPrintThreshold, family.getDataBlockEncoding(),
family.getCompressionType());
cacheOnWriteLogged = false;
}
private StoreContext initializeStoreContext(ColumnFamilyDescriptor family) throws IOException {
return new StoreContext.Builder()
.withBlockSize(family.getBlocksize())
.withEncryptionContext(EncryptionUtil.createEncryptionContext(conf, family))
.withBloomType(family.getBloomFilterType())
.withCacheConfig(createCacheConf(family))
.withCellComparator(region.getCellComparator())
.withColumnFamilyDescriptor(family)
.withCompactedFilesSupplier(this::getCompactedFiles)
.withRegionFileSystem(region.getRegionFileSystem())
.withFavoredNodesSupplier(this::getFavoredNodes)
.withFamilyStoreDirectoryPath(region.getRegionFileSystem()
.getStoreDir(family.getNameAsString()))
.withRegionCoprocessorHost(region.getCoprocessorHost())
.build();
}
private InetSocketAddress[] getFavoredNodes() {
InetSocketAddress[] favoredNodes = null;
if (region.getRegionServerServices() != null) {
favoredNodes = region.getRegionServerServices().getFavoredNodesForRegion(
region.getRegionInfo().getEncodedName());
}
return favoredNodes;
}
/**
* @return MemStore Instance to use in this store.
*/
private MemStore getMemstore() {
MemStore ms = null;
// Check if in-memory-compaction configured. Note MemoryCompactionPolicy is an enum!
MemoryCompactionPolicy inMemoryCompaction = null;
if (this.getTableName().isSystemTable()) {
inMemoryCompaction = MemoryCompactionPolicy.valueOf(
conf.get("hbase.systemtables.compacting.memstore.type", "NONE"));
} else {
inMemoryCompaction = getColumnFamilyDescriptor().getInMemoryCompaction();
}
if (inMemoryCompaction == null) {
inMemoryCompaction =
MemoryCompactionPolicy.valueOf(conf.get(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY,
CompactingMemStore.COMPACTING_MEMSTORE_TYPE_DEFAULT).toUpperCase());
}
switch (inMemoryCompaction) {
case NONE:
ms = ReflectionUtils.newInstance(DefaultMemStore.class,
new Object[] { conf, getComparator(),
this.getHRegion().getRegionServicesForStores()});
break;
default:
Class<? extends CompactingMemStore> clz = conf.getClass(MEMSTORE_CLASS_NAME,
CompactingMemStore.class, CompactingMemStore.class);
ms = ReflectionUtils.newInstance(clz, new Object[]{conf, getComparator(), this,
this.getHRegion().getRegionServicesForStores(), inMemoryCompaction});
}
return ms;
}
/**
* Creates the cache config.
* @param family The current column family.
*/
protected CacheConfig createCacheConf(final ColumnFamilyDescriptor family) {
CacheConfig cacheConf = new CacheConfig(conf, family, region.getBlockCache(),
region.getRegionServicesForStores().getByteBuffAllocator());
LOG.info("Created cacheConfig: {}, for column family {} of region {} ", cacheConf,
family.getNameAsString(), region.getRegionInfo().getEncodedName());
return cacheConf;
}
/**
* Creates the store engine configured for the given Store.
* @param store The store. An unfortunate dependency needed due to it
* being passed to coprocessors via the compactor.
* @param conf Store configuration.
* @param kvComparator KVComparator for storeFileManager.
* @return StoreEngine to use.
*/
protected StoreEngine<?, ?, ?, ?> createStoreEngine(HStore store, Configuration conf,
CellComparator kvComparator) throws IOException {
return StoreEngine.create(store, conf, kvComparator);
}
/**
* @return TTL in seconds of the specified family
*/
public static long determineTTLFromFamily(final ColumnFamilyDescriptor family) {
// HCD.getTimeToLive returns ttl in seconds. Convert to milliseconds.
long ttl = family.getTimeToLive();
if (ttl == HConstants.FOREVER) {
// Default is unlimited ttl.
ttl = Long.MAX_VALUE;
} else if (ttl == -1) {
ttl = Long.MAX_VALUE;
} else {
// Second -> ms adjust for user data
ttl *= 1000;
}
return ttl;
}
StoreContext getStoreContext() {
return storeContext;
}
@Override
public String getColumnFamilyName() {
return this.storeContext.getFamily().getNameAsString();
}
@Override
public TableName getTableName() {
return this.getRegionInfo().getTable();
}
@Override
public FileSystem getFileSystem() {
return storeContext.getRegionFileSystem().getFileSystem();
}
public HRegionFileSystem getRegionFileSystem() {
return storeContext.getRegionFileSystem();
}
/* Implementation of StoreConfigInformation */
@Override
public long getStoreFileTtl() {
// TTL only applies if there's no MIN_VERSIONs setting on the column.
return (this.scanInfo.getMinVersions() == 0) ? this.scanInfo.getTtl() : Long.MAX_VALUE;
}
@Override
public long getMemStoreFlushSize() {
// TODO: Why is this in here? The flushsize of the region rather than the store? St.Ack
return this.region.memstoreFlushSize;
}
@Override
public MemStoreSize getFlushableSize() {
return this.memstore.getFlushableSize();
}
@Override
public MemStoreSize getSnapshotSize() {
return this.memstore.getSnapshotSize();
}
@Override
public long getCompactionCheckMultiplier() {
return this.compactionCheckMultiplier;
}
@Override
public long getBlockingFileCount() {
return blockingFileCount;
}
/* End implementation of StoreConfigInformation */
@Override
public ColumnFamilyDescriptor getColumnFamilyDescriptor() {
return this.storeContext.getFamily();
}
@Override
public OptionalLong getMaxSequenceId() {
return StoreUtils.getMaxSequenceIdInList(this.getStorefiles());
}
@Override
public OptionalLong getMaxMemStoreTS() {
return StoreUtils.getMaxMemStoreTSInList(this.getStorefiles());
}
/**
* @return the data block encoder
*/
public HFileDataBlockEncoder getDataBlockEncoder() {
return dataBlockEncoder;
}
/**
* Should be used only in tests.
* @param blockEncoder the block delta encoder to use
*/
void setDataBlockEncoderInTest(HFileDataBlockEncoder blockEncoder) {
this.dataBlockEncoder = blockEncoder;
}
/**
* Creates an unsorted list of StoreFile loaded in parallel
* from the given directory.
*/
private List<HStoreFile> loadStoreFiles(boolean warmup) throws IOException {
Collection<StoreFileInfo> files = getRegionFileSystem().getStoreFiles(getColumnFamilyName());
return openStoreFiles(files, warmup);
}
private List<HStoreFile> openStoreFiles(Collection<StoreFileInfo> files, boolean warmup)
throws IOException {
if (CollectionUtils.isEmpty(files)) {
return Collections.emptyList();
}
// initialize the thread pool for opening store files in parallel..
ThreadPoolExecutor storeFileOpenerThreadPool =
this.region.getStoreFileOpenAndCloseThreadPool("StoreFileOpener-"
+ this.region.getRegionInfo().getEncodedName() + "-" + this.getColumnFamilyName());
CompletionService<HStoreFile> completionService =
new ExecutorCompletionService<>(storeFileOpenerThreadPool);
int totalValidStoreFile = 0;
for (StoreFileInfo storeFileInfo : files) {
// The StoreFileInfo will carry store configuration down to HFile, we need to set it to
// our store's CompoundConfiguration here.
storeFileInfo.setConf(conf);
// open each store file in parallel
completionService.submit(() -> this.createStoreFileAndReader(storeFileInfo));
totalValidStoreFile++;
}
Set<String> compactedStoreFiles = new HashSet<>();
ArrayList<HStoreFile> results = new ArrayList<>(files.size());
IOException ioe = null;
try {
for (int i = 0; i < totalValidStoreFile; i++) {
try {
HStoreFile storeFile = completionService.take().get();
if (storeFile != null) {
LOG.debug("loaded {}", storeFile);
results.add(storeFile);
compactedStoreFiles.addAll(storeFile.getCompactedStoreFiles());
}
} catch (InterruptedException e) {
if (ioe == null) {
ioe = new InterruptedIOException(e.getMessage());
}
} catch (ExecutionException e) {
if (ioe == null) {
ioe = new IOException(e.getCause());
}
}
}
} finally {
storeFileOpenerThreadPool.shutdownNow();
}
if (ioe != null) {
// close StoreFile readers
boolean evictOnClose =
getCacheConfig() != null? getCacheConfig().shouldEvictOnClose(): true;
for (HStoreFile file : results) {
try {
if (file != null) {
file.closeStoreFile(evictOnClose);
}
} catch (IOException e) {
LOG.warn("Could not close store file {}", file, e);
}
}
throw ioe;
}
// Should not archive the compacted store files when region warmup. See HBASE-22163.
if (!warmup) {
// Remove the compacted files from result
List<HStoreFile> filesToRemove = new ArrayList<>(compactedStoreFiles.size());
for (HStoreFile storeFile : results) {
if (compactedStoreFiles.contains(storeFile.getPath().getName())) {
LOG.warn("Clearing the compacted storefile {} from {}", storeFile, this);
storeFile.getReader().close(storeFile.getCacheConf() != null ?
storeFile.getCacheConf().shouldEvictOnClose() : true);
filesToRemove.add(storeFile);
}
}
results.removeAll(filesToRemove);
if (!filesToRemove.isEmpty() && this.isPrimaryReplicaStore()) {
LOG.debug("Moving the files {} to archive", filesToRemove);
getRegionFileSystem().removeStoreFiles(this.getColumnFamilyDescriptor().getNameAsString(),
filesToRemove);
}
}
return results;
}
@Override
public void refreshStoreFiles() throws IOException {
Collection<StoreFileInfo> newFiles = getRegionFileSystem().getStoreFiles(getColumnFamilyName());
refreshStoreFilesInternal(newFiles);
}
/**
* Replaces the store files that the store has with the given files. Mainly used by secondary
* region replicas to keep up to date with the primary region files.
*/
public void refreshStoreFiles(Collection<String> newFiles) throws IOException {
List<StoreFileInfo> storeFiles = new ArrayList<>(newFiles.size());
for (String file : newFiles) {
storeFiles.add(getRegionFileSystem().getStoreFileInfo(getColumnFamilyName(), file));
}
refreshStoreFilesInternal(storeFiles);
}
/**
* Checks the underlying store files, and opens the files that have not
* been opened, and removes the store file readers for store files no longer
* available. Mainly used by secondary region replicas to keep up to date with
* the primary region files.
*/
private void refreshStoreFilesInternal(Collection<StoreFileInfo> newFiles) throws IOException {
StoreFileManager sfm = storeEngine.getStoreFileManager();
Collection<HStoreFile> currentFiles = sfm.getStorefiles();
Collection<HStoreFile> compactedFiles = sfm.getCompactedfiles();
if (currentFiles == null) {
currentFiles = Collections.emptySet();
}
if (newFiles == null) {
newFiles = Collections.emptySet();
}
if (compactedFiles == null) {
compactedFiles = Collections.emptySet();
}
HashMap<StoreFileInfo, HStoreFile> currentFilesSet = new HashMap<>(currentFiles.size());
for (HStoreFile sf : currentFiles) {
currentFilesSet.put(sf.getFileInfo(), sf);
}
HashMap<StoreFileInfo, HStoreFile> compactedFilesSet = new HashMap<>(compactedFiles.size());
for (HStoreFile sf : compactedFiles) {
compactedFilesSet.put(sf.getFileInfo(), sf);
}
Set<StoreFileInfo> newFilesSet = new HashSet<StoreFileInfo>(newFiles);
// Exclude the files that have already been compacted
newFilesSet = Sets.difference(newFilesSet, compactedFilesSet.keySet());
Set<StoreFileInfo> toBeAddedFiles = Sets.difference(newFilesSet, currentFilesSet.keySet());
Set<StoreFileInfo> toBeRemovedFiles = Sets.difference(currentFilesSet.keySet(), newFilesSet);
if (toBeAddedFiles.isEmpty() && toBeRemovedFiles.isEmpty()) {
return;
}
LOG.info("Refreshing store files for " + this + " files to add: "
+ toBeAddedFiles + " files to remove: " + toBeRemovedFiles);
Set<HStoreFile> toBeRemovedStoreFiles = new HashSet<>(toBeRemovedFiles.size());
for (StoreFileInfo sfi : toBeRemovedFiles) {
toBeRemovedStoreFiles.add(currentFilesSet.get(sfi));
}
// try to open the files
List<HStoreFile> openedFiles = openStoreFiles(toBeAddedFiles, false);
// propogate the file changes to the underlying store file manager
replaceStoreFiles(toBeRemovedStoreFiles, openedFiles); //won't throw an exception
// Advance the memstore read point to be at least the new store files seqIds so that
// readers might pick it up. This assumes that the store is not getting any writes (otherwise
// in-flight transactions might be made visible)
if (!toBeAddedFiles.isEmpty()) {
// we must have the max sequence id here as we do have several store files
region.getMVCC().advanceTo(this.getMaxSequenceId().getAsLong());
}
refreshStoreSizeAndTotalBytes();
}
protected HStoreFile createStoreFileAndReader(final Path p) throws IOException {
StoreFileInfo info = new StoreFileInfo(conf, this.getFileSystem(),
p, isPrimaryReplicaStore());
return createStoreFileAndReader(info);
}
private HStoreFile createStoreFileAndReader(StoreFileInfo info) throws IOException {
info.setRegionCoprocessorHost(this.region.getCoprocessorHost());
HStoreFile storeFile = new HStoreFile(info, getColumnFamilyDescriptor().getBloomFilterType(),
getCacheConfig());
storeFile.initReader();
return storeFile;
}
/**
* This message intends to inform the MemStore that next coming updates
* are going to be part of the replaying edits from WAL
*/
public void startReplayingFromWAL(){
this.memstore.startReplayingFromWAL();
}
/**
* This message intends to inform the MemStore that the replaying edits from WAL
* are done
*/
public void stopReplayingFromWAL(){
this.memstore.stopReplayingFromWAL();
}
/**
* Adds a value to the memstore
*/
public void add(final Cell cell, MemStoreSizing memstoreSizing) {
lock.readLock().lock();
try {
if (this.currentParallelPutCount.getAndIncrement() > this.parallelPutCountPrintThreshold) {
LOG.trace("tableName={}, encodedName={}, columnFamilyName={} is too busy!",
this.getTableName(), this.getRegionInfo().getEncodedName(), this.getColumnFamilyName());
}
this.memstore.add(cell, memstoreSizing);
} finally {
lock.readLock().unlock();
currentParallelPutCount.decrementAndGet();
}
}
/**
* Adds the specified value to the memstore
*/
public void add(final Iterable<Cell> cells, MemStoreSizing memstoreSizing) {
lock.readLock().lock();
try {
if (this.currentParallelPutCount.getAndIncrement() > this.parallelPutCountPrintThreshold) {
LOG.trace("tableName={}, encodedName={}, columnFamilyName={} is too busy!",
this.getTableName(), this.getRegionInfo().getEncodedName(), this.getColumnFamilyName());
}
memstore.add(cells, memstoreSizing);
} finally {
lock.readLock().unlock();
currentParallelPutCount.decrementAndGet();
}
}
@Override
public long timeOfOldestEdit() {
return memstore.timeOfOldestEdit();
}
/**
* @return All store files.
*/
@Override
public Collection<HStoreFile> getStorefiles() {
return this.storeEngine.getStoreFileManager().getStorefiles();
}
@Override
public Collection<HStoreFile> getCompactedFiles() {
return this.storeEngine.getStoreFileManager().getCompactedfiles();
}
/**
* This throws a WrongRegionException if the HFile does not fit in this region, or an
* InvalidHFileException if the HFile is not valid.
*/
public void assertBulkLoadHFileOk(Path srcPath) throws IOException {
HFile.Reader reader = null;
try {
LOG.info("Validating hfile at " + srcPath + " for inclusion in " + this);
FileSystem srcFs = srcPath.getFileSystem(conf);
srcFs.access(srcPath, FsAction.READ_WRITE);
reader = HFile.createReader(srcFs, srcPath, getCacheConfig(), isPrimaryReplicaStore(), conf);
Optional<byte[]> firstKey = reader.getFirstRowKey();
Preconditions.checkState(firstKey.isPresent(), "First key can not be null");
Optional<Cell> lk = reader.getLastKey();
Preconditions.checkState(lk.isPresent(), "Last key can not be null");
byte[] lastKey = CellUtil.cloneRow(lk.get());
if (LOG.isDebugEnabled()) {
LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey.get()) +
" last=" + Bytes.toStringBinary(lastKey));
LOG.debug("Region bounds: first=" +
Bytes.toStringBinary(getRegionInfo().getStartKey()) +
" last=" + Bytes.toStringBinary(getRegionInfo().getEndKey()));
}
if (!this.getRegionInfo().containsRange(firstKey.get(), lastKey)) {
throw new WrongRegionException(
"Bulk load file " + srcPath.toString() + " does not fit inside region "
+ this.getRegionInfo().getRegionNameAsString());
}
if(reader.length() > conf.getLong(HConstants.HREGION_MAX_FILESIZE,
HConstants.DEFAULT_MAX_FILE_SIZE)) {
LOG.warn("Trying to bulk load hfile " + srcPath + " with size: " +
reader.length() + " bytes can be problematic as it may lead to oversplitting.");
}
if (verifyBulkLoads) {
long verificationStartTime = EnvironmentEdgeManager.currentTime();
LOG.info("Full verification started for bulk load hfile: {}", srcPath);
Cell prevCell = null;
HFileScanner scanner = reader.getScanner(conf, false, false, false);
scanner.seekTo();
do {
Cell cell = scanner.getCell();
if (prevCell != null) {
if (getComparator().compareRows(prevCell, cell) > 0) {
throw new InvalidHFileException("Previous row is greater than"
+ " current row: path=" + srcPath + " previous="
+ CellUtil.getCellKeyAsString(prevCell) + " current="
+ CellUtil.getCellKeyAsString(cell));
}
if (CellComparator.getInstance().compareFamilies(prevCell, cell) != 0) {
throw new InvalidHFileException("Previous key had different"
+ " family compared to current key: path=" + srcPath
+ " previous="
+ Bytes.toStringBinary(prevCell.getFamilyArray(), prevCell.getFamilyOffset(),
prevCell.getFamilyLength())
+ " current="
+ Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
cell.getFamilyLength()));
}
}
prevCell = cell;
} while (scanner.next());
LOG.info("Full verification complete for bulk load hfile: " + srcPath.toString() +
" took " + (EnvironmentEdgeManager.currentTime() - verificationStartTime) + " ms");
}
} finally {
if (reader != null) {
reader.close();
}
}
}
/**
* This method should only be called from Region. It is assumed that the ranges of values in the
* HFile fit within the stores assigned region. (assertBulkLoadHFileOk checks this)
*
* @param seqNum sequence Id associated with the HFile
*/
public Pair<Path, Path> preBulkLoadHFile(String srcPathStr, long seqNum) throws IOException {
Path srcPath = new Path(srcPathStr);
return getRegionFileSystem().bulkLoadStoreFile(getColumnFamilyName(), srcPath, seqNum);
}
public Path bulkLoadHFile(byte[] family, String srcPathStr, Path dstPath) throws IOException {
Path srcPath = new Path(srcPathStr);
try {
getRegionFileSystem().commitStoreFile(srcPath, dstPath);
} finally {
if (this.getCoprocessorHost() != null) {
this.getCoprocessorHost().postCommitStoreFile(family, srcPath, dstPath);
}
}
LOG.info("Loaded HFile " + srcPath + " into " + this + " as "
+ dstPath + " - updating store file list.");
HStoreFile sf = createStoreFileAndReader(dstPath);
bulkLoadHFile(sf);
LOG.info("Successfully loaded {} into {} (new location: {})",
srcPath, this, dstPath);
return dstPath;
}
public void bulkLoadHFile(StoreFileInfo fileInfo) throws IOException {
HStoreFile sf = createStoreFileAndReader(fileInfo);
bulkLoadHFile(sf);
}
private void bulkLoadHFile(HStoreFile sf) throws IOException {
StoreFileReader r = sf.getReader();
this.storeSize.addAndGet(r.length());
this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes());
// Append the new storefile into the list
this.lock.writeLock().lock();
try {
this.storeEngine.getStoreFileManager().insertNewFiles(Lists.newArrayList(sf));
} finally {
// We need the lock, as long as we are updating the storeFiles
// or changing the memstore. Let us release it before calling
// notifyChangeReadersObservers. See HBASE-4485 for a possible
// deadlock scenario that could have happened if continue to hold
// the lock.
this.lock.writeLock().unlock();
}
LOG.info("Loaded HFile " + sf.getFileInfo() + " into " + this);
if (LOG.isTraceEnabled()) {
String traceMessage = "BULK LOAD time,size,store size,store files ["
+ EnvironmentEdgeManager.currentTime() + "," + r.length() + "," + storeSize
+ "," + storeEngine.getStoreFileManager().getStorefileCount() + "]";
LOG.trace(traceMessage);
}
}
/**
* Close all the readers We don't need to worry about subsequent requests because the Region holds
* a write lock that will prevent any more reads or writes.
* @return the {@link StoreFile StoreFiles} that were previously being used.
* @throws IOException on failure
*/
public ImmutableCollection<HStoreFile> close() throws IOException {
this.archiveLock.lock();
this.lock.writeLock().lock();
try {
// Clear so metrics doesn't find them.
ImmutableCollection<HStoreFile> result = storeEngine.getStoreFileManager().clearFiles();
Collection<HStoreFile> compactedfiles =
storeEngine.getStoreFileManager().clearCompactedFiles();
// clear the compacted files
if (CollectionUtils.isNotEmpty(compactedfiles)) {
removeCompactedfiles(compactedfiles, getCacheConfig() != null ?
getCacheConfig().shouldEvictOnClose() : true);
}
if (!result.isEmpty()) {
// initialize the thread pool for closing store files in parallel.
ThreadPoolExecutor storeFileCloserThreadPool = this.region
.getStoreFileOpenAndCloseThreadPool("StoreFileCloser-"
+ this.region.getRegionInfo().getEncodedName() + "-" + this.getColumnFamilyName());
// close each store file in parallel
CompletionService<Void> completionService =
new ExecutorCompletionService<>(storeFileCloserThreadPool);
for (HStoreFile f : result) {
completionService.submit(new Callable<Void>() {
@Override
public Void call() throws IOException {
boolean evictOnClose =
getCacheConfig() != null? getCacheConfig().shouldEvictOnClose(): true;
f.closeStoreFile(evictOnClose);
return null;
}
});
}
IOException ioe = null;
try {
for (int i = 0; i < result.size(); i++) {
try {
Future<Void> future = completionService.take();
future.get();
} catch (InterruptedException e) {
if (ioe == null) {
ioe = new InterruptedIOException();
ioe.initCause(e);
}
} catch (ExecutionException e) {
if (ioe == null) {
ioe = new IOException(e.getCause());
}
}
}
} finally {
storeFileCloserThreadPool.shutdownNow();
}
if (ioe != null) {
throw ioe;
}
}
LOG.trace("Closed {}", this);
return result;
} finally {
this.lock.writeLock().unlock();
this.archiveLock.unlock();
}
}
/**
* Write out current snapshot. Presumes {@code StoreFlusherImpl.prepare()} has been called
* previously.
* @param logCacheFlushId flush sequence number
* @return The path name of the tmp file to which the store was flushed
* @throws IOException if exception occurs during process
*/
protected List<Path> flushCache(final long logCacheFlushId, MemStoreSnapshot snapshot,
MonitoredTask status, ThroughputController throughputController,
FlushLifeCycleTracker tracker) throws IOException {
// If an exception happens flushing, we let it out without clearing
// the memstore snapshot. The old snapshot will be returned when we say
// 'snapshot', the next time flush comes around.
// Retry after catching exception when flushing, otherwise server will abort
// itself
StoreFlusher flusher = storeEngine.getStoreFlusher();
IOException lastException = null;
for (int i = 0; i < flushRetriesNumber; i++) {
try {
List<Path> pathNames =
flusher.flushSnapshot(snapshot, logCacheFlushId, status, throughputController, tracker);