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.regionserver;
019
020import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTED_INPUT_BYTES;
021import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTED_INPUT_BYTES_DESC;
022import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTED_OUTPUT_BYTES;
023import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTED_OUTPUT_BYTES_DESC;
024import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_INPUT_FILE_COUNT;
025import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_INPUT_FILE_COUNT_DESC;
026import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_INPUT_SIZE;
027import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_INPUT_SIZE_DESC;
028import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_OUTPUT_FILE_COUNT;
029import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_OUTPUT_FILE_COUNT_DESC;
030import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_OUTPUT_SIZE;
031import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_OUTPUT_SIZE_DESC;
032import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_TIME;
033import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_TIME_DESC;
034import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSHED_MEMSTORE_BYTES;
035import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSHED_MEMSTORE_BYTES_DESC;
036import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSHED_OUTPUT_BYTES;
037import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSHED_OUTPUT_BYTES_DESC;
038import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSH_MEMSTORE_SIZE;
039import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSH_MEMSTORE_SIZE_DESC;
040import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSH_OUTPUT_SIZE;
041import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSH_OUTPUT_SIZE_DESC;
042import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSH_TIME;
043import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSH_TIME_DESC;
044import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTED_INPUT_BYTES;
045import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTED_INPUT_BYTES_DESC;
046import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTED_OUTPUT_BYTES;
047import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTED_OUTPUT_BYTES_DESC;
048import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_INPUT_FILE_COUNT;
049import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_INPUT_FILE_COUNT_DESC;
050import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_INPUT_SIZE;
051import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_INPUT_SIZE_DESC;
052import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_OUTPUT_FILE_COUNT;
053import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_OUTPUT_FILE_COUNT_DESC;
054import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_OUTPUT_SIZE;
055import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_OUTPUT_SIZE_DESC;
056import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_TIME;
057import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_TIME_DESC;
058import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.SPLIT_KEY;
059import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.SPLIT_REQUEST_DESC;
060import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.SPLIT_REQUEST_KEY;
061import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.SPLIT_SUCCESS_DESC;
062import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.SPLIT_SUCCESS_KEY;
063
064import java.util.Map;
065import java.util.Map.Entry;
066import java.util.concurrent.atomic.AtomicBoolean;
067import org.apache.hadoop.hbase.TableName;
068import org.apache.hadoop.hbase.metrics.Interns;
069import org.apache.hadoop.metrics2.MetricHistogram;
070import org.apache.hadoop.metrics2.MetricsRecordBuilder;
071import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry;
072import org.apache.hadoop.metrics2.lib.MutableFastCounter;
073import org.apache.yetus.audience.InterfaceAudience;
074import org.slf4j.Logger;
075import org.slf4j.LoggerFactory;
076
077import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
078import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
079
080@InterfaceAudience.Private
081public class MetricsTableSourceImpl implements MetricsTableSource {
082
083  private static final String _COLUMNFAMILY = "_columnfamily_";
084
085  private static final Logger LOG = LoggerFactory.getLogger(MetricsTableSourceImpl.class);
086
087  private AtomicBoolean closed = new AtomicBoolean(false);
088
089  // Non-final so that we can null out the wrapper
090  // This is just paranoia. We really really don't want to
091  // leak a whole table by way of keeping the
092  // tableWrapper around too long.
093  private MetricsTableWrapperAggregate tableWrapperAgg;
094  private final MetricsTableAggregateSourceImpl agg;
095  private final DynamicMetricsRegistry registry;
096  private final String tableNamePrefix;
097  private final String tableNamePrefixPart1;
098  private final String tableNamePrefixPart2;
099  private final TableName tableName;
100  private final int hashCode;
101
102  // split related metrics
103  private MutableFastCounter splitRequest;
104  private MutableFastCounter splitSuccess;
105  private MetricHistogram splitTimeHisto;
106
107  // flush related metrics
108  private MetricHistogram flushTimeHisto;
109  private MetricHistogram flushMemstoreSizeHisto;
110  private MetricHistogram flushOutputSizeHisto;
111  private MutableFastCounter flushedMemstoreBytes;
112  private MutableFastCounter flushedOutputBytes;
113
114  // compaction related metrics
115  private MetricHistogram compactionTimeHisto;
116  private MetricHistogram compactionInputFileCountHisto;
117  private MetricHistogram compactionInputSizeHisto;
118  private MetricHistogram compactionOutputFileCountHisto;
119  private MetricHistogram compactionOutputSizeHisto;
120  private MutableFastCounter compactedInputBytes;
121  private MutableFastCounter compactedOutputBytes;
122
123  private MetricHistogram majorCompactionTimeHisto;
124  private MetricHistogram majorCompactionInputFileCountHisto;
125  private MetricHistogram majorCompactionInputSizeHisto;
126  private MetricHistogram majorCompactionOutputFileCountHisto;
127  private MetricHistogram majorCompactionOutputSizeHisto;
128  private MutableFastCounter majorCompactedInputBytes;
129  private MutableFastCounter majorCompactedOutputBytes;
130
131  public MetricsTableSourceImpl(String tblName, MetricsTableAggregateSourceImpl aggregate,
132    MetricsTableWrapperAggregate tblWrapperAgg) {
133    LOG.debug("Creating new MetricsTableSourceImpl for table '{}'", tblName);
134    this.tableName = TableName.valueOf(tblName);
135    this.agg = aggregate;
136
137    this.tableWrapperAgg = tblWrapperAgg;
138    this.registry = agg.getMetricsRegistry();
139    this.tableNamePrefixPart1 = "Namespace_" + this.tableName.getNamespaceAsString() + "_table_"
140      + this.tableName.getQualifierAsString();
141    this.tableNamePrefixPart2 = "_metric_";
142    this.tableNamePrefix = tableNamePrefixPart1 + tableNamePrefixPart2;
143    this.hashCode = this.tableName.hashCode();
144  }
145
146  @Override
147  public synchronized void registerMetrics() {
148    flushTimeHisto = registry.newTimeHistogram(tableNamePrefix + FLUSH_TIME, FLUSH_TIME_DESC);
149    flushMemstoreSizeHisto =
150      registry.newSizeHistogram(tableNamePrefix + FLUSH_MEMSTORE_SIZE, FLUSH_MEMSTORE_SIZE_DESC);
151    flushOutputSizeHisto =
152      registry.newSizeHistogram(tableNamePrefix + FLUSH_OUTPUT_SIZE, FLUSH_OUTPUT_SIZE_DESC);
153    flushedOutputBytes =
154      registry.newCounter(tableNamePrefix + FLUSHED_OUTPUT_BYTES, FLUSHED_OUTPUT_BYTES_DESC, 0L);
155    flushedMemstoreBytes = registry.newCounter(tableNamePrefix + FLUSHED_MEMSTORE_BYTES,
156      FLUSHED_MEMSTORE_BYTES_DESC, 0L);
157
158    compactionTimeHisto =
159      registry.newTimeHistogram(tableNamePrefix + COMPACTION_TIME, COMPACTION_TIME_DESC);
160    compactionInputFileCountHisto = registry.newHistogram(
161      tableNamePrefix + COMPACTION_INPUT_FILE_COUNT, COMPACTION_INPUT_FILE_COUNT_DESC);
162    compactionInputSizeHisto = registry.newSizeHistogram(tableNamePrefix + COMPACTION_INPUT_SIZE,
163      COMPACTION_INPUT_SIZE_DESC);
164    compactionOutputFileCountHisto = registry.newHistogram(
165      tableNamePrefix + COMPACTION_OUTPUT_FILE_COUNT, COMPACTION_OUTPUT_FILE_COUNT_DESC);
166    compactionOutputSizeHisto = registry.newSizeHistogram(tableNamePrefix + COMPACTION_OUTPUT_SIZE,
167      COMPACTION_OUTPUT_SIZE_DESC);
168    compactedInputBytes =
169      registry.newCounter(tableNamePrefix + COMPACTED_INPUT_BYTES, COMPACTED_INPUT_BYTES_DESC, 0L);
170    compactedOutputBytes = registry.newCounter(tableNamePrefix + COMPACTED_OUTPUT_BYTES,
171      COMPACTED_OUTPUT_BYTES_DESC, 0L);
172
173    majorCompactionTimeHisto = registry.newTimeHistogram(tableNamePrefix + MAJOR_COMPACTION_TIME,
174      MAJOR_COMPACTION_TIME_DESC);
175    majorCompactionInputFileCountHisto = registry.newHistogram(
176      tableNamePrefix + MAJOR_COMPACTION_INPUT_FILE_COUNT, MAJOR_COMPACTION_INPUT_FILE_COUNT_DESC);
177    majorCompactionInputSizeHisto = registry.newSizeHistogram(
178      tableNamePrefix + MAJOR_COMPACTION_INPUT_SIZE, MAJOR_COMPACTION_INPUT_SIZE_DESC);
179    majorCompactionOutputFileCountHisto =
180      registry.newHistogram(tableNamePrefix + MAJOR_COMPACTION_OUTPUT_FILE_COUNT,
181        MAJOR_COMPACTION_OUTPUT_FILE_COUNT_DESC);
182    majorCompactionOutputSizeHisto = registry.newSizeHistogram(
183      tableNamePrefix + MAJOR_COMPACTION_OUTPUT_SIZE, MAJOR_COMPACTION_OUTPUT_SIZE_DESC);
184    majorCompactedInputBytes = registry.newCounter(tableNamePrefix + MAJOR_COMPACTED_INPUT_BYTES,
185      MAJOR_COMPACTED_INPUT_BYTES_DESC, 0L);
186    majorCompactedOutputBytes = registry.newCounter(tableNamePrefix + MAJOR_COMPACTED_OUTPUT_BYTES,
187      MAJOR_COMPACTED_OUTPUT_BYTES_DESC, 0L);
188
189    splitTimeHisto = registry.newTimeHistogram(tableNamePrefix + SPLIT_KEY);
190    splitRequest = registry.newCounter(tableNamePrefix + SPLIT_REQUEST_KEY, SPLIT_REQUEST_DESC, 0L);
191    splitSuccess = registry.newCounter(tableNamePrefix + SPLIT_SUCCESS_KEY, SPLIT_SUCCESS_DESC, 0L);
192  }
193
194  private void deregisterMetrics() {
195    registry.removeHistogramMetrics(tableNamePrefix + FLUSH_TIME);
196    registry.removeHistogramMetrics(tableNamePrefix + FLUSH_MEMSTORE_SIZE);
197    registry.removeHistogramMetrics(tableNamePrefix + FLUSH_OUTPUT_SIZE);
198    registry.removeMetric(tableNamePrefix + FLUSHED_OUTPUT_BYTES);
199    registry.removeMetric(tableNamePrefix + FLUSHED_MEMSTORE_BYTES);
200    registry.removeHistogramMetrics(tableNamePrefix + COMPACTION_TIME);
201    registry.removeHistogramMetrics(tableNamePrefix + COMPACTION_INPUT_FILE_COUNT);
202    registry.removeHistogramMetrics(tableNamePrefix + COMPACTION_INPUT_SIZE);
203    registry.removeHistogramMetrics(tableNamePrefix + COMPACTION_OUTPUT_FILE_COUNT);
204    registry.removeHistogramMetrics(tableNamePrefix + COMPACTION_OUTPUT_SIZE);
205    registry.removeMetric(tableNamePrefix + COMPACTED_INPUT_BYTES);
206    registry.removeMetric(tableNamePrefix + COMPACTED_OUTPUT_BYTES);
207    registry.removeHistogramMetrics(tableNamePrefix + MAJOR_COMPACTION_TIME);
208    registry.removeHistogramMetrics(tableNamePrefix + MAJOR_COMPACTION_INPUT_FILE_COUNT);
209    registry.removeHistogramMetrics(tableNamePrefix + MAJOR_COMPACTION_INPUT_SIZE);
210    registry.removeHistogramMetrics(tableNamePrefix + MAJOR_COMPACTION_OUTPUT_FILE_COUNT);
211    registry.removeHistogramMetrics(tableNamePrefix + MAJOR_COMPACTION_OUTPUT_SIZE);
212    registry.removeMetric(tableNamePrefix + MAJOR_COMPACTED_INPUT_BYTES);
213    registry.removeMetric(tableNamePrefix + MAJOR_COMPACTED_OUTPUT_BYTES);
214    registry.removeHistogramMetrics(tableNamePrefix + SPLIT_KEY);
215    registry.removeMetric(tableNamePrefix + SPLIT_REQUEST_KEY);
216    registry.removeMetric(tableNamePrefix + SPLIT_SUCCESS_KEY);
217  }
218
219  @Override
220  public void close() {
221    boolean wasClosed = closed.getAndSet(true);
222
223    // Has someone else already closed this for us?
224    if (wasClosed) {
225      return;
226    }
227
228    // Before removing the metrics remove this table from the aggregate table bean.
229    // This should mean that it's unlikely that snapshot and close happen at the same time.
230    agg.deleteTableSource(tableName.getNameAsString());
231
232    // While it's un-likely that snapshot and close happen at the same time it's still possible.
233    // So grab the lock to ensure that all calls to snapshot are done before we remove the metrics
234    synchronized (this) {
235      if (LOG.isTraceEnabled()) {
236        LOG.trace("Removing table Metrics for table ");
237      }
238      deregisterMetrics();
239      tableWrapperAgg = null;
240    }
241  }
242
243  @Override
244  public MetricsTableAggregateSource getAggregateSource() {
245    return agg;
246  }
247
248  @Override
249  public int compareTo(MetricsTableSource source) {
250    if (!(source instanceof MetricsTableSourceImpl)) {
251      return -1;
252    }
253    MetricsTableSourceImpl impl = (MetricsTableSourceImpl) source;
254    return Long.compare(hashCode, impl.hashCode);
255  }
256
257  void snapshot(MetricsRecordBuilder mrb, boolean ignored) {
258    // If there is a close that started be double extra sure
259    // that we're not getting any locks and not putting data
260    // into the metrics that should be removed. So early out
261    // before even getting the lock.
262    if (closed.get()) {
263      return;
264    }
265
266    // Grab the read
267    // This ensures that removes of the metrics
268    // can't happen while we are putting them back in.
269    synchronized (this) {
270      // It's possible that a close happened between checking
271      // the closed variable and getting the lock.
272      if (closed.get()) {
273        return;
274      }
275
276      if (this.tableWrapperAgg != null) {
277        mrb.addCounter(
278          Interns.info(tableNamePrefix + MetricsRegionServerSource.CP_REQUEST_COUNT,
279            MetricsRegionServerSource.CP_REQUEST_COUNT_DESC),
280          tableWrapperAgg.getCpRequestsCount(tableName.getNameAsString()));
281        mrb.addCounter(
282          Interns.info(tableNamePrefix + MetricsRegionServerSource.READ_REQUEST_COUNT,
283            MetricsRegionServerSource.READ_REQUEST_COUNT_DESC),
284          tableWrapperAgg.getReadRequestCount(tableName.getNameAsString()));
285        mrb.addCounter(
286          Interns.info(tableNamePrefix + MetricsRegionServerSource.FILTERED_READ_REQUEST_COUNT,
287            MetricsRegionServerSource.FILTERED_READ_REQUEST_COUNT_DESC),
288          tableWrapperAgg.getFilteredReadRequestCount(tableName.getNameAsString()));
289        mrb.addCounter(
290          Interns.info(tableNamePrefix + MetricsRegionServerSource.WRITE_REQUEST_COUNT,
291            MetricsRegionServerSource.WRITE_REQUEST_COUNT_DESC),
292          tableWrapperAgg.getWriteRequestCount(tableName.getNameAsString()));
293        mrb.addCounter(
294          Interns.info(tableNamePrefix + MetricsRegionServerSource.TOTAL_REQUEST_COUNT,
295            MetricsRegionServerSource.TOTAL_REQUEST_COUNT_DESC),
296          tableWrapperAgg.getTotalRequestsCount(tableName.getNameAsString()));
297        mrb.addGauge(
298          Interns.info(tableNamePrefix + MetricsRegionServerSource.MEMSTORE_SIZE,
299            MetricsRegionServerSource.MEMSTORE_SIZE_DESC),
300          tableWrapperAgg.getMemStoreSize(tableName.getNameAsString()));
301        mrb.addGauge(
302          Interns.info(tableNamePrefix + MetricsRegionServerSource.STOREFILE_COUNT,
303            MetricsRegionServerSource.STOREFILE_COUNT_DESC),
304          tableWrapperAgg.getNumStoreFiles(tableName.getNameAsString()));
305        mrb.addGauge(
306          Interns.info(tableNamePrefix + MetricsRegionServerSource.MAX_STOREFILE_COUNT,
307            MetricsRegionServerSource.MAX_STOREFILE_COUNT_DESC),
308          tableWrapperAgg.getMaxStoreFiles(tableName.getNameAsString()));
309        mrb.addGauge(
310          Interns.info(tableNamePrefix + MetricsRegionServerSource.STOREFILE_SIZE,
311            MetricsRegionServerSource.STOREFILE_SIZE_DESC),
312          tableWrapperAgg.getStoreFileSize(tableName.getNameAsString()));
313        mrb.addGauge(
314          Interns.info(tableNamePrefix + MetricsTableSource.TABLE_SIZE,
315            MetricsTableSource.TABLE_SIZE_DESC),
316          tableWrapperAgg.getTableSize(tableName.getNameAsString()));
317        mrb.addGauge(
318          Interns.info(tableNamePrefix + MetricsRegionServerSource.AVERAGE_REGION_SIZE,
319            MetricsRegionServerSource.AVERAGE_REGION_SIZE_DESC),
320          tableWrapperAgg.getAvgRegionSize(tableName.getNameAsString()));
321        mrb.addGauge(
322          Interns.info(tableNamePrefix + MetricsRegionServerSource.REGION_COUNT,
323            MetricsRegionServerSource.REGION_COUNT_DESC),
324          tableWrapperAgg.getNumRegions(tableName.getNameAsString()));
325        mrb.addGauge(
326          Interns.info(tableNamePrefix + MetricsRegionServerSource.STORE_COUNT,
327            MetricsRegionServerSource.STORE_COUNT_DESC),
328          tableWrapperAgg.getNumStores(tableName.getNameAsString()));
329        mrb.addGauge(
330          Interns.info(tableNamePrefix + MetricsRegionServerSource.MAX_STORE_FILE_AGE,
331            MetricsRegionServerSource.MAX_STORE_FILE_AGE_DESC),
332          tableWrapperAgg.getMaxStoreFileAge(tableName.getNameAsString()));
333        mrb.addGauge(
334          Interns.info(tableNamePrefix + MetricsRegionServerSource.MIN_STORE_FILE_AGE,
335            MetricsRegionServerSource.MIN_STORE_FILE_AGE_DESC),
336          tableWrapperAgg.getMinStoreFileAge(tableName.getNameAsString()));
337        mrb.addGauge(
338          Interns.info(tableNamePrefix + MetricsRegionServerSource.AVG_STORE_FILE_AGE,
339            MetricsRegionServerSource.AVG_STORE_FILE_AGE_DESC),
340          tableWrapperAgg.getAvgStoreFileAge(tableName.getNameAsString()));
341        mrb.addGauge(
342          Interns.info(tableNamePrefix + MetricsRegionServerSource.STATIC_BLOOM_SIZE,
343            MetricsRegionServerSource.STATIC_BLOOM_SIZE_DESC),
344          tableWrapperAgg.getStaticBloomSize(tableName.getNameAsString()));
345        mrb.addGauge(
346          Interns.info(tableNamePrefix + MetricsRegionServerSource.STATIC_INDEX_SIZE,
347            MetricsRegionServerSource.STATIC_INDEX_SIZE),
348          tableWrapperAgg.getStaticIndexSize(tableName.getNameAsString()));
349        mrb.addCounter(
350          Interns.info(tableNamePrefix + MetricsRegionServerSource.BLOOM_FILTER_REQUESTS_COUNT,
351            MetricsRegionServerSource.BLOOM_FILTER_REQUESTS_COUNT_DESC),
352          tableWrapperAgg.getBloomFilterRequestsCount(tableName.getNameAsString()));
353        mrb.addCounter(
354          Interns.info(
355            tableNamePrefix + MetricsRegionServerSource.BLOOM_FILTER_NEGATIVE_RESULTS_COUNT,
356            MetricsRegionServerSource.BLOOM_FILTER_NEGATIVE_RESULTS_COUNT_DESC),
357          tableWrapperAgg.getBloomFilterNegativeResultsCount(tableName.getNameAsString()));
358        mrb.addCounter(
359          Interns.info(
360            tableNamePrefix + MetricsRegionServerSource.BLOOM_FILTER_ELIGIBLE_REQUESTS_COUNT,
361            MetricsRegionServerSource.BLOOM_FILTER_ELIGIBLE_REQUESTS_COUNT_DESC),
362          tableWrapperAgg.getBloomFilterEligibleRequestsCount(tableName.getNameAsString()));
363        mrb.addGauge(
364          Interns.info(tableNamePrefix + MetricsRegionServerSource.NUM_REFERENCE_FILES,
365            MetricsRegionServerSource.NUM_REFERENCE_FILES_DESC),
366          tableWrapperAgg.getNumReferenceFiles(tableName.getNameAsString()));
367        addGauge(mrb, tableWrapperAgg.getMemstoreOnlyRowReadsCount(tableName.getNameAsString()),
368          MetricsRegionSource.ROW_READS_ONLY_ON_MEMSTORE,
369          MetricsRegionSource.ROW_READS_ONLY_ON_MEMSTORE_DESC);
370        addGauge(mrb, tableWrapperAgg.getMixedRowReadsCount(tableName.getNameAsString()),
371          MetricsRegionSource.MIXED_ROW_READS, MetricsRegionSource.MIXED_ROW_READS_ON_STORE_DESC);
372      }
373    }
374  }
375
376  private void addGauge(MetricsRecordBuilder mrb, Map<String, Long> metricMap, String metricName,
377    String metricDesc) {
378    if (metricMap != null) {
379      for (Entry<String, Long> entry : metricMap.entrySet()) {
380        // append 'store' and its name to the metric
381        mrb.addGauge(Interns.info(this.tableNamePrefixPart1 + _COLUMNFAMILY
382          + Iterables
383            .get(Splitter.onPattern(MetricsTableWrapperAggregate.HASH).split(entry.getKey()), 1)
384          + this.tableNamePrefixPart2 + metricName, metricDesc), entry.getValue());
385      }
386    }
387  }
388
389  @Override
390  public String getTableName() {
391    return tableName.getNameAsString();
392  }
393
394  @Override
395  public int hashCode() {
396    return hashCode;
397  }
398
399  @Override
400  public boolean equals(Object o) {
401    if (this == o) {
402      return true;
403    }
404    if (!(o instanceof MetricsTableSourceImpl)) {
405      return false;
406    }
407    return (compareTo((MetricsTableSourceImpl) o) == 0);
408  }
409
410  public MetricsTableWrapperAggregate getTableWrapper() {
411    return tableWrapperAgg;
412  }
413
414  public String getTableNamePrefix() {
415    return tableNamePrefix;
416  }
417
418  @Override
419  public void incrSplitRequest() {
420    splitRequest.incr();
421  }
422
423  @Override
424  public void incrSplitSuccess() {
425    splitSuccess.incr();
426  }
427
428  @Override
429  public void updateSplitTime(long t) {
430    splitTimeHisto.add(t);
431  }
432
433  @Override
434  public void updateFlushTime(long t) {
435    flushTimeHisto.add(t);
436  }
437
438  @Override
439  public synchronized void updateFlushMemstoreSize(long bytes) {
440    flushMemstoreSizeHisto.add(bytes);
441    flushedMemstoreBytes.incr(bytes);
442  }
443
444  @Override
445  public synchronized void updateFlushOutputSize(long bytes) {
446    flushOutputSizeHisto.add(bytes);
447    flushedOutputBytes.incr(bytes);
448  }
449
450  @Override
451  public synchronized void updateCompactionTime(boolean isMajor, long t) {
452    compactionTimeHisto.add(t);
453    if (isMajor) {
454      majorCompactionTimeHisto.add(t);
455    }
456  }
457
458  @Override
459  public synchronized void updateCompactionInputFileCount(boolean isMajor, long c) {
460    compactionInputFileCountHisto.add(c);
461    if (isMajor) {
462      majorCompactionInputFileCountHisto.add(c);
463    }
464  }
465
466  @Override
467  public synchronized void updateCompactionInputSize(boolean isMajor, long bytes) {
468    compactionInputSizeHisto.add(bytes);
469    compactedInputBytes.incr(bytes);
470    if (isMajor) {
471      majorCompactionInputSizeHisto.add(bytes);
472      majorCompactedInputBytes.incr(bytes);
473    }
474  }
475
476  @Override
477  public synchronized void updateCompactionOutputFileCount(boolean isMajor, long c) {
478    compactionOutputFileCountHisto.add(c);
479    if (isMajor) {
480      majorCompactionOutputFileCountHisto.add(c);
481    }
482  }
483
484  @Override
485  public synchronized void updateCompactionOutputSize(boolean isMajor, long bytes) {
486    compactionOutputSizeHisto.add(bytes);
487    compactedOutputBytes.incr(bytes);
488    if (isMajor) {
489      majorCompactionOutputSizeHisto.add(bytes);
490      majorCompactedOutputBytes.incr(bytes);
491    }
492  }
493}