1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.regionserver.throttle;
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.Configured;
26 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
27 import org.apache.hadoop.hbase.Stoppable;
28 import org.apache.hadoop.hbase.classification.InterfaceAudience;
29 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
30 import org.apache.hadoop.hbase.regionserver.compactions.OffPeakHours;
31 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
32
33 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
34 public abstract class PressureAwareThroughputController extends Configured implements
35 ThroughputController, Stoppable {
36 private static final Log LOG = LogFactory.getLog(PressureAwareThroughputController.class);
37
38
39
40
41 private static final class ActiveOperation {
42
43 private final long startTime;
44
45 private long lastControlTime;
46
47 private long lastControlSize;
48
49 private long totalSize;
50
51 private long numberOfSleeps;
52
53 private long totalSleepTime;
54
55
56 private long lastLogTime;
57
58 ActiveOperation() {
59 long currentTime = EnvironmentEdgeManager.currentTime();
60 this.startTime = currentTime;
61 this.lastControlTime = currentTime;
62 this.lastLogTime = currentTime;
63 }
64 }
65
66 protected long maxThroughputUpperBound;
67
68 protected long maxThroughputLowerBound;
69
70 protected OffPeakHours offPeakHours;
71
72 protected long controlPerSize;
73
74 protected int tuningPeriod;
75
76 private volatile double maxThroughput;
77 private volatile double maxThroughputPerOperation;
78
79 protected final ConcurrentMap<String, ActiveOperation> activeOperations =
80 new ConcurrentHashMap<String, ActiveOperation>();
81
82 @Override
83 public abstract void setup(final RegionServerServices server);
84
85 protected String throughputDesc(long deltaSize, long elapsedTime) {
86 return throughputDesc((double) deltaSize / elapsedTime * 1000);
87 }
88
89 protected String throughputDesc(double speed) {
90 if (speed >= 1E15) {
91 return "unlimited";
92 } else {
93 return String.format("%.2f MB/sec", speed / 1024 / 1024);
94 }
95 }
96
97 @Override
98 public void start(String opName) {
99 activeOperations.put(opName, new ActiveOperation());
100 maxThroughputPerOperation = getMaxThroughput() / activeOperations.size();
101 }
102
103 @Override
104 public long control(String opName, long size) throws InterruptedException {
105 ActiveOperation operation = activeOperations.get(opName);
106 operation.totalSize += size;
107 long deltaSize = operation.totalSize - operation.lastControlSize;
108 if (deltaSize < controlPerSize) {
109 return 0;
110 }
111 long now = EnvironmentEdgeManager.currentTime();
112 long minTimeAllowed = (long) (deltaSize / maxThroughputPerOperation * 1000);
113 long elapsedTime = now - operation.lastControlTime;
114 operation.lastControlSize = operation.totalSize;
115 if (elapsedTime >= minTimeAllowed) {
116 operation.lastControlTime = EnvironmentEdgeManager.currentTime();
117 return 0;
118 }
119
120 long sleepTime = minTimeAllowed - elapsedTime;
121 if (LOG.isDebugEnabled()) {
122
123 if (now - operation.lastLogTime > 5L * 1000) {
124 LOG.debug("deltaSize: " + deltaSize + " bytes; elapseTime: " + elapsedTime + " ns");
125 LOG.debug(opName + " sleep " + sleepTime + " ms because current throughput is "
126 + throughputDesc(deltaSize, elapsedTime) + ", max allowed is "
127 + throughputDesc(maxThroughputPerOperation) + ", already slept "
128 + operation.numberOfSleeps + " time(s) and total slept time is "
129 + operation.totalSleepTime + " ms till now.");
130 operation.lastLogTime = now;
131 }
132 }
133 Thread.sleep(sleepTime);
134 operation.numberOfSleeps++;
135 operation.totalSleepTime += sleepTime;
136 operation.lastControlTime = EnvironmentEdgeManager.currentTime();
137 return sleepTime;
138 }
139
140 @Override
141 public void finish(String opName) {
142 ActiveOperation operation = activeOperations.remove(opName);
143 maxThroughputPerOperation = getMaxThroughput() / activeOperations.size();
144 long elapsedTime = EnvironmentEdgeManager.currentTime() - operation.startTime;
145 LOG.info(opName + " average throughput is "
146 + throughputDesc(operation.totalSize, elapsedTime) + ", slept "
147 + operation.numberOfSleeps + " time(s) and total slept time is "
148 + operation.totalSleepTime + " ms. " + activeOperations.size()
149 + " active operations remaining, total limit is " + throughputDesc(getMaxThroughput()));
150 }
151
152 private volatile boolean stopped = false;
153
154 @Override
155 public void stop(String why) {
156 stopped = true;
157 }
158
159 @Override
160 public boolean isStopped() {
161 return stopped;
162 }
163
164 public double getMaxThroughput() {
165 return maxThroughput;
166 }
167
168 public void setMaxThroughput(double maxThroughput) {
169 this.maxThroughput = maxThroughput;
170 maxThroughputPerOperation = getMaxThroughput() / activeOperations.size();
171 }
172 }