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