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.ipc;
019
020import org.apache.hadoop.hbase.metrics.ExceptionTrackingSourceImpl;
021import org.apache.hadoop.hbase.metrics.Interns;
022import org.apache.hadoop.hbase.util.Pair;
023import org.apache.hadoop.metrics2.MetricHistogram;
024import org.apache.hadoop.metrics2.MetricsCollector;
025import org.apache.hadoop.metrics2.MetricsRecordBuilder;
026import org.apache.hadoop.metrics2.lib.MutableFastCounter;
027import org.apache.yetus.audience.InterfaceAudience;
028
029@InterfaceAudience.Private
030public class MetricsHBaseServerSourceImpl extends ExceptionTrackingSourceImpl
031  implements MetricsHBaseServerSource {
032  private final MetricsHBaseServerWrapper wrapper;
033  private final MutableFastCounter authorizationSuccesses;
034  private final MutableFastCounter authorizationFailures;
035  private final MutableFastCounter authenticationSuccesses;
036  private final MutableFastCounter authenticationFailures;
037  private final MutableFastCounter authenticationFallbacks;
038  private final MutableFastCounter sentBytes;
039  private final MutableFastCounter receivedBytes;
040  private final MutableFastCounter maxOutboundBytesExceeded;
041
042  private MetricHistogram queueCallTime;
043  private MetricHistogram processCallTime;
044  private MetricHistogram totalCallTime;
045  private MetricHistogram unwritableTime;
046  private MetricHistogram requestSize;
047  private MetricHistogram responseSize;
048
049  public MetricsHBaseServerSourceImpl(String metricsName, String metricsDescription,
050    String metricsContext, String metricsJmxContext, MetricsHBaseServerWrapper wrapper) {
051    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
052    this.wrapper = wrapper;
053
054    this.authorizationSuccesses = this.getMetricsRegistry().newCounter(AUTHORIZATION_SUCCESSES_NAME,
055      AUTHORIZATION_SUCCESSES_DESC, 0L);
056    this.authorizationFailures = this.getMetricsRegistry().newCounter(AUTHORIZATION_FAILURES_NAME,
057      AUTHORIZATION_FAILURES_DESC, 0L);
058    this.authenticationSuccesses = this.getMetricsRegistry()
059      .newCounter(AUTHENTICATION_SUCCESSES_NAME, AUTHENTICATION_SUCCESSES_DESC, 0L);
060    this.authenticationFailures = this.getMetricsRegistry().newCounter(AUTHENTICATION_FAILURES_NAME,
061      AUTHENTICATION_FAILURES_DESC, 0L);
062    this.authenticationFallbacks = this.getMetricsRegistry()
063      .newCounter(AUTHENTICATION_FALLBACKS_NAME, AUTHENTICATION_FALLBACKS_DESC, 0L);
064    this.sentBytes = this.getMetricsRegistry().newCounter(SENT_BYTES_NAME, SENT_BYTES_DESC, 0L);
065    this.receivedBytes =
066      this.getMetricsRegistry().newCounter(RECEIVED_BYTES_NAME, RECEIVED_BYTES_DESC, 0L);
067    this.queueCallTime =
068      this.getMetricsRegistry().newTimeHistogram(QUEUE_CALL_TIME_NAME, QUEUE_CALL_TIME_DESC);
069    this.processCallTime =
070      this.getMetricsRegistry().newTimeHistogram(PROCESS_CALL_TIME_NAME, PROCESS_CALL_TIME_DESC);
071    this.totalCallTime =
072      this.getMetricsRegistry().newTimeHistogram(TOTAL_CALL_TIME_NAME, TOTAL_CALL_TIME_DESC);
073    this.unwritableTime =
074      this.getMetricsRegistry().newTimeHistogram(UNWRITABLE_TIME_NAME, UNWRITABLE_TIME_DESC);
075    this.maxOutboundBytesExceeded = this.getMetricsRegistry()
076      .newCounter(MAX_OUTBOUND_BYTES_EXCEEDED_NAME, MAX_OUTBOUND_BYTES_EXCEEDED_DESC, 0);
077    this.requestSize =
078      this.getMetricsRegistry().newSizeHistogram(REQUEST_SIZE_NAME, REQUEST_SIZE_DESC);
079    this.responseSize =
080      this.getMetricsRegistry().newSizeHistogram(RESPONSE_SIZE_NAME, RESPONSE_SIZE_DESC);
081  }
082
083  @Override
084  public void authorizationSuccess() {
085    authorizationSuccesses.incr();
086  }
087
088  @Override
089  public void authorizationFailure() {
090    authorizationFailures.incr();
091  }
092
093  @Override
094  public void authenticationFailure() {
095    authenticationFailures.incr();
096  }
097
098  @Override
099  public void authenticationFallback() {
100    authenticationFallbacks.incr();
101  }
102
103  @Override
104  public void authenticationSuccess() {
105    authenticationSuccesses.incr();
106  }
107
108  @Override
109  public void sentBytes(long count) {
110    this.sentBytes.incr(count);
111  }
112
113  @Override
114  public void receivedBytes(int count) {
115    this.receivedBytes.incr(count);
116  }
117
118  @Override
119  public void sentResponse(long count) {
120    this.responseSize.add(count);
121  }
122
123  @Override
124  public void receivedRequest(long count) {
125    this.requestSize.add(count);
126  }
127
128  @Override
129  public void dequeuedCall(int qTime) {
130    queueCallTime.add(qTime);
131  }
132
133  @Override
134  public void processedCall(int processingTime) {
135    processCallTime.add(processingTime);
136  }
137
138  @Override
139  public void queuedAndProcessedCall(int totalTime) {
140    totalCallTime.add(totalTime);
141  }
142
143  @Override
144  public void unwritableTime(long unwritableTime) {
145    this.unwritableTime.add(unwritableTime);
146  }
147
148  @Override
149  public void maxOutboundBytesExceeded() {
150    maxOutboundBytesExceeded.incr();
151  }
152
153  @Override
154  public void getMetrics(MetricsCollector metricsCollector, boolean all) {
155    MetricsRecordBuilder mrb = metricsCollector.addRecord(metricsName);
156
157    if (wrapper != null) {
158      mrb.addGauge(Interns.info(QUEUE_SIZE_NAME, QUEUE_SIZE_DESC), wrapper.getTotalQueueSize())
159        .addGauge(Interns.info(GENERAL_QUEUE_NAME, GENERAL_QUEUE_DESC),
160          wrapper.getGeneralQueueLength())
161        .addGauge(Interns.info(REPLICATION_QUEUE_NAME, REPLICATION_QUEUE_DESC),
162          wrapper.getReplicationQueueLength())
163        .addGauge(Interns.info(BULKLOAD_QUEUE_NAME, BULKLOAD_QUEUE_DESC),
164          wrapper.getBulkLoadQueueLength())
165        .addGauge(Interns.info(PRIORITY_QUEUE_NAME, PRIORITY_QUEUE_DESC),
166          wrapper.getPriorityQueueLength())
167        .addGauge(Interns.info(METAPRIORITY_QUEUE_NAME, METAPRIORITY_QUEUE_DESC),
168          wrapper.getMetaPriorityQueueLength())
169        .addGauge(Interns.info(NUM_OPEN_CONNECTIONS_NAME, NUM_OPEN_CONNECTIONS_DESC),
170          wrapper.getNumOpenConnections())
171        .addGauge(Interns.info(NUM_ACTIVE_HANDLER_NAME, NUM_ACTIVE_HANDLER_DESC),
172          wrapper.getActiveRpcHandlerCount())
173        .addGauge(Interns.info(NUM_ACTIVE_GENERAL_HANDLER_NAME, NUM_ACTIVE_GENERAL_HANDLER_DESC),
174          wrapper.getActiveGeneralRpcHandlerCount())
175        .addGauge(Interns.info(NUM_ACTIVE_PRIORITY_HANDLER_NAME, NUM_ACTIVE_PRIORITY_HANDLER_DESC),
176          wrapper.getActivePriorityRpcHandlerCount())
177        .addGauge(
178          Interns.info(NUM_ACTIVE_REPLICATION_HANDLER_NAME, NUM_ACTIVE_REPLICATION_HANDLER_DESC),
179          wrapper.getActiveReplicationRpcHandlerCount())
180        .addCounter(Interns.info(NUM_GENERAL_CALLS_DROPPED_NAME, NUM_GENERAL_CALLS_DROPPED_DESC),
181          wrapper.getNumGeneralCallsDropped())
182        .addCounter(Interns.info(NUM_LIFO_MODE_SWITCHES_NAME, NUM_LIFO_MODE_SWITCHES_DESC),
183          wrapper.getNumLifoModeSwitches())
184        .addGauge(Interns.info(WRITE_QUEUE_NAME, WRITE_QUEUE_DESC), wrapper.getWriteQueueLength())
185        .addGauge(Interns.info(NUM_ACTIVE_BULKLOAD_HANDLER_NAME, NUM_ACTIVE_BULKLOAD_HANDLER_DESC),
186          wrapper.getActiveBulkLoadRpcHandlerCount())
187        .addGauge(Interns.info(READ_QUEUE_NAME, READ_QUEUE_DESC), wrapper.getReadQueueLength())
188        .addGauge(Interns.info(SCAN_QUEUE_NAME, SCAN_QUEUE_DESC), wrapper.getScanQueueLength())
189        .addGauge(Interns.info(NUM_ACTIVE_WRITE_HANDLER_NAME, NUM_ACTIVE_WRITE_HANDLER_DESC),
190          wrapper.getActiveWriteRpcHandlerCount())
191        .addGauge(Interns.info(NUM_ACTIVE_READ_HANDLER_NAME, NUM_ACTIVE_READ_HANDLER_DESC),
192          wrapper.getActiveReadRpcHandlerCount())
193        .addGauge(Interns.info(NUM_ACTIVE_SCAN_HANDLER_NAME, NUM_ACTIVE_SCAN_HANDLER_DESC),
194          wrapper.getActiveScanRpcHandlerCount())
195        .addGauge(Interns.info(NETTY_DM_USAGE_NAME, NETTY_DM_USAGE_DESC),
196          wrapper.getNettyDmUsage());
197
198      Pair<Long, Long> totalAndMax = wrapper.getTotalAndMaxNettyOutboundBytes();
199      mrb.addGauge(
200        Interns.info(NETTY_TOTAL_PENDING_OUTBOUND_NAME, NETTY_TOTAL_PENDING_OUTBOUND_DESC),
201        totalAndMax.getFirst());
202      mrb.addGauge(Interns.info(NETTY_MAX_PENDING_OUTBOUND_NAME, NETTY_MAX_PENDING_OUTBOUND_DESC),
203        totalAndMax.getSecond());
204    }
205
206    metricsRegistry.snapshot(mrb, all);
207  }
208}