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