HBASE-5434 [REST] Include more metrics in cluster status request (Mubarak Seyed)
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1304919 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
53712ce495
commit
accf8ee862
|
@ -274,7 +274,29 @@ implements WritableComparable<HServerLoad> {
|
||||||
public long getWriteRequestsCount() {
|
public long getWriteRequestsCount() {
|
||||||
return writeRequestsCount;
|
return writeRequestsCount;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The current total size of root-level indexes for the region, in KB.
|
||||||
|
*/
|
||||||
|
public int getRootIndexSizeKB() {
|
||||||
|
return rootIndexSizeKB;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The total size of all index blocks, not just the root level, in KB.
|
||||||
|
*/
|
||||||
|
public int getTotalStaticIndexSizeKB() {
|
||||||
|
return totalStaticIndexSizeKB;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The total size of all Bloom filter blocks, not just loaded into the
|
||||||
|
* block cache, in KB.
|
||||||
|
*/
|
||||||
|
public int getTotalStaticBloomSizeKB() {
|
||||||
|
return totalStaticBloomSizeKB;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return the total number of kvs in current compaction
|
* @return the total number of kvs in current compaction
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -87,7 +87,11 @@ public class StorageClusterStatusResource extends ResourceBase {
|
||||||
for (HServerLoad.RegionLoad region: load.getRegionsLoad().values()) {
|
for (HServerLoad.RegionLoad region: load.getRegionsLoad().values()) {
|
||||||
node.addRegion(region.getName(), region.getStores(),
|
node.addRegion(region.getName(), region.getStores(),
|
||||||
region.getStorefiles(), region.getStorefileSizeMB(),
|
region.getStorefiles(), region.getStorefileSizeMB(),
|
||||||
region.getMemStoreSizeMB(), region.getStorefileIndexSizeMB());
|
region.getMemStoreSizeMB(), region.getStorefileIndexSizeMB(),
|
||||||
|
region.getReadRequestsCount(), region.getWriteRequestsCount(),
|
||||||
|
region.getRootIndexSizeKB(), region.getTotalStaticIndexSizeKB(),
|
||||||
|
region.getTotalStaticBloomSizeKB(), region.getTotalCompactingKVs(),
|
||||||
|
region.getCurrentCompactedKVs());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
for (ServerName name: status.getDeadServerNames()) {
|
for (ServerName name: status.getDeadServerNames()) {
|
||||||
|
|
|
@ -83,6 +83,13 @@ import com.google.protobuf.ByteString;
|
||||||
* <attribute name="storefileSizeMB" type="int"></attribute>
|
* <attribute name="storefileSizeMB" type="int"></attribute>
|
||||||
* <attribute name="memstoreSizeMB" type="int"></attribute>
|
* <attribute name="memstoreSizeMB" type="int"></attribute>
|
||||||
* <attribute name="storefileIndexSizeMB" type="int"></attribute>
|
* <attribute name="storefileIndexSizeMB" type="int"></attribute>
|
||||||
|
* <attribute name="readRequestsCount" type="int"></attribute>
|
||||||
|
* <attribute name="writeRequestsCount" type="int"></attribute>
|
||||||
|
* <attribute name="rootIndexSizeKB" type="int"></attribute>
|
||||||
|
* <attribute name="totalStaticIndexSizeKB" type="int"></attribute>
|
||||||
|
* <attribute name="totalStaticBloomSizeKB" type="int"></attribute>
|
||||||
|
* <attribute name="totalCompactingKVs" type="int"></attribute>
|
||||||
|
* <attribute name="currentCompactedKVs" type="int"></attribute>
|
||||||
* </complexType>
|
* </complexType>
|
||||||
* </pre>
|
* </pre>
|
||||||
*/
|
*/
|
||||||
|
@ -107,6 +114,13 @@ public class StorageClusterStatusModel
|
||||||
private int storefileSizeMB;
|
private int storefileSizeMB;
|
||||||
private int memstoreSizeMB;
|
private int memstoreSizeMB;
|
||||||
private int storefileIndexSizeMB;
|
private int storefileIndexSizeMB;
|
||||||
|
private long readRequestsCount;
|
||||||
|
private long writeRequestsCount;
|
||||||
|
private int rootIndexSizeKB;
|
||||||
|
private int totalStaticIndexSizeKB;
|
||||||
|
private int totalStaticBloomSizeKB;
|
||||||
|
private long totalCompactingKVs;
|
||||||
|
private long currentCompactedKVs;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Default constructor
|
* Default constructor
|
||||||
|
@ -131,13 +145,23 @@ public class StorageClusterStatusModel
|
||||||
* @param storefileIndexSizeMB total size of store file indexes, in MB
|
* @param storefileIndexSizeMB total size of store file indexes, in MB
|
||||||
*/
|
*/
|
||||||
public Region(byte[] name, int stores, int storefiles,
|
public Region(byte[] name, int stores, int storefiles,
|
||||||
int storefileSizeMB, int memstoreSizeMB, int storefileIndexSizeMB) {
|
int storefileSizeMB, int memstoreSizeMB, int storefileIndexSizeMB,
|
||||||
|
long readRequestsCount, long writeRequestsCount, int rootIndexSizeKB,
|
||||||
|
int totalStaticIndexSizeKB, int totalStaticBloomSizeKB,
|
||||||
|
long totalCompactingKVs, long currentCompactedKVs) {
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.stores = stores;
|
this.stores = stores;
|
||||||
this.storefiles = storefiles;
|
this.storefiles = storefiles;
|
||||||
this.storefileSizeMB = storefileSizeMB;
|
this.storefileSizeMB = storefileSizeMB;
|
||||||
this.memstoreSizeMB = memstoreSizeMB;
|
this.memstoreSizeMB = memstoreSizeMB;
|
||||||
this.storefileIndexSizeMB = storefileIndexSizeMB;
|
this.storefileIndexSizeMB = storefileIndexSizeMB;
|
||||||
|
this.readRequestsCount = readRequestsCount;
|
||||||
|
this.writeRequestsCount = writeRequestsCount;
|
||||||
|
this.rootIndexSizeKB = rootIndexSizeKB;
|
||||||
|
this.totalStaticIndexSizeKB = totalStaticIndexSizeKB;
|
||||||
|
this.totalStaticBloomSizeKB = totalStaticBloomSizeKB;
|
||||||
|
this.totalCompactingKVs = totalCompactingKVs;
|
||||||
|
this.currentCompactedKVs = currentCompactedKVs;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -187,7 +211,117 @@ public class StorageClusterStatusModel
|
||||||
public int getStorefileIndexSizeMB() {
|
public int getStorefileIndexSizeMB() {
|
||||||
return storefileIndexSizeMB;
|
return storefileIndexSizeMB;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the current total read requests made to region
|
||||||
|
*/
|
||||||
|
@XmlAttribute
|
||||||
|
public long getReadRequestsCount() {
|
||||||
|
return readRequestsCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the current total write requests made to region
|
||||||
|
*/
|
||||||
|
@XmlAttribute
|
||||||
|
public long getWriteRequestsCount() {
|
||||||
|
return writeRequestsCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The current total size of root-level indexes for the region, in KB.
|
||||||
|
*/
|
||||||
|
@XmlAttribute
|
||||||
|
public int getRootIndexSizeKB() {
|
||||||
|
return rootIndexSizeKB;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The total size of static index, in KB
|
||||||
|
*/
|
||||||
|
@XmlAttribute
|
||||||
|
public int getTotalStaticIndexSizeKB() {
|
||||||
|
return totalStaticIndexSizeKB;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The total size of static bloom, in KB
|
||||||
|
*/
|
||||||
|
@XmlAttribute
|
||||||
|
public int getTotalStaticBloomSizeKB() {
|
||||||
|
return totalStaticBloomSizeKB;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The total number of compacting key-values
|
||||||
|
*/
|
||||||
|
@XmlAttribute
|
||||||
|
public long getTotalCompactingKVs() {
|
||||||
|
return totalCompactingKVs;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The number of current compacted key-values
|
||||||
|
*/
|
||||||
|
@XmlAttribute
|
||||||
|
public long getCurrentCompactedKVs() {
|
||||||
|
return currentCompactedKVs;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param readRequestsCount The current total read requests made to region
|
||||||
|
*/
|
||||||
|
public void setReadRequestsCount(long readRequestsCount) {
|
||||||
|
this.readRequestsCount = readRequestsCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param rootIndexSizeKB The current total size of root-level indexes
|
||||||
|
* for the region, in KB
|
||||||
|
*/
|
||||||
|
public void setRootIndexSizeKB(int rootIndexSizeKB) {
|
||||||
|
this.rootIndexSizeKB = rootIndexSizeKB;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param writeRequestsCount The current total write requests made to region
|
||||||
|
*/
|
||||||
|
public void setWriteRequestsCount(long writeRequestsCount) {
|
||||||
|
this.writeRequestsCount = writeRequestsCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param currentCompactedKVs The completed count of key values
|
||||||
|
* in currently running compaction
|
||||||
|
*/
|
||||||
|
public void setCurrentCompactedKVs(long currentCompactedKVs) {
|
||||||
|
this.currentCompactedKVs = currentCompactedKVs;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param totalCompactingKVs The total compacting key values
|
||||||
|
* in currently running compaction
|
||||||
|
*/
|
||||||
|
public void setTotalCompactingKVs(long totalCompactingKVs) {
|
||||||
|
this.totalCompactingKVs = totalCompactingKVs;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param totalStaticBloomSizeKB The total size of all Bloom filter blocks,
|
||||||
|
* not just loaded into the block cache, in KB.
|
||||||
|
*/
|
||||||
|
public void setTotalStaticBloomSizeKB(int totalStaticBloomSizeKB) {
|
||||||
|
this.totalStaticBloomSizeKB = totalStaticBloomSizeKB;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param totalStaticIndexSizeKB The total size of all index blocks,
|
||||||
|
* not just the root level, in KB.
|
||||||
|
*/
|
||||||
|
public void setTotalStaticIndexSizeKB(int totalStaticIndexSizeKB) {
|
||||||
|
this.totalStaticIndexSizeKB = totalStaticIndexSizeKB;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param name the region name
|
* @param name the region name
|
||||||
*/
|
*/
|
||||||
|
@ -243,9 +377,14 @@ public class StorageClusterStatusModel
|
||||||
* @param name the region name
|
* @param name the region name
|
||||||
*/
|
*/
|
||||||
public void addRegion(byte[] name, int stores, int storefiles,
|
public void addRegion(byte[] name, int stores, int storefiles,
|
||||||
int storefileSizeMB, int memstoreSizeMB, int storefileIndexSizeMB) {
|
int storefileSizeMB, int memstoreSizeMB, int storefileIndexSizeMB,
|
||||||
|
long readRequestsCount, long writeRequestsCount, int rootIndexSizeKB,
|
||||||
|
int totalStaticIndexSizeKB, int totalStaticBloomSizeKB,
|
||||||
|
long totalCompactingKVs, long currentCompactedKVs) {
|
||||||
regions.add(new Region(name, stores, storefiles, storefileSizeMB,
|
regions.add(new Region(name, stores, storefiles, storefileSizeMB,
|
||||||
memstoreSizeMB, storefileIndexSizeMB));
|
memstoreSizeMB, storefileIndexSizeMB, readRequestsCount,
|
||||||
|
writeRequestsCount, rootIndexSizeKB, totalStaticIndexSizeKB,
|
||||||
|
totalStaticBloomSizeKB, totalCompactingKVs, currentCompactedKVs));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -532,6 +671,20 @@ public class StorageClusterStatusModel
|
||||||
sb.append(region.memstoreSizeMB);
|
sb.append(region.memstoreSizeMB);
|
||||||
sb.append("\n storefileIndexSizeMB=");
|
sb.append("\n storefileIndexSizeMB=");
|
||||||
sb.append(region.storefileIndexSizeMB);
|
sb.append(region.storefileIndexSizeMB);
|
||||||
|
sb.append("\n readRequestsCount=");
|
||||||
|
sb.append(region.readRequestsCount);
|
||||||
|
sb.append("\n writeRequestsCount=");
|
||||||
|
sb.append(region.writeRequestsCount);
|
||||||
|
sb.append("\n rootIndexSizeKB=");
|
||||||
|
sb.append(region.rootIndexSizeKB);
|
||||||
|
sb.append("\n totalStaticIndexSizeKB=");
|
||||||
|
sb.append(region.totalStaticIndexSizeKB);
|
||||||
|
sb.append("\n totalStaticBloomSizeKB=");
|
||||||
|
sb.append(region.totalStaticBloomSizeKB);
|
||||||
|
sb.append("\n totalCompactingKVs=");
|
||||||
|
sb.append(region.totalCompactingKVs);
|
||||||
|
sb.append("\n currentCompactedKVs=");
|
||||||
|
sb.append(region.currentCompactedKVs);
|
||||||
sb.append('\n');
|
sb.append('\n');
|
||||||
}
|
}
|
||||||
sb.append('\n');
|
sb.append('\n');
|
||||||
|
@ -549,7 +702,7 @@ public class StorageClusterStatusModel
|
||||||
}
|
}
|
||||||
return sb.toString();
|
return sb.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public byte[] createProtobufOutput() {
|
public byte[] createProtobufOutput() {
|
||||||
StorageClusterStatus.Builder builder = StorageClusterStatus.newBuilder();
|
StorageClusterStatus.Builder builder = StorageClusterStatus.newBuilder();
|
||||||
|
@ -573,6 +726,13 @@ public class StorageClusterStatusModel
|
||||||
regionBuilder.setStorefileSizeMB(region.storefileSizeMB);
|
regionBuilder.setStorefileSizeMB(region.storefileSizeMB);
|
||||||
regionBuilder.setMemstoreSizeMB(region.memstoreSizeMB);
|
regionBuilder.setMemstoreSizeMB(region.memstoreSizeMB);
|
||||||
regionBuilder.setStorefileIndexSizeMB(region.storefileIndexSizeMB);
|
regionBuilder.setStorefileIndexSizeMB(region.storefileIndexSizeMB);
|
||||||
|
regionBuilder.setReadRequestsCount(region.readRequestsCount);
|
||||||
|
regionBuilder.setWriteRequestsCount(region.writeRequestsCount);
|
||||||
|
regionBuilder.setRootIndexSizeKB(region.rootIndexSizeKB);
|
||||||
|
regionBuilder.setTotalStaticIndexSizeKB(region.totalStaticIndexSizeKB);
|
||||||
|
regionBuilder.setTotalStaticBloomSizeKB(region.totalStaticBloomSizeKB);
|
||||||
|
regionBuilder.setTotalCompactingKVs(region.totalCompactingKVs);
|
||||||
|
regionBuilder.setCurrentCompactedKVs(region.currentCompactedKVs);
|
||||||
nodeBuilder.addRegions(regionBuilder);
|
nodeBuilder.addRegions(regionBuilder);
|
||||||
}
|
}
|
||||||
builder.addLiveNodes(nodeBuilder);
|
builder.addLiveNodes(nodeBuilder);
|
||||||
|
@ -611,7 +771,14 @@ public class StorageClusterStatusModel
|
||||||
region.getStorefiles(),
|
region.getStorefiles(),
|
||||||
region.getStorefileSizeMB(),
|
region.getStorefileSizeMB(),
|
||||||
region.getMemstoreSizeMB(),
|
region.getMemstoreSizeMB(),
|
||||||
region.getStorefileIndexSizeMB());
|
region.getStorefileIndexSizeMB(),
|
||||||
|
region.getReadRequestsCount(),
|
||||||
|
region.getWriteRequestsCount(),
|
||||||
|
region.getRootIndexSizeKB(),
|
||||||
|
region.getTotalStaticIndexSizeKB(),
|
||||||
|
region.getTotalStaticBloomSizeKB(),
|
||||||
|
region.getTotalCompactingKVs(),
|
||||||
|
region.getCurrentCompactedKVs());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
for (String node: builder.getDeadNodesList()) {
|
for (String node: builder.getDeadNodesList()) {
|
||||||
|
|
|
@ -92,6 +92,34 @@ public final class StorageClusterStatusMessage {
|
||||||
// optional int32 storefileIndexSizeMB = 6;
|
// optional int32 storefileIndexSizeMB = 6;
|
||||||
boolean hasStorefileIndexSizeMB();
|
boolean hasStorefileIndexSizeMB();
|
||||||
int getStorefileIndexSizeMB();
|
int getStorefileIndexSizeMB();
|
||||||
|
|
||||||
|
// optional int64 readRequestsCount = 7;
|
||||||
|
boolean hasReadRequestsCount();
|
||||||
|
long getReadRequestsCount();
|
||||||
|
|
||||||
|
// optional int64 writeRequestsCount = 8;
|
||||||
|
boolean hasWriteRequestsCount();
|
||||||
|
long getWriteRequestsCount();
|
||||||
|
|
||||||
|
// optional int32 rootIndexSizeKB = 9;
|
||||||
|
boolean hasRootIndexSizeKB();
|
||||||
|
int getRootIndexSizeKB();
|
||||||
|
|
||||||
|
// optional int32 totalStaticIndexSizeKB = 10;
|
||||||
|
boolean hasTotalStaticIndexSizeKB();
|
||||||
|
int getTotalStaticIndexSizeKB();
|
||||||
|
|
||||||
|
// optional int32 totalStaticBloomSizeKB = 11;
|
||||||
|
boolean hasTotalStaticBloomSizeKB();
|
||||||
|
int getTotalStaticBloomSizeKB();
|
||||||
|
|
||||||
|
// optional int64 totalCompactingKVs = 12;
|
||||||
|
boolean hasTotalCompactingKVs();
|
||||||
|
long getTotalCompactingKVs();
|
||||||
|
|
||||||
|
// optional int64 currentCompactedKVs = 13;
|
||||||
|
boolean hasCurrentCompactedKVs();
|
||||||
|
long getCurrentCompactedKVs();
|
||||||
}
|
}
|
||||||
public static final class Region extends
|
public static final class Region extends
|
||||||
com.google.protobuf.GeneratedMessage
|
com.google.protobuf.GeneratedMessage
|
||||||
|
@ -182,6 +210,76 @@ public final class StorageClusterStatusMessage {
|
||||||
return storefileIndexSizeMB_;
|
return storefileIndexSizeMB_;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// optional int64 readRequestsCount = 7;
|
||||||
|
public static final int READREQUESTSCOUNT_FIELD_NUMBER = 7;
|
||||||
|
private long readRequestsCount_;
|
||||||
|
public boolean hasReadRequestsCount() {
|
||||||
|
return ((bitField0_ & 0x00000040) == 0x00000040);
|
||||||
|
}
|
||||||
|
public long getReadRequestsCount() {
|
||||||
|
return readRequestsCount_;
|
||||||
|
}
|
||||||
|
|
||||||
|
// optional int64 writeRequestsCount = 8;
|
||||||
|
public static final int WRITEREQUESTSCOUNT_FIELD_NUMBER = 8;
|
||||||
|
private long writeRequestsCount_;
|
||||||
|
public boolean hasWriteRequestsCount() {
|
||||||
|
return ((bitField0_ & 0x00000080) == 0x00000080);
|
||||||
|
}
|
||||||
|
public long getWriteRequestsCount() {
|
||||||
|
return writeRequestsCount_;
|
||||||
|
}
|
||||||
|
|
||||||
|
// optional int32 rootIndexSizeKB = 9;
|
||||||
|
public static final int ROOTINDEXSIZEKB_FIELD_NUMBER = 9;
|
||||||
|
private int rootIndexSizeKB_;
|
||||||
|
public boolean hasRootIndexSizeKB() {
|
||||||
|
return ((bitField0_ & 0x00000100) == 0x00000100);
|
||||||
|
}
|
||||||
|
public int getRootIndexSizeKB() {
|
||||||
|
return rootIndexSizeKB_;
|
||||||
|
}
|
||||||
|
|
||||||
|
// optional int32 totalStaticIndexSizeKB = 10;
|
||||||
|
public static final int TOTALSTATICINDEXSIZEKB_FIELD_NUMBER = 10;
|
||||||
|
private int totalStaticIndexSizeKB_;
|
||||||
|
public boolean hasTotalStaticIndexSizeKB() {
|
||||||
|
return ((bitField0_ & 0x00000200) == 0x00000200);
|
||||||
|
}
|
||||||
|
public int getTotalStaticIndexSizeKB() {
|
||||||
|
return totalStaticIndexSizeKB_;
|
||||||
|
}
|
||||||
|
|
||||||
|
// optional int32 totalStaticBloomSizeKB = 11;
|
||||||
|
public static final int TOTALSTATICBLOOMSIZEKB_FIELD_NUMBER = 11;
|
||||||
|
private int totalStaticBloomSizeKB_;
|
||||||
|
public boolean hasTotalStaticBloomSizeKB() {
|
||||||
|
return ((bitField0_ & 0x00000400) == 0x00000400);
|
||||||
|
}
|
||||||
|
public int getTotalStaticBloomSizeKB() {
|
||||||
|
return totalStaticBloomSizeKB_;
|
||||||
|
}
|
||||||
|
|
||||||
|
// optional int64 totalCompactingKVs = 12;
|
||||||
|
public static final int TOTALCOMPACTINGKVS_FIELD_NUMBER = 12;
|
||||||
|
private long totalCompactingKVs_;
|
||||||
|
public boolean hasTotalCompactingKVs() {
|
||||||
|
return ((bitField0_ & 0x00000800) == 0x00000800);
|
||||||
|
}
|
||||||
|
public long getTotalCompactingKVs() {
|
||||||
|
return totalCompactingKVs_;
|
||||||
|
}
|
||||||
|
|
||||||
|
// optional int64 currentCompactedKVs = 13;
|
||||||
|
public static final int CURRENTCOMPACTEDKVS_FIELD_NUMBER = 13;
|
||||||
|
private long currentCompactedKVs_;
|
||||||
|
public boolean hasCurrentCompactedKVs() {
|
||||||
|
return ((bitField0_ & 0x00001000) == 0x00001000);
|
||||||
|
}
|
||||||
|
public long getCurrentCompactedKVs() {
|
||||||
|
return currentCompactedKVs_;
|
||||||
|
}
|
||||||
|
|
||||||
private void initFields() {
|
private void initFields() {
|
||||||
name_ = com.google.protobuf.ByteString.EMPTY;
|
name_ = com.google.protobuf.ByteString.EMPTY;
|
||||||
stores_ = 0;
|
stores_ = 0;
|
||||||
|
@ -189,6 +287,13 @@ public final class StorageClusterStatusMessage {
|
||||||
storefileSizeMB_ = 0;
|
storefileSizeMB_ = 0;
|
||||||
memstoreSizeMB_ = 0;
|
memstoreSizeMB_ = 0;
|
||||||
storefileIndexSizeMB_ = 0;
|
storefileIndexSizeMB_ = 0;
|
||||||
|
readRequestsCount_ = 0L;
|
||||||
|
writeRequestsCount_ = 0L;
|
||||||
|
rootIndexSizeKB_ = 0;
|
||||||
|
totalStaticIndexSizeKB_ = 0;
|
||||||
|
totalStaticBloomSizeKB_ = 0;
|
||||||
|
totalCompactingKVs_ = 0L;
|
||||||
|
currentCompactedKVs_ = 0L;
|
||||||
}
|
}
|
||||||
private byte memoizedIsInitialized = -1;
|
private byte memoizedIsInitialized = -1;
|
||||||
public final boolean isInitialized() {
|
public final boolean isInitialized() {
|
||||||
|
@ -224,6 +329,27 @@ public final class StorageClusterStatusMessage {
|
||||||
if (((bitField0_ & 0x00000020) == 0x00000020)) {
|
if (((bitField0_ & 0x00000020) == 0x00000020)) {
|
||||||
output.writeInt32(6, storefileIndexSizeMB_);
|
output.writeInt32(6, storefileIndexSizeMB_);
|
||||||
}
|
}
|
||||||
|
if (((bitField0_ & 0x00000040) == 0x00000040)) {
|
||||||
|
output.writeInt64(7, readRequestsCount_);
|
||||||
|
}
|
||||||
|
if (((bitField0_ & 0x00000080) == 0x00000080)) {
|
||||||
|
output.writeInt64(8, writeRequestsCount_);
|
||||||
|
}
|
||||||
|
if (((bitField0_ & 0x00000100) == 0x00000100)) {
|
||||||
|
output.writeInt32(9, rootIndexSizeKB_);
|
||||||
|
}
|
||||||
|
if (((bitField0_ & 0x00000200) == 0x00000200)) {
|
||||||
|
output.writeInt32(10, totalStaticIndexSizeKB_);
|
||||||
|
}
|
||||||
|
if (((bitField0_ & 0x00000400) == 0x00000400)) {
|
||||||
|
output.writeInt32(11, totalStaticBloomSizeKB_);
|
||||||
|
}
|
||||||
|
if (((bitField0_ & 0x00000800) == 0x00000800)) {
|
||||||
|
output.writeInt64(12, totalCompactingKVs_);
|
||||||
|
}
|
||||||
|
if (((bitField0_ & 0x00001000) == 0x00001000)) {
|
||||||
|
output.writeInt64(13, currentCompactedKVs_);
|
||||||
|
}
|
||||||
getUnknownFields().writeTo(output);
|
getUnknownFields().writeTo(output);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -257,6 +383,34 @@ public final class StorageClusterStatusMessage {
|
||||||
size += com.google.protobuf.CodedOutputStream
|
size += com.google.protobuf.CodedOutputStream
|
||||||
.computeInt32Size(6, storefileIndexSizeMB_);
|
.computeInt32Size(6, storefileIndexSizeMB_);
|
||||||
}
|
}
|
||||||
|
if (((bitField0_ & 0x00000040) == 0x00000040)) {
|
||||||
|
size += com.google.protobuf.CodedOutputStream
|
||||||
|
.computeInt64Size(7, readRequestsCount_);
|
||||||
|
}
|
||||||
|
if (((bitField0_ & 0x00000080) == 0x00000080)) {
|
||||||
|
size += com.google.protobuf.CodedOutputStream
|
||||||
|
.computeInt64Size(8, writeRequestsCount_);
|
||||||
|
}
|
||||||
|
if (((bitField0_ & 0x00000100) == 0x00000100)) {
|
||||||
|
size += com.google.protobuf.CodedOutputStream
|
||||||
|
.computeInt32Size(9, rootIndexSizeKB_);
|
||||||
|
}
|
||||||
|
if (((bitField0_ & 0x00000200) == 0x00000200)) {
|
||||||
|
size += com.google.protobuf.CodedOutputStream
|
||||||
|
.computeInt32Size(10, totalStaticIndexSizeKB_);
|
||||||
|
}
|
||||||
|
if (((bitField0_ & 0x00000400) == 0x00000400)) {
|
||||||
|
size += com.google.protobuf.CodedOutputStream
|
||||||
|
.computeInt32Size(11, totalStaticBloomSizeKB_);
|
||||||
|
}
|
||||||
|
if (((bitField0_ & 0x00000800) == 0x00000800)) {
|
||||||
|
size += com.google.protobuf.CodedOutputStream
|
||||||
|
.computeInt64Size(12, totalCompactingKVs_);
|
||||||
|
}
|
||||||
|
if (((bitField0_ & 0x00001000) == 0x00001000)) {
|
||||||
|
size += com.google.protobuf.CodedOutputStream
|
||||||
|
.computeInt64Size(13, currentCompactedKVs_);
|
||||||
|
}
|
||||||
size += getUnknownFields().getSerializedSize();
|
size += getUnknownFields().getSerializedSize();
|
||||||
memoizedSerializedSize = size;
|
memoizedSerializedSize = size;
|
||||||
return size;
|
return size;
|
||||||
|
@ -393,6 +547,20 @@ public final class StorageClusterStatusMessage {
|
||||||
bitField0_ = (bitField0_ & ~0x00000010);
|
bitField0_ = (bitField0_ & ~0x00000010);
|
||||||
storefileIndexSizeMB_ = 0;
|
storefileIndexSizeMB_ = 0;
|
||||||
bitField0_ = (bitField0_ & ~0x00000020);
|
bitField0_ = (bitField0_ & ~0x00000020);
|
||||||
|
readRequestsCount_ = 0L;
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000040);
|
||||||
|
writeRequestsCount_ = 0L;
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000080);
|
||||||
|
rootIndexSizeKB_ = 0;
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000100);
|
||||||
|
totalStaticIndexSizeKB_ = 0;
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000200);
|
||||||
|
totalStaticBloomSizeKB_ = 0;
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000400);
|
||||||
|
totalCompactingKVs_ = 0L;
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000800);
|
||||||
|
currentCompactedKVs_ = 0L;
|
||||||
|
bitField0_ = (bitField0_ & ~0x00001000);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -455,6 +623,34 @@ public final class StorageClusterStatusMessage {
|
||||||
to_bitField0_ |= 0x00000020;
|
to_bitField0_ |= 0x00000020;
|
||||||
}
|
}
|
||||||
result.storefileIndexSizeMB_ = storefileIndexSizeMB_;
|
result.storefileIndexSizeMB_ = storefileIndexSizeMB_;
|
||||||
|
if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
|
||||||
|
to_bitField0_ |= 0x00000040;
|
||||||
|
}
|
||||||
|
result.readRequestsCount_ = readRequestsCount_;
|
||||||
|
if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
|
||||||
|
to_bitField0_ |= 0x00000080;
|
||||||
|
}
|
||||||
|
result.writeRequestsCount_ = writeRequestsCount_;
|
||||||
|
if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
|
||||||
|
to_bitField0_ |= 0x00000100;
|
||||||
|
}
|
||||||
|
result.rootIndexSizeKB_ = rootIndexSizeKB_;
|
||||||
|
if (((from_bitField0_ & 0x00000200) == 0x00000200)) {
|
||||||
|
to_bitField0_ |= 0x00000200;
|
||||||
|
}
|
||||||
|
result.totalStaticIndexSizeKB_ = totalStaticIndexSizeKB_;
|
||||||
|
if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
|
||||||
|
to_bitField0_ |= 0x00000400;
|
||||||
|
}
|
||||||
|
result.totalStaticBloomSizeKB_ = totalStaticBloomSizeKB_;
|
||||||
|
if (((from_bitField0_ & 0x00000800) == 0x00000800)) {
|
||||||
|
to_bitField0_ |= 0x00000800;
|
||||||
|
}
|
||||||
|
result.totalCompactingKVs_ = totalCompactingKVs_;
|
||||||
|
if (((from_bitField0_ & 0x00001000) == 0x00001000)) {
|
||||||
|
to_bitField0_ |= 0x00001000;
|
||||||
|
}
|
||||||
|
result.currentCompactedKVs_ = currentCompactedKVs_;
|
||||||
result.bitField0_ = to_bitField0_;
|
result.bitField0_ = to_bitField0_;
|
||||||
onBuilt();
|
onBuilt();
|
||||||
return result;
|
return result;
|
||||||
|
@ -489,6 +685,27 @@ public final class StorageClusterStatusMessage {
|
||||||
if (other.hasStorefileIndexSizeMB()) {
|
if (other.hasStorefileIndexSizeMB()) {
|
||||||
setStorefileIndexSizeMB(other.getStorefileIndexSizeMB());
|
setStorefileIndexSizeMB(other.getStorefileIndexSizeMB());
|
||||||
}
|
}
|
||||||
|
if (other.hasReadRequestsCount()) {
|
||||||
|
setReadRequestsCount(other.getReadRequestsCount());
|
||||||
|
}
|
||||||
|
if (other.hasWriteRequestsCount()) {
|
||||||
|
setWriteRequestsCount(other.getWriteRequestsCount());
|
||||||
|
}
|
||||||
|
if (other.hasRootIndexSizeKB()) {
|
||||||
|
setRootIndexSizeKB(other.getRootIndexSizeKB());
|
||||||
|
}
|
||||||
|
if (other.hasTotalStaticIndexSizeKB()) {
|
||||||
|
setTotalStaticIndexSizeKB(other.getTotalStaticIndexSizeKB());
|
||||||
|
}
|
||||||
|
if (other.hasTotalStaticBloomSizeKB()) {
|
||||||
|
setTotalStaticBloomSizeKB(other.getTotalStaticBloomSizeKB());
|
||||||
|
}
|
||||||
|
if (other.hasTotalCompactingKVs()) {
|
||||||
|
setTotalCompactingKVs(other.getTotalCompactingKVs());
|
||||||
|
}
|
||||||
|
if (other.hasCurrentCompactedKVs()) {
|
||||||
|
setCurrentCompactedKVs(other.getCurrentCompactedKVs());
|
||||||
|
}
|
||||||
this.mergeUnknownFields(other.getUnknownFields());
|
this.mergeUnknownFields(other.getUnknownFields());
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
@ -554,6 +771,41 @@ public final class StorageClusterStatusMessage {
|
||||||
storefileIndexSizeMB_ = input.readInt32();
|
storefileIndexSizeMB_ = input.readInt32();
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
case 56: {
|
||||||
|
bitField0_ |= 0x00000040;
|
||||||
|
readRequestsCount_ = input.readInt64();
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case 64: {
|
||||||
|
bitField0_ |= 0x00000080;
|
||||||
|
writeRequestsCount_ = input.readInt64();
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case 72: {
|
||||||
|
bitField0_ |= 0x00000100;
|
||||||
|
rootIndexSizeKB_ = input.readInt32();
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case 80: {
|
||||||
|
bitField0_ |= 0x00000200;
|
||||||
|
totalStaticIndexSizeKB_ = input.readInt32();
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case 88: {
|
||||||
|
bitField0_ |= 0x00000400;
|
||||||
|
totalStaticBloomSizeKB_ = input.readInt32();
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case 96: {
|
||||||
|
bitField0_ |= 0x00000800;
|
||||||
|
totalCompactingKVs_ = input.readInt64();
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case 104: {
|
||||||
|
bitField0_ |= 0x00001000;
|
||||||
|
currentCompactedKVs_ = input.readInt64();
|
||||||
|
break;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -689,6 +941,153 @@ public final class StorageClusterStatusMessage {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// optional int64 readRequestsCount = 7;
|
||||||
|
private long readRequestsCount_ ;
|
||||||
|
public boolean hasReadRequestsCount() {
|
||||||
|
return ((bitField0_ & 0x00000040) == 0x00000040);
|
||||||
|
}
|
||||||
|
public long getReadRequestsCount() {
|
||||||
|
return readRequestsCount_;
|
||||||
|
}
|
||||||
|
public Builder setReadRequestsCount(long value) {
|
||||||
|
bitField0_ |= 0x00000040;
|
||||||
|
readRequestsCount_ = value;
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public Builder clearReadRequestsCount() {
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000040);
|
||||||
|
readRequestsCount_ = 0L;
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
// optional int64 writeRequestsCount = 8;
|
||||||
|
private long writeRequestsCount_ ;
|
||||||
|
public boolean hasWriteRequestsCount() {
|
||||||
|
return ((bitField0_ & 0x00000080) == 0x00000080);
|
||||||
|
}
|
||||||
|
public long getWriteRequestsCount() {
|
||||||
|
return writeRequestsCount_;
|
||||||
|
}
|
||||||
|
public Builder setWriteRequestsCount(long value) {
|
||||||
|
bitField0_ |= 0x00000080;
|
||||||
|
writeRequestsCount_ = value;
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public Builder clearWriteRequestsCount() {
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000080);
|
||||||
|
writeRequestsCount_ = 0L;
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
// optional int32 rootIndexSizeKB = 9;
|
||||||
|
private int rootIndexSizeKB_ ;
|
||||||
|
public boolean hasRootIndexSizeKB() {
|
||||||
|
return ((bitField0_ & 0x00000100) == 0x00000100);
|
||||||
|
}
|
||||||
|
public int getRootIndexSizeKB() {
|
||||||
|
return rootIndexSizeKB_;
|
||||||
|
}
|
||||||
|
public Builder setRootIndexSizeKB(int value) {
|
||||||
|
bitField0_ |= 0x00000100;
|
||||||
|
rootIndexSizeKB_ = value;
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public Builder clearRootIndexSizeKB() {
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000100);
|
||||||
|
rootIndexSizeKB_ = 0;
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
// optional int32 totalStaticIndexSizeKB = 10;
|
||||||
|
private int totalStaticIndexSizeKB_ ;
|
||||||
|
public boolean hasTotalStaticIndexSizeKB() {
|
||||||
|
return ((bitField0_ & 0x00000200) == 0x00000200);
|
||||||
|
}
|
||||||
|
public int getTotalStaticIndexSizeKB() {
|
||||||
|
return totalStaticIndexSizeKB_;
|
||||||
|
}
|
||||||
|
public Builder setTotalStaticIndexSizeKB(int value) {
|
||||||
|
bitField0_ |= 0x00000200;
|
||||||
|
totalStaticIndexSizeKB_ = value;
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public Builder clearTotalStaticIndexSizeKB() {
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000200);
|
||||||
|
totalStaticIndexSizeKB_ = 0;
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
// optional int32 totalStaticBloomSizeKB = 11;
|
||||||
|
private int totalStaticBloomSizeKB_ ;
|
||||||
|
public boolean hasTotalStaticBloomSizeKB() {
|
||||||
|
return ((bitField0_ & 0x00000400) == 0x00000400);
|
||||||
|
}
|
||||||
|
public int getTotalStaticBloomSizeKB() {
|
||||||
|
return totalStaticBloomSizeKB_;
|
||||||
|
}
|
||||||
|
public Builder setTotalStaticBloomSizeKB(int value) {
|
||||||
|
bitField0_ |= 0x00000400;
|
||||||
|
totalStaticBloomSizeKB_ = value;
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public Builder clearTotalStaticBloomSizeKB() {
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000400);
|
||||||
|
totalStaticBloomSizeKB_ = 0;
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
// optional int64 totalCompactingKVs = 12;
|
||||||
|
private long totalCompactingKVs_ ;
|
||||||
|
public boolean hasTotalCompactingKVs() {
|
||||||
|
return ((bitField0_ & 0x00000800) == 0x00000800);
|
||||||
|
}
|
||||||
|
public long getTotalCompactingKVs() {
|
||||||
|
return totalCompactingKVs_;
|
||||||
|
}
|
||||||
|
public Builder setTotalCompactingKVs(long value) {
|
||||||
|
bitField0_ |= 0x00000800;
|
||||||
|
totalCompactingKVs_ = value;
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public Builder clearTotalCompactingKVs() {
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000800);
|
||||||
|
totalCompactingKVs_ = 0L;
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
// optional int64 currentCompactedKVs = 13;
|
||||||
|
private long currentCompactedKVs_ ;
|
||||||
|
public boolean hasCurrentCompactedKVs() {
|
||||||
|
return ((bitField0_ & 0x00001000) == 0x00001000);
|
||||||
|
}
|
||||||
|
public long getCurrentCompactedKVs() {
|
||||||
|
return currentCompactedKVs_;
|
||||||
|
}
|
||||||
|
public Builder setCurrentCompactedKVs(long value) {
|
||||||
|
bitField0_ |= 0x00001000;
|
||||||
|
currentCompactedKVs_ = value;
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public Builder clearCurrentCompactedKVs() {
|
||||||
|
bitField0_ = (bitField0_ & ~0x00001000);
|
||||||
|
currentCompactedKVs_ = 0L;
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
// @@protoc_insertion_point(builder_scope:org.apache.hadoop.hbase.rest.protobuf.generated.StorageClusterStatus.Region)
|
// @@protoc_insertion_point(builder_scope:org.apache.hadoop.hbase.rest.protobuf.generated.StorageClusterStatus.Region)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2412,20 +2811,25 @@ public final class StorageClusterStatusMessage {
|
||||||
java.lang.String[] descriptorData = {
|
java.lang.String[] descriptorData = {
|
||||||
"\n!StorageClusterStatusMessage.proto\022/org" +
|
"\n!StorageClusterStatusMessage.proto\022/org" +
|
||||||
".apache.hadoop.hbase.rest.protobuf.gener" +
|
".apache.hadoop.hbase.rest.protobuf.gener" +
|
||||||
"ated\"\222\004\n\024StorageClusterStatus\022]\n\tliveNod" +
|
"ated\"\333\005\n\024StorageClusterStatus\022]\n\tliveNod" +
|
||||||
"es\030\001 \003(\0132J.org.apache.hadoop.hbase.rest." +
|
"es\030\001 \003(\0132J.org.apache.hadoop.hbase.rest." +
|
||||||
"protobuf.generated.StorageClusterStatus." +
|
"protobuf.generated.StorageClusterStatus." +
|
||||||
"Node\022\021\n\tdeadNodes\030\002 \003(\t\022\017\n\007regions\030\003 \001(\005" +
|
"Node\022\021\n\tdeadNodes\030\002 \003(\t\022\017\n\007regions\030\003 \001(\005" +
|
||||||
"\022\020\n\010requests\030\004 \001(\005\022\023\n\013averageLoad\030\005 \001(\001\032" +
|
"\022\020\n\010requests\030\004 \001(\005\022\023\n\013averageLoad\030\005 \001(\001\032" +
|
||||||
"\211\001\n\006Region\022\014\n\004name\030\001 \002(\014\022\016\n\006stores\030\002 \001(\005" +
|
"\322\002\n\006Region\022\014\n\004name\030\001 \002(\014\022\016\n\006stores\030\002 \001(\005" +
|
||||||
"\022\022\n\nstorefiles\030\003 \001(\005\022\027\n\017storefileSizeMB\030" +
|
"\022\022\n\nstorefiles\030\003 \001(\005\022\027\n\017storefileSizeMB\030" +
|
||||||
"\004 \001(\005\022\026\n\016memstoreSizeMB\030\005 \001(\005\022\034\n\024storefi",
|
"\004 \001(\005\022\026\n\016memstoreSizeMB\030\005 \001(\005\022\034\n\024storefi",
|
||||||
"leIndexSizeMB\030\006 \001(\005\032\303\001\n\004Node\022\014\n\004name\030\001 \002" +
|
"leIndexSizeMB\030\006 \001(\005\022\031\n\021readRequestsCount" +
|
||||||
"(\t\022\021\n\tstartCode\030\002 \001(\003\022\020\n\010requests\030\003 \001(\005\022" +
|
"\030\007 \001(\003\022\032\n\022writeRequestsCount\030\010 \001(\003\022\027\n\017ro" +
|
||||||
"\022\n\nheapSizeMB\030\004 \001(\005\022\025\n\rmaxHeapSizeMB\030\005 \001" +
|
"otIndexSizeKB\030\t \001(\005\022\036\n\026totalStaticIndexS" +
|
||||||
"(\005\022]\n\007regions\030\006 \003(\0132L.org.apache.hadoop." +
|
"izeKB\030\n \001(\005\022\036\n\026totalStaticBloomSizeKB\030\013 " +
|
||||||
"hbase.rest.protobuf.generated.StorageClu" +
|
"\001(\005\022\032\n\022totalCompactingKVs\030\014 \001(\003\022\033\n\023curre" +
|
||||||
"sterStatus.Region"
|
"ntCompactedKVs\030\r \001(\003\032\303\001\n\004Node\022\014\n\004name\030\001 " +
|
||||||
|
"\002(\t\022\021\n\tstartCode\030\002 \001(\003\022\020\n\010requests\030\003 \001(\005" +
|
||||||
|
"\022\022\n\nheapSizeMB\030\004 \001(\005\022\025\n\rmaxHeapSizeMB\030\005 " +
|
||||||
|
"\001(\005\022]\n\007regions\030\006 \003(\0132L.org.apache.hadoop" +
|
||||||
|
".hbase.rest.protobuf.generated.StorageCl",
|
||||||
|
"usterStatus.Region"
|
||||||
};
|
};
|
||||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||||
|
@ -2445,7 +2849,7 @@ public final class StorageClusterStatusMessage {
|
||||||
internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_StorageClusterStatus_Region_fieldAccessorTable = new
|
internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_StorageClusterStatus_Region_fieldAccessorTable = new
|
||||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||||
internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_StorageClusterStatus_Region_descriptor,
|
internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_StorageClusterStatus_Region_descriptor,
|
||||||
new java.lang.String[] { "Name", "Stores", "Storefiles", "StorefileSizeMB", "MemstoreSizeMB", "StorefileIndexSizeMB", },
|
new java.lang.String[] { "Name", "Stores", "Storefiles", "StorefileSizeMB", "MemstoreSizeMB", "StorefileIndexSizeMB", "ReadRequestsCount", "WriteRequestsCount", "RootIndexSizeKB", "TotalStaticIndexSizeKB", "TotalStaticBloomSizeKB", "TotalCompactingKVs", "CurrentCompactedKVs", },
|
||||||
org.apache.hadoop.hbase.rest.protobuf.generated.StorageClusterStatusMessage.StorageClusterStatus.Region.class,
|
org.apache.hadoop.hbase.rest.protobuf.generated.StorageClusterStatusMessage.StorageClusterStatus.Region.class,
|
||||||
org.apache.hadoop.hbase.rest.protobuf.generated.StorageClusterStatusMessage.StorageClusterStatus.Region.Builder.class);
|
org.apache.hadoop.hbase.rest.protobuf.generated.StorageClusterStatusMessage.StorageClusterStatus.Region.Builder.class);
|
||||||
internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_StorageClusterStatus_Node_descriptor =
|
internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_StorageClusterStatus_Node_descriptor =
|
||||||
|
|
|
@ -166,6 +166,13 @@
|
||||||
<attribute name="storefileSizeMB" type="int"></attribute>
|
<attribute name="storefileSizeMB" type="int"></attribute>
|
||||||
<attribute name="memstoreSizeMB" type="int"></attribute>
|
<attribute name="memstoreSizeMB" type="int"></attribute>
|
||||||
<attribute name="storefileIndexSizeMB" type="int"></attribute>
|
<attribute name="storefileIndexSizeMB" type="int"></attribute>
|
||||||
|
<attribute name="readRequestsCount" type="int"></attribute>
|
||||||
|
<attribute name="writeRequestsCount" type="int"></attribute>
|
||||||
|
<attribute name="rootIndexSizeKB" type="int"></attribute>
|
||||||
|
<attribute name="totalStaticIndexSizeKB" type="int"></attribute>
|
||||||
|
<attribute name="totalStaticBloomSizeKB" type="int"></attribute>
|
||||||
|
<attribute name="totalCompactingKVs" type="int"></attribute>
|
||||||
|
<attribute name="currentCompactedKVs" type="int"></attribute>
|
||||||
</complexType>
|
</complexType>
|
||||||
|
|
||||||
</schema>
|
</schema>
|
||||||
|
|
|
@ -26,6 +26,13 @@ message StorageClusterStatus {
|
||||||
optional int32 storefileSizeMB = 4;
|
optional int32 storefileSizeMB = 4;
|
||||||
optional int32 memstoreSizeMB = 5;
|
optional int32 memstoreSizeMB = 5;
|
||||||
optional int32 storefileIndexSizeMB = 6;
|
optional int32 storefileIndexSizeMB = 6;
|
||||||
|
optional int64 readRequestsCount = 7;
|
||||||
|
optional int64 writeRequestsCount = 8;
|
||||||
|
optional int32 rootIndexSizeKB = 9;
|
||||||
|
optional int32 totalStaticIndexSizeKB = 10;
|
||||||
|
optional int32 totalStaticBloomSizeKB = 11;
|
||||||
|
optional int64 totalCompactingKVs = 12;
|
||||||
|
optional int64 currentCompactedKVs = 13;
|
||||||
}
|
}
|
||||||
message Node {
|
message Node {
|
||||||
required string name = 1; // name:port
|
required string name = 1; // name:port
|
||||||
|
|
|
@ -45,19 +45,25 @@ public class TestStorageClusterStatusModel extends TestCase {
|
||||||
" name=\"test1\" maxHeapSizeMB=\"1024\" heapSizeMB=\"128\">" +
|
" name=\"test1\" maxHeapSizeMB=\"1024\" heapSizeMB=\"128\">" +
|
||||||
"<Region stores=\"1\" storefiles=\"1\" storefileSizeMB=\"0\"" +
|
"<Region stores=\"1\" storefiles=\"1\" storefileSizeMB=\"0\"" +
|
||||||
" storefileIndexSizeMB=\"0\" name=\"LVJPT1QtLCww\"" +
|
" storefileIndexSizeMB=\"0\" name=\"LVJPT1QtLCww\"" +
|
||||||
" memstoreSizeMB=\"0\"/></Node>" +
|
" memstoreSizeMB=\"0\" readRequestsCount=\"1\"" +
|
||||||
|
" writeRequestsCount=\"2\" rootIndexSizeKB=\"1\"" +
|
||||||
|
" totalStaticIndexSizeKB=\"1\" totalStaticBloomSizeKB=\"1\"" +
|
||||||
|
" totalCompactingKVs=\"1\" currentCompactedKVs=\"1\"/></Node>" +
|
||||||
"<Node startCode=\"1245239331198\" requests=\"0\" name=\"test2\"" +
|
"<Node startCode=\"1245239331198\" requests=\"0\" name=\"test2\"" +
|
||||||
" maxHeapSizeMB=\"1024\" heapSizeMB=\"512\">" +
|
" maxHeapSizeMB=\"1024\" heapSizeMB=\"512\">" +
|
||||||
"<Region stores=\"1\" storefiles=\"1\" storefileSizeMB=\"0\"" +
|
"<Region stores=\"1\" storefiles=\"1\" storefileSizeMB=\"0\"" +
|
||||||
" storefileIndexSizeMB=\"0\" name=\"Lk1FVEEuLCwxMjQ2MDAwMDQzNzI0\"" +
|
" storefileIndexSizeMB=\"0\" name=\"Lk1FVEEuLCwxMjQ2MDAwMDQzNzI0\"" +
|
||||||
" memstoreSizeMB=\"0\"/></Node>"+
|
" memstoreSizeMB=\"0\" readRequestsCount=\"1\"" +
|
||||||
|
" writeRequestsCount=\"2\" rootIndexSizeKB=\"1\"" +
|
||||||
|
" totalStaticIndexSizeKB=\"1\" totalStaticBloomSizeKB=\"1\"" +
|
||||||
|
" totalCompactingKVs=\"1\" currentCompactedKVs=\"1\"/></Node>"+
|
||||||
"</LiveNodes></ClusterStatus>";
|
"</LiveNodes></ClusterStatus>";
|
||||||
|
|
||||||
private static final String AS_PB =
|
private static final String AS_PB =
|
||||||
"Ci0KBXRlc3QxEOO6i+eeJBgAIIABKIAIMhUKCS1ST09ULSwsMBABGAEgACgAMAAKOQoFdGVzdDIQ"+
|
"CjsKBXRlc3QxEOO6i+eeJBgAIIABKIAIMiMKCS1ST09ULSwsMBABGAEgACgAMAA4AUACSAFQAVgB" +
|
||||||
"/pKx8J4kGAAggAQogAgyIQoVLk1FVEEuLCwxMjQ2MDAwMDQzNzI0EAEYASAAKAAwABgCIAApAAAA"+
|
"YAFoAQpHCgV0ZXN0MhD+krHwniQYACCABCiACDIvChUuTUVUQS4sLDEyNDYwMDAwNDM3MjQQARgB" +
|
||||||
"AAAA8D8=";
|
"IAAoADAAOAFAAkgBUAFYAWABaAEYAiAAKQAAAAAAAPA/";
|
||||||
|
|
||||||
private JAXBContext context;
|
private JAXBContext context;
|
||||||
|
|
||||||
public TestStorageClusterStatusModel() throws JAXBException {
|
public TestStorageClusterStatusModel() throws JAXBException {
|
||||||
|
@ -71,9 +77,10 @@ public class TestStorageClusterStatusModel extends TestCase {
|
||||||
model.setRequests(0);
|
model.setRequests(0);
|
||||||
model.setAverageLoad(1.0);
|
model.setAverageLoad(1.0);
|
||||||
model.addLiveNode("test1", 1245219839331L, 128, 1024)
|
model.addLiveNode("test1", 1245219839331L, 128, 1024)
|
||||||
.addRegion(Bytes.toBytes("-ROOT-,,0"), 1, 1, 0, 0, 0);
|
.addRegion(Bytes.toBytes("-ROOT-,,0"), 1, 1, 0, 0, 0, 1, 2, 1, 1, 1, 1, 1);
|
||||||
model.addLiveNode("test2", 1245239331198L, 512, 1024)
|
model.addLiveNode("test2", 1245239331198L, 512, 1024)
|
||||||
.addRegion(Bytes.toBytes(".META.,,1246000043724"),1, 1, 0, 0, 0);
|
.addRegion(Bytes.toBytes(".META.,,1246000043724"),1, 1, 0, 0, 0,
|
||||||
|
1, 2, 1, 1, 1, 1, 1);
|
||||||
return model;
|
return model;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -119,6 +126,13 @@ public class TestStorageClusterStatusModel extends TestCase {
|
||||||
assertEquals(region.getStorefileSizeMB(), 0);
|
assertEquals(region.getStorefileSizeMB(), 0);
|
||||||
assertEquals(region.getMemstoreSizeMB(), 0);
|
assertEquals(region.getMemstoreSizeMB(), 0);
|
||||||
assertEquals(region.getStorefileIndexSizeMB(), 0);
|
assertEquals(region.getStorefileIndexSizeMB(), 0);
|
||||||
|
assertEquals(region.getReadRequestsCount(), 1);
|
||||||
|
assertEquals(region.getWriteRequestsCount(), 2);
|
||||||
|
assertEquals(region.getRootIndexSizeKB(), 1);
|
||||||
|
assertEquals(region.getTotalStaticIndexSizeKB(), 1);
|
||||||
|
assertEquals(region.getTotalStaticBloomSizeKB(), 1);
|
||||||
|
assertEquals(region.getTotalCompactingKVs(), 1);
|
||||||
|
assertEquals(region.getCurrentCompactedKVs(), 1);
|
||||||
assertFalse(regions.hasNext());
|
assertFalse(regions.hasNext());
|
||||||
node = nodes.next();
|
node = nodes.next();
|
||||||
assertEquals(node.getName(), "test2");
|
assertEquals(node.getName(), "test2");
|
||||||
|
@ -133,6 +147,14 @@ public class TestStorageClusterStatusModel extends TestCase {
|
||||||
assertEquals(region.getStorefileSizeMB(), 0);
|
assertEquals(region.getStorefileSizeMB(), 0);
|
||||||
assertEquals(region.getMemstoreSizeMB(), 0);
|
assertEquals(region.getMemstoreSizeMB(), 0);
|
||||||
assertEquals(region.getStorefileIndexSizeMB(), 0);
|
assertEquals(region.getStorefileIndexSizeMB(), 0);
|
||||||
|
assertEquals(region.getReadRequestsCount(), 1);
|
||||||
|
assertEquals(region.getWriteRequestsCount(), 2);
|
||||||
|
assertEquals(region.getRootIndexSizeKB(), 1);
|
||||||
|
assertEquals(region.getTotalStaticIndexSizeKB(), 1);
|
||||||
|
assertEquals(region.getTotalStaticBloomSizeKB(), 1);
|
||||||
|
assertEquals(region.getTotalCompactingKVs(), 1);
|
||||||
|
assertEquals(region.getCurrentCompactedKVs(), 1);
|
||||||
|
|
||||||
assertFalse(regions.hasNext());
|
assertFalse(regions.hasNext());
|
||||||
assertFalse(nodes.hasNext());
|
assertFalse(nodes.hasNext());
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue