Skip to content

[PHOENIX-6825] Metric to track phoenix connection time #2062

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 2 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_FAILED_PHOENIX_CONNECTIONS;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_PHOENIX_CONNECTIONS_ATTEMPTED_COUNTER;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_PHOENIX_CONNECTION_TIME;
import static org.apache.phoenix.thirdparty.com.google.common.base.Preconditions.checkNotNull;

import java.sql.Connection;
Expand All @@ -44,6 +45,7 @@
import org.apache.phoenix.query.QueryServices;
import org.apache.phoenix.query.QueryServicesImpl;
import org.apache.phoenix.query.QueryServicesOptions;
import org.apache.phoenix.util.EnvironmentEdgeManager;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -218,6 +220,7 @@ public boolean acceptsURL(String url) throws SQLException {

@Override
public Connection connect(String url, Properties info) throws SQLException {
final long connectionStartTime = EnvironmentEdgeManager.currentTimeMillis();
GLOBAL_PHOENIX_CONNECTIONS_ATTEMPTED_COUNTER.increment();
if (!acceptsURL(url)) {
GLOBAL_FAILED_PHOENIX_CONNECTIONS.increment();
Expand All @@ -237,6 +240,8 @@ public Connection connect(String url, Properties info) throws SQLException {
throw e;
} finally {
unlock(LockMode.READ);
GLOBAL_PHOENIX_CONNECTION_TIME.update(
EnvironmentEdgeManager.currentTimeMillis() - connectionStartTime);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@
import static org.apache.phoenix.monitoring.MetricType.PHOENIX_CONNECTIONS_THROTTLED_COUNTER;
import static org.apache.phoenix.monitoring.MetricType.PHOENIX_CONNECTIONS_ATTEMPTED_COUNTER;
import static org.apache.phoenix.monitoring.MetricType.PHOENIX_CONNECTIONS_FAILED_COUNTER;
import static org.apache.phoenix.monitoring.MetricType.PHOENIX_CONNECTION_TIME;
import static org.apache.phoenix.monitoring.MetricType.TASK_REJECTED_COUNTER;
import static org.apache.phoenix.monitoring.MetricType.HA_PARALLEL_POOL1_TASK_END_TO_END_TIME;
import static org.apache.phoenix.monitoring.MetricType.HA_PARALLEL_POOL1_TASK_EXECUTED_COUNTER;
Expand Down Expand Up @@ -132,6 +133,7 @@ public enum GlobalClientMetrics {
GLOBAL_HCONNECTIONS_COUNTER(HCONNECTIONS_COUNTER),
GLOBAL_PHOENIX_CONNECTIONS_THROTTLED_COUNTER(PHOENIX_CONNECTIONS_THROTTLED_COUNTER),
GLOBAL_PHOENIX_CONNECTIONS_ATTEMPTED_COUNTER(PHOENIX_CONNECTIONS_ATTEMPTED_COUNTER),
GLOBAL_PHOENIX_CONNECTION_TIME(PHOENIX_CONNECTION_TIME),
GLOBAL_PAGED_ROWS_COUNTER(PAGED_ROWS_COUNTER),
GLOBAL_HBASE_COUNT_RPC_CALLS(COUNT_RPC_CALLS),
GLOBAL_HBASE_COUNT_REMOTE_RPC_CALLS(COUNT_REMOTE_RPC_CALLS),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -147,6 +147,7 @@ public enum MetricType {
PHOENIX_CONNECTIONS_ATTEMPTED_COUNTER("ca","Number of requests for Phoenix connections, whether successful or not.",LogLevel.OFF, PLong.INSTANCE),
PHOENIX_CONNECTIONS_FAILED_COUNTER("cf", "Number of client Phoenix Connections Failed to open" +
", not including throttled connections", LogLevel.OFF, PLong.INSTANCE),
PHOENIX_CONNECTION_TIME("pct", "Phoenix connection creation time", LogLevel.OFF, PLong.INSTANCE),
CLIENT_METADATA_CACHE_MISS_COUNTER("cmcm", "Number of cache misses for the CQSI cache.", LogLevel.DEBUG, PLong.INSTANCE),
CLIENT_METADATA_CACHE_HIT_COUNTER("cmch", "Number of cache hits for the CQSI cache.", LogLevel.DEBUG, PLong.INSTANCE),
CLIENT_METADATA_CACHE_EVICTION_COUNTER("cmce", "Number of cache evictions for the CQSI cache" +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_OPEN_PHOENIX_CONNECTIONS;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_PHOENIX_CONNECTIONS_ATTEMPTED_COUNTER;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_PHOENIX_CONNECTIONS_THROTTLED_COUNTER;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_PHOENIX_CONNECTION_TIME;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_SERVICES_COUNTER;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_TIME;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_TIMEOUT_COUNTER;
Expand Down Expand Up @@ -1070,6 +1071,7 @@ public void testGetConnectionsThrottledForSameUrl() throws Exception {
Properties props = new Properties();
props.setProperty(QueryServices.CLIENT_CONNECTION_MAX_ALLOWED_CONNECTIONS, Integer.toString(maxConnections));

GLOBAL_PHOENIX_CONNECTION_TIME.getMetric().reset();
GLOBAL_HCONNECTIONS_COUNTER.getMetric().reset();
GLOBAL_QUERY_SERVICES_COUNTER.getMetric().reset();
GLOBAL_PHOENIX_CONNECTIONS_ATTEMPTED_COUNTER.getMetric().reset();
Expand All @@ -1087,6 +1089,7 @@ public void testGetConnectionsThrottledForSameUrl() throws Exception {
c.close();
}
}
assert(GLOBAL_PHOENIX_CONNECTION_TIME.getMetric().getValue() > 0);
assertEquals(1, GLOBAL_QUERY_SERVICES_COUNTER.getMetric().getValue());
assertTrue("No connection was throttled!", wasThrottled);
assertEquals(1, GLOBAL_PHOENIX_CONNECTIONS_THROTTLED_COUNTER.getMetric().getValue());
Expand Down Expand Up @@ -1127,7 +1130,8 @@ public void testGetConnectionsFailedCounter() throws Exception {
Properties props2 = new Properties(props);
//Will create IllegalArgumentException while parsing loglevel
props2.setProperty(QueryServices.LOG_LEVEL, "notKnown");


GLOBAL_PHOENIX_CONNECTION_TIME.getMetric().reset();
GLOBAL_QUERY_SERVICES_COUNTER.getMetric().reset();
GLOBAL_PHOENIX_CONNECTIONS_ATTEMPTED_COUNTER.getMetric().reset();
GLOBAL_PHOENIX_CONNECTIONS_THROTTLED_COUNTER.getMetric().reset();
Expand Down Expand Up @@ -1157,6 +1161,7 @@ public void testGetConnectionsFailedCounter() throws Exception {
c.close();
}
}
assert(GLOBAL_PHOENIX_CONNECTION_TIME.getMetric().getValue() > 0);
assertEquals(1, GLOBAL_QUERY_SERVICES_COUNTER.getMetric().getValue());
assertEquals(1, GLOBAL_PHOENIX_CONNECTIONS_THROTTLED_COUNTER.getMetric().getValue());
assertEquals(3, GLOBAL_FAILED_PHOENIX_CONNECTIONS.getMetric().getValue());
Expand Down Expand Up @@ -1205,6 +1210,7 @@ public void testGetConnectionsForDifferentTenantsConcurrently() throws Exceptio
c.close();
} catch (Exception ignore) {}
}
assert(GLOBAL_PHOENIX_CONNECTION_TIME.getMetric().getValue() > 0);
assertEquals(expectedHConnections, GLOBAL_HCONNECTIONS_COUNTER.getMetric().getValue());
assertEquals(expectedHConnections, GLOBAL_QUERY_SERVICES_COUNTER.getMetric().getValue());
} finally {
Expand Down Expand Up @@ -1236,6 +1242,7 @@ public void testGetConnectionsWithDifferentJDBCParamsConcurrently() throws Exce
for (int i = 0; i < futures.size(); i++) {
futures.get(i).get();
}
assert(GLOBAL_PHOENIX_CONNECTION_TIME.getMetric().getValue() > 0);
assertEquals(numConnections, GLOBAL_HCONNECTIONS_COUNTER.getMetric().getValue());
assertEquals(numConnections, GLOBAL_QUERY_SERVICES_COUNTER.getMetric().getValue());
} finally {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,9 +32,11 @@
import org.apache.phoenix.query.ConnectionlessQueryServicesImpl;
import org.apache.phoenix.query.QueryServices;
import org.apache.phoenix.query.QueryServicesTestImpl;
import org.apache.phoenix.util.EnvironmentEdgeManager;
import org.apache.phoenix.util.PropertiesUtil;
import org.apache.phoenix.util.ReadOnlyProps;

import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_PHOENIX_CONNECTION_TIME;


/**
Expand All @@ -49,7 +51,7 @@
public class PhoenixTestDriver extends PhoenixEmbeddedDriver {

private final ReadOnlyProps overrideProps;

@GuardedBy("this")
private final QueryServices queryServices;

Expand Down Expand Up @@ -84,8 +86,11 @@ public boolean acceptsURL(String url) throws SQLException {

@Override
public synchronized Connection connect(String url, Properties info) throws SQLException {
final long connectionStartTime = EnvironmentEdgeManager.currentTimeMillis();
checkClosed();
return super.connect(url, info);
Connection testConnect = super.connect(url, info);
GLOBAL_PHOENIX_CONNECTION_TIME.update(EnvironmentEdgeManager.currentTimeMillis() - connectionStartTime);
return testConnect;
}

@Override // public for testing
Expand Down