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.READ_REQUEST_COUNT, 279 MetricsRegionServerSource.READ_REQUEST_COUNT_DESC), 280 tableWrapperAgg.getReadRequestCount(tableName.getNameAsString())); 281 mrb.addCounter( 282 Interns.info(tableNamePrefix + MetricsRegionServerSource.FILTERED_READ_REQUEST_COUNT, 283 MetricsRegionServerSource.FILTERED_READ_REQUEST_COUNT_DESC), 284 tableWrapperAgg.getFilteredReadRequestCount(tableName.getNameAsString())); 285 mrb.addCounter( 286 Interns.info(tableNamePrefix + MetricsRegionServerSource.WRITE_REQUEST_COUNT, 287 MetricsRegionServerSource.WRITE_REQUEST_COUNT_DESC), 288 tableWrapperAgg.getWriteRequestCount(tableName.getNameAsString())); 289 mrb.addCounter( 290 Interns.info(tableNamePrefix + MetricsRegionServerSource.TOTAL_REQUEST_COUNT, 291 MetricsRegionServerSource.TOTAL_REQUEST_COUNT_DESC), 292 tableWrapperAgg.getTotalRequestsCount(tableName.getNameAsString())); 293 mrb.addGauge( 294 Interns.info(tableNamePrefix + MetricsRegionServerSource.MEMSTORE_SIZE, 295 MetricsRegionServerSource.MEMSTORE_SIZE_DESC), 296 tableWrapperAgg.getMemStoreSize(tableName.getNameAsString())); 297 mrb.addGauge( 298 Interns.info(tableNamePrefix + MetricsRegionServerSource.STOREFILE_COUNT, 299 MetricsRegionServerSource.STOREFILE_COUNT_DESC), 300 tableWrapperAgg.getNumStoreFiles(tableName.getNameAsString())); 301 mrb.addGauge( 302 Interns.info(tableNamePrefix + MetricsRegionServerSource.MAX_STOREFILE_COUNT, 303 MetricsRegionServerSource.MAX_STOREFILE_COUNT_DESC), 304 tableWrapperAgg.getMaxStoreFiles(tableName.getNameAsString())); 305 mrb.addGauge( 306 Interns.info(tableNamePrefix + MetricsRegionServerSource.STOREFILE_SIZE, 307 MetricsRegionServerSource.STOREFILE_SIZE_DESC), 308 tableWrapperAgg.getStoreFileSize(tableName.getNameAsString())); 309 mrb.addGauge( 310 Interns.info(tableNamePrefix + MetricsTableSource.TABLE_SIZE, 311 MetricsTableSource.TABLE_SIZE_DESC), 312 tableWrapperAgg.getTableSize(tableName.getNameAsString())); 313 mrb.addGauge( 314 Interns.info(tableNamePrefix + MetricsRegionServerSource.AVERAGE_REGION_SIZE, 315 MetricsRegionServerSource.AVERAGE_REGION_SIZE_DESC), 316 tableWrapperAgg.getAvgRegionSize(tableName.getNameAsString())); 317 mrb.addGauge( 318 Interns.info(tableNamePrefix + MetricsRegionServerSource.REGION_COUNT, 319 MetricsRegionServerSource.REGION_COUNT_DESC), 320 tableWrapperAgg.getNumRegions(tableName.getNameAsString())); 321 mrb.addGauge( 322 Interns.info(tableNamePrefix + MetricsRegionServerSource.STORE_COUNT, 323 MetricsRegionServerSource.STORE_COUNT_DESC), 324 tableWrapperAgg.getNumStores(tableName.getNameAsString())); 325 mrb.addGauge( 326 Interns.info(tableNamePrefix + MetricsRegionServerSource.MAX_STORE_FILE_AGE, 327 MetricsRegionServerSource.MAX_STORE_FILE_AGE_DESC), 328 tableWrapperAgg.getMaxStoreFileAge(tableName.getNameAsString())); 329 mrb.addGauge( 330 Interns.info(tableNamePrefix + MetricsRegionServerSource.MIN_STORE_FILE_AGE, 331 MetricsRegionServerSource.MIN_STORE_FILE_AGE_DESC), 332 tableWrapperAgg.getMinStoreFileAge(tableName.getNameAsString())); 333 mrb.addGauge( 334 Interns.info(tableNamePrefix + MetricsRegionServerSource.AVG_STORE_FILE_AGE, 335 MetricsRegionServerSource.AVG_STORE_FILE_AGE_DESC), 336 tableWrapperAgg.getAvgStoreFileAge(tableName.getNameAsString())); 337 mrb.addGauge( 338 Interns.info(tableNamePrefix + MetricsRegionServerSource.STATIC_BLOOM_SIZE, 339 MetricsRegionServerSource.STATIC_BLOOM_SIZE_DESC), 340 tableWrapperAgg.getStaticBloomSize(tableName.getNameAsString())); 341 mrb.addGauge( 342 Interns.info(tableNamePrefix + MetricsRegionServerSource.STATIC_INDEX_SIZE, 343 MetricsRegionServerSource.STATIC_INDEX_SIZE), 344 tableWrapperAgg.getStaticIndexSize(tableName.getNameAsString())); 345 mrb.addCounter( 346 Interns.info(tableNamePrefix + MetricsRegionServerSource.BLOOM_FILTER_REQUESTS_COUNT, 347 MetricsRegionServerSource.BLOOM_FILTER_REQUESTS_COUNT_DESC), 348 tableWrapperAgg.getBloomFilterRequestsCount(tableName.getNameAsString())); 349 mrb.addCounter( 350 Interns.info( 351 tableNamePrefix + MetricsRegionServerSource.BLOOM_FILTER_NEGATIVE_RESULTS_COUNT, 352 MetricsRegionServerSource.BLOOM_FILTER_NEGATIVE_RESULTS_COUNT_DESC), 353 tableWrapperAgg.getBloomFilterNegativeResultsCount(tableName.getNameAsString())); 354 mrb.addCounter( 355 Interns.info( 356 tableNamePrefix + MetricsRegionServerSource.BLOOM_FILTER_ELIGIBLE_REQUESTS_COUNT, 357 MetricsRegionServerSource.BLOOM_FILTER_ELIGIBLE_REQUESTS_COUNT_DESC), 358 tableWrapperAgg.getBloomFilterEligibleRequestsCount(tableName.getNameAsString())); 359 mrb.addGauge( 360 Interns.info(tableNamePrefix + MetricsRegionServerSource.NUM_REFERENCE_FILES, 361 MetricsRegionServerSource.NUM_REFERENCE_FILES_DESC), 362 tableWrapperAgg.getNumReferenceFiles(tableName.getNameAsString())); 363 addGauge(mrb, tableWrapperAgg.getMemstoreOnlyRowReadsCount(tableName.getNameAsString()), 364 MetricsRegionSource.ROW_READS_ONLY_ON_MEMSTORE, 365 MetricsRegionSource.ROW_READS_ONLY_ON_MEMSTORE_DESC); 366 addGauge(mrb, tableWrapperAgg.getMixedRowReadsCount(tableName.getNameAsString()), 367 MetricsRegionSource.MIXED_ROW_READS, MetricsRegionSource.MIXED_ROW_READS_ON_STORE_DESC); 368 } 369 } 370 } 371 372 private void addGauge(MetricsRecordBuilder mrb, Map<String, Long> metricMap, String metricName, 373 String metricDesc) { 374 if (metricMap != null) { 375 for (Entry<String, Long> entry : metricMap.entrySet()) { 376 // append 'store' and its name to the metric 377 mrb.addGauge(Interns.info(this.tableNamePrefixPart1 + _COLUMNFAMILY 378 + Iterables 379 .get(Splitter.onPattern(MetricsTableWrapperAggregate.HASH).split(entry.getKey()), 1) 380 + this.tableNamePrefixPart2 + metricName, metricDesc), entry.getValue()); 381 } 382 } 383 } 384 385 @Override 386 public String getTableName() { 387 return tableName.getNameAsString(); 388 } 389 390 @Override 391 public int hashCode() { 392 return hashCode; 393 } 394 395 @Override 396 public boolean equals(Object o) { 397 if (this == o) { 398 return true; 399 } 400 if (!(o instanceof MetricsTableSourceImpl)) { 401 return false; 402 } 403 return (compareTo((MetricsTableSourceImpl) o) == 0); 404 } 405 406 public MetricsTableWrapperAggregate getTableWrapper() { 407 return tableWrapperAgg; 408 } 409 410 public String getTableNamePrefix() { 411 return tableNamePrefix; 412 } 413 414 @Override 415 public void incrSplitRequest() { 416 splitRequest.incr(); 417 } 418 419 @Override 420 public void incrSplitSuccess() { 421 splitSuccess.incr(); 422 } 423 424 @Override 425 public void updateSplitTime(long t) { 426 splitTimeHisto.add(t); 427 } 428 429 @Override 430 public void updateFlushTime(long t) { 431 flushTimeHisto.add(t); 432 } 433 434 @Override 435 public synchronized void updateFlushMemstoreSize(long bytes) { 436 flushMemstoreSizeHisto.add(bytes); 437 flushedMemstoreBytes.incr(bytes); 438 } 439 440 @Override 441 public synchronized void updateFlushOutputSize(long bytes) { 442 flushOutputSizeHisto.add(bytes); 443 flushedOutputBytes.incr(bytes); 444 } 445 446 @Override 447 public synchronized void updateCompactionTime(boolean isMajor, long t) { 448 compactionTimeHisto.add(t); 449 if (isMajor) { 450 majorCompactionTimeHisto.add(t); 451 } 452 } 453 454 @Override 455 public synchronized void updateCompactionInputFileCount(boolean isMajor, long c) { 456 compactionInputFileCountHisto.add(c); 457 if (isMajor) { 458 majorCompactionInputFileCountHisto.add(c); 459 } 460 } 461 462 @Override 463 public synchronized void updateCompactionInputSize(boolean isMajor, long bytes) { 464 compactionInputSizeHisto.add(bytes); 465 compactedInputBytes.incr(bytes); 466 if (isMajor) { 467 majorCompactionInputSizeHisto.add(bytes); 468 majorCompactedInputBytes.incr(bytes); 469 } 470 } 471 472 @Override 473 public synchronized void updateCompactionOutputFileCount(boolean isMajor, long c) { 474 compactionOutputFileCountHisto.add(c); 475 if (isMajor) { 476 majorCompactionOutputFileCountHisto.add(c); 477 } 478 } 479 480 @Override 481 public synchronized void updateCompactionOutputSize(boolean isMajor, long bytes) { 482 compactionOutputSizeHisto.add(bytes); 483 compactedOutputBytes.incr(bytes); 484 if (isMajor) { 485 majorCompactionOutputSizeHisto.add(bytes); 486 majorCompactedOutputBytes.incr(bytes); 487 } 488 } 489}