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