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.util.concurrent.ConcurrentHashMap;
21 import java.util.concurrent.ConcurrentMap;
22
23 import org.apache.commons.logging.Log;
24 import org.apache.commons.logging.LogFactory;
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.conf.Configured;
27 import org.apache.hadoop.hbase.Chore;
28 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
29 import org.apache.hadoop.hbase.Stoppable;
30 import org.apache.hadoop.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
32 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
48 public class PressureAwareCompactionThroughputController extends Configured implements
49 CompactionThroughputController, Stoppable {
50
51 private final static Log LOG = LogFactory
52 .getLog(PressureAwareCompactionThroughputController.class);
53
54 public static final String HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_HIGHER_BOUND =
55 "hbase.hstore.compaction.throughput.higher.bound";
56
57 private static final long DEFAULT_HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_HIGHER_BOUND =
58 20L * 1024 * 1024;
59
60 public static final String HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_LOWER_BOUND =
61 "hbase.hstore.compaction.throughput.lower.bound";
62
63 private static final long DEFAULT_HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_LOWER_BOUND =
64 10L * 1024 * 1024;
65
66 public static final String HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_OFFPEAK =
67 "hbase.hstore.compaction.throughput.offpeak";
68
69 private static final long DEFAULT_HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_OFFPEAK = Long.MAX_VALUE;
70
71 public static final String HBASE_HSTORE_COMPACTION_THROUGHPUT_TUNE_PERIOD =
72 "hbase.hstore.compaction.throughput.tune.period";
73
74 private static final int DEFAULT_HSTORE_COMPACTION_THROUGHPUT_TUNE_PERIOD = 60 * 1000;
75
76
77
78
79 private static final class ActiveCompaction {
80
81 private final long startTime;
82
83 private long lastControlTime;
84
85 private long lastControlSize;
86
87 private long totalSize;
88
89 private long numberOfSleeps;
90
91 private long totalSleepTime;
92
93
94 private long lastLogTime;
95
96 ActiveCompaction() {
97 long currentTime = EnvironmentEdgeManager.currentTimeMillis();
98 this.startTime = currentTime;
99 this.lastControlTime = currentTime;
100 this.lastLogTime = currentTime;
101 }
102 }
103
104 private long maxThroughputHigherBound;
105
106 private long maxThroughputLowerBound;
107
108 private long maxThroughputOffpeak;
109
110 private OffPeakHours offPeakHours;
111
112 private long controlPerSize;
113
114 private int tuningPeriod;
115
116 volatile double maxThroughput;
117
118 private final ConcurrentMap<String, ActiveCompaction> activeCompactions =
119 new ConcurrentHashMap<String, ActiveCompaction>();
120
121 @Override
122 public void setup(final RegionServerServices server) {
123 Chore tuner = new Chore("CompactionThroughputTuner", tuningPeriod, this) {
124
125 @Override
126 protected void chore() {
127 tune(server.getCompactionPressure());
128 }
129 };
130 tuner.setDaemon(true);
131 tuner.start();
132 }
133
134 private void tune(double compactionPressure) {
135 double maxThroughputToSet;
136 if (compactionPressure > 1.0) {
137
138 maxThroughputToSet = Double.MAX_VALUE;
139 } else if (offPeakHours.isOffPeakHour()) {
140 maxThroughputToSet = maxThroughputOffpeak;
141 } else {
142
143
144 maxThroughputToSet =
145 maxThroughputLowerBound + (maxThroughputHigherBound - maxThroughputLowerBound)
146 * compactionPressure;
147 }
148 if (LOG.isDebugEnabled()) {
149 LOG.debug("compactionPressure is " + compactionPressure + ", tune compaction throughput to "
150 + throughputDesc(maxThroughputToSet));
151 }
152 this.maxThroughput = maxThroughputToSet;
153 }
154
155 @Override
156 public void setConf(Configuration conf) {
157 super.setConf(conf);
158 if (conf == null) {
159 return;
160 }
161 this.maxThroughputHigherBound =
162 conf.getLong(HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_HIGHER_BOUND,
163 DEFAULT_HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_HIGHER_BOUND);
164 this.maxThroughputLowerBound =
165 conf.getLong(HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_LOWER_BOUND,
166 DEFAULT_HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_LOWER_BOUND);
167 this.maxThroughputOffpeak =
168 conf.getLong(HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_OFFPEAK,
169 DEFAULT_HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_OFFPEAK);
170 this.offPeakHours = OffPeakHours.getInstance(conf);
171 this.controlPerSize = this.maxThroughputLowerBound;
172 this.maxThroughput = this.maxThroughputLowerBound;
173 this.tuningPeriod =
174 getConf().getInt(HBASE_HSTORE_COMPACTION_THROUGHPUT_TUNE_PERIOD,
175 DEFAULT_HSTORE_COMPACTION_THROUGHPUT_TUNE_PERIOD);
176 LOG.info("Compaction throughput configurations, higher bound: "
177 + throughputDesc(maxThroughputHigherBound) + ", lower bound "
178 + throughputDesc(maxThroughputLowerBound) + ", off peak: "
179 + throughputDesc(maxThroughputOffpeak) + ", tuning period: " + tuningPeriod + " ms");
180 }
181
182 private String throughputDesc(long deltaSize, long elapsedTime) {
183 return throughputDesc((double) deltaSize / elapsedTime * 1000);
184 }
185
186 private String throughputDesc(double speed) {
187 if (speed >= 1E15) {
188 return "unlimited";
189 } else {
190 return String.format("%.2f MB/sec", speed / 1024 / 1024);
191 }
192 }
193
194 @Override
195 public void start(String compactionName) {
196 activeCompactions.put(compactionName, new ActiveCompaction());
197 }
198
199 @Override
200 public long control(String compactionName, long size) throws InterruptedException {
201 ActiveCompaction compaction = activeCompactions.get(compactionName);
202 compaction.totalSize += size;
203 long deltaSize = compaction.totalSize - compaction.lastControlSize;
204 if (deltaSize < controlPerSize) {
205 return 0;
206 }
207 long now = EnvironmentEdgeManager.currentTimeMillis();
208 double maxThroughputPerCompaction = this.maxThroughput / activeCompactions.size();
209 long minTimeAllowed = (long) (deltaSize / maxThroughputPerCompaction * 1000);
210 long elapsedTime = now - compaction.lastControlTime;
211 compaction.lastControlSize = compaction.totalSize;
212 if (elapsedTime >= minTimeAllowed) {
213 compaction.lastControlTime = EnvironmentEdgeManager.currentTimeMillis();
214 return 0;
215 }
216
217 long sleepTime = minTimeAllowed - elapsedTime;
218 if (LOG.isDebugEnabled()) {
219
220 if (now - compaction.lastLogTime > 60L * 1000) {
221 LOG.debug(compactionName + " sleep " + sleepTime + " ms because current throughput is "
222 + throughputDesc(deltaSize, elapsedTime) + ", max allowed is "
223 + throughputDesc(maxThroughputPerCompaction) + ", already slept "
224 + compaction.numberOfSleeps + " time(s) and total slept time is "
225 + compaction.totalSleepTime + " ms till now.");
226 compaction.lastLogTime = now;
227 }
228 }
229 Thread.sleep(sleepTime);
230 compaction.numberOfSleeps++;
231 compaction.totalSleepTime += sleepTime;
232 compaction.lastControlTime = EnvironmentEdgeManager.currentTimeMillis();
233 return sleepTime;
234 }
235
236 @Override
237 public void finish(String compactionName) {
238 ActiveCompaction compaction = activeCompactions.remove(compactionName);
239 long elapsedTime =
240 Math.max(1, EnvironmentEdgeManager.currentTimeMillis() - compaction.startTime);
241 LOG.info(compactionName + " average throughput is "
242 + throughputDesc(compaction.totalSize, elapsedTime) + ", slept "
243 + compaction.numberOfSleeps + " time(s) and total slept time is "
244 + compaction.totalSleepTime + " ms. " + activeCompactions.size()
245 + " active compactions remaining, total limit is " + throughputDesc(maxThroughput));
246 }
247
248 private volatile boolean stopped = false;
249
250 @Override
251 public void stop(String why) {
252 stopped = true;
253 }
254
255 @Override
256 public boolean isStopped() {
257 return stopped;
258 }
259
260 @Override
261 public String toString() {
262 return "DefaultCompactionThroughputController [maxThroughput=" + throughputDesc(maxThroughput)
263 + ", activeCompactions=" + activeCompactions.size() + "]";
264 }
265 }