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