1
2
3
4
5
6
7
8
9
10
11 package org.apache.hadoop.hbase.regionserver.compactions;
12
13 import com.google.common.base.Preconditions;
14 import com.google.common.base.Predicate;
15 import com.google.common.collect.Collections2;
16 import com.google.common.collect.Lists;
17
18 import java.io.IOException;
19 import java.util.ArrayList;
20 import java.util.Collection;
21 import java.util.List;
22 import java.util.Random;
23
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.hbase.classification.InterfaceAudience;
28 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
29 import org.apache.hadoop.hbase.regionserver.StoreFile;
30 import org.apache.hadoop.hbase.regionserver.StoreUtils;
31
32
33
34
35 @InterfaceAudience.Private
36 public abstract class SortedCompactionPolicy extends CompactionPolicy {
37
38 private static final Log LOG = LogFactory.getLog(SortedCompactionPolicy.class);
39
40 public SortedCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo) {
41 super(conf, storeConfigInfo);
42 }
43
44 public List<StoreFile> preSelectCompactionForCoprocessor(final Collection<StoreFile> candidates,
45 final List<StoreFile> filesCompacting) {
46 return getCurrentEligibleFiles(new ArrayList<StoreFile>(candidates), filesCompacting);
47 }
48
49
50
51
52
53
54
55 public CompactionRequest selectCompaction(Collection<StoreFile> candidateFiles,
56 final List<StoreFile> filesCompacting, final boolean isUserCompaction,
57 final boolean mayUseOffPeak, final boolean forceMajor) throws IOException {
58
59 ArrayList<StoreFile> candidateSelection = new ArrayList<StoreFile>(candidateFiles);
60
61
62
63 int futureFiles = filesCompacting.isEmpty() ? 0 : 1;
64 boolean mayBeStuck = (candidateFiles.size() - filesCompacting.size() + futureFiles)
65 >= storeConfigInfo.getBlockingFileCount();
66
67 candidateSelection = getCurrentEligibleFiles(candidateSelection, filesCompacting);
68 LOG.debug("Selecting compaction from " + candidateFiles.size() + " store files, " +
69 filesCompacting.size() + " compacting, " + candidateSelection.size() +
70 " eligible, " + storeConfigInfo.getBlockingFileCount() + " blocking");
71
72
73 boolean isAllFiles = candidateFiles.size() == candidateSelection.size();
74 if (!(forceMajor && isAllFiles)) {
75 candidateSelection = skipLargeFiles(candidateSelection, mayUseOffPeak);
76 isAllFiles = candidateFiles.size() == candidateSelection.size();
77 }
78
79
80
81 boolean isTryingMajor = (forceMajor && isAllFiles && isUserCompaction)
82 || (((forceMajor && isAllFiles) || shouldPerformMajorCompaction(candidateSelection))
83 && (candidateSelection.size() < comConf.getMaxFilesToCompact()));
84
85 boolean isAfterSplit = StoreUtils.hasReferences(candidateSelection);
86
87 CompactionRequest result = createCompactionRequest(candidateSelection,
88 isTryingMajor || isAfterSplit, mayUseOffPeak, mayBeStuck);
89 result.setAfterSplit(isAfterSplit);
90
91 ArrayList<StoreFile> filesToCompact = Lists.newArrayList(result.getFiles());
92 removeExcessFiles(filesToCompact, isUserCompaction, isTryingMajor);
93 result.updateFiles(filesToCompact);
94
95 isAllFiles = (candidateFiles.size() == filesToCompact.size());
96 result.setOffPeak(!filesToCompact.isEmpty() && !isAllFiles && mayUseOffPeak);
97 result.setIsMajor(isTryingMajor && isAllFiles, isAllFiles);
98
99 return result;
100 }
101
102 protected abstract CompactionRequest createCompactionRequest(ArrayList<StoreFile>
103 candidateSelection, boolean tryingMajor, boolean mayUseOffPeak, boolean mayBeStuck)
104 throws IOException;
105
106
107
108
109
110 @Override
111 public abstract boolean shouldPerformMajorCompaction(final Collection<StoreFile> filesToCompact)
112 throws IOException;
113
114
115
116
117 private final Random random = new Random();
118
119
120
121
122
123 public long getNextMajorCompactTime(final Collection<StoreFile> filesToCompact) {
124
125 long ret = comConf.getMajorCompactionPeriod();
126 if (ret > 0) {
127
128 double jitterPct = comConf.getMajorCompactionJitter();
129 if (jitterPct > 0) {
130 long jitter = Math.round(ret * jitterPct);
131
132 Integer seed = StoreUtils.getDeterministicRandomSeed(filesToCompact);
133 if (seed != null) {
134
135 double rnd = -1;
136 synchronized (this) {
137 this.random.setSeed(seed);
138 rnd = this.random.nextDouble();
139 }
140 ret += jitter - Math.round(2L * jitter * rnd);
141 } else {
142 ret = 0;
143 }
144 }
145 }
146 return ret;
147 }
148
149
150
151
152
153 @Override
154 public boolean throttleCompaction(long compactionSize) {
155 return compactionSize > comConf.getThrottlePoint();
156 }
157
158 public abstract boolean needsCompaction(final Collection<StoreFile> storeFiles,
159 final List<StoreFile> filesCompacting);
160
161 protected ArrayList<StoreFile> getCurrentEligibleFiles(ArrayList<StoreFile> candidateFiles,
162 final List<StoreFile> filesCompacting) {
163
164 if (!filesCompacting.isEmpty()) {
165
166
167 StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
168 int idx = candidateFiles.indexOf(last);
169 Preconditions.checkArgument(idx != -1);
170 candidateFiles.subList(0, idx + 1).clear();
171 }
172 return candidateFiles;
173 }
174
175
176
177
178
179
180 protected ArrayList<StoreFile> skipLargeFiles(ArrayList<StoreFile> candidates,
181 boolean mayUseOffpeak) {
182 int pos = 0;
183 while (pos < candidates.size() && !candidates.get(pos).isReference()
184 && (candidates.get(pos).getReader().length() > comConf.getMaxCompactSize(mayUseOffpeak))) {
185 ++pos;
186 }
187 if (pos > 0) {
188 LOG.debug("Some files are too large. Excluding " + pos
189 + " files from compaction candidates");
190 candidates.subList(0, pos).clear();
191 }
192 return candidates;
193 }
194
195
196
197
198
199 protected ArrayList<StoreFile> filterBulk(ArrayList<StoreFile> candidates) {
200 candidates.removeAll(Collections2.filter(candidates, new Predicate<StoreFile>() {
201 @Override
202 public boolean apply(StoreFile input) {
203 return input.excludeFromMinorCompaction();
204 }
205 }));
206 return candidates;
207 }
208
209
210
211
212 protected void removeExcessFiles(ArrayList<StoreFile> candidates,
213 boolean isUserCompaction, boolean isMajorCompaction) {
214 int excess = candidates.size() - comConf.getMaxFilesToCompact();
215 if (excess > 0) {
216 if (isMajorCompaction && isUserCompaction) {
217 LOG.debug("Warning, compacting more than " + comConf.getMaxFilesToCompact()
218 + " files because of a user-requested major compaction");
219 } else {
220 LOG.debug("Too many admissible files. Excluding " + excess
221 + " files from compaction candidates");
222 candidates.subList(comConf.getMaxFilesToCompact(), candidates.size()).clear();
223 }
224 }
225 }
226
227
228
229
230
231 protected ArrayList<StoreFile> checkMinFilesCriteria(ArrayList<StoreFile> candidates,
232 int minFiles) {
233 if (candidates.size() < minFiles) {
234 if (LOG.isDebugEnabled()) {
235 LOG.debug("Not compacting files because we only have " + candidates.size()
236 + " files ready for compaction. Need " + minFiles + " to initiate.");
237 }
238 candidates.clear();
239 }
240 return candidates;
241 }
242 }