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