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.Calendar;
25 import java.util.Collection;
26 import java.util.GregorianCalendar;
27 import java.util.List;
28 import java.util.Random;
29
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.classification.InterfaceAudience;
33 import org.apache.hadoop.conf.Configuration;
34 import org.apache.hadoop.hbase.HConstants;
35 import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
36 import org.apache.hadoop.hbase.regionserver.HStore;
37 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
38 import org.apache.hadoop.hbase.regionserver.StoreFile;
39 import org.apache.hadoop.hbase.regionserver.StoreFileManager;
40 import org.apache.hadoop.hbase.regionserver.StoreUtils;
41 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
42
43 import com.google.common.base.Preconditions;
44 import com.google.common.base.Predicate;
45 import com.google.common.collect.Collections2;
46
47
48
49
50
51
52 @InterfaceAudience.Private
53 public class DefaultCompactionPolicy extends CompactionPolicy {
54 private static final Log LOG = LogFactory.getLog(DefaultCompactionPolicy.class);
55
56 public DefaultCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo) {
57 super(conf, storeConfigInfo);
58 }
59
60 private ArrayList<StoreFile> getCurrentEligibleFiles(
61 ArrayList<StoreFile> candidateFiles, final List<StoreFile> filesCompacting) {
62
63 if (!filesCompacting.isEmpty()) {
64
65
66 StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
67 int idx = candidateFiles.indexOf(last);
68 Preconditions.checkArgument(idx != -1);
69 candidateFiles.subList(0, idx + 1).clear();
70 }
71 return candidateFiles;
72 }
73
74 public List<StoreFile> preSelectCompactionForCoprocessor(
75 final Collection<StoreFile> candidates, final List<StoreFile> filesCompacting) {
76 return getCurrentEligibleFiles(new ArrayList<StoreFile>(candidates), filesCompacting);
77 }
78
79
80
81
82
83
84 public CompactionRequest selectCompaction(Collection<StoreFile> candidateFiles,
85 final List<StoreFile> filesCompacting, final boolean isUserCompaction,
86 final boolean mayUseOffPeak, final boolean forceMajor) throws IOException {
87
88 ArrayList<StoreFile> candidateSelection = new ArrayList<StoreFile>(candidateFiles);
89 candidateSelection = getCurrentEligibleFiles(candidateSelection, filesCompacting);
90 long cfTtl = this.storeConfigInfo.getStoreFileTtl();
91 if (!forceMajor) {
92
93 if (comConf.shouldDeleteExpired() && (cfTtl != Long.MAX_VALUE)) {
94 ArrayList<StoreFile> expiredSelection = selectExpiredStoreFiles(
95 candidateSelection, EnvironmentEdgeManager.currentTimeMillis() - cfTtl);
96 if (expiredSelection != null) {
97 return new CompactionRequest(expiredSelection);
98 }
99 }
100 candidateSelection = skipLargeFiles(candidateSelection);
101 }
102
103
104
105
106
107 boolean majorCompaction = (
108 (forceMajor && isUserCompaction)
109 || ((forceMajor || isMajorCompaction(candidateSelection))
110 && (candidateSelection.size() < comConf.getMaxFilesToCompact()))
111 || StoreUtils.hasReferences(candidateSelection)
112 );
113
114 if (!majorCompaction) {
115
116 candidateSelection = filterBulk(candidateSelection);
117 candidateSelection = applyCompactionPolicy(candidateSelection, mayUseOffPeak);
118 candidateSelection = checkMinFilesCriteria(candidateSelection);
119 }
120 candidateSelection = removeExcessFiles(candidateSelection, isUserCompaction, majorCompaction);
121 CompactionRequest result = new CompactionRequest(candidateSelection);
122 result.setOffPeak(!candidateSelection.isEmpty() && !majorCompaction && mayUseOffPeak);
123 return result;
124 }
125
126
127
128
129
130
131
132
133
134
135
136 private ArrayList<StoreFile> selectExpiredStoreFiles(
137 ArrayList<StoreFile> candidates, long maxExpiredTimeStamp) {
138 if (candidates == null || candidates.size() == 0) return null;
139 ArrayList<StoreFile> expiredStoreFiles = null;
140
141 for (StoreFile storeFile : candidates) {
142 if (storeFile.getReader().getMaxTimestamp() < maxExpiredTimeStamp) {
143 LOG.info("Deleting the expired store file by compaction: "
144 + storeFile.getPath() + " whose maxTimeStamp is "
145 + storeFile.getReader().getMaxTimestamp()
146 + " while the max expired timestamp is " + maxExpiredTimeStamp);
147 if (expiredStoreFiles == null) {
148 expiredStoreFiles = new ArrayList<StoreFile>();
149 }
150 expiredStoreFiles.add(storeFile);
151 }
152 }
153
154 return expiredStoreFiles;
155 }
156
157
158
159
160
161
162
163 private ArrayList<StoreFile> skipLargeFiles(ArrayList<StoreFile> candidates) {
164 int pos = 0;
165 while (pos < candidates.size() && !candidates.get(pos).isReference()
166 && (candidates.get(pos).getReader().length() > comConf.getMaxCompactSize())) {
167 ++pos;
168 }
169 if (pos > 0) {
170 LOG.debug("Some files are too large. Excluding " + pos
171 + " files from compaction candidates");
172 candidates.subList(0, pos).clear();
173 }
174 return candidates;
175 }
176
177
178
179
180
181
182 private ArrayList<StoreFile> filterBulk(ArrayList<StoreFile> candidates) {
183 candidates.removeAll(Collections2.filter(candidates,
184 new Predicate<StoreFile>() {
185 @Override
186 public boolean apply(StoreFile input) {
187 return input.excludeFromMinorCompaction();
188 }
189 }));
190 return candidates;
191 }
192
193
194
195
196
197
198 private ArrayList<StoreFile> removeExcessFiles(ArrayList<StoreFile> candidates,
199 boolean isUserCompaction, boolean isMajorCompaction) {
200 int excess = candidates.size() - comConf.getMaxFilesToCompact();
201 if (excess > 0) {
202 if (isMajorCompaction && isUserCompaction) {
203 LOG.debug("Warning, compacting more than " + comConf.getMaxFilesToCompact() +
204 " files because of a user-requested major compaction");
205 } else {
206 LOG.debug("Too many admissible files. Excluding " + excess
207 + " files from compaction candidates");
208 candidates.subList(comConf.getMaxFilesToCompact(), candidates.size()).clear();
209 }
210 }
211 return candidates;
212 }
213
214
215
216
217
218 private ArrayList<StoreFile> checkMinFilesCriteria(ArrayList<StoreFile> candidates) {
219 int minFiles = comConf.getMinFilesToCompact();
220 if (candidates.size() < minFiles) {
221 if(LOG.isDebugEnabled()) {
222 LOG.debug("Not compacting files because we only have " + candidates.size() +
223 " files ready for compaction. Need " + minFiles + " to initiate.");
224 }
225 candidates.clear();
226 }
227 return candidates;
228 }
229
230
231
232
233
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 ArrayList<StoreFile> applyCompactionPolicy(
261 ArrayList<StoreFile> candidates, boolean mayUseOffPeak) throws IOException {
262 if (candidates.isEmpty()) {
263 return candidates;
264 }
265
266
267 int start = 0;
268 double ratio = comConf.getCompactionRatio();
269 if (mayUseOffPeak) {
270 ratio = comConf.getCompactionRatioOffPeak();
271 LOG.info("Running an off-peak compaction, selection ratio = " + ratio);
272 }
273
274
275 final int countOfFiles = candidates.size();
276 long[] fileSizes = new long[countOfFiles];
277 long[] sumSize = new long[countOfFiles];
278 for (int i = countOfFiles - 1; i >= 0; --i) {
279 StoreFile file = candidates.get(i);
280 fileSizes[i] = file.getReader().length();
281
282 int tooFar = i + comConf.getMaxFilesToCompact() - 1;
283 sumSize[i] = fileSizes[i]
284 + ((i + 1 < countOfFiles) ? sumSize[i + 1] : 0)
285 - ((tooFar < countOfFiles) ? fileSizes[tooFar] : 0);
286 }
287
288
289 while (countOfFiles - start >= comConf.getMinFilesToCompact() &&
290 fileSizes[start] > Math.max(comConf.getMinCompactSize(),
291 (long) (sumSize[start + 1] * ratio))) {
292 ++start;
293 }
294 if (start < countOfFiles) {
295 LOG.info("Default compaction algorithm has selected " + (countOfFiles - start)
296 + " files from " + countOfFiles + " candidates");
297 }
298
299 if (start > 0) {
300 candidates.subList(0, start).clear();
301 }
302 return candidates;
303 }
304
305
306
307
308
309 public boolean isMajorCompaction(final Collection<StoreFile> filesToCompact)
310 throws IOException {
311 boolean result = false;
312 long mcTime = getNextMajorCompactTime(filesToCompact);
313 if (filesToCompact == null || filesToCompact.isEmpty() || mcTime == 0) {
314 return result;
315 }
316
317 long lowTimestamp = StoreUtils.getLowestTimestamp(filesToCompact);
318 long now = System.currentTimeMillis();
319 if (lowTimestamp > 0l && lowTimestamp < (now - mcTime)) {
320
321 long cfTtl = this.storeConfigInfo.getStoreFileTtl();
322 if (filesToCompact.size() == 1) {
323
324 StoreFile sf = filesToCompact.iterator().next();
325 Long minTimestamp = sf.getMinimumTimestamp();
326 long oldest = (minTimestamp == null)
327 ? Long.MIN_VALUE
328 : now - minTimestamp.longValue();
329 if (sf.isMajorCompaction() &&
330 (cfTtl == HConstants.FOREVER || oldest < cfTtl)) {
331 if (LOG.isDebugEnabled()) {
332 LOG.debug("Skipping major compaction of " + this +
333 " because one (major) compacted file only and oldestTime " +
334 oldest + "ms is < ttl=" + cfTtl);
335 }
336 } else if (cfTtl != HConstants.FOREVER && oldest > cfTtl) {
337 LOG.debug("Major compaction triggered on store " + this +
338 ", because keyvalues outdated; time since last major compaction " +
339 (now - lowTimestamp) + "ms");
340 result = true;
341 }
342 } else {
343 if (LOG.isDebugEnabled()) {
344 LOG.debug("Major compaction triggered on store " + this +
345 "; time since last major compaction " + (now - lowTimestamp) + "ms");
346 }
347 result = true;
348 }
349 }
350 return result;
351 }
352
353 public long getNextMajorCompactTime(final Collection<StoreFile> filesToCompact) {
354
355 long ret = comConf.getMajorCompactionPeriod();
356 if (ret > 0) {
357
358 double jitterPct = comConf.getMajorCompactionJitter();
359 if (jitterPct > 0) {
360 long jitter = Math.round(ret * jitterPct);
361
362 Integer seed = StoreUtils.getDeterministicRandomSeed(filesToCompact);
363 if (seed != null) {
364 double rnd = (new Random(seed)).nextDouble();
365 ret += jitter - Math.round(2L * jitter * rnd);
366 } else {
367 ret = 0;
368 }
369 }
370 }
371 return ret;
372 }
373
374
375
376
377
378 public boolean throttleCompaction(long compactionSize) {
379 return compactionSize > comConf.getThrottlePoint();
380 }
381
382 @Override
383 public boolean needsCompaction(final Collection<StoreFile> storeFiles,
384 final List<StoreFile> filesCompacting) {
385 int numCandidates = storeFiles.size() - filesCompacting.size();
386 return numCandidates > comConf.getMinFilesToCompact();
387 }
388 }