1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21 package org.apache.hadoop.hbase;
22
23 import com.google.common.base.Objects;
24 import org.apache.hadoop.hbase.classification.InterfaceAudience;
25 import org.apache.hadoop.hbase.classification.InterfaceStability;
26 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
27 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
28 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor;
29 import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
30 import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
31 import org.apache.hadoop.hbase.util.Bytes;
32 import org.apache.hadoop.hbase.util.Strings;
33
34 import java.util.Arrays;
35 import java.util.List;
36 import java.util.Map;
37 import java.util.TreeMap;
38 import java.util.TreeSet;
39
40
41
42
43 @InterfaceAudience.Public
44 @InterfaceStability.Evolving
45 public class ServerLoad {
46 private int stores = 0;
47 private int storefiles = 0;
48 private int storeUncompressedSizeMB = 0;
49 private int storefileSizeMB = 0;
50 private int memstoreSizeMB = 0;
51 private int storefileIndexSizeMB = 0;
52 private long readRequestsCount = 0;
53 private long writeRequestsCount = 0;
54 private int rootIndexSizeKB = 0;
55 private int totalStaticIndexSizeKB = 0;
56 private int totalStaticBloomSizeKB = 0;
57 private long totalCompactingKVs = 0;
58 private long currentCompactedKVs = 0;
59 private long reportTime = 0;
60
61 public ServerLoad(ClusterStatusProtos.ServerLoad serverLoad) {
62 this.serverLoad = serverLoad;
63 this.reportTime = System.currentTimeMillis();
64 for (ClusterStatusProtos.RegionLoad rl: serverLoad.getRegionLoadsList()) {
65 stores += rl.getStores();
66 storefiles += rl.getStorefiles();
67 storeUncompressedSizeMB += rl.getStoreUncompressedSizeMB();
68 storefileSizeMB += rl.getStorefileSizeMB();
69 memstoreSizeMB += rl.getMemstoreSizeMB();
70 storefileIndexSizeMB += rl.getStorefileIndexSizeMB();
71 readRequestsCount += rl.getReadRequestsCount();
72 writeRequestsCount += rl.getWriteRequestsCount();
73 rootIndexSizeKB += rl.getRootIndexSizeKB();
74 totalStaticIndexSizeKB += rl.getTotalStaticIndexSizeKB();
75 totalStaticBloomSizeKB += rl.getTotalStaticBloomSizeKB();
76 totalCompactingKVs += rl.getTotalCompactingKVs();
77 currentCompactedKVs += rl.getCurrentCompactedKVs();
78 }
79
80 }
81
82
83
84
85 public ClusterStatusProtos.ServerLoad obtainServerLoadPB() {
86 return serverLoad;
87 }
88
89 protected ClusterStatusProtos.ServerLoad serverLoad;
90
91
92 public long getNumberOfRequests() {
93 return serverLoad.getNumberOfRequests();
94 }
95 public boolean hasNumberOfRequests() {
96 return serverLoad.hasNumberOfRequests();
97 }
98
99
100 public long getTotalNumberOfRequests() {
101 return serverLoad.getTotalNumberOfRequests();
102 }
103 public boolean hasTotalNumberOfRequests() {
104 return serverLoad.hasTotalNumberOfRequests();
105 }
106
107
108 public int getUsedHeapMB() {
109 return serverLoad.getUsedHeapMB();
110 }
111 public boolean hasUsedHeapMB() {
112 return serverLoad.hasUsedHeapMB();
113 }
114
115
116 public int getMaxHeapMB() {
117 return serverLoad.getMaxHeapMB();
118 }
119 public boolean hasMaxHeapMB() {
120 return serverLoad.hasMaxHeapMB();
121 }
122
123 public int getStores() {
124 return stores;
125 }
126
127 public int getStorefiles() {
128 return storefiles;
129 }
130
131 public int getStoreUncompressedSizeMB() {
132 return storeUncompressedSizeMB;
133 }
134
135 public int getStorefileSizeInMB() {
136 return storefileSizeMB;
137 }
138
139 public int getMemstoreSizeInMB() {
140 return memstoreSizeMB;
141 }
142
143 public int getStorefileIndexSizeInMB() {
144 return storefileIndexSizeMB;
145 }
146
147 public long getReadRequestsCount() {
148 return readRequestsCount;
149 }
150
151 public long getWriteRequestsCount() {
152 return writeRequestsCount;
153 }
154
155 public int getRootIndexSizeKB() {
156 return rootIndexSizeKB;
157 }
158
159 public int getTotalStaticIndexSizeKB() {
160 return totalStaticIndexSizeKB;
161 }
162
163 public int getTotalStaticBloomSizeKB() {
164 return totalStaticBloomSizeKB;
165 }
166
167 public long getTotalCompactingKVs() {
168 return totalCompactingKVs;
169 }
170
171 public long getCurrentCompactedKVs() {
172 return currentCompactedKVs;
173 }
174
175
176
177
178 public int getNumberOfRegions() {
179 return serverLoad.getRegionLoadsCount();
180 }
181
182 public int getInfoServerPort() {
183 return serverLoad.getInfoServerPort();
184 }
185
186
187
188
189
190 public List<ReplicationLoadSource> getReplicationLoadSourceList() {
191 return ProtobufUtil.toReplicationLoadSourceList(serverLoad.getReplLoadSourceList());
192 }
193
194
195
196
197
198 public ReplicationLoadSink getReplicationLoadSink() {
199 if (serverLoad.hasReplLoadSink()) {
200 return ProtobufUtil.toReplicationLoadSink(serverLoad.getReplLoadSink());
201 } else {
202 return null;
203 }
204 }
205
206
207
208
209
210
211
212
213
214
215 public int getLoad() {
216
217
218
219
220 return getNumberOfRegions();
221 }
222
223
224
225
226 public Map<byte[], RegionLoad> getRegionsLoad() {
227 Map<byte[], RegionLoad> regionLoads =
228 new TreeMap<byte[], RegionLoad>(Bytes.BYTES_COMPARATOR);
229 for (ClusterStatusProtos.RegionLoad rl : serverLoad.getRegionLoadsList()) {
230 RegionLoad regionLoad = new RegionLoad(rl);
231 regionLoads.put(regionLoad.getName(), regionLoad);
232 }
233 return regionLoads;
234 }
235
236
237
238
239
240 public String[] getRegionServerCoprocessors() {
241 List<Coprocessor> list = obtainServerLoadPB().getCoprocessorsList();
242 String [] ret = new String[list.size()];
243 int i = 0;
244 for (Coprocessor elem : list) {
245 ret[i++] = elem.getName();
246 }
247
248 return ret;
249 }
250
251
252
253
254
255
256 public String[] getRsCoprocessors() {
257
258
259 TreeSet<String> coprocessSet = new TreeSet<String>();
260 for (Coprocessor coprocessor : obtainServerLoadPB().getCoprocessorsList()) {
261 coprocessSet.add(coprocessor.getName());
262 }
263 return coprocessSet.toArray(new String[coprocessSet.size()]);
264 }
265
266
267
268
269 public double getRequestsPerSecond() {
270 return getNumberOfRequests();
271 }
272
273
274
275
276 @Override
277 public String toString() {
278 StringBuilder sb =
279 Strings.appendKeyValue(new StringBuilder(), "requestsPerSecond",
280 Double.valueOf(getRequestsPerSecond()));
281 Strings.appendKeyValue(sb, "numberOfOnlineRegions", Integer.valueOf(getNumberOfRegions()));
282 sb = Strings.appendKeyValue(sb, "usedHeapMB", Integer.valueOf(this.getUsedHeapMB()));
283 sb = Strings.appendKeyValue(sb, "maxHeapMB", Integer.valueOf(getMaxHeapMB()));
284 sb = Strings.appendKeyValue(sb, "numberOfStores", Integer.valueOf(this.stores));
285 sb = Strings.appendKeyValue(sb, "numberOfStorefiles", Integer.valueOf(this.storefiles));
286 sb =
287 Strings.appendKeyValue(sb, "storefileUncompressedSizeMB",
288 Integer.valueOf(this.storeUncompressedSizeMB));
289 sb = Strings.appendKeyValue(sb, "storefileSizeMB", Integer.valueOf(this.storefileSizeMB));
290 if (this.storeUncompressedSizeMB != 0) {
291 sb =
292 Strings.appendKeyValue(
293 sb,
294 "compressionRatio",
295 String.format("%.4f", (float) this.storefileSizeMB
296 / (float) this.storeUncompressedSizeMB));
297 }
298 sb = Strings.appendKeyValue(sb, "memstoreSizeMB", Integer.valueOf(this.memstoreSizeMB));
299 sb =
300 Strings.appendKeyValue(sb, "storefileIndexSizeMB",
301 Integer.valueOf(this.storefileIndexSizeMB));
302 sb = Strings.appendKeyValue(sb, "readRequestsCount", Long.valueOf(this.readRequestsCount));
303 sb = Strings.appendKeyValue(sb, "writeRequestsCount", Long.valueOf(this.writeRequestsCount));
304 sb = Strings.appendKeyValue(sb, "rootIndexSizeKB", Integer.valueOf(this.rootIndexSizeKB));
305 sb =
306 Strings.appendKeyValue(sb, "totalStaticIndexSizeKB",
307 Integer.valueOf(this.totalStaticIndexSizeKB));
308 sb =
309 Strings.appendKeyValue(sb, "totalStaticBloomSizeKB",
310 Integer.valueOf(this.totalStaticBloomSizeKB));
311 sb = Strings.appendKeyValue(sb, "totalCompactingKVs", Long.valueOf(this.totalCompactingKVs));
312 sb = Strings.appendKeyValue(sb, "currentCompactedKVs", Long.valueOf(this.currentCompactedKVs));
313 float compactionProgressPct = Float.NaN;
314 if (this.totalCompactingKVs > 0) {
315 compactionProgressPct =
316 Float.valueOf((float) this.currentCompactedKVs / this.totalCompactingKVs);
317 }
318 sb = Strings.appendKeyValue(sb, "compactionProgressPct", compactionProgressPct);
319
320 String[] coprocessorStrings = getRsCoprocessors();
321 if (coprocessorStrings != null) {
322 sb = Strings.appendKeyValue(sb, "coprocessors", Arrays.toString(coprocessorStrings));
323 }
324 return sb.toString();
325 }
326
327 public static final ServerLoad EMPTY_SERVERLOAD =
328 new ServerLoad(ClusterStatusProtos.ServerLoad.newBuilder().build());
329
330 public long getReportTime() {
331 return reportTime;
332 }
333
334 @Override
335 public int hashCode() {
336 return Objects.hashCode(stores, storefiles, storeUncompressedSizeMB,
337 storefileSizeMB, memstoreSizeMB, storefileIndexSizeMB, readRequestsCount,
338 writeRequestsCount, rootIndexSizeKB, totalStaticIndexSizeKB,
339 totalStaticBloomSizeKB, totalCompactingKVs, currentCompactedKVs);
340 }
341
342 @Override
343 public boolean equals(Object other) {
344 if (other == this) {
345 return true;
346 }
347 if (other instanceof ServerLoad) {
348 ServerLoad sl = ((ServerLoad) other);
349 return stores == sl.stores && storefiles == sl.storefiles
350 && storeUncompressedSizeMB == sl.storeUncompressedSizeMB
351 && storefileSizeMB == sl.storefileSizeMB
352 && memstoreSizeMB == sl.memstoreSizeMB
353 && storefileIndexSizeMB == sl.storefileIndexSizeMB
354 && readRequestsCount == sl.readRequestsCount
355 && writeRequestsCount == sl.writeRequestsCount
356 && rootIndexSizeKB == sl.rootIndexSizeKB
357 && totalStaticIndexSizeKB == sl.totalStaticIndexSizeKB
358 && totalStaticBloomSizeKB == sl.totalStaticBloomSizeKB
359 && totalCompactingKVs == sl.totalCompactingKVs
360 && currentCompactedKVs == sl.currentCompactedKVs;
361 }
362 return false;
363 }
364 }