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 */ 018 019package org.apache.hadoop.hbase.regionserver; 020 021import java.util.concurrent.ConcurrentHashMap; 022 023import org.apache.hadoop.hbase.CompatibilitySingletonFactory; 024import org.apache.hadoop.hbase.metrics.BaseSourceImpl; 025import org.apache.hadoop.hbase.metrics.Interns; 026import org.apache.hadoop.metrics2.MetricsCollector; 027import org.apache.hadoop.metrics2.MetricsRecordBuilder; 028import org.apache.yetus.audience.InterfaceAudience; 029import org.slf4j.Logger; 030import org.slf4j.LoggerFactory; 031 032@InterfaceAudience.Private 033public class MetricsTableAggregateSourceImpl extends BaseSourceImpl 034 implements MetricsTableAggregateSource { 035 036 private static final Logger LOG = LoggerFactory.getLogger(MetricsTableAggregateSourceImpl.class); 037 private ConcurrentHashMap<String, MetricsTableSource> tableSources = new ConcurrentHashMap<>(); 038 039 public MetricsTableAggregateSourceImpl() { 040 this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT); 041 } 042 043 public MetricsTableAggregateSourceImpl(String metricsName, 044 String metricsDescription, 045 String metricsContext, 046 String metricsJmxContext) { 047 super(metricsName, metricsDescription, metricsContext, metricsJmxContext); 048 } 049 050 private void register(MetricsTableSource source) { 051 source.registerMetrics(); 052 } 053 054 @Override 055 public void deleteTableSource(String table) { 056 try { 057 MetricsTableSource source = tableSources.remove(table); 058 if (source != null) { 059 source.close(); 060 } 061 } catch (Exception e) { 062 // Ignored. If this errors out it means that someone is double 063 // closing the user source and the user metrics is already nulled out. 064 LOG.info("Error trying to remove " + table + " from " + getClass().getSimpleName(), e); 065 } 066 } 067 068 @Override 069 public MetricsTableSource getOrCreateTableSource(String table, 070 MetricsTableWrapperAggregate wrapper) { 071 MetricsTableSource source = tableSources.get(table); 072 if (source != null) { 073 return source; 074 } 075 MetricsTableSource newSource = CompatibilitySingletonFactory 076 .getInstance(MetricsRegionServerSourceFactory.class).createTable(table, wrapper); 077 return tableSources.computeIfAbsent(table, k -> { 078 // register the new metrics now 079 newSource.registerMetrics(); 080 return newSource; 081 }); 082 } 083 084 /** 085 * Yes this is a get function that doesn't return anything. Thanks Hadoop for breaking all 086 * expectations of java programmers. Instead of returning anything Hadoop metrics expects 087 * getMetrics to push the metrics into the collector. 088 * 089 * @param collector the collector 090 * @param all get all the metrics regardless of when they last changed. 091 */ 092 @Override 093 public void getMetrics(MetricsCollector collector, boolean all) { 094 MetricsRecordBuilder mrb = collector.addRecord(metricsName); 095 if (tableSources != null) { 096 for (MetricsTableSource tableMetricSource : tableSources.values()) { 097 if (tableMetricSource instanceof MetricsTableSourceImpl) { 098 ((MetricsTableSourceImpl) tableMetricSource).snapshot(mrb, all); 099 } 100 } 101 mrb.addGauge(Interns.info(NUM_TABLES, NUMBER_OF_TABLES_DESC), tableSources.size()); 102 metricsRegistry.snapshot(mrb, all); 103 } 104 } 105}