View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.replication.regionserver;
19  
20  import org.apache.hadoop.metrics2.lib.MutableCounterLong;
21  import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
22  
23  public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSourceSource {
24  
25    private final MetricsReplicationSourceImpl rms;
26    private final String id;
27    private final String keyPrefix;
28    private final String sizeOfLogQueueKey;
29    private final String ageOfLastShippedOpKey;
30    private final String logReadInEditsKey;
31    private final String logEditsFilteredKey;
32    private final String shippedBatchesKey;
33    private final String shippedOpsKey;
34    private final String shippedKBsKey;
35    private final String logReadInBytesKey;
36  
37    private final MutableGaugeLong ageOfLastShippedOpGauge;
38    private final MutableGaugeLong sizeOfLogQueueGauge;
39    private final MutableCounterLong logReadInEditsCounter;
40    private final MutableCounterLong logEditsFilteredCounter;
41    private final MutableCounterLong shippedBatchesCounter;
42    private final MutableCounterLong shippedOpsCounter;
43    private final MutableCounterLong shippedKBsCounter;
44    private final MutableCounterLong logReadInBytesCounter;
45  
46    private final String unknownFileLengthKey;
47    private final String uncleanlyClosedKey;
48    private final String uncleanlySkippedBytesKey;
49    private final String restartedKey;
50    private final String repeatedBytesKey;
51    private final String completedLogsKey;
52    private final String completedRecoveryKey;
53    private final MutableCounterLong unknownFileLengthForClosedWAL;
54    private final MutableCounterLong uncleanlyClosedWAL;
55    private final MutableCounterLong uncleanlyClosedSkippedBytes;
56    private final MutableCounterLong restartWALReading;
57    private final MutableCounterLong repeatedFileBytes;
58    private final MutableCounterLong completedWAL;
59    private final MutableCounterLong completedRecoveryQueue;
60  
61    public MetricsReplicationSourceSourceImpl(MetricsReplicationSourceImpl rms, String id) {
62      this.rms = rms;
63      this.id = id;
64      this.keyPrefix = "source." + this.id + ".";
65  
66      ageOfLastShippedOpKey = "source." + id + ".ageOfLastShippedOp";
67      ageOfLastShippedOpGauge = rms.getMetricsRegistry().getLongGauge(ageOfLastShippedOpKey, 0L);
68  
69      sizeOfLogQueueKey = "source." + id + ".sizeOfLogQueue";
70      sizeOfLogQueueGauge = rms.getMetricsRegistry().getLongGauge(sizeOfLogQueueKey, 0L);
71  
72      shippedBatchesKey = "source." + this.id + ".shippedBatches";
73      shippedBatchesCounter = rms.getMetricsRegistry().getLongCounter(shippedBatchesKey, 0L);
74  
75      shippedOpsKey = "source." + this.id + ".shippedOps";
76      shippedOpsCounter = rms.getMetricsRegistry().getLongCounter(shippedOpsKey, 0L);
77  
78      shippedKBsKey = "source." + this.id + ".shippedKBs";
79      shippedKBsCounter = rms.getMetricsRegistry().getLongCounter(shippedKBsKey, 0L);
80  
81      logReadInBytesKey = "source." + this.id + ".logReadInBytes";
82      logReadInBytesCounter = rms.getMetricsRegistry().getLongCounter(logReadInBytesKey, 0L);
83  
84      logReadInEditsKey = "source." + id + ".logEditsRead";
85      logReadInEditsCounter = rms.getMetricsRegistry().getLongCounter(logReadInEditsKey, 0L);
86  
87      logEditsFilteredKey = "source." + id + ".logEditsFiltered";
88      logEditsFilteredCounter = rms.getMetricsRegistry().getLongCounter(logEditsFilteredKey, 0L);
89      unknownFileLengthKey = this.keyPrefix + "closedLogsWithUnknownFileLength";
90      unknownFileLengthForClosedWAL = rms.getMetricsRegistry().getLongCounter(unknownFileLengthKey, 0L);
91  
92      uncleanlyClosedKey = this.keyPrefix + "uncleanlyClosedLogs";
93      uncleanlyClosedWAL = rms.getMetricsRegistry().getLongCounter(uncleanlyClosedKey, 0L);
94  
95      uncleanlySkippedBytesKey = this.keyPrefix + "ignoredUncleanlyClosedLogContentsInBytes";
96      uncleanlyClosedSkippedBytes = rms.getMetricsRegistry().getLongCounter(uncleanlySkippedBytesKey, 0L);
97  
98      restartedKey = this.keyPrefix + "restartedLogReading";
99      restartWALReading = rms.getMetricsRegistry().getLongCounter(restartedKey, 0L);
100 
101     repeatedBytesKey = this.keyPrefix + "repeatedLogFileBytes";
102     repeatedFileBytes = rms.getMetricsRegistry().getLongCounter(repeatedBytesKey, 0L);
103 
104     completedLogsKey = this.keyPrefix + "completedLogs";
105     completedWAL = rms.getMetricsRegistry().getLongCounter(completedLogsKey, 0L);
106 
107     completedRecoveryKey = this.keyPrefix + "completedRecoverQueues";
108     completedRecoveryQueue = rms.getMetricsRegistry().getLongCounter(completedRecoveryKey, 0L);
109   }
110 
111   @Override public void setLastShippedAge(long age) {
112     ageOfLastShippedOpGauge.set(age);
113   }
114 
115   @Override public void setSizeOfLogQueue(int size) {
116     sizeOfLogQueueGauge.set(size);
117   }
118 
119   @Override public void incrSizeOfLogQueue(int size) {
120     sizeOfLogQueueGauge.incr(size);
121   }
122 
123   @Override public void decrSizeOfLogQueue(int size) {
124     sizeOfLogQueueGauge.decr(size);
125   }
126 
127   @Override public void incrLogReadInEdits(long size) {
128     logReadInEditsCounter.incr(size);
129   }
130 
131   @Override public void incrLogEditsFiltered(long size) {
132     logEditsFilteredCounter.incr(size);
133   }
134 
135   @Override public void incrBatchesShipped(int batches) {
136     shippedBatchesCounter.incr(batches);
137   }
138 
139   @Override public void incrOpsShipped(long ops) {
140     shippedOpsCounter.incr(ops);
141   }
142 
143   @Override public void incrShippedKBs(long size) {
144     shippedKBsCounter.incr(size);
145   }
146 
147   @Override public void incrLogReadInBytes(long size) {
148     logReadInBytesCounter.incr(size);
149   }
150 
151   @Override public void clear() {
152     rms.removeMetric(ageOfLastShippedOpKey);
153 
154     rms.removeMetric(sizeOfLogQueueKey);
155 
156     rms.removeMetric(shippedBatchesKey);
157     rms.removeMetric(shippedOpsKey);
158     rms.removeMetric(shippedKBsKey);
159 
160     rms.removeMetric(logReadInBytesKey);
161     rms.removeMetric(logReadInEditsKey);
162 
163     rms.removeMetric(logEditsFilteredKey);
164 
165     rms.removeMetric(unknownFileLengthKey);
166     rms.removeMetric(uncleanlyClosedKey);
167     rms.removeMetric(uncleanlySkippedBytesKey);
168     rms.removeMetric(restartedKey);
169     rms.removeMetric(repeatedBytesKey);
170     rms.removeMetric(completedLogsKey);
171     rms.removeMetric(completedRecoveryKey);
172   }
173 
174   @Override
175   public long getLastShippedAge() {
176     return ageOfLastShippedOpGauge.value();
177   }
178 
179   @Override
180   public void incrUnknownFileLengthForClosedWAL() {
181     unknownFileLengthForClosedWAL.incr(1L);
182   }
183 
184   @Override
185   public void incrUncleanlyClosedWALs() {
186     uncleanlyClosedWAL.incr(1L);
187   }
188 
189   @Override
190   public void incrBytesSkippedInUncleanlyClosedWALs(final long bytes) {
191     uncleanlyClosedSkippedBytes.incr(bytes);
192   }
193 
194   @Override
195   public void incrRestartedWALReading() {
196     restartWALReading.incr(1L);
197   }
198 
199   @Override
200   public void incrRepeatedFileBytes(final long bytes) {
201     repeatedFileBytes.incr(bytes);
202   }
203 
204   @Override
205   public void incrCompletedWAL() {
206     completedWAL.incr(1L);
207   }
208 
209   @Override
210   public void incrCompletedRecoveryQueue() {
211     completedRecoveryQueue.incr(1L);
212   }
213 }