001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.rest.model; 019 020import com.fasterxml.jackson.annotation.JsonProperty; 021import java.io.IOException; 022import java.io.Serializable; 023import java.util.ArrayList; 024import java.util.List; 025import javax.xml.bind.annotation.XmlAttribute; 026import javax.xml.bind.annotation.XmlElement; 027import javax.xml.bind.annotation.XmlElementWrapper; 028import javax.xml.bind.annotation.XmlRootElement; 029import org.apache.hadoop.hbase.rest.ProtobufMessageHandler; 030import org.apache.hadoop.hbase.rest.RestUtil; 031import org.apache.hadoop.hbase.util.Bytes; 032import org.apache.yetus.audience.InterfaceAudience; 033 034import org.apache.hbase.thirdparty.com.google.protobuf.CodedInputStream; 035import org.apache.hbase.thirdparty.com.google.protobuf.Message; 036import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; 037 038import org.apache.hadoop.hbase.shaded.rest.protobuf.generated.StorageClusterStatusMessage.StorageClusterStatus; 039 040/** 041 * Representation of the status of a storage cluster: 042 * <p> 043 * <ul> 044 * <li>regions: the total number of regions served by the cluster</li> 045 * <li>requests: the total number of requests per second handled by the cluster in the last 046 * reporting interval</li> 047 * <li>averageLoad: the average load of the region servers in the cluster</li> 048 * <li>liveNodes: detailed status of the live region servers</li> 049 * <li>deadNodes: the names of region servers declared dead</li> 050 * </ul> 051 * 052 * <pre> 053 * <complexType name="StorageClusterStatus"> 054 * <sequence> 055 * <element name="liveNode" type="tns:Node" 056 * maxOccurs="unbounded" minOccurs="0"> 057 * </element> 058 * <element name="deadNode" type="string" maxOccurs="unbounded" 059 * minOccurs="0"> 060 * </element> 061 * </sequence> 062 * <attribute name="regions" type="int"></attribute> 063 * <attribute name="requests" type="int"></attribute> 064 * <attribute name="averageLoad" type="float"></attribute> 065 * </complexType> 066 * 067 * <complexType name="Node"> 068 * <sequence> 069 * <element name="region" type="tns:Region" 070 * maxOccurs="unbounded" minOccurs="0"></element> 071 * </sequence> 072 * <attribute name="name" type="string"></attribute> 073 * <attribute name="startCode" type="int"></attribute> 074 * <attribute name="requests" type="int"></attribute> 075 * <attribute name="heapSizeMB" type="int"></attribute> 076 * <attribute name="maxHeapSizeMB" type="int"></attribute> 077 * </complexType> 078 * 079 * <complexType name="Region"> 080 * <attribute name="name" type="base64Binary"></attribute> 081 * <attribute name="stores" type="int"></attribute> 082 * <attribute name="storefiles" type="int"></attribute> 083 * <attribute name="storefileSizeMB" type="int"></attribute> 084 * <attribute name="memstoreSizeMB" type="int"></attribute> 085 * <attribute name="storefileIndexSizeMB" type="int"></attribute> 086 * <attribute name="readRequestsCount" type="int"></attribute> 087 * <attribute name="cpRequestsCount" type="int"></attribute> 088 * <attribute name="writeRequestsCount" type="int"></attribute> 089 * <attribute name="rootIndexSizeKB" type="int"></attribute> 090 * <attribute name="totalStaticIndexSizeKB" type="int"></attribute> 091 * <attribute name="totalStaticBloomSizeKB" type="int"></attribute> 092 * <attribute name="totalCompactingKVs" type="int"></attribute> 093 * <attribute name="currentCompactedKVs" type="int"></attribute> 094 * </complexType> 095 * </pre> 096 */ 097@XmlRootElement(name = "ClusterStatus") 098@InterfaceAudience.Private 099public class StorageClusterStatusModel implements Serializable, ProtobufMessageHandler { 100 private static final long serialVersionUID = 1L; 101 102 /** 103 * Represents a region server. 104 */ 105 public static class Node implements Serializable { 106 private static final long serialVersionUID = 1L; 107 108 /** 109 * Represents a region hosted on a region server. 110 */ 111 public static class Region implements Serializable { 112 private static final long serialVersionUID = -1326683840086398193L; 113 114 private byte[] name; 115 private int stores; 116 private int storefiles; 117 private int storefileSizeMB; 118 private int memstoreSizeMB; 119 private long storefileIndexSizeKB; 120 private long readRequestsCount; 121 private long cpRequestsCount; 122 private long writeRequestsCount; 123 private int rootIndexSizeKB; 124 private int totalStaticIndexSizeKB; 125 private int totalStaticBloomSizeKB; 126 private long totalCompactingKVs; 127 private long currentCompactedKVs; 128 129 /** 130 * Default constructor 131 */ 132 public Region() { 133 } 134 135 /** 136 * Constructor 137 * @param name the region name 138 */ 139 public Region(byte[] name) { 140 this.name = name; 141 } 142 143 /** 144 * Constructor 145 * @param name the region name 146 * @param stores the number of stores 147 * @param storefiles the number of store files 148 * @param storefileSizeMB total size of store files, in MB 149 * @param memstoreSizeMB total size of memstore, in MB 150 * @param storefileIndexSizeKB total size of store file indexes, in KB 151 */ 152 public Region(byte[] name, int stores, int storefiles, int storefileSizeMB, 153 int memstoreSizeMB, long storefileIndexSizeKB, long readRequestsCount, long cpRequestsCount, 154 long writeRequestsCount, int rootIndexSizeKB, int totalStaticIndexSizeKB, 155 int totalStaticBloomSizeKB, long totalCompactingKVs, long currentCompactedKVs) { 156 this.name = name; 157 this.stores = stores; 158 this.storefiles = storefiles; 159 this.storefileSizeMB = storefileSizeMB; 160 this.memstoreSizeMB = memstoreSizeMB; 161 this.storefileIndexSizeKB = storefileIndexSizeKB; 162 this.readRequestsCount = readRequestsCount; 163 this.cpRequestsCount = cpRequestsCount; 164 this.writeRequestsCount = writeRequestsCount; 165 this.rootIndexSizeKB = rootIndexSizeKB; 166 this.totalStaticIndexSizeKB = totalStaticIndexSizeKB; 167 this.totalStaticBloomSizeKB = totalStaticBloomSizeKB; 168 this.totalCompactingKVs = totalCompactingKVs; 169 this.currentCompactedKVs = currentCompactedKVs; 170 } 171 172 /** Returns the region name */ 173 @XmlAttribute 174 public byte[] getName() { 175 return name; 176 } 177 178 /** Returns the number of stores */ 179 @XmlAttribute 180 public int getStores() { 181 return stores; 182 } 183 184 /** Returns the number of store files */ 185 @XmlAttribute 186 public int getStorefiles() { 187 return storefiles; 188 } 189 190 /** Returns the total size of store files, in MB */ 191 @XmlAttribute 192 public int getStorefileSizeMB() { 193 return storefileSizeMB; 194 } 195 196 /** Returns memstore size, in MB */ 197 @XmlAttribute 198 public int getMemStoreSizeMB() { 199 return memstoreSizeMB; 200 } 201 202 /** Returns the total size of store file indexes, in KB */ 203 @XmlAttribute 204 public long getStorefileIndexSizeKB() { 205 return storefileIndexSizeKB; 206 } 207 208 /** Returns the current total read requests made to region */ 209 @XmlAttribute 210 public long getReadRequestsCount() { 211 return readRequestsCount; 212 } 213 214 /** Returns the current total read requests made to region */ 215 @XmlAttribute 216 public long getCpRequestsCount() { 217 return cpRequestsCount; 218 } 219 220 /** Returns the current total write requests made to region */ 221 @XmlAttribute 222 public long getWriteRequestsCount() { 223 return writeRequestsCount; 224 } 225 226 /** Returns The current total size of root-level indexes for the region, in KB. */ 227 @XmlAttribute 228 public int getRootIndexSizeKB() { 229 return rootIndexSizeKB; 230 } 231 232 /** Returns The total size of static index, in KB */ 233 @XmlAttribute 234 public int getTotalStaticIndexSizeKB() { 235 return totalStaticIndexSizeKB; 236 } 237 238 /** Returns The total size of static bloom, in KB */ 239 @XmlAttribute 240 public int getTotalStaticBloomSizeKB() { 241 return totalStaticBloomSizeKB; 242 } 243 244 /** Returns The total number of compacting key-values */ 245 @XmlAttribute 246 public long getTotalCompactingKVs() { 247 return totalCompactingKVs; 248 } 249 250 /** Returns The number of current compacted key-values */ 251 @XmlAttribute 252 public long getCurrentCompactedKVs() { 253 return currentCompactedKVs; 254 } 255 256 /** 257 * @param readRequestsCount The current total read requests made to region 258 */ 259 public void setReadRequestsCount(long readRequestsCount) { 260 this.readRequestsCount = readRequestsCount; 261 } 262 263 /** 264 * @param cpRequestsCount The current total read requests made to region 265 */ 266 public void setCpRequestsCount(long cpRequestsCount) { 267 this.cpRequestsCount = cpRequestsCount; 268 } 269 270 /** 271 * @param rootIndexSizeKB The current total size of root-level indexes for the region, in KB 272 */ 273 public void setRootIndexSizeKB(int rootIndexSizeKB) { 274 this.rootIndexSizeKB = rootIndexSizeKB; 275 } 276 277 /** 278 * @param writeRequestsCount The current total write requests made to region 279 */ 280 public void setWriteRequestsCount(long writeRequestsCount) { 281 this.writeRequestsCount = writeRequestsCount; 282 } 283 284 /** 285 * @param currentCompactedKVs The completed count of key values in currently running 286 * compaction 287 */ 288 public void setCurrentCompactedKVs(long currentCompactedKVs) { 289 this.currentCompactedKVs = currentCompactedKVs; 290 } 291 292 /** 293 * @param totalCompactingKVs The total compacting key values in currently running compaction 294 */ 295 public void setTotalCompactingKVs(long totalCompactingKVs) { 296 this.totalCompactingKVs = totalCompactingKVs; 297 } 298 299 /** 300 * @param totalStaticBloomSizeKB The total size of all Bloom filter blocks, not just loaded 301 * into the block cache, in KB. 302 */ 303 public void setTotalStaticBloomSizeKB(int totalStaticBloomSizeKB) { 304 this.totalStaticBloomSizeKB = totalStaticBloomSizeKB; 305 } 306 307 /** 308 * @param totalStaticIndexSizeKB The total size of all index blocks, not just the root level, 309 * in KB. 310 */ 311 public void setTotalStaticIndexSizeKB(int totalStaticIndexSizeKB) { 312 this.totalStaticIndexSizeKB = totalStaticIndexSizeKB; 313 } 314 315 /** 316 * @param name the region name 317 */ 318 public void setName(byte[] name) { 319 this.name = name; 320 } 321 322 /** 323 * @param stores the number of stores 324 */ 325 public void setStores(int stores) { 326 this.stores = stores; 327 } 328 329 /** 330 * @param storefiles the number of store files 331 */ 332 public void setStorefiles(int storefiles) { 333 this.storefiles = storefiles; 334 } 335 336 /** 337 * @param storefileSizeMB total size of store files, in MB 338 */ 339 public void setStorefileSizeMB(int storefileSizeMB) { 340 this.storefileSizeMB = storefileSizeMB; 341 } 342 343 /** 344 * @param memstoreSizeMB memstore size, in MB 345 */ 346 public void setMemStoreSizeMB(int memstoreSizeMB) { 347 this.memstoreSizeMB = memstoreSizeMB; 348 } 349 350 /** 351 * @param storefileIndexSizeKB total size of store file indexes, in KB 352 */ 353 public void setStorefileIndexSizeKB(long storefileIndexSizeKB) { 354 this.storefileIndexSizeKB = storefileIndexSizeKB; 355 } 356 } 357 358 private String name; 359 private long startCode; 360 private long requests; 361 private int heapSizeMB; 362 private int maxHeapSizeMB; 363 private List<Region> regions = new ArrayList<>(); 364 365 /** 366 * Add a region name to the list 367 * @param name the region name 368 */ 369 public void addRegion(byte[] name, int stores, int storefiles, int storefileSizeMB, 370 int memstoreSizeMB, long storefileIndexSizeKB, long readRequestsCount, long cpRequestsCount, 371 long writeRequestsCount, int rootIndexSizeKB, int totalStaticIndexSizeKB, 372 int totalStaticBloomSizeKB, long totalCompactingKVs, long currentCompactedKVs) { 373 regions.add( 374 new Region(name, stores, storefiles, storefileSizeMB, memstoreSizeMB, storefileIndexSizeKB, 375 readRequestsCount, cpRequestsCount, writeRequestsCount, rootIndexSizeKB, 376 totalStaticIndexSizeKB, totalStaticBloomSizeKB, totalCompactingKVs, currentCompactedKVs)); 377 } 378 379 /** 380 * @param index the index 381 * @return the region name 382 */ 383 public Region getRegion(int index) { 384 return regions.get(index); 385 } 386 387 /** 388 * Default constructor 389 */ 390 public Node() { 391 } 392 393 /** 394 * Constructor 395 * @param name the region server name 396 * @param startCode the region server's start code 397 */ 398 public Node(String name, long startCode) { 399 this.name = name; 400 this.startCode = startCode; 401 } 402 403 /** Returns the region server's name */ 404 @XmlAttribute 405 public String getName() { 406 return name; 407 } 408 409 /** Returns the region server's start code */ 410 @XmlAttribute 411 public long getStartCode() { 412 return startCode; 413 } 414 415 /** Returns the current heap size, in MB */ 416 @XmlAttribute 417 public int getHeapSizeMB() { 418 return heapSizeMB; 419 } 420 421 /** Returns the maximum heap size, in MB */ 422 @XmlAttribute 423 public int getMaxHeapSizeMB() { 424 return maxHeapSizeMB; 425 } 426 427 /** Returns the list of regions served by the region server */ 428 @XmlElement(name = "Region") 429 public List<Region> getRegions() { 430 return regions; 431 } 432 433 /** Returns the number of requests per second processed by the region server */ 434 @XmlAttribute 435 public long getRequests() { 436 return requests; 437 } 438 439 /** 440 * @param name the region server's hostname 441 */ 442 public void setName(String name) { 443 this.name = name; 444 } 445 446 /** 447 * @param startCode the region server's start code 448 */ 449 public void setStartCode(long startCode) { 450 this.startCode = startCode; 451 } 452 453 /** 454 * @param heapSizeMB the current heap size, in MB 455 */ 456 public void setHeapSizeMB(int heapSizeMB) { 457 this.heapSizeMB = heapSizeMB; 458 } 459 460 /** 461 * @param maxHeapSizeMB the maximum heap size, in MB 462 */ 463 public void setMaxHeapSizeMB(int maxHeapSizeMB) { 464 this.maxHeapSizeMB = maxHeapSizeMB; 465 } 466 467 /** 468 * @param regions a list of regions served by the region server 469 */ 470 public void setRegions(List<Region> regions) { 471 this.regions = regions; 472 } 473 474 /** 475 * @param requests the number of requests per second processed by the region server 476 */ 477 public void setRequests(long requests) { 478 this.requests = requests; 479 } 480 } 481 482 private List<Node> liveNodes = new ArrayList<>(); 483 private List<String> deadNodes = new ArrayList<>(); 484 private int regions; 485 private long requests; 486 private double averageLoad; 487 488 /** 489 * Add a live node to the cluster representation. 490 * @param name the region server name 491 * @param startCode the region server's start code 492 * @param heapSizeMB the current heap size, in MB 493 * @param maxHeapSizeMB the maximum heap size, in MB 494 */ 495 public Node addLiveNode(String name, long startCode, int heapSizeMB, int maxHeapSizeMB) { 496 Node node = new Node(name, startCode); 497 node.setHeapSizeMB(heapSizeMB); 498 node.setMaxHeapSizeMB(maxHeapSizeMB); 499 liveNodes.add(node); 500 return node; 501 } 502 503 /** 504 * @param index the index 505 * @return the region server model 506 */ 507 public Node getLiveNode(int index) { 508 return liveNodes.get(index); 509 } 510 511 /** 512 * Add a dead node to the cluster representation. 513 * @param node the dead region server's name 514 */ 515 public void addDeadNode(String node) { 516 deadNodes.add(node); 517 } 518 519 /** 520 * @param index the index 521 * @return the dead region server's name 522 */ 523 public String getDeadNode(int index) { 524 return deadNodes.get(index); 525 } 526 527 /** 528 * Default constructor 529 */ 530 public StorageClusterStatusModel() { 531 } 532 533 /** Returns the list of live nodes */ 534 @XmlElement(name = "Node") 535 @XmlElementWrapper(name = "LiveNodes") 536 // workaround https://github.com/FasterXML/jackson-dataformat-xml/issues/192 537 @JsonProperty("LiveNodes") 538 public List<Node> getLiveNodes() { 539 return liveNodes; 540 } 541 542 /** Returns the list of dead nodes */ 543 @XmlElement(name = "Node") 544 @XmlElementWrapper(name = "DeadNodes") 545 // workaround https://github.com/FasterXML/jackson-dataformat-xml/issues/192 546 @JsonProperty("DeadNodes") 547 public List<String> getDeadNodes() { 548 return deadNodes; 549 } 550 551 /** Returns the total number of regions served by the cluster */ 552 @XmlAttribute 553 public int getRegions() { 554 return regions; 555 } 556 557 /** 558 * @return the total number of requests per second handled by the cluster in the last reporting 559 * interval 560 */ 561 @XmlAttribute 562 public long getRequests() { 563 return requests; 564 } 565 566 /** Returns the average load of the region servers in the cluster */ 567 @XmlAttribute 568 public double getAverageLoad() { 569 return averageLoad; 570 } 571 572 /** 573 * @param nodes the list of live node models 574 */ 575 public void setLiveNodes(List<Node> nodes) { 576 this.liveNodes = nodes; 577 } 578 579 /** 580 * @param nodes the list of dead node names 581 */ 582 public void setDeadNodes(List<String> nodes) { 583 this.deadNodes = nodes; 584 } 585 586 /** 587 * @param regions the total number of regions served by the cluster 588 */ 589 public void setRegions(int regions) { 590 this.regions = regions; 591 } 592 593 /** 594 * @param requests the total number of requests per second handled by the cluster 595 */ 596 public void setRequests(long requests) { 597 this.requests = requests; 598 } 599 600 /** 601 * @param averageLoad the average load of region servers in the cluster 602 */ 603 public void setAverageLoad(double averageLoad) { 604 this.averageLoad = averageLoad; 605 } 606 607 @Override 608 public String toString() { 609 StringBuilder sb = new StringBuilder(); 610 sb.append(String.format("%d live servers, %d dead servers, " + "%.4f average load%n%n", 611 liveNodes.size(), deadNodes.size(), averageLoad)); 612 if (!liveNodes.isEmpty()) { 613 sb.append(liveNodes.size()); 614 sb.append(" live servers\n"); 615 for (Node node : liveNodes) { 616 sb.append(" "); 617 sb.append(node.name); 618 sb.append(' '); 619 sb.append(node.startCode); 620 sb.append("\n requests="); 621 sb.append(node.requests); 622 sb.append(", regions="); 623 sb.append(node.regions.size()); 624 sb.append("\n heapSizeMB="); 625 sb.append(node.heapSizeMB); 626 sb.append("\n maxHeapSizeMB="); 627 sb.append(node.maxHeapSizeMB); 628 sb.append("\n\n"); 629 for (Node.Region region : node.regions) { 630 sb.append(" "); 631 sb.append(Bytes.toString(region.name)); 632 sb.append("\n stores="); 633 sb.append(region.stores); 634 sb.append("\n storefiless="); 635 sb.append(region.storefiles); 636 sb.append("\n storefileSizeMB="); 637 sb.append(region.storefileSizeMB); 638 sb.append("\n memstoreSizeMB="); 639 sb.append(region.memstoreSizeMB); 640 sb.append("\n storefileIndexSizeKB="); 641 sb.append(region.storefileIndexSizeKB); 642 sb.append("\n readRequestsCount="); 643 sb.append(region.readRequestsCount); 644 sb.append("\n cpRequestsCount="); 645 sb.append(region.cpRequestsCount); 646 sb.append("\n writeRequestsCount="); 647 sb.append(region.writeRequestsCount); 648 sb.append("\n rootIndexSizeKB="); 649 sb.append(region.rootIndexSizeKB); 650 sb.append("\n totalStaticIndexSizeKB="); 651 sb.append(region.totalStaticIndexSizeKB); 652 sb.append("\n totalStaticBloomSizeKB="); 653 sb.append(region.totalStaticBloomSizeKB); 654 sb.append("\n totalCompactingKVs="); 655 sb.append(region.totalCompactingKVs); 656 sb.append("\n currentCompactedKVs="); 657 sb.append(region.currentCompactedKVs); 658 sb.append('\n'); 659 } 660 sb.append('\n'); 661 } 662 } 663 if (!deadNodes.isEmpty()) { 664 sb.append('\n'); 665 sb.append(deadNodes.size()); 666 sb.append(" dead servers\n"); 667 for (String node : deadNodes) { 668 sb.append(" "); 669 sb.append(node); 670 sb.append('\n'); 671 } 672 } 673 return sb.toString(); 674 } 675 676 @Override 677 public Message messageFromObject() { 678 StorageClusterStatus.Builder builder = StorageClusterStatus.newBuilder(); 679 builder.setRegions(regions); 680 builder.setRequests(requests); 681 builder.setAverageLoad(averageLoad); 682 for (Node node : liveNodes) { 683 StorageClusterStatus.Node.Builder nodeBuilder = StorageClusterStatus.Node.newBuilder(); 684 nodeBuilder.setName(node.name); 685 nodeBuilder.setStartCode(node.startCode); 686 nodeBuilder.setRequests(node.requests); 687 nodeBuilder.setHeapSizeMB(node.heapSizeMB); 688 nodeBuilder.setMaxHeapSizeMB(node.maxHeapSizeMB); 689 for (Node.Region region : node.regions) { 690 StorageClusterStatus.Region.Builder regionBuilder = 691 StorageClusterStatus.Region.newBuilder(); 692 regionBuilder.setName(UnsafeByteOperations.unsafeWrap(region.name)); 693 regionBuilder.setStores(region.stores); 694 regionBuilder.setStorefiles(region.storefiles); 695 regionBuilder.setStorefileSizeMB(region.storefileSizeMB); 696 regionBuilder.setMemStoreSizeMB(region.memstoreSizeMB); 697 regionBuilder.setStorefileIndexSizeKB(region.storefileIndexSizeKB); 698 regionBuilder.setReadRequestsCount(region.readRequestsCount); 699 regionBuilder.setCpRequestsCount(region.cpRequestsCount); 700 regionBuilder.setWriteRequestsCount(region.writeRequestsCount); 701 regionBuilder.setRootIndexSizeKB(region.rootIndexSizeKB); 702 regionBuilder.setTotalStaticIndexSizeKB(region.totalStaticIndexSizeKB); 703 regionBuilder.setTotalStaticBloomSizeKB(region.totalStaticBloomSizeKB); 704 regionBuilder.setTotalCompactingKVs(region.totalCompactingKVs); 705 regionBuilder.setCurrentCompactedKVs(region.currentCompactedKVs); 706 nodeBuilder.addRegions(regionBuilder); 707 } 708 builder.addLiveNodes(nodeBuilder); 709 } 710 for (String node : deadNodes) { 711 builder.addDeadNodes(node); 712 } 713 return builder.build(); 714 } 715 716 @Override 717 public ProtobufMessageHandler getObjectFromMessage(CodedInputStream cis) throws IOException { 718 StorageClusterStatus.Builder builder = StorageClusterStatus.newBuilder(); 719 RestUtil.mergeFrom(builder, cis); 720 if (builder.hasRegions()) { 721 regions = builder.getRegions(); 722 } 723 if (builder.hasRequests()) { 724 requests = builder.getRequests(); 725 } 726 if (builder.hasAverageLoad()) { 727 averageLoad = builder.getAverageLoad(); 728 } 729 for (StorageClusterStatus.Node node : builder.getLiveNodesList()) { 730 long startCode = node.hasStartCode() ? node.getStartCode() : -1; 731 StorageClusterStatusModel.Node nodeModel = 732 addLiveNode(node.getName(), startCode, node.getHeapSizeMB(), node.getMaxHeapSizeMB()); 733 long requests = node.hasRequests() ? node.getRequests() : 0; 734 nodeModel.setRequests(requests); 735 for (StorageClusterStatus.Region region : node.getRegionsList()) { 736 nodeModel.addRegion(region.getName().toByteArray(), region.getStores(), 737 region.getStorefiles(), region.getStorefileSizeMB(), region.getMemStoreSizeMB(), 738 region.getStorefileIndexSizeKB(), region.getReadRequestsCount(), 739 region.getCpRequestsCount(), region.getWriteRequestsCount(), region.getRootIndexSizeKB(), 740 region.getTotalStaticIndexSizeKB(), region.getTotalStaticBloomSizeKB(), 741 region.getTotalCompactingKVs(), region.getCurrentCompactedKVs()); 742 } 743 } 744 for (String node : builder.getDeadNodesList()) { 745 addDeadNode(node); 746 } 747 return this; 748 } 749}