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.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 long cfTtl = this.storeConfigInfo.getStoreFileTtl();
97 if (!forceMajor) {
98
99 if (comConf.shouldDeleteExpired() && (cfTtl != Long.MAX_VALUE)) {
100 ArrayList<StoreFile> expiredSelection = selectExpiredStoreFiles(
101 candidateSelection, EnvironmentEdgeManager.currentTimeMillis() - cfTtl);
102 if (expiredSelection != null) {
103 return new CompactionRequest(expiredSelection);
104 }
105 }
106 candidateSelection = skipLargeFiles(candidateSelection);
107 }
108
109
110
111
112
113 boolean majorCompaction = (
114 (forceMajor && isUserCompaction)
115 || ((forceMajor || isMajorCompaction(candidateSelection))
116 && (candidateSelection.size() < comConf.getMaxFilesToCompact()))
117 || StoreUtils.hasReferences(candidateSelection)
118 );
119
120 if (!majorCompaction) {
121
122 candidateSelection = filterBulk(candidateSelection);
123 candidateSelection = applyCompactionPolicy(candidateSelection, mayUseOffPeak, mayBeStuck);
124 candidateSelection = checkMinFilesCriteria(candidateSelection);
125 }
126 candidateSelection = removeExcessFiles(candidateSelection, isUserCompaction, majorCompaction);
127 CompactionRequest result = new CompactionRequest(candidateSelection);
128 result.setOffPeak(!candidateSelection.isEmpty() && !majorCompaction && mayUseOffPeak);
129 return result;
130 }
131
132
133
134
135
136
137
138
139
140
141
142 private ArrayList<StoreFile> selectExpiredStoreFiles(
143 ArrayList<StoreFile> candidates, long maxExpiredTimeStamp) {
144 if (candidates == null || candidates.size() == 0) return null;
145 ArrayList<StoreFile> expiredStoreFiles = null;
146
147 for (StoreFile storeFile : candidates) {
148 if (storeFile.getReader().getMaxTimestamp() < maxExpiredTimeStamp) {
149 LOG.info("Deleting the expired store file by compaction: "
150 + storeFile.getPath() + " whose maxTimeStamp is "
151 + storeFile.getReader().getMaxTimestamp()
152 + " while the max expired timestamp is " + maxExpiredTimeStamp);
153 if (expiredStoreFiles == null) {
154 expiredStoreFiles = new ArrayList<StoreFile>();
155 }
156 expiredStoreFiles.add(storeFile);
157 }
158 }
159
160 return expiredStoreFiles;
161 }
162
163
164
165
166
167
168
169 private ArrayList<StoreFile> skipLargeFiles(ArrayList<StoreFile> candidates) {
170 int pos = 0;
171 while (pos < candidates.size() && !candidates.get(pos).isReference()
172 && (candidates.get(pos).getReader().length() > comConf.getMaxCompactSize())) {
173 ++pos;
174 }
175 if (pos > 0) {
176 LOG.debug("Some files are too large. Excluding " + pos
177 + " files from compaction candidates");
178 candidates.subList(0, pos).clear();
179 }
180 return candidates;
181 }
182
183
184
185
186
187
188 private ArrayList<StoreFile> filterBulk(ArrayList<StoreFile> candidates) {
189 candidates.removeAll(Collections2.filter(candidates,
190 new Predicate<StoreFile>() {
191 @Override
192 public boolean apply(StoreFile input) {
193 return input.excludeFromMinorCompaction();
194 }
195 }));
196 return candidates;
197 }
198
199
200
201
202
203
204 private ArrayList<StoreFile> removeExcessFiles(ArrayList<StoreFile> candidates,
205 boolean isUserCompaction, boolean isMajorCompaction) {
206 int excess = candidates.size() - comConf.getMaxFilesToCompact();
207 if (excess > 0) {
208 if (isMajorCompaction && isUserCompaction) {
209 LOG.debug("Warning, compacting more than " + comConf.getMaxFilesToCompact() +
210 " files because of a user-requested major compaction");
211 } else {
212 LOG.debug("Too many admissible files. Excluding " + excess
213 + " files from compaction candidates");
214 candidates.subList(comConf.getMaxFilesToCompact(), candidates.size()).clear();
215 }
216 }
217 return candidates;
218 }
219
220
221
222
223
224 private ArrayList<StoreFile> checkMinFilesCriteria(ArrayList<StoreFile> candidates) {
225 int minFiles = comConf.getMinFilesToCompact();
226 if (candidates.size() < minFiles) {
227 if(LOG.isDebugEnabled()) {
228 LOG.debug("Not compacting files because we only have " + candidates.size() +
229 " files ready for compaction. Need " + minFiles + " to initiate.");
230 }
231 candidates.clear();
232 }
233 return candidates;
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 ArrayList<StoreFile> applyCompactionPolicy(ArrayList<StoreFile> candidates,
267 boolean mayUseOffPeak, boolean mayBeStuck) throws IOException {
268 if (candidates.isEmpty()) {
269 return candidates;
270 }
271
272
273 int start = 0;
274 double ratio = comConf.getCompactionRatio();
275 if (mayUseOffPeak) {
276 ratio = comConf.getCompactionRatioOffPeak();
277 LOG.info("Running an off-peak compaction, selection ratio = " + ratio);
278 }
279
280
281 final int countOfFiles = candidates.size();
282 long[] fileSizes = new long[countOfFiles];
283 long[] sumSize = new long[countOfFiles];
284 for (int i = countOfFiles - 1; i >= 0; --i) {
285 StoreFile file = candidates.get(i);
286 fileSizes[i] = file.getReader().length();
287
288 int tooFar = i + comConf.getMaxFilesToCompact() - 1;
289 sumSize[i] = fileSizes[i]
290 + ((i + 1 < countOfFiles) ? sumSize[i + 1] : 0)
291 - ((tooFar < countOfFiles) ? fileSizes[tooFar] : 0);
292 }
293
294
295 while (countOfFiles - start >= comConf.getMinFilesToCompact() &&
296 fileSizes[start] > Math.max(comConf.getMinCompactSize(),
297 (long) (sumSize[start + 1] * ratio))) {
298 ++start;
299 }
300 if (start < countOfFiles) {
301 LOG.info("Default compaction algorithm has selected " + (countOfFiles - start)
302 + " files from " + countOfFiles + " candidates");
303 } else if (mayBeStuck) {
304
305 int filesToLeave = candidates.size() - comConf.getMinFilesToCompact();
306 if (filesToLeave >= 0) {
307 start = filesToLeave;
308 }
309 }
310 candidates.subList(0, start).clear();
311 return candidates;
312 }
313
314
315
316
317
318 public boolean isMajorCompaction(final Collection<StoreFile> filesToCompact)
319 throws IOException {
320 boolean result = false;
321 long mcTime = getNextMajorCompactTime(filesToCompact);
322 if (filesToCompact == null || filesToCompact.isEmpty() || mcTime == 0) {
323 return result;
324 }
325
326 long lowTimestamp = StoreUtils.getLowestTimestamp(filesToCompact);
327 long now = System.currentTimeMillis();
328 if (lowTimestamp > 0l && lowTimestamp < (now - mcTime)) {
329
330 long cfTtl = this.storeConfigInfo.getStoreFileTtl();
331 if (filesToCompact.size() == 1) {
332
333 StoreFile sf = filesToCompact.iterator().next();
334 Long minTimestamp = sf.getMinimumTimestamp();
335 long oldest = (minTimestamp == null)
336 ? Long.MIN_VALUE
337 : now - minTimestamp.longValue();
338 if (sf.isMajorCompaction() &&
339 (cfTtl == HConstants.FOREVER || oldest < cfTtl)) {
340 if (LOG.isDebugEnabled()) {
341 LOG.debug("Skipping major compaction of " + this +
342 " because one (major) compacted file only and oldestTime " +
343 oldest + "ms is < ttl=" + cfTtl);
344 }
345 } else if (cfTtl != HConstants.FOREVER && oldest > cfTtl) {
346 LOG.debug("Major compaction triggered on store " + this +
347 ", because keyvalues outdated; time since last major compaction " +
348 (now - lowTimestamp) + "ms");
349 result = true;
350 }
351 } else {
352 if (LOG.isDebugEnabled()) {
353 LOG.debug("Major compaction triggered on store " + this +
354 "; time since last major compaction " + (now - lowTimestamp) + "ms");
355 }
356 result = true;
357 }
358 }
359 return result;
360 }
361
362 public long getNextMajorCompactTime(final Collection<StoreFile> filesToCompact) {
363
364 long ret = comConf.getMajorCompactionPeriod();
365 if (ret > 0) {
366
367 double jitterPct = comConf.getMajorCompactionJitter();
368 if (jitterPct > 0) {
369 long jitter = Math.round(ret * jitterPct);
370
371 Integer seed = StoreUtils.getDeterministicRandomSeed(filesToCompact);
372 if (seed != null) {
373 double rnd = (new Random(seed)).nextDouble();
374 ret += jitter - Math.round(2L * jitter * rnd);
375 } else {
376 ret = 0;
377 }
378 }
379 }
380 return ret;
381 }
382
383
384
385
386
387 public boolean throttleCompaction(long compactionSize) {
388 return compactionSize > comConf.getThrottlePoint();
389 }
390
391 @Override
392 public boolean needsCompaction(final Collection<StoreFile> storeFiles,
393 final List<StoreFile> filesCompacting) {
394 int numCandidates = storeFiles.size() - filesCompacting.size();
395 return numCandidates >= comConf.getMinFilesToCompact();
396 }
397 }