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 */
018
019package org.apache.hadoop.hbase.regionserver;
020
021import java.util.Map;
022import java.util.Map.Entry;
023import java.util.concurrent.atomic.AtomicBoolean;
024
025import org.apache.hadoop.hbase.metrics.Interns;
026import org.apache.hadoop.metrics2.MetricsRecordBuilder;
027import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry;
028import org.apache.hadoop.metrics2.lib.MutableFastCounter;
029import org.apache.yetus.audience.InterfaceAudience;
030import org.slf4j.Logger;
031import org.slf4j.LoggerFactory;
032
033@InterfaceAudience.Private
034public class MetricsRegionSourceImpl implements MetricsRegionSource {
035
036  private static final Logger LOG = LoggerFactory.getLogger(MetricsRegionSourceImpl.class);
037
038  private static final String _STORE = "_store_";
039
040  private AtomicBoolean closed = new AtomicBoolean(false);
041
042  // Non-final so that we can null out the wrapper
043  // This is just paranoia. We really really don't want to
044  // leak a whole region by way of keeping the
045  // regionWrapper around too long.
046  private MetricsRegionWrapper regionWrapper;
047
048  private final MetricsRegionAggregateSourceImpl agg;
049  private final DynamicMetricsRegistry registry;
050
051  private final String regionNamePrefix;
052  private final String regionNamePrefix1;
053  private final String regionNamePrefix2;
054  private final String regionPutKey;
055  private final String regionDeleteKey;
056  private final String regionGetKey;
057  private final String regionIncrementKey;
058  private final String regionAppendKey;
059  private final String regionScanKey;
060
061  /*
062   * Implementation note: Do not put histograms per region. With hundreds of regions in a server
063   * histograms allocate too many counters. See HBASE-17016.
064   */
065  private final MutableFastCounter regionPut;
066  private final MutableFastCounter regionDelete;
067  private final MutableFastCounter regionIncrement;
068  private final MutableFastCounter regionAppend;
069  private final MutableFastCounter regionGet;
070  private final MutableFastCounter regionScan;
071
072  private final int hashCode;
073
074  public MetricsRegionSourceImpl(MetricsRegionWrapper regionWrapper,
075                                 MetricsRegionAggregateSourceImpl aggregate) {
076    this.regionWrapper = regionWrapper;
077    agg = aggregate;
078    hashCode = regionWrapper.getRegionHashCode();
079    agg.register(this);
080
081    LOG.debug("Creating new MetricsRegionSourceImpl for table " +
082        regionWrapper.getTableName() + " " + regionWrapper.getRegionName());
083
084    registry = agg.getMetricsRegistry();
085
086    regionNamePrefix1 = "Namespace_" + regionWrapper.getNamespace() + "_table_"
087        + regionWrapper.getTableName() + "_region_" + regionWrapper.getRegionName();
088    regionNamePrefix2 = "_metric_";
089    regionNamePrefix = regionNamePrefix1 + regionNamePrefix2;
090
091    String suffix = "Count";
092
093    regionPutKey = regionNamePrefix + MetricsRegionServerSource.PUT_KEY + suffix;
094    regionPut = registry.getCounter(regionPutKey, 0L);
095
096    regionDeleteKey = regionNamePrefix + MetricsRegionServerSource.DELETE_KEY + suffix;
097    regionDelete = registry.getCounter(regionDeleteKey, 0L);
098
099    regionIncrementKey = regionNamePrefix + MetricsRegionServerSource.INCREMENT_KEY + suffix;
100    regionIncrement = registry.getCounter(regionIncrementKey, 0L);
101
102    regionAppendKey = regionNamePrefix + MetricsRegionServerSource.APPEND_KEY + suffix;
103    regionAppend = registry.getCounter(regionAppendKey, 0L);
104
105    regionGetKey = regionNamePrefix + MetricsRegionServerSource.GET_KEY + suffix;
106    regionGet = registry.getCounter(regionGetKey, 0L);
107
108    regionScanKey = regionNamePrefix + MetricsRegionServerSource.SCAN_KEY + suffix;
109    regionScan = registry.getCounter(regionScanKey, 0L);
110  }
111
112  @Override
113  public void close() {
114    boolean wasClosed = closed.getAndSet(true);
115
116    // Has someone else already closed this for us?
117    if (wasClosed) {
118      return;
119    }
120
121    // Before removing the metrics remove this region from the aggregate region bean.
122    // This should mean that it's unlikely that snapshot and close happen at the same time.
123    agg.deregister(this);
124
125    // While it's un-likely that snapshot and close happen at the same time it's still possible.
126    // So grab the lock to ensure that all calls to snapshot are done before we remove the metrics
127    synchronized (this) {
128      if (LOG.isTraceEnabled()) {
129        LOG.trace("Removing region Metrics: " + regionWrapper.getRegionName());
130      }
131
132      registry.removeMetric(regionPutKey);
133      registry.removeMetric(regionDeleteKey);
134      registry.removeMetric(regionIncrementKey);
135      registry.removeMetric(regionAppendKey);
136      registry.removeMetric(regionGetKey);
137      registry.removeMetric(regionScanKey);
138
139      regionWrapper = null;
140    }
141  }
142
143  @Override
144  public void updatePut() {
145    regionPut.incr();
146  }
147
148  @Override
149  public void updateDelete() {
150    regionDelete.incr();
151  }
152
153  @Override
154  public void updateGet(long mills) {
155    regionGet.incr();
156  }
157
158  @Override
159  public void updateScanTime(long mills) {
160    regionScan.incr();
161  }
162
163  @Override
164  public void updateIncrement() {
165    regionIncrement.incr();
166  }
167
168  @Override
169  public void updateAppend() {
170    regionAppend.incr();
171  }
172
173  @Override
174  public MetricsRegionAggregateSource getAggregateSource() {
175    return agg;
176  }
177
178  @Override
179  public int compareTo(MetricsRegionSource source) {
180    if (!(source instanceof MetricsRegionSourceImpl)) {
181      return -1;
182    }
183
184    MetricsRegionSourceImpl impl = (MetricsRegionSourceImpl) source;
185    if (impl == null) {
186      return -1;
187    }
188
189    return Long.compare(hashCode, impl.hashCode);
190  }
191
192  void snapshot(MetricsRecordBuilder mrb, boolean ignored) {
193
194    // If there is a close that started be double extra sure
195    // that we're not getting any locks and not putting data
196    // into the metrics that should be removed. So early out
197    // before even getting the lock.
198    if (closed.get()) {
199      return;
200    }
201
202    // Grab the read
203    // This ensures that removes of the metrics
204    // can't happen while we are putting them back in.
205    synchronized (this) {
206
207      // It's possible that a close happened between checking
208      // the closed variable and getting the lock.
209      if (closed.get()) {
210        return;
211      }
212
213      mrb.addGauge(
214          Interns.info(
215              regionNamePrefix + MetricsRegionServerSource.STORE_COUNT,
216              MetricsRegionServerSource.STORE_COUNT_DESC),
217          this.regionWrapper.getNumStores());
218      mrb.addGauge(Interns.info(
219              regionNamePrefix + MetricsRegionServerSource.STOREFILE_COUNT,
220              MetricsRegionServerSource.STOREFILE_COUNT_DESC),
221          this.regionWrapper.getNumStoreFiles());
222      mrb.addGauge(Interns.info(
223              regionNamePrefix + MetricsRegionServerSource.STORE_REF_COUNT,
224              MetricsRegionServerSource.STORE_REF_COUNT),
225          this.regionWrapper.getStoreRefCount());
226      mrb.addGauge(Interns.info(
227        regionNamePrefix + MetricsRegionServerSource.MAX_COMPACTED_STORE_FILE_REF_COUNT,
228        MetricsRegionServerSource.MAX_COMPACTED_STORE_FILE_REF_COUNT),
229        this.regionWrapper.getMaxCompactedStoreFileRefCount()
230      );
231      mrb.addGauge(Interns.info(
232              regionNamePrefix + MetricsRegionServerSource.MEMSTORE_SIZE,
233              MetricsRegionServerSource.MEMSTORE_SIZE_DESC),
234          this.regionWrapper.getMemStoreSize());
235      mrb.addGauge(Interns.info(
236        regionNamePrefix + MetricsRegionServerSource.MAX_STORE_FILE_AGE,
237        MetricsRegionServerSource.MAX_STORE_FILE_AGE_DESC),
238        this.regionWrapper.getMaxStoreFileAge());
239      mrb.addGauge(Interns.info(
240        regionNamePrefix + MetricsRegionServerSource.MIN_STORE_FILE_AGE,
241        MetricsRegionServerSource.MIN_STORE_FILE_AGE_DESC),
242        this.regionWrapper.getMinStoreFileAge());
243      mrb.addGauge(Interns.info(
244        regionNamePrefix + MetricsRegionServerSource.AVG_STORE_FILE_AGE,
245        MetricsRegionServerSource.AVG_STORE_FILE_AGE_DESC),
246        this.regionWrapper.getAvgStoreFileAge());
247      mrb.addGauge(Interns.info(
248        regionNamePrefix + MetricsRegionServerSource.NUM_REFERENCE_FILES,
249        MetricsRegionServerSource.NUM_REFERENCE_FILES_DESC),
250        this.regionWrapper.getNumReferenceFiles());
251      mrb.addGauge(Interns.info(
252              regionNamePrefix + MetricsRegionServerSource.STOREFILE_SIZE,
253              MetricsRegionServerSource.STOREFILE_SIZE_DESC),
254          this.regionWrapper.getStoreFileSize());
255      mrb.addCounter(Interns.info(
256              regionNamePrefix + MetricsRegionSource.COMPACTIONS_COMPLETED_COUNT,
257              MetricsRegionSource.COMPACTIONS_COMPLETED_DESC),
258          this.regionWrapper.getNumCompactionsCompleted());
259      mrb.addCounter(Interns.info(
260          regionNamePrefix + MetricsRegionSource.COMPACTIONS_FAILED_COUNT,
261          MetricsRegionSource.COMPACTIONS_FAILED_DESC),
262          this.regionWrapper.getNumCompactionsFailed());
263      mrb.addCounter(Interns.info(
264              regionNamePrefix + MetricsRegionSource.LAST_MAJOR_COMPACTION_AGE,
265              MetricsRegionSource.LAST_MAJOR_COMPACTION_DESC),
266          this.regionWrapper.getLastMajorCompactionAge());
267      mrb.addCounter(Interns.info(
268              regionNamePrefix + MetricsRegionSource.NUM_BYTES_COMPACTED_COUNT,
269              MetricsRegionSource.NUM_BYTES_COMPACTED_DESC),
270          this.regionWrapper.getNumBytesCompacted());
271      mrb.addCounter(Interns.info(
272              regionNamePrefix + MetricsRegionSource.NUM_FILES_COMPACTED_COUNT,
273              MetricsRegionSource.NUM_FILES_COMPACTED_DESC),
274          this.regionWrapper.getNumFilesCompacted());
275      mrb.addCounter(Interns.info(
276              regionNamePrefix + MetricsRegionServerSource.READ_REQUEST_COUNT,
277              MetricsRegionServerSource.READ_REQUEST_COUNT_DESC),
278          this.regionWrapper.getReadRequestCount());
279      mrb.addCounter(Interns.info(
280              regionNamePrefix + MetricsRegionServerSource.FILTERED_READ_REQUEST_COUNT,
281              MetricsRegionServerSource.FILTERED_READ_REQUEST_COUNT_DESC),
282          this.regionWrapper.getFilteredReadRequestCount());
283      mrb.addCounter(Interns.info(
284              regionNamePrefix + MetricsRegionServerSource.WRITE_REQUEST_COUNT,
285              MetricsRegionServerSource.WRITE_REQUEST_COUNT_DESC),
286          this.regionWrapper.getWriteRequestCount());
287      mrb.addCounter(Interns.info(
288              regionNamePrefix + MetricsRegionSource.REPLICA_ID,
289              MetricsRegionSource.REPLICA_ID_DESC),
290          this.regionWrapper.getReplicaId());
291      mrb.addCounter(Interns.info(
292              regionNamePrefix + MetricsRegionSource.COMPACTIONS_QUEUED_COUNT,
293              MetricsRegionSource.COMPACTIONS_QUEUED_DESC),
294          this.regionWrapper.getNumCompactionsQueued());
295      mrb.addCounter(Interns.info(
296              regionNamePrefix + MetricsRegionSource.FLUSHES_QUEUED_COUNT,
297              MetricsRegionSource.FLUSHES_QUEUED_DESC),
298          this.regionWrapper.getNumFlushesQueued());
299      mrb.addCounter(Interns.info(
300              regionNamePrefix + MetricsRegionSource.MAX_COMPACTION_QUEUE_SIZE,
301              MetricsRegionSource.MAX_COMPACTION_QUEUE_DESC),
302          this.regionWrapper.getMaxCompactionQueueSize());
303      mrb.addCounter(Interns.info(
304              regionNamePrefix + MetricsRegionSource.MAX_FLUSH_QUEUE_SIZE,
305              MetricsRegionSource.MAX_FLUSH_QUEUE_DESC),
306          this.regionWrapper.getMaxFlushQueueSize());
307      addCounter(mrb, this.regionWrapper.getMemstoreOnlyRowReadsCount(),
308        MetricsRegionSource.ROW_READS_ONLY_ON_MEMSTORE,
309        MetricsRegionSource.ROW_READS_ONLY_ON_MEMSTORE_DESC);
310      addCounter(mrb, this.regionWrapper.getMixedRowReadsCount(),
311        MetricsRegionSource.MIXED_ROW_READS,
312        MetricsRegionSource.MIXED_ROW_READS_ON_STORE_DESC);
313    }
314  }
315
316  private void addCounter(MetricsRecordBuilder mrb, Map<String, Long> metricMap, String metricName,
317      String metricDesc) {
318    if (metricMap != null) {
319      for (Entry<String, Long> entry : metricMap.entrySet()) {
320        // append 'store' and its name to the metric
321        mrb.addCounter(Interns.info(
322          this.regionNamePrefix1 + _STORE + entry.getKey() + this.regionNamePrefix2 + metricName,
323          metricDesc), entry.getValue());
324      }
325    }
326  }
327
328  @Override
329  public int hashCode() {
330    return hashCode;
331  }
332
333  @Override
334  public boolean equals(Object obj) {
335    return obj == this ||
336        (obj instanceof MetricsRegionSourceImpl && compareTo((MetricsRegionSourceImpl) obj) == 0);
337  }
338}