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