HBASE-26891 Make MetricsConnection scope configurable (#4285)
Signed-off-by: Nick Dimiduk <ndimiduk@apache.org> Signed-off-by: Andrew Purtell <apurtell@apache.org>
This commit is contained in:
parent
f2f2a89592
commit
9a88092817
|
@ -132,7 +132,8 @@ public class AsyncConnectionImpl implements AsyncConnection {
|
|||
this.connConf = new AsyncConnectionConfiguration(conf);
|
||||
this.registry = registry;
|
||||
if (conf.getBoolean(CLIENT_SIDE_METRICS_ENABLED_KEY, false)) {
|
||||
this.metrics = Optional.of(new MetricsConnection(this.toString(), () -> null, () -> null));
|
||||
String scope = MetricsConnection.getScope(conf, clusterId, this);
|
||||
this.metrics = Optional.of(new MetricsConnection(scope, () -> null, () -> null));
|
||||
} else {
|
||||
this.metrics = Optional.empty();
|
||||
}
|
||||
|
|
|
@ -33,14 +33,16 @@ import java.util.concurrent.ConcurrentSkipListMap;
|
|||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
* This class is for maintaining the various connection statistics and publishing them through
|
||||
|
@ -57,6 +59,34 @@ public class MetricsConnection implements StatisticTrackable {
|
|||
/** Set this key to {@code true} to enable metrics collection of client requests. */
|
||||
public static final String CLIENT_SIDE_METRICS_ENABLED_KEY = "hbase.client.metrics.enable";
|
||||
|
||||
/**
|
||||
* Set to specify a custom scope for the metrics published through {@link MetricsConnection}.
|
||||
* The scope is added to JMX MBean objectName, and defaults to a combination of the Connection's
|
||||
* clusterId and hashCode. For example, a default value for a connection to cluster "foo" might
|
||||
* be "foo-7d9d0818", where "7d9d0818" is the hashCode of the underlying AsyncConnectionImpl.
|
||||
* Users may set this key to give a more contextual name for this scope. For example, one might
|
||||
* want to differentiate a read connection from a write connection by setting the scopes to
|
||||
* "foo-read" and "foo-write" respectively.
|
||||
*
|
||||
* Scope is the only thing that lends any uniqueness to the metrics. Care should be taken to
|
||||
* avoid using the same scope for multiple Connections, otherwise the metrics may aggregate in
|
||||
* unforeseen ways.
|
||||
*/
|
||||
public static final String METRICS_SCOPE_KEY = "hbase.client.metrics.scope";
|
||||
|
||||
/**
|
||||
* Returns the scope for a MetricsConnection based on the configured {@link #METRICS_SCOPE_KEY}
|
||||
* or by generating a default from the passed clusterId and connectionObj's hashCode.
|
||||
* @param conf configuration for the connection
|
||||
* @param clusterId clusterId for the connection
|
||||
* @param connectionObj either a Connection or AsyncConnectionImpl, the instance
|
||||
* creating this MetricsConnection.
|
||||
*/
|
||||
static String getScope(Configuration conf, String clusterId, Object connectionObj) {
|
||||
return conf.get(METRICS_SCOPE_KEY,
|
||||
clusterId + "@" + Integer.toHexString(connectionObj.hashCode()));
|
||||
}
|
||||
|
||||
private static final String CNT_BASE = "rpcCount_";
|
||||
private static final String DRTN_BASE = "rpcCallDurationMs_";
|
||||
private static final String REQ_BASE = "rpcCallRequestSizeBytes_";
|
||||
|
@ -251,7 +281,7 @@ public class MetricsConnection implements StatisticTrackable {
|
|||
|
||||
private final MetricRegistry registry;
|
||||
private final JmxReporter reporter;
|
||||
private final String scope;
|
||||
protected final String scope;
|
||||
|
||||
private final NewMetric<Timer> timerFactory = new NewMetric<Timer>() {
|
||||
@Override public Timer newMetric(Class<?> clazz, String name, String scope) {
|
||||
|
|
|
@ -23,9 +23,12 @@ import static org.junit.Assert.assertTrue;
|
|||
import com.codahale.metrics.RatioGauge;
|
||||
import com.codahale.metrics.RatioGauge.Ratio;
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MetricsTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
|
@ -67,6 +70,25 @@ public class TestMetricsConnection {
|
|||
METRICS.shutdown();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMetricsConnectionScope() throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
String clusterId = "foo";
|
||||
String scope = "testScope";
|
||||
conf.setBoolean(MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY, true);
|
||||
|
||||
AsyncConnectionImpl impl = new AsyncConnectionImpl(conf, null, "foo", null, User.getCurrent());
|
||||
Optional<MetricsConnection> metrics = impl.getConnectionMetrics();
|
||||
assertTrue("Metrics should be present", metrics.isPresent());
|
||||
assertEquals(clusterId + "@" + Integer.toHexString(impl.hashCode()), metrics.get().scope);
|
||||
conf.set(MetricsConnection.METRICS_SCOPE_KEY, scope);
|
||||
impl = new AsyncConnectionImpl(conf, null, "foo", null, User.getCurrent());
|
||||
|
||||
metrics = impl.getConnectionMetrics();
|
||||
assertTrue("Metrics should be present", metrics.isPresent());
|
||||
assertEquals(scope, metrics.get().scope);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStaticMetrics() throws IOException {
|
||||
final byte[] foo = Bytes.toBytes("foo");
|
||||
|
|
Loading…
Reference in New Issue