1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.regionserver.compactions;
19
20 import java.io.IOException;
21 import java.util.ArrayList;
22 import java.util.Collection;
23 import java.util.List;
24
25 import org.apache.commons.logging.Log;
26 import org.apache.commons.logging.LogFactory;
27 import org.apache.hadoop.conf.Configuration;
28 import org.apache.hadoop.hbase.HConstants;
29 import org.apache.hadoop.hbase.classification.InterfaceAudience;
30 import org.apache.hadoop.hbase.regionserver.HRegionServer;
31 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
32 import org.apache.hadoop.hbase.regionserver.StoreFile;
33 import org.apache.hadoop.hbase.regionserver.StoreUtils;
34
35
36
37
38
39
40 @InterfaceAudience.Private
41 public class RatioBasedCompactionPolicy extends SortedCompactionPolicy {
42 private static final Log LOG = LogFactory.getLog(RatioBasedCompactionPolicy.class);
43
44 public RatioBasedCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo) {
45 super(conf, storeConfigInfo);
46 }
47
48
49
50
51
52 @Override
53 public boolean shouldPerformMajorCompaction(final Collection<StoreFile> filesToCompact)
54 throws IOException {
55 boolean result = false;
56 long mcTime = getNextMajorCompactTime(filesToCompact);
57 if (filesToCompact == null || filesToCompact.isEmpty() || mcTime == 0) {
58 return result;
59 }
60
61 long lowTimestamp = StoreUtils.getLowestTimestamp(filesToCompact);
62 long now = System.currentTimeMillis();
63 if (lowTimestamp > 0l && lowTimestamp < (now - mcTime)) {
64
65 long cfTtl = this.storeConfigInfo.getStoreFileTtl();
66 if (filesToCompact.size() == 1) {
67
68 StoreFile sf = filesToCompact.iterator().next();
69 Long minTimestamp = sf.getMinimumTimestamp();
70 long oldest = (minTimestamp == null) ? Long.MIN_VALUE : now - minTimestamp.longValue();
71 if (sf.isMajorCompaction() && (cfTtl == Long.MAX_VALUE || oldest < cfTtl)) {
72 float blockLocalityIndex =
73 sf.getHDFSBlockDistribution().getBlockLocalityIndex(
74 HRegionServer.getHostname(comConf.conf));
75 if (blockLocalityIndex < comConf.getMinLocalityToForceCompact()) {
76 if (LOG.isDebugEnabled()) {
77 LOG.debug("Major compaction triggered on only store " + this
78 + "; to make hdfs blocks local, current blockLocalityIndex is "
79 + blockLocalityIndex + " (min " + comConf.getMinLocalityToForceCompact() + ")");
80 }
81 result = true;
82 } else {
83 if (LOG.isDebugEnabled()) {
84 LOG.debug("Skipping major compaction of " + this
85 + " because one (major) compacted file only, oldestTime " + oldest
86 + "ms is < ttl=" + cfTtl + " and blockLocalityIndex is " + blockLocalityIndex
87 + " (min " + comConf.getMinLocalityToForceCompact() + ")");
88 }
89 }
90 } else if (cfTtl != HConstants.FOREVER && oldest > cfTtl) {
91 LOG.debug("Major compaction triggered on store " + this
92 + ", because keyvalues outdated; time since last major compaction "
93 + (now - lowTimestamp) + "ms");
94 result = true;
95 }
96 } else {
97 if (LOG.isDebugEnabled()) {
98 LOG.debug("Major compaction triggered on store " + this
99 + "; time since last major compaction " + (now - lowTimestamp) + "ms");
100 }
101 result = true;
102 }
103 }
104 return result;
105 }
106
107 @Override
108 protected CompactionRequest getCompactionRequest(ArrayList<StoreFile> candidateSelection,
109 boolean tryingMajor, boolean isUserCompaction, boolean mayUseOffPeak, boolean mayBeStuck)
110 throws IOException {
111 if (!tryingMajor) {
112 candidateSelection = filterBulk(candidateSelection);
113 candidateSelection = applyCompactionPolicy(candidateSelection, mayUseOffPeak, mayBeStuck);
114 candidateSelection =
115 checkMinFilesCriteria(candidateSelection, comConf.getMinFilesToCompact());
116 }
117 removeExcessFiles(candidateSelection, isUserCompaction, tryingMajor);
118 CompactionRequest result = new CompactionRequest(candidateSelection);
119 result.setOffPeak(!candidateSelection.isEmpty() && !tryingMajor && mayUseOffPeak);
120 return result;
121 }
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139 protected ArrayList<StoreFile> applyCompactionPolicy(ArrayList<StoreFile> candidates,
140 boolean mayUseOffPeak, boolean mayBeStuck) throws IOException {
141 if (candidates.isEmpty()) {
142 return candidates;
143 }
144
145
146 int start = 0;
147 double ratio = comConf.getCompactionRatio();
148 if (mayUseOffPeak) {
149 ratio = comConf.getCompactionRatioOffPeak();
150 LOG.info("Running an off-peak compaction, selection ratio = " + ratio);
151 }
152
153
154 final int countOfFiles = candidates.size();
155 long[] fileSizes = new long[countOfFiles];
156 long[] sumSize = new long[countOfFiles];
157 for (int i = countOfFiles - 1; i >= 0; --i) {
158 StoreFile file = candidates.get(i);
159 fileSizes[i] = file.getReader().length();
160
161 int tooFar = i + comConf.getMaxFilesToCompact() - 1;
162 sumSize[i] =
163 fileSizes[i] + ((i + 1 < countOfFiles) ? sumSize[i + 1] : 0)
164 - ((tooFar < countOfFiles) ? fileSizes[tooFar] : 0);
165 }
166
167 while (countOfFiles - start >= comConf.getMinFilesToCompact()
168 && fileSizes[start] > Math.max(comConf.getMinCompactSize(),
169 (long) (sumSize[start + 1] * ratio))) {
170 ++start;
171 }
172 if (start < countOfFiles) {
173 LOG.info("Default compaction algorithm has selected " + (countOfFiles - start)
174 + " files from " + countOfFiles + " candidates");
175 } else if (mayBeStuck) {
176
177 int filesToLeave = candidates.size() - comConf.getMinFilesToCompact();
178 if (filesToLeave >= 0) {
179 start = filesToLeave;
180 }
181 }
182 candidates.subList(0, start).clear();
183 return candidates;
184 }
185
186
187
188
189
190 public void setMinThreshold(int minThreshold) {
191 comConf.setMinFilesToCompact(minThreshold);
192 }
193 }