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;
21
22 import java.io.DataInput;
23 import java.io.DataOutput;
24 import java.io.IOException;
25 import java.util.ArrayList;
26 import java.util.Arrays;
27 import java.util.Collections;
28 import java.util.List;
29 import java.util.Map;
30 import java.util.Set;
31 import java.util.TreeMap;
32 import java.util.TreeSet;
33
34 import org.apache.hadoop.hbase.util.Bytes;
35 import org.apache.hadoop.hbase.util.Strings;
36 import org.apache.hadoop.io.VersionedWritable;
37 import org.apache.hadoop.io.WritableComparable;
38 import org.apache.hadoop.io.WritableUtils;
39
40
41
42
43 public class HServerLoad extends VersionedWritable
44 implements WritableComparable<HServerLoad> {
45 private static final byte VERSION = 2;
46
47 public static final HServerLoad EMPTY_HSERVERLOAD = new HServerLoad();
48
49
50
51
52 private int numberOfRequests = 0;
53
54
55
56 private int totalNumberOfRequests = 0;
57
58
59 private int usedHeapMB = 0;
60
61
62 private int maxHeapMB = 0;
63
64
65 private Set<String> coprocessors = new TreeSet<String>();
66
67
68
69
70
71 public String[] getRsCoprocessors() {
72 return coprocessors.toArray(new String[0]);
73 }
74
75
76 Map<byte[], RegionLoad> regionLoad =
77 new TreeMap<byte[], RegionLoad>(Bytes.BYTES_COMPARATOR);
78
79
80 public byte getVersion() {
81 return VERSION;
82 }
83
84
85
86
87 public static class RegionLoad extends VersionedWritable {
88 private static final byte VERSION = 2;
89
90
91 public byte getVersion() {
92 return VERSION;
93 }
94
95
96 private byte[] name;
97
98 private int stores;
99
100 private int storefiles;
101
102 private int storeUncompressedSizeMB;
103
104 private int storefileSizeMB;
105
106 private int memstoreSizeMB;
107
108
109
110
111
112 private int storefileIndexSizeMB;
113
114 private long readRequestsCount;
115
116 private long writeRequestsCount;
117
118 private long totalCompactingKVs;
119
120 private long currentCompactedKVs;
121
122
123 private int rootIndexSizeKB;
124
125
126 private int totalStaticIndexSizeKB;
127
128
129
130
131
132 private int totalStaticBloomSizeKB;
133
134
135
136
137
138 private long completeSequenceId;
139
140
141
142
143 public RegionLoad() {
144 super();
145 }
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160 public RegionLoad(final byte[] name, final int stores,
161 final int storefiles, final int storeUncompressedSizeMB,
162 final int storefileSizeMB,
163 final int memstoreSizeMB, final int storefileIndexSizeMB,
164 final int rootIndexSizeKB, final int totalStaticIndexSizeKB,
165 final int totalStaticBloomSizeKB,
166 final long readRequestsCount, final long writeRequestsCount,
167 final long totalCompactingKVs, final long currentCompactedKVs) {
168 this.name = name;
169 this.stores = stores;
170 this.storefiles = storefiles;
171 this.storeUncompressedSizeMB = storeUncompressedSizeMB;
172 this.storefileSizeMB = storefileSizeMB;
173 this.memstoreSizeMB = memstoreSizeMB;
174 this.storefileIndexSizeMB = storefileIndexSizeMB;
175 this.rootIndexSizeKB = rootIndexSizeKB;
176 this.totalStaticIndexSizeKB = totalStaticIndexSizeKB;
177 this.totalStaticBloomSizeKB = totalStaticBloomSizeKB;
178 this.readRequestsCount = readRequestsCount;
179 this.writeRequestsCount = writeRequestsCount;
180 this.totalCompactingKVs = totalCompactingKVs;
181 this.currentCompactedKVs = currentCompactedKVs;
182 }
183
184
185
186
187 public byte[] getName() {
188 return name;
189 }
190
191
192
193
194 public String getNameAsString() {
195 return Bytes.toString(name);
196 }
197
198
199
200
201 public int getStores() {
202 return stores;
203 }
204
205
206
207
208 public int getStorefiles() {
209 return storefiles;
210 }
211
212
213
214
215 public int getStorefileSizeMB() {
216 return storefileSizeMB;
217 }
218
219
220
221
222 public int getMemStoreSizeMB() {
223 return memstoreSizeMB;
224 }
225
226
227
228
229 public int getStorefileIndexSizeMB() {
230 return storefileIndexSizeMB;
231 }
232
233
234
235
236 public long getRequestsCount() {
237 return readRequestsCount + writeRequestsCount;
238 }
239
240
241
242
243 public long getReadRequestsCount() {
244 return readRequestsCount;
245 }
246
247
248
249
250 public long getWriteRequestsCount() {
251 return writeRequestsCount;
252 }
253
254
255
256
257 public int getRootIndexSizeKB() {
258 return rootIndexSizeKB;
259 }
260
261
262
263
264 public int getTotalStaticIndexSizeKB() {
265 return totalStaticIndexSizeKB;
266 }
267
268
269
270
271
272 public int getTotalStaticBloomSizeKB() {
273 return totalStaticBloomSizeKB;
274 }
275
276
277
278
279 public long getTotalCompactingKVs() {
280 return totalCompactingKVs;
281 }
282
283
284
285
286 public long getCurrentCompactedKVs() {
287 return currentCompactedKVs;
288 }
289
290
291
292
293 public long getCompleteSequenceId() {
294 return this.completeSequenceId;
295 }
296
297
298
299
300
301
302 public void setName(byte[] name) {
303 this.name = name;
304 }
305
306
307
308
309 public void setStores(int stores) {
310 this.stores = stores;
311 }
312
313
314
315
316 public void setStorefiles(int storefiles) {
317 this.storefiles = storefiles;
318 }
319
320
321
322
323 public void setMemStoreSizeMB(int memstoreSizeMB) {
324 this.memstoreSizeMB = memstoreSizeMB;
325 }
326
327
328
329
330
331 public void setStorefileIndexSizeMB(int storefileIndexSizeMB) {
332 this.storefileIndexSizeMB = storefileIndexSizeMB;
333 }
334
335
336
337
338 public void setReadRequestsCount(int requestsCount) {
339 this.readRequestsCount = requestsCount;
340 }
341
342
343
344
345 public void setWriteRequestsCount(int requestsCount) {
346 this.writeRequestsCount = requestsCount;
347 }
348
349
350
351
352 public void setTotalCompactingKVs(long totalCompactingKVs) {
353 this.totalCompactingKVs = totalCompactingKVs;
354 }
355
356
357
358
359
360 public void setCurrentCompactedKVs(long currentCompactedKVs) {
361 this.currentCompactedKVs = currentCompactedKVs;
362 }
363
364
365
366
367 public void setCompleteSequenceId(long value) {
368 this.completeSequenceId = value;
369 }
370
371
372
373
374
375
376
377 private void readFields92(DataInput in) throws IOException {
378
379 int version = in.readByte();
380 int namelen = in.readInt();
381 this.name = new byte[namelen];
382 in.readFully(this.name);
383 this.stores = in.readInt();
384 this.storefiles = in.readInt();
385 this.storeUncompressedSizeMB = in.readInt();
386 this.storefileSizeMB = in.readInt();
387 this.memstoreSizeMB = in.readInt();
388 this.storefileIndexSizeMB = in.readInt();
389 this.readRequestsCount = in.readInt();
390 this.writeRequestsCount = in.readInt();
391 this.rootIndexSizeKB = in.readInt();
392 this.totalStaticIndexSizeKB = in.readInt();
393 this.totalStaticBloomSizeKB = in.readInt();
394 this.totalCompactingKVs = in.readLong();
395 this.currentCompactedKVs = in.readLong();
396 int coprocessorsSize = in.readInt();
397
398 for (int i = 0; i < coprocessorsSize; i++) {
399 in.readUTF();
400 }
401 }
402
403
404 public void readFields(DataInput in) throws IOException {
405 int version = in.readByte();
406 if (version > VERSION) throw new IOException("Version mismatch; " + version);
407 if (version == 1) {
408 readFields92(in);
409 return;
410 }
411 int namelen = WritableUtils.readVInt(in);
412 this.name = new byte[namelen];
413 in.readFully(this.name);
414 this.stores = WritableUtils.readVInt(in);
415 this.storefiles = WritableUtils.readVInt(in);
416 this.storeUncompressedSizeMB = WritableUtils.readVInt(in);
417 this.storefileSizeMB = WritableUtils.readVInt(in);
418 this.memstoreSizeMB = WritableUtils.readVInt(in);
419 this.storefileIndexSizeMB = WritableUtils.readVInt(in);
420 this.readRequestsCount = WritableUtils.readVLong(in);
421 this.writeRequestsCount = WritableUtils.readVLong(in);
422 this.rootIndexSizeKB = WritableUtils.readVInt(in);
423 this.totalStaticIndexSizeKB = WritableUtils.readVInt(in);
424 this.totalStaticBloomSizeKB = WritableUtils.readVInt(in);
425 this.totalCompactingKVs = WritableUtils.readVLong(in);
426 this.currentCompactedKVs = WritableUtils.readVLong(in);
427 int coprocessorsSize = WritableUtils.readVInt(in);
428
429 for (int i = 0; i < coprocessorsSize; i++) {
430 in.readUTF();
431 }
432 }
433
434 public void write(DataOutput out) throws IOException {
435 super.write(out);
436 WritableUtils.writeVInt(out, name.length);
437 out.write(name);
438 WritableUtils.writeVInt(out, stores);
439 WritableUtils.writeVInt(out, storefiles);
440 WritableUtils.writeVInt(out, storeUncompressedSizeMB);
441 WritableUtils.writeVInt(out, storefileSizeMB);
442 WritableUtils.writeVInt(out, memstoreSizeMB);
443 WritableUtils.writeVInt(out, storefileIndexSizeMB);
444 WritableUtils.writeVLong(out, readRequestsCount);
445 WritableUtils.writeVLong(out, writeRequestsCount);
446 WritableUtils.writeVInt(out, rootIndexSizeKB);
447 WritableUtils.writeVInt(out, totalStaticIndexSizeKB);
448 WritableUtils.writeVInt(out, totalStaticBloomSizeKB);
449 WritableUtils.writeVLong(out, totalCompactingKVs);
450 WritableUtils.writeVLong(out, currentCompactedKVs);
451
452
453 WritableUtils.writeVInt(out, 0);
454 }
455
456
457
458
459 @Override
460 public String toString() {
461 StringBuilder sb = Strings.appendKeyValue(new StringBuilder(), "numberOfStores",
462 Integer.valueOf(this.stores));
463 sb = Strings.appendKeyValue(sb, "numberOfStorefiles",
464 Integer.valueOf(this.storefiles));
465 sb = Strings.appendKeyValue(sb, "storefileUncompressedSizeMB",
466 Integer.valueOf(this.storeUncompressedSizeMB));
467 sb = Strings.appendKeyValue(sb, "storefileSizeMB",
468 Integer.valueOf(this.storefileSizeMB));
469 if (this.storeUncompressedSizeMB != 0) {
470 sb = Strings.appendKeyValue(sb, "compressionRatio",
471 String.format("%.4f", (float)this.storefileSizeMB/
472 (float)this.storeUncompressedSizeMB));
473 }
474 sb = Strings.appendKeyValue(sb, "memstoreSizeMB",
475 Integer.valueOf(this.memstoreSizeMB));
476 sb = Strings.appendKeyValue(sb, "storefileIndexSizeMB",
477 Integer.valueOf(this.storefileIndexSizeMB));
478 sb = Strings.appendKeyValue(sb, "readRequestsCount",
479 Long.valueOf(this.readRequestsCount));
480 sb = Strings.appendKeyValue(sb, "writeRequestsCount",
481 Long.valueOf(this.writeRequestsCount));
482 sb = Strings.appendKeyValue(sb, "rootIndexSizeKB",
483 Integer.valueOf(this.rootIndexSizeKB));
484 sb = Strings.appendKeyValue(sb, "totalStaticIndexSizeKB",
485 Integer.valueOf(this.totalStaticIndexSizeKB));
486 sb = Strings.appendKeyValue(sb, "totalStaticBloomSizeKB",
487 Integer.valueOf(this.totalStaticBloomSizeKB));
488 sb = Strings.appendKeyValue(sb, "totalCompactingKVs",
489 Long.valueOf(this.totalCompactingKVs));
490 sb = Strings.appendKeyValue(sb, "currentCompactedKVs",
491 Long.valueOf(this.currentCompactedKVs));
492 float compactionProgressPct = Float.NaN;
493 if( this.totalCompactingKVs > 0 ) {
494 compactionProgressPct = Float.valueOf(
495 this.currentCompactedKVs / this.totalCompactingKVs);
496 }
497 sb = Strings.appendKeyValue(sb, "compactionProgressPct",
498 compactionProgressPct);
499 return sb.toString();
500 }
501 }
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517 public HServerLoad() {
518 super();
519 }
520
521
522
523
524
525
526
527
528 public HServerLoad(final int totalNumberOfRequests,
529 final int numberOfRequests, final int usedHeapMB, final int maxHeapMB,
530 final Map<byte[], RegionLoad> regionLoad,
531 final Set<String> coprocessors) {
532 this.numberOfRequests = numberOfRequests;
533 this.usedHeapMB = usedHeapMB;
534 this.maxHeapMB = maxHeapMB;
535 this.regionLoad = regionLoad;
536 this.totalNumberOfRequests = totalNumberOfRequests;
537 this.coprocessors = coprocessors;
538 }
539
540
541
542
543
544 public HServerLoad(final HServerLoad hsl) {
545 this(hsl.totalNumberOfRequests, hsl.numberOfRequests, hsl.usedHeapMB,
546 hsl.maxHeapMB, hsl.getRegionsLoad(), hsl.coprocessors);
547 for (Map.Entry<byte[], RegionLoad> e : hsl.regionLoad.entrySet()) {
548 this.regionLoad.put(e.getKey(), e.getValue());
549 }
550 }
551
552
553
554
555
556
557
558
559
560
561 public int getLoad() {
562
563
564
565 return this.regionLoad.size();
566 }
567
568
569
570
571 @Override
572 public String toString() {
573 return toString(1);
574 }
575
576
577
578
579
580
581
582 public String toString(int msgInterval) {
583 int numberOfRegions = this.regionLoad.size();
584 StringBuilder sb = new StringBuilder();
585 sb = Strings.appendKeyValue(sb, "requestsPerSecond",
586 Integer.valueOf(numberOfRequests/msgInterval));
587 sb = Strings.appendKeyValue(sb, "numberOfOnlineRegions",
588 Integer.valueOf(numberOfRegions));
589 sb = Strings.appendKeyValue(sb, "usedHeapMB",
590 Integer.valueOf(this.usedHeapMB));
591 sb = Strings.appendKeyValue(sb, "maxHeapMB", Integer.valueOf(maxHeapMB));
592 return sb.toString();
593 }
594
595
596
597
598 @Override
599 public boolean equals(Object o) {
600 if (this == o) {
601 return true;
602 }
603 if (o == null) {
604 return false;
605 }
606 if (getClass() != o.getClass()) {
607 return false;
608 }
609 return compareTo((HServerLoad)o) == 0;
610 }
611
612
613
614
615
616
617 public int getNumberOfRegions() {
618 return this.regionLoad.size();
619 }
620
621
622
623
624 public int getNumberOfRequests() {
625 return numberOfRequests;
626 }
627
628
629
630
631 public int getTotalNumberOfRequests() {
632 return totalNumberOfRequests;
633 }
634
635
636
637
638 public int getUsedHeapMB() {
639 return usedHeapMB;
640 }
641
642
643
644
645 public int getMaxHeapMB() {
646 return maxHeapMB;
647 }
648
649
650
651
652 public Map<byte[], RegionLoad> getRegionsLoad() {
653 return Collections.unmodifiableMap(regionLoad);
654 }
655
656
657
658
659 public int getStorefiles() {
660 int count = 0;
661 for (RegionLoad info: regionLoad.values())
662 count += info.getStorefiles();
663 return count;
664 }
665
666
667
668
669 public int getStorefileSizeInMB() {
670 int count = 0;
671 for (RegionLoad info: regionLoad.values())
672 count += info.getStorefileSizeMB();
673 return count;
674 }
675
676
677
678
679 public int getMemStoreSizeInMB() {
680 int count = 0;
681 for (RegionLoad info: regionLoad.values())
682 count += info.getMemStoreSizeMB();
683 return count;
684 }
685
686
687
688
689 public int getStorefileIndexSizeInMB() {
690 int count = 0;
691 for (RegionLoad info: regionLoad.values())
692 count += info.getStorefileIndexSizeMB();
693 return count;
694 }
695
696
697 public void readFields(DataInput in) throws IOException {
698 readFieldsGetRegionKeys(in);
699 }
700
701 public List<byte[]> readFieldsGetRegionKeys(DataInput in) throws IOException {
702 super.readFields(in);
703 int version = in.readByte();
704 if (version > VERSION) throw new IOException("Version mismatch; " + version);
705 numberOfRequests = in.readInt();
706 usedHeapMB = in.readInt();
707 maxHeapMB = in.readInt();
708 int numberOfRegions = in.readInt();
709
710
711 List<byte[]> regionKeys = new ArrayList<byte[]>(numberOfRegions);
712 regionLoad.clear();
713 for (int i = 0; i < numberOfRegions; i++) {
714 RegionLoad rl = new RegionLoad();
715 rl.readFields(in);
716 regionLoad.put(rl.getName(), rl);
717 regionKeys.add(rl.getName());
718 }
719 totalNumberOfRequests = in.readInt();
720 int coprocessorsSize = in.readInt();
721 coprocessors.clear();
722 for(int i = 0; i < coprocessorsSize; i++) {
723 coprocessors.add(in.readUTF());
724 }
725 return regionKeys;
726 }
727
728 public void write(DataOutput out) throws IOException {
729 super.write(out);
730 out.writeByte(VERSION);
731 out.writeInt(numberOfRequests);
732 out.writeInt(usedHeapMB);
733 out.writeInt(maxHeapMB);
734 out.writeInt(this.regionLoad.size());
735 for (RegionLoad rl: regionLoad.values())
736 rl.write(out);
737 out.writeInt(totalNumberOfRequests);
738 out.writeInt(coprocessors.size());
739 for (String coprocessor: coprocessors) {
740 out.writeUTF(coprocessor);
741 }
742 }
743
744
745
746 public int compareTo(HServerLoad o) {
747 return this.getLoad() - o.getLoad();
748 }
749 }