summaryrefslogtreecommitdiff
path: root/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientMetrics.java
blob: a4304009add0716c0ea9a3f3203356db7fec6646 (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
/**
 * Licensed to the Apache Software Foundation (ASF) under one
 * or more contributor license agreements.  See the NOTICE file
 * distributed with this work for additional information
 * regarding copyright ownership.  The ASF licenses this file
 * to you under the Apache License, Version 2.0 (the
 * "License"); you may not use this file except in compliance
 * with the License.  You may obtain a copy of the License at
 *
 *     http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */
package org.apache.hadoop.hdds.scm;

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.metrics2.MetricsSystem;
import org.apache.hadoop.metrics2.annotation.Metric;
import org.apache.hadoop.metrics2.annotation.Metrics;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.metrics2.lib.MetricsRegistry;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableRate;

/**
 * The client metrics for the Storage Container protocol.
 */
@InterfaceAudience.Private
@Metrics(about = "Storage Container Client Metrics", context = "dfs")
public class XceiverClientMetrics {
  public static final String SOURCE_NAME = XceiverClientMetrics.class
      .getSimpleName();

  private @Metric MutableCounterLong pendingOps;
  private MutableCounterLong[] pendingOpsArray;
  private MutableRate[] containerOpsLatency;
  private MetricsRegistry registry;

  public XceiverClientMetrics() {
    int numEnumEntries = ContainerProtos.Type.values().length;
    this.registry = new MetricsRegistry(SOURCE_NAME);

    this.pendingOpsArray = new MutableCounterLong[numEnumEntries];
    this.containerOpsLatency = new MutableRate[numEnumEntries];
    for (int i = 0; i < numEnumEntries; i++) {
      pendingOpsArray[i] = registry.newCounter(
          "numPending" + ContainerProtos.Type.forNumber(i + 1),
          "number of pending" + ContainerProtos.Type.forNumber(i + 1) + " ops",
          (long) 0);

      containerOpsLatency[i] = registry.newRate(
          ContainerProtos.Type.forNumber(i + 1) + "Latency",
          "latency of " + ContainerProtos.Type.forNumber(i + 1)
          + " ops");
    }
  }

  public static XceiverClientMetrics create() {
    MetricsSystem ms = DefaultMetricsSystem.instance();
    return ms.register(SOURCE_NAME, "Storage Container Client Metrics",
        new XceiverClientMetrics());
  }

  public void incrPendingContainerOpsMetrics(ContainerProtos.Type type) {
    pendingOps.incr();
    pendingOpsArray[type.ordinal()].incr();
  }

  public void decrPendingContainerOpsMetrics(ContainerProtos.Type type) {
    pendingOps.incr(-1);
    pendingOpsArray[type.ordinal()].incr(-1);
  }

  public void addContainerOpsLatency(ContainerProtos.Type type,
      long latencyNanos) {
    containerOpsLatency[type.ordinal()].add(latencyNanos);
  }

  public long getContainerOpsMetrics(ContainerProtos.Type type) {
    return pendingOpsArray[type.ordinal()].value();
  }

  public void unRegister() {
    MetricsSystem ms = DefaultMetricsSystem.instance();
    ms.unregisterSource(SOURCE_NAME);
  }
}