1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.regionserver.compactions;
21
22 import java.io.IOException;
23 import java.util.ArrayList;
24 import java.util.Collection;
25 import java.util.List;
26 import java.util.Random;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.hbase.HConstants;
33 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
34 import org.apache.hadoop.hbase.regionserver.StoreFile;
35 import org.apache.hadoop.hbase.regionserver.StoreUtils;
36 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
37
38 import com.google.common.base.Preconditions;
39 import com.google.common.base.Predicate;
40 import com.google.common.collect.Collections2;
41
42
43
44
45
46
47 @InterfaceAudience.Private
48 public class RatioBasedCompactionPolicy extends CompactionPolicy {
49 private static final Log LOG = LogFactory.getLog(RatioBasedCompactionPolicy.class);
50
51 public RatioBasedCompactionPolicy(Configuration conf,
52 StoreConfigInformation storeConfigInfo) {
53 super(conf, storeConfigInfo);
54 }
55
56 private ArrayList<StoreFile> getCurrentEligibleFiles(
57 ArrayList<StoreFile> candidateFiles, final List<StoreFile> filesCompacting) {
58
59 if (!filesCompacting.isEmpty()) {
60
61
62 StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
63 int idx = candidateFiles.indexOf(last);
64 Preconditions.checkArgument(idx != -1);
65 candidateFiles.subList(0, idx + 1).clear();
66 }
67 return candidateFiles;
68 }
69
70 public List<StoreFile> preSelectCompactionForCoprocessor(
71 final Collection<StoreFile> candidates, final List<StoreFile> filesCompacting) {
72 return getCurrentEligibleFiles(new ArrayList<StoreFile>(candidates), filesCompacting);
73 }
74
75
76
77
78
79
80 public CompactionRequest selectCompaction(Collection<StoreFile> candidateFiles,
81 final List<StoreFile> filesCompacting, final boolean isUserCompaction,
82 final boolean mayUseOffPeak, final boolean forceMajor) throws IOException {
83
84 ArrayList<StoreFile> candidateSelection = new ArrayList<StoreFile>(candidateFiles);
85
86
87
88 int futureFiles = filesCompacting.isEmpty() ? 0 : 1;
89 boolean mayBeStuck = (candidateFiles.size() - filesCompacting.size() + futureFiles)
90 >= storeConfigInfo.getBlockingFileCount();
91 candidateSelection = getCurrentEligibleFiles(candidateSelection, filesCompacting);
92 LOG.debug("Selecting compaction from " + candidateFiles.size() + " store files, " +
93 filesCompacting.size() + " compacting, " + candidateSelection.size() +
94 " eligible, " + storeConfigInfo.getBlockingFileCount() + " blocking");
95
96 if (!forceMajor) {
97 candidateSelection = skipLargeFiles(candidateSelection);
98 }
99
100
101
102
103
104 boolean majorCompaction = (
105 (forceMajor && isUserCompaction)
106 || ((forceMajor || isMajorCompaction(candidateSelection))
107 && (candidateSelection.size() < comConf.getMaxFilesToCompact()))
108 || StoreUtils.hasReferences(candidateSelection)
109 );
110
111 if (!majorCompaction) {
112
113 candidateSelection = filterBulk(candidateSelection);
114 candidateSelection = applyCompactionPolicy(candidateSelection, mayUseOffPeak, mayBeStuck);
115 candidateSelection = checkMinFilesCriteria(candidateSelection);
116 }
117 candidateSelection = removeExcessFiles(candidateSelection, isUserCompaction, majorCompaction);
118 CompactionRequest result = new CompactionRequest(candidateSelection);
119 result.setOffPeak(!candidateSelection.isEmpty() && !majorCompaction && mayUseOffPeak);
120 return result;
121 }
122
123
124
125
126
127
128
129 private ArrayList<StoreFile> skipLargeFiles(ArrayList<StoreFile> candidates) {
130 int pos = 0;
131 while (pos < candidates.size() && !candidates.get(pos).isReference()
132 && (candidates.get(pos).getReader().length() > comConf.getMaxCompactSize())) {
133 ++pos;
134 }
135 if (pos > 0) {
136 LOG.debug("Some files are too large. Excluding " + pos
137 + " files from compaction candidates");
138 candidates.subList(0, pos).clear();
139 }
140 return candidates;
141 }
142
143
144
145
146
147
148 private ArrayList<StoreFile> filterBulk(ArrayList<StoreFile> candidates) {
149 candidates.removeAll(Collections2.filter(candidates,
150 new Predicate<StoreFile>() {
151 @Override
152 public boolean apply(StoreFile input) {
153 return input.excludeFromMinorCompaction();
154 }
155 }));
156 return candidates;
157 }
158
159
160
161
162
163
164 private ArrayList<StoreFile> removeExcessFiles(ArrayList<StoreFile> candidates,
165 boolean isUserCompaction, boolean isMajorCompaction) {
166 int excess = candidates.size() - comConf.getMaxFilesToCompact();
167 if (excess > 0) {
168 if (isMajorCompaction && isUserCompaction) {
169 LOG.debug("Warning, compacting more than " + comConf.getMaxFilesToCompact() +
170 " files because of a user-requested major compaction");
171 } else {
172 LOG.debug("Too many admissible files. Excluding " + excess
173 + " files from compaction candidates");
174 candidates.subList(comConf.getMaxFilesToCompact(), candidates.size()).clear();
175 }
176 }
177 return candidates;
178 }
179
180
181
182
183
184 private ArrayList<StoreFile> checkMinFilesCriteria(ArrayList<StoreFile> candidates) {
185 int minFiles = comConf.getMinFilesToCompact();
186 if (candidates.size() < minFiles) {
187 if(LOG.isDebugEnabled()) {
188 LOG.debug("Not compacting files because we only have " + candidates.size() +
189 " files ready for compaction. Need " + minFiles + " to initiate.");
190 }
191 candidates.clear();
192 }
193 return candidates;
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 ArrayList<StoreFile> applyCompactionPolicy(ArrayList<StoreFile> candidates,
227 boolean mayUseOffPeak, boolean mayBeStuck) throws IOException {
228 if (candidates.isEmpty()) {
229 return candidates;
230 }
231
232
233 int start = 0;
234 double ratio = comConf.getCompactionRatio();
235 if (mayUseOffPeak) {
236 ratio = comConf.getCompactionRatioOffPeak();
237 LOG.info("Running an off-peak compaction, selection ratio = " + ratio);
238 }
239
240
241 final int countOfFiles = candidates.size();
242 long[] fileSizes = new long[countOfFiles];
243 long[] sumSize = new long[countOfFiles];
244 for (int i = countOfFiles - 1; i >= 0; --i) {
245 StoreFile file = candidates.get(i);
246 fileSizes[i] = file.getReader().length();
247
248 int tooFar = i + comConf.getMaxFilesToCompact() - 1;
249 sumSize[i] = fileSizes[i]
250 + ((i + 1 < countOfFiles) ? sumSize[i + 1] : 0)
251 - ((tooFar < countOfFiles) ? fileSizes[tooFar] : 0);
252 }
253
254
255 while (countOfFiles - start >= comConf.getMinFilesToCompact() &&
256 fileSizes[start] > Math.max(comConf.getMinCompactSize(),
257 (long) (sumSize[start + 1] * ratio))) {
258 ++start;
259 }
260 if (start < countOfFiles) {
261 LOG.info("Default compaction algorithm has selected " + (countOfFiles - start)
262 + " files from " + countOfFiles + " candidates");
263 } else if (mayBeStuck) {
264
265 int filesToLeave = candidates.size() - comConf.getMinFilesToCompact();
266 if (filesToLeave >= 0) {
267 start = filesToLeave;
268 }
269 }
270 candidates.subList(0, start).clear();
271 return candidates;
272 }
273
274
275
276
277
278 public boolean isMajorCompaction(final Collection<StoreFile> filesToCompact)
279 throws IOException {
280 boolean result = false;
281 long mcTime = getNextMajorCompactTime(filesToCompact);
282 if (filesToCompact == null || filesToCompact.isEmpty() || mcTime == 0) {
283 return result;
284 }
285
286 long lowTimestamp = StoreUtils.getLowestTimestamp(filesToCompact);
287 long now = System.currentTimeMillis();
288 if (lowTimestamp > 0l && lowTimestamp < (now - mcTime)) {
289
290 long cfTtl = this.storeConfigInfo.getStoreFileTtl();
291 if (filesToCompact.size() == 1) {
292
293 StoreFile sf = filesToCompact.iterator().next();
294 Long minTimestamp = sf.getMinimumTimestamp();
295 long oldest = (minTimestamp == null)
296 ? Long.MIN_VALUE
297 : now - minTimestamp.longValue();
298 if (sf.isMajorCompaction() &&
299 (cfTtl == HConstants.FOREVER || oldest < cfTtl)) {
300 if (LOG.isDebugEnabled()) {
301 LOG.debug("Skipping major compaction of " + this +
302 " because one (major) compacted file only and oldestTime " +
303 oldest + "ms is < ttl=" + cfTtl);
304 }
305 } else if (cfTtl != HConstants.FOREVER && oldest > cfTtl) {
306 LOG.debug("Major compaction triggered on store " + this +
307 ", because keyvalues outdated; time since last major compaction " +
308 (now - lowTimestamp) + "ms");
309 result = true;
310 }
311 } else {
312 if (LOG.isDebugEnabled()) {
313 LOG.debug("Major compaction triggered on store " + this +
314 "; time since last major compaction " + (now - lowTimestamp) + "ms");
315 }
316 result = true;
317 }
318 }
319 return result;
320 }
321
322 public long getNextMajorCompactTime(final Collection<StoreFile> filesToCompact) {
323
324 long ret = comConf.getMajorCompactionPeriod();
325 if (ret > 0) {
326
327 double jitterPct = comConf.getMajorCompactionJitter();
328 if (jitterPct > 0) {
329 long jitter = Math.round(ret * jitterPct);
330
331 Integer seed = StoreUtils.getDeterministicRandomSeed(filesToCompact);
332 if (seed != null) {
333 double rnd = (new Random(seed)).nextDouble();
334 ret += jitter - Math.round(2L * jitter * rnd);
335 } else {
336 ret = 0;
337 }
338 }
339 }
340 return ret;
341 }
342
343
344
345
346
347 public boolean throttleCompaction(long compactionSize) {
348 return compactionSize > comConf.getThrottlePoint();
349 }
350
351 public boolean needsCompaction(final Collection<StoreFile> storeFiles,
352 final List<StoreFile> filesCompacting) {
353 int numCandidates = storeFiles.size() - filesCompacting.size();
354 return numCandidates >= comConf.getMinFilesToCompact();
355 }
356 }