You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@phoenix.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2022/07/07 09:49:00 UTC

[jira] [Commented] (PHOENIX-6564) Add Phoenix Connection Failed Global Metric

    [ https://issues.apache.org/jira/browse/PHOENIX-6564?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17563663#comment-17563663 ] 

ASF GitHub Bot commented on PHOENIX-6564:
-----------------------------------------

dbwong commented on code in PR #1459:
URL: https://github.com/apache/phoenix/pull/1459#discussion_r915680872


##########
phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java:
##########
@@ -1099,6 +1100,53 @@ public void testGetConnectionsThrottledForSameUrl() throws Exception {
         assertEquals(maxConnections, connections.size());
     }
 
+    @Test
+    public void testGetConnectionsFailedCounter() throws Exception {
+        int attemptedPhoenixConnections = 7;
+        //3 Failed connections and 1 throttled connection
+        int maxConnections = attemptedPhoenixConnections - 4;
+        List<Connection> connections = Lists.newArrayList();
+        String zkQuorum = "localhost:" + getUtility().getZkCluster().getClientPort();
+        String url = PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum +
+                ':' +  CUSTOM_URL_STRING + '=' + "FailedCounterTest";
+        Properties props = new Properties();
+        props.setProperty(QueryServices.CLIENT_CONNECTION_MAX_ALLOWED_CONNECTIONS, Integer.toString(maxConnections));
+        Properties props1 = new Properties(props);
+        props1.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Integer.toString(-1));
+
+        GLOBAL_QUERY_SERVICES_COUNTER.getMetric().reset();
+        GLOBAL_PHOENIX_CONNECTIONS_ATTEMPTED_COUNTER.getMetric().reset();
+        GLOBAL_PHOENIX_CONNECTIONS_THROTTLED_COUNTER.getMetric().reset();
+        GLOBAL_FAILED_PHOENIX_CONNECTIONS.getMetric().reset();
+        try {
+            for (int i = 0; i < attemptedPhoenixConnections; i++) {
+                try {
+                    if (i % 3 == 0) {
+                        connections.add(DriverManager.getConnection(url, props1));
+                    } else {
+                        connections.add(DriverManager.getConnection(url, props));
+                    }
+                } catch (SQLException se) {
+                    if (i % 3 == 0) {
+                        assertEquals(SQLExceptionCode.INVALID_SCN.getErrorCode(), se.getErrorCode());
+                    } else {
+                        assertEquals(SQLExceptionCode.NEW_CONNECTION_THROTTLED.getErrorCode(), se.getErrorCode());
+                    }
+                }
+            }
+        } finally {
+            for (Connection c : connections) {
+                c.close();
+            }
+        }
+        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());
+        assertTrue("Not all connections were attempted!",
+                attemptedPhoenixConnections <= GLOBAL_PHOENIX_CONNECTIONS_ATTEMPTED_COUNTER.getMetric().getValue());

Review Comment:
   Shouldn't we have equality here? 



##########
phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java:
##########
@@ -254,163 +255,170 @@ private PhoenixConnection(ConnectionQueryServices services, String url,
             Properties info, PMetaData metaData, MutationState mutationState,
             boolean isDescVarLengthRowKeyUpgrade, boolean isRunningUpgrade,
             boolean buildingIndex, boolean isInternalConnection) throws SQLException {
-        GLOBAL_PHOENIX_CONNECTIONS_ATTEMPTED_COUNTER.increment();
-        this.url = url;
-        this.isDescVarLengthRowKeyUpgrade = isDescVarLengthRowKeyUpgrade;
-        this.isInternalConnection = isInternalConnection;
-
-        // Filter user provided properties based on property policy, if
-        // provided and QueryServices.PROPERTY_POLICY_PROVIDER_ENABLED is true
-        if (Boolean.valueOf(info.getProperty(QueryServices.PROPERTY_POLICY_PROVIDER_ENABLED,
-                String.valueOf(QueryServicesOptions.DEFAULT_PROPERTY_POLICY_PROVIDER_ENABLED)))) {
-            PropertyPolicyProvider.getPropertyPolicy().evaluate(info);
-        }
-
-        // Copy so client cannot change
-        this.info = PropertiesUtil.deepCopy(info);
-        final PName tenantId = JDBCUtil.getTenantId(url, info);
-        if (this.info.isEmpty() && tenantId == null) {
-            this.services = services;
-        } else {
-            // Create child services keyed by tenantId to track resource usage
-            // for
-            // a tenantId for all connections on this JVM.
-            if (tenantId != null) {
-                services = services.getChildQueryServices(tenantId
-                        .getBytesPtr());
+        try {
+            GLOBAL_PHOENIX_CONNECTIONS_ATTEMPTED_COUNTER.increment();
+            this.url = url;
+            this.isDescVarLengthRowKeyUpgrade = isDescVarLengthRowKeyUpgrade;
+            this.isInternalConnection = isInternalConnection;
+
+            // Filter user provided properties based on property policy, if
+            // provided and QueryServices.PROPERTY_POLICY_PROVIDER_ENABLED is true
+            if (Boolean.valueOf(info.getProperty(QueryServices.PROPERTY_POLICY_PROVIDER_ENABLED,
+                    String.valueOf(QueryServicesOptions.DEFAULT_PROPERTY_POLICY_PROVIDER_ENABLED)))) {
+                PropertyPolicyProvider.getPropertyPolicy().evaluate(info);
             }
-            ReadOnlyProps currentProps = services.getProps();
-            final ReadOnlyProps augmentedProps = currentProps
-                    .addAll(filterKnownNonProperties(this.info));
-            this.services = augmentedProps == currentProps ? services
-                    : new DelegateConnectionQueryServices(services) {
-                @Override
-                public ReadOnlyProps getProps() {
-                    return augmentedProps;
+
+            // Copy so client cannot change
+            this.info = PropertiesUtil.deepCopy(info);
+            final PName tenantId = JDBCUtil.getTenantId(url, info);
+            if (this.info.isEmpty() && tenantId == null) {
+                this.services = services;
+            } else {
+                // Create child services keyed by tenantId to track resource usage
+                // for
+                // a tenantId for all connections on this JVM.
+                if (tenantId != null) {
+                    services = services.getChildQueryServices(tenantId
+                            .getBytesPtr());
                 }
-            };
-        }
+                ReadOnlyProps currentProps = services.getProps();
+                final ReadOnlyProps augmentedProps = currentProps
+                        .addAll(filterKnownNonProperties(this.info));
+                this.services = augmentedProps == currentProps ? services
+                        : new DelegateConnectionQueryServices(services) {
+                    @Override
+                    public ReadOnlyProps getProps() {
+                        return augmentedProps;
+                    }
+                };
+            }
 
-        Long scnParam = JDBCUtil.getCurrentSCN(url, this.info);
-        checkScn(scnParam);
-        Long buildIndexAtParam = JDBCUtil.getBuildIndexSCN(url, this.info);
-        checkBuildIndexAt(buildIndexAtParam);
-        checkScnAndBuildIndexAtEquality(scnParam, buildIndexAtParam);
+            Long scnParam = JDBCUtil.getCurrentSCN(url, this.info);
+            checkScn(scnParam);
+            Long buildIndexAtParam = JDBCUtil.getBuildIndexSCN(url, this.info);
+            checkBuildIndexAt(buildIndexAtParam);
+            checkScnAndBuildIndexAtEquality(scnParam, buildIndexAtParam);
+
+            this.scn = scnParam != null ? scnParam : buildIndexAtParam;
+            this.buildingIndex = buildingIndex || buildIndexAtParam != null;
+            this.isAutoFlush = this.services.getProps().getBoolean(
+                    QueryServices.TRANSACTIONS_ENABLED,
+                    QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED)
+                    && this.services.getProps().getBoolean(
+                    QueryServices.AUTO_FLUSH_ATTRIB,
+                    QueryServicesOptions.DEFAULT_AUTO_FLUSH);
+            this.isAutoCommit = JDBCUtil.getAutoCommit(
+                    url,
+                    this.info,
+                    this.services.getProps().getBoolean(
+                            QueryServices.AUTO_COMMIT_ATTRIB,
+                            QueryServicesOptions.DEFAULT_AUTO_COMMIT));
+            this.consistency = JDBCUtil.getConsistencyLevel(
+                    url,
+                    this.info,
+                    this.services.getProps().get(QueryServices.CONSISTENCY_ATTRIB,
+                            QueryServicesOptions.DEFAULT_CONSISTENCY_LEVEL));
+            // currently we are not resolving schema set through property, so if
+            // schema doesn't exists ,connection will not fail
+            // but queries may fail
+            this.schema = JDBCUtil.getSchema(
+                    url,
+                    this.info,
+                    this.services.getProps().get(QueryServices.SCHEMA_ATTRIB,
+                            QueryServicesOptions.DEFAULT_SCHEMA));
+            this.tenantId = tenantId;
+            this.mutateBatchSize = JDBCUtil.getMutateBatchSize(url, this.info,
+                    this.services.getProps());
+            this.mutateBatchSizeBytes = JDBCUtil.getMutateBatchSizeBytes(url,
+                    this.info, this.services.getProps());
+            datePattern = this.services.getProps().get(
+                    QueryServices.DATE_FORMAT_ATTRIB, DateUtil.DEFAULT_DATE_FORMAT);
+            timePattern = this.services.getProps().get(
+                    QueryServices.TIME_FORMAT_ATTRIB, DateUtil.DEFAULT_TIME_FORMAT);
+            timestampPattern = this.services.getProps().get(
+                    QueryServices.TIMESTAMP_FORMAT_ATTRIB,
+                    DateUtil.DEFAULT_TIMESTAMP_FORMAT);
+            String numberPattern = this.services.getProps().get(
+                    QueryServices.NUMBER_FORMAT_ATTRIB,
+                    NumberUtil.DEFAULT_NUMBER_FORMAT);
+            int maxSize = this.services.getProps().getInt(
+                    QueryServices.MAX_MUTATION_SIZE_ATTRIB,
+                    QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
+            long maxSizeBytes = this.services.getProps().getLong(
+                    QueryServices.MAX_MUTATION_SIZE_BYTES_ATTRIB,
+                    QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE_BYTES);
+            String timeZoneID = this.services.getProps().get(QueryServices.DATE_FORMAT_TIMEZONE_ATTRIB,
+                    DateUtil.DEFAULT_TIME_ZONE_ID);
+            Format dateFormat = DateUtil.getDateFormatter(datePattern, timeZoneID);
+            Format timeFormat = DateUtil.getDateFormatter(timePattern, timeZoneID);
+            Format timestampFormat = DateUtil.getDateFormatter(timestampPattern, timeZoneID);
+            formatters.put(PDate.INSTANCE, dateFormat);
+            formatters.put(PTime.INSTANCE, timeFormat);
+            formatters.put(PTimestamp.INSTANCE, timestampFormat);
+            formatters.put(PUnsignedDate.INSTANCE, dateFormat);
+            formatters.put(PUnsignedTime.INSTANCE, timeFormat);
+            formatters.put(PUnsignedTimestamp.INSTANCE, timestampFormat);
+            formatters.put(PDecimal.INSTANCE,
+                    FunctionArgumentType.NUMERIC.getFormatter(numberPattern));
+            formatters.put(PVarbinary.INSTANCE, VarBinaryFormatter.INSTANCE);
+            // We do not limit the metaData on a connection less than the global
+            // one,
+            // as there's not much that will be cached here.
+            Pruner pruner = new Pruner() {
 
-        this.scn = scnParam != null ? scnParam : buildIndexAtParam;
-        this.buildingIndex = buildingIndex || buildIndexAtParam != null;
-        this.isAutoFlush = this.services.getProps().getBoolean(
-                QueryServices.TRANSACTIONS_ENABLED,
-                QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED)
-                && this.services.getProps().getBoolean(
-                        QueryServices.AUTO_FLUSH_ATTRIB,
-                        QueryServicesOptions.DEFAULT_AUTO_FLUSH);
-        this.isAutoCommit = JDBCUtil.getAutoCommit(
-                url,
-                this.info,
-                this.services.getProps().getBoolean(
-                        QueryServices.AUTO_COMMIT_ATTRIB,
-                        QueryServicesOptions.DEFAULT_AUTO_COMMIT));
-        this.consistency = JDBCUtil.getConsistencyLevel(
-                url,
-                this.info,
-                this.services.getProps().get(QueryServices.CONSISTENCY_ATTRIB,
-                        QueryServicesOptions.DEFAULT_CONSISTENCY_LEVEL));
-        // currently we are not resolving schema set through property, so if
-        // schema doesn't exists ,connection will not fail
-        // but queries may fail
-        this.schema = JDBCUtil.getSchema(
-                url,
-                this.info,
-                this.services.getProps().get(QueryServices.SCHEMA_ATTRIB,
-                        QueryServicesOptions.DEFAULT_SCHEMA));
-        this.tenantId = tenantId;
-        this.mutateBatchSize = JDBCUtil.getMutateBatchSize(url, this.info,
-                this.services.getProps());
-        this.mutateBatchSizeBytes = JDBCUtil.getMutateBatchSizeBytes(url,
-                this.info, this.services.getProps());
-        datePattern = this.services.getProps().get(
-                QueryServices.DATE_FORMAT_ATTRIB, DateUtil.DEFAULT_DATE_FORMAT);
-        timePattern = this.services.getProps().get(
-                QueryServices.TIME_FORMAT_ATTRIB, DateUtil.DEFAULT_TIME_FORMAT);
-        timestampPattern = this.services.getProps().get(
-                QueryServices.TIMESTAMP_FORMAT_ATTRIB,
-                DateUtil.DEFAULT_TIMESTAMP_FORMAT);
-        String numberPattern = this.services.getProps().get(
-                QueryServices.NUMBER_FORMAT_ATTRIB,
-                NumberUtil.DEFAULT_NUMBER_FORMAT);
-        int maxSize = this.services.getProps().getInt(
-                QueryServices.MAX_MUTATION_SIZE_ATTRIB,
-                QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
-        long maxSizeBytes = this.services.getProps().getLong(
-                QueryServices.MAX_MUTATION_SIZE_BYTES_ATTRIB,
-                QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE_BYTES);
-        String timeZoneID = this.services.getProps().get(QueryServices.DATE_FORMAT_TIMEZONE_ATTRIB,
-                DateUtil.DEFAULT_TIME_ZONE_ID);
-        Format dateFormat = DateUtil.getDateFormatter(datePattern, timeZoneID);
-        Format timeFormat = DateUtil.getDateFormatter(timePattern, timeZoneID);
-        Format timestampFormat = DateUtil.getDateFormatter(timestampPattern, timeZoneID);
-        formatters.put(PDate.INSTANCE, dateFormat);
-        formatters.put(PTime.INSTANCE, timeFormat);
-        formatters.put(PTimestamp.INSTANCE, timestampFormat);
-        formatters.put(PUnsignedDate.INSTANCE, dateFormat);
-        formatters.put(PUnsignedTime.INSTANCE, timeFormat);
-        formatters.put(PUnsignedTimestamp.INSTANCE, timestampFormat);
-        formatters.put(PDecimal.INSTANCE,
-                FunctionArgumentType.NUMERIC.getFormatter(numberPattern));
-        formatters.put(PVarbinary.INSTANCE, VarBinaryFormatter.INSTANCE);
-        // We do not limit the metaData on a connection less than the global
-        // one,
-        // as there's not much that will be cached here.
-        Pruner pruner = new Pruner() {
+                @Override
+                public boolean prune(PTable table) {
+                    long maxTimestamp = scn == null ? HConstants.LATEST_TIMESTAMP
+                            : scn;
+                    return (table.getType() != PTableType.SYSTEM && (table
+                            .getTimeStamp() >= maxTimestamp || (table.getTenantId() != null && !Objects
+                            .equal(tenantId, table.getTenantId()))));
+                }
 
-            @Override
-            public boolean prune(PTable table) {
-                long maxTimestamp = scn == null ? HConstants.LATEST_TIMESTAMP
-                        : scn;
-                return (table.getType() != PTableType.SYSTEM && (table
-                        .getTimeStamp() >= maxTimestamp || (table.getTenantId() != null && !Objects
-                        .equal(tenantId, table.getTenantId()))));
+                @Override
+                public boolean prune(PFunction function) {
+                    long maxTimestamp = scn == null ? HConstants.LATEST_TIMESTAMP
+                            : scn;
+                    return (function.getTimeStamp() >= maxTimestamp || (function
+                            .getTenantId() != null && !Objects.equal(tenantId,
+                            function.getTenantId())));
+                }
+            };
+            this.logLevel = LogLevel.valueOf(this.services.getProps().get(QueryServices.LOG_LEVEL,
+                    QueryServicesOptions.DEFAULT_LOGGING_LEVEL));
+            this.auditLogLevel = LogLevel.valueOf(this.services.getProps().get(QueryServices.AUDIT_LOG_LEVEL,
+                    QueryServicesOptions.DEFAULT_AUDIT_LOGGING_LEVEL));
+            this.isRequestLevelMetricsEnabled = JDBCUtil.isCollectingRequestLevelMetricsEnabled(url, info,
+                    this.services.getProps());
+            this.mutationState = mutationState == null ? newMutationState(maxSize,
+                    maxSizeBytes) : new MutationState(mutationState, this);
+            this.metaData = metaData;
+            this.metaData.pruneTables(pruner);
+            this.metaData.pruneFunctions(pruner);
+            this.services.addConnection(this);
+
+            // setup tracing, if its enabled
+            this.sampler = Tracing.getConfiguredSampler(this);
+            this.customTracingAnnotations = getImmutableCustomTracingAnnotations();
+            this.scannerQueue = new LinkedBlockingQueue<>();
+            this.tableResultIteratorFactory = new DefaultTableResultIteratorFactory();
+            this.isRunningUpgrade = isRunningUpgrade;
+
+            this.logSamplingRate = Double.parseDouble(this.services.getProps().get(QueryServices.LOG_SAMPLE_RATE,
+                    QueryServicesOptions.DEFAULT_LOG_SAMPLE_RATE));
+            if (isInternalConnection) {
+                GLOBAL_OPEN_INTERNAL_PHOENIX_CONNECTIONS.increment();
+            } else {
+                GLOBAL_OPEN_PHOENIX_CONNECTIONS.increment();
             }
-
-            @Override
-            public boolean prune(PFunction function) {
-                long maxTimestamp = scn == null ? HConstants.LATEST_TIMESTAMP
-                        : scn;
-                return (function.getTimeStamp() >= maxTimestamp || (function
-                        .getTenantId() != null && !Objects.equal(tenantId,
-                                function.getTenantId())));
+            this.sourceOfOperation =
+                    this.services.getProps().get(QueryServices.SOURCE_OPERATION_ATTRIB, null);
+        } catch (SQLException sqlException) {
+            if (!isInternalConnection && sqlException.getErrorCode() != SQLExceptionCode.NEW_CONNECTION_THROTTLED.getErrorCode()) {
+                GLOBAL_FAILED_PHOENIX_CONNECTIONS.increment();
             }
-        };
-        this.logLevel= LogLevel.valueOf(this.services.getProps().get(QueryServices.LOG_LEVEL,
-                QueryServicesOptions.DEFAULT_LOGGING_LEVEL));
-        this.auditLogLevel= LogLevel.valueOf(this.services.getProps().get(QueryServices.AUDIT_LOG_LEVEL,
-                QueryServicesOptions.DEFAULT_AUDIT_LOGGING_LEVEL));
-        this.isRequestLevelMetricsEnabled = JDBCUtil.isCollectingRequestLevelMetricsEnabled(url, info,
-                this.services.getProps());
-        this.mutationState = mutationState == null ? newMutationState(maxSize,
-                maxSizeBytes) : new MutationState(mutationState, this);
-        this.metaData = metaData;
-        this.metaData.pruneTables(pruner);
-        this.metaData.pruneFunctions(pruner);
-        this.services.addConnection(this);
-
-        // setup tracing, if its enabled
-        this.sampler = Tracing.getConfiguredSampler(this);
-        this.customTracingAnnotations = getImmutableCustomTracingAnnotations();
-        this.scannerQueue = new LinkedBlockingQueue<>();
-        this.tableResultIteratorFactory = new DefaultTableResultIteratorFactory();
-        this.isRunningUpgrade = isRunningUpgrade;
-        
-        this.logSamplingRate = Double.parseDouble(this.services.getProps().get(QueryServices.LOG_SAMPLE_RATE,
-                QueryServicesOptions.DEFAULT_LOG_SAMPLE_RATE));
-        if(isInternalConnection) {
-            GLOBAL_OPEN_INTERNAL_PHOENIX_CONNECTIONS.increment();
-        } else {
-            GLOBAL_OPEN_PHOENIX_CONNECTIONS.increment();
+            throw sqlException;
         }

Review Comment:
   What about non-SQLExceptions that we might accidentally throw?





> Add Phoenix Connection Failed Global Metric
> -------------------------------------------
>
>                 Key: PHOENIX-6564
>                 URL: https://issues.apache.org/jira/browse/PHOENIX-6564
>             Project: Phoenix
>          Issue Type: New Feature
>          Components: core
>    Affects Versions: 4.16.1, 5.1.2
>            Reporter: Daniel Wong
>            Assignee: Lokesh Khurana
>            Priority: Minor
>
> Add Phoenix Connection Failed Global Metric.  We have PHOENIX_CONNECTIONS_ATTEMPTED_COUNTER as well as a PHOENIX_CONNECTIONS_THROTTLED_COUNTER but this doesn't cover all possible failures under attempted during DriverManager.getConnection.  This Jira is to provide a counter that covers all failures and to ensure that attempted also covers as wide of a net as possible.  This would be a counter metric that increments when DriverManager.getConnection fails to return a jdbc connection to the user.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)