HBASE-14976 Add RPC call queues to the web ui (Pallavi Adusumilli)

This commit is contained in:
Mikhail Antonov 2015-12-19 12:44:30 -08:00
parent b68542e35a
commit 8c921ea94f
3 changed files with 15 additions and 2 deletions

View File

@ -104,7 +104,8 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
<section>
<h2>Server Metrics</h2>
<& ServerMetricsTmpl; mWrap = regionServer.getRegionServerMetrics().getRegionServerWrapper(); &>
<& ServerMetricsTmpl; mWrap = regionServer.getRegionServerMetrics().getRegionServerWrapper();
mServerWrap = regionServer.getRpcServer().getMetrics().getHBaseServerWrapper(); &>
</section>
<section>

View File

@ -18,10 +18,12 @@ limitations under the License.
</%doc>
<%args>
MetricsRegionServerWrapper mWrap;
MetricsHBaseServerWrapper mServerWrap;
</%args>
<%import>
java.util.*;
org.apache.hadoop.hbase.regionserver.HRegionServer;
org.apache.hadoop.hbase.ipc.MetricsHBaseServerWrapper;
org.apache.hadoop.hbase.regionserver.MetricsRegionServerWrapper;
org.apache.hadoop.hbase.util.Bytes;
org.apache.hadoop.hbase.HRegionInfo;
@ -61,7 +63,7 @@ java.lang.management.ManagementFactory;
<& storeStats; mWrap = mWrap &>
</div>
<div class="tab-pane" id="tab_queueStats">
<& queueStats; mWrap = mWrap &>
<& queueStats; mWrap = mWrap; mServerWrap = mServerWrap; &>
</div>
</div>
</div>
@ -184,16 +186,20 @@ MetricsRegionServerWrapper mWrap;
<%def queueStats>
<%args>
MetricsRegionServerWrapper mWrap;
MetricsHBaseServerWrapper mServerWrap;
</%args>
<table class="table table-striped">
<tr>
<th>Compaction Queue Size</th>
<th>Flush Queue Size</th>
<th>Call Queue Size (bytes)</th>
</tr>
<tr>
<td><% mWrap.getCompactionQueueSize() %></td>
<td><% mWrap.getFlushQueueSize() %></td>
<td><% StringUtils.TraditionalBinaryPrefix.long2String(mServerWrap.getTotalQueueSize(),
"", 1) %></td>
</tr>
</table>
</%def>

View File

@ -32,8 +32,10 @@ import org.apache.hadoop.hbase.exceptions.RegionMovedException;
@InterfaceAudience.Private
public class MetricsHBaseServer {
private MetricsHBaseServerSource source;
private MetricsHBaseServerWrapper serverWrapper;
public MetricsHBaseServer(String serverName, MetricsHBaseServerWrapper wrapper) {
serverWrapper = wrapper;
source = CompatibilitySingletonFactory.getInstance(MetricsHBaseServerSourceFactory.class)
.create(serverName, wrapper);
}
@ -115,4 +117,8 @@ public class MetricsHBaseServer {
public MetricsHBaseServerSource getMetricsSource() {
return source;
}
public MetricsHBaseServerWrapper getHBaseServerWrapper() {
return serverWrapper;
}
}