1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.regionserver;
19
20 import java.io.IOException;
21 import java.util.List;
22
23 import org.apache.hadoop.conf.Configuration;
24 import org.apache.hadoop.fs.Path;
25 import org.apache.hadoop.hbase.KeyValue.KVComparator;
26 import org.apache.hadoop.hbase.classification.InterfaceAudience;
27 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
28 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
29 import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
30 import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionPolicy;
31 import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequest;
32 import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactor;
33 import org.apache.hadoop.hbase.security.User;
34
35
36
37
38
39
40
41
42 @InterfaceAudience.Private
43 public class DateTieredStoreEngine extends
44 StoreEngine<DefaultStoreFlusher, DateTieredCompactionPolicy, DateTieredCompactor, DefaultStoreFileManager> {
45
46 @Override
47 public boolean needsCompaction(List<StoreFile> filesCompacting) {
48 return compactionPolicy.needsCompaction(storeFileManager.getStorefiles(),
49 filesCompacting);
50 }
51
52 @Override
53 public CompactionContext createCompaction() throws IOException {
54 return new DateTieredCompactionContext();
55 }
56
57 @Override
58 protected void createComponents(Configuration conf, Store store, KVComparator kvComparator)
59 throws IOException {
60 this.compactionPolicy = new DateTieredCompactionPolicy(conf, store);
61 this.storeFileManager =
62 new DefaultStoreFileManager(kvComparator, StoreFile.Comparators.SEQ_ID_MAX_TIMESTAMP, conf,
63 compactionPolicy.getConf());
64 this.storeFlusher = new DefaultStoreFlusher(conf, store);
65 this.compactor = new DateTieredCompactor(conf, store);
66 }
67
68 private final class DateTieredCompactionContext extends CompactionContext {
69
70 @Override
71 public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
72 return compactionPolicy.preSelectCompactionForCoprocessor(storeFileManager.getStorefiles(),
73 filesCompacting);
74 }
75
76 @Override
77 public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
78 boolean mayUseOffPeak, boolean forceMajor) throws IOException {
79 request = compactionPolicy.selectCompaction(storeFileManager.getStorefiles(), filesCompacting,
80 isUserCompaction, mayUseOffPeak, forceMajor);
81 return request != null;
82 }
83
84 @Override
85 public void forceSelect(CompactionRequest request) {
86 if (!(request instanceof DateTieredCompactionRequest)) {
87 throw new IllegalArgumentException("DateTieredCompactionRequest is expected. Actual: "
88 + request.getClass().getCanonicalName());
89 }
90 super.forceSelect(request);
91 }
92
93 @Override
94 public List<Path> compact(CompactionThroughputController throughputController) throws IOException {
95 return compact(throughputController, null);
96 }
97
98 public List<Path> compact(CompactionThroughputController throughputController, User user)
99 throws IOException {
100 if (request instanceof DateTieredCompactionRequest) {
101 return compactor.compact(request, ((DateTieredCompactionRequest) request).getBoundaries(),
102 throughputController, user);
103 } else {
104 throw new IllegalArgumentException("DateTieredCompactionRequest is expected. Actual: "
105 + request.getClass().getCanonicalName());
106 }
107 }
108 }
109 }