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.ArrayList;
22 import java.util.List;
23
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.hbase.classification.InterfaceAudience;
27 import org.apache.hadoop.conf.Configuration;
28 import org.apache.hadoop.fs.Path;
29 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
30 import org.apache.hadoop.hbase.KeyValue.KVComparator;
31 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
32 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
33 import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy;
34 import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactor;
35 import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
36 import org.apache.hadoop.hbase.security.User;
37
38 import com.google.common.base.Preconditions;
39
40
41
42
43 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
44 public class StripeStoreEngine extends StoreEngine<StripeStoreFlusher,
45 StripeCompactionPolicy, StripeCompactor, StripeStoreFileManager> {
46 private static final Log LOG = LogFactory.getLog(StripeStoreEngine.class);
47 private StripeStoreConfig config;
48
49 @Override
50 public boolean needsCompaction(List<StoreFile> filesCompacting) {
51 return this.compactionPolicy.needsCompactions(this.storeFileManager, filesCompacting);
52 }
53
54 @Override
55 public CompactionContext createCompaction() {
56 return new StripeCompaction();
57 }
58
59 @Override
60 protected void createComponents(
61 Configuration conf, Store store, KVComparator comparator) throws IOException {
62 this.config = new StripeStoreConfig(conf, store);
63 this.compactionPolicy = new StripeCompactionPolicy(conf, store, config);
64 this.storeFileManager = new StripeStoreFileManager(comparator, conf, this.config);
65 this.storeFlusher = new StripeStoreFlusher(
66 conf, store, this.compactionPolicy, this.storeFileManager);
67 this.compactor = new StripeCompactor(conf, store);
68 }
69
70
71
72
73 private class StripeCompaction extends CompactionContext {
74 private StripeCompactionPolicy.StripeCompactionRequest stripeRequest = null;
75
76 @Override
77 public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
78 return compactionPolicy.preSelectFilesForCoprocessor(storeFileManager, filesCompacting);
79 }
80
81 @Override
82 public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
83 boolean mayUseOffPeak, boolean forceMajor) throws IOException {
84 this.stripeRequest = compactionPolicy.selectCompaction(
85 storeFileManager, filesCompacting, mayUseOffPeak);
86 this.request = (this.stripeRequest == null)
87 ? new CompactionRequest(new ArrayList<StoreFile>()) : this.stripeRequest.getRequest();
88 return this.stripeRequest != null;
89 }
90
91 @Override
92 public void forceSelect(CompactionRequest request) {
93 super.forceSelect(request);
94 if (this.stripeRequest != null) {
95 this.stripeRequest.setRequest(this.request);
96 } else {
97 LOG.warn("Stripe store is forced to take an arbitrary file list and compact it.");
98 this.stripeRequest = compactionPolicy.createEmptyRequest(storeFileManager, this.request);
99 }
100 }
101
102 @Override
103 public List<Path> compact(CompactionThroughputController throughputController)
104 throws IOException {
105 Preconditions.checkArgument(this.stripeRequest != null, "Cannot compact without selection");
106 return this.stripeRequest.execute(compactor, throughputController, null);
107 }
108
109 @Override
110 public List<Path> compact(CompactionThroughputController throughputController, User user)
111 throws IOException {
112 Preconditions.checkArgument(this.stripeRequest != null, "Cannot compact without selection");
113 return this.stripeRequest.execute(compactor, throughputController, user);
114 }
115 }
116 }