You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/01/21 04:09:39 UTC

[GitHub] [pulsar] codelipenghui commented on a change in pull request #9244: [Authentication]Add authentication metrics

codelipenghui commented on a change in pull request #9244:
URL: https://github.com/apache/pulsar/pull/9244#discussion_r561485790



##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/metrics/AuthenticationMetrics.java
##########
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication.metrics;
+
+import io.prometheus.client.Counter;
+
+public class AuthenticationMetrics {
+    private static final Counter authSuccessMetrics = Counter.build()
+            .name("pulsar_auth_success")

Review comment:
       ```suggestion
               .name("pulsar_authentication_success_count")
   ```

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/metrics/AuthenticationMetrics.java
##########
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication.metrics;
+
+import io.prometheus.client.Counter;
+
+public class AuthenticationMetrics {
+    private static final Counter authSuccessMetrics = Counter.build()
+            .name("pulsar_auth_success")
+            .help("Pulsar authentication success")
+            .labelNames("authMethod")
+            .register();
+    private static final Counter authFailuresMetrics = Counter.build()
+            .name("pulsar_auth_failures")

Review comment:
       ```suggestion
               .name("pulsar_authentication_failures_count")
   ```

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/metrics/AuthenticationMetrics.java
##########
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication.metrics;
+
+import io.prometheus.client.Counter;
+
+public class AuthenticationMetrics {
+    private static final Counter authSuccessMetrics = Counter.build()
+            .name("pulsar_auth_success")
+            .help("Pulsar authentication success")
+            .labelNames("authMethod")

Review comment:
       ```suggestion
               .labelNames("auth_method")
   ```

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -313,6 +317,17 @@ public BrokerService(PulsarService pulsar) throws Exception {
         this.backlogQuotaManager = new BacklogQuotaManager(pulsar);
         this.backlogQuotaChecker = Executors
                 .newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-backlog-quota-checker"));
+        this.pulsar.addPrometheusRawMetricsProvider(stream -> {

Review comment:
       From the broker side, I think you can generate the metrics directly. The RawMetricsProvider is used by some plugins who want to expose metrics through the /metrics endpoint.

##########
File path: pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/metrics/AuthenticationMetrics.java
##########
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.authentication.metrics;
+
+import io.prometheus.client.Counter;
+
+public class AuthenticationMetrics {
+    private static final Counter authSuccessMetrics = Counter.build()
+            .name("pulsar_auth_success")
+            .help("Pulsar authentication success")
+            .labelNames("authMethod")
+            .register();
+    private static final Counter authFailuresMetrics = Counter.build()
+            .name("pulsar_auth_failures")
+            .help("Pulsar authentication failures")
+            .labelNames("authMethod", "reason")

Review comment:
       ```suggestion
               .labelNames("auth_method", "reason")
   ```

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -313,6 +317,17 @@ public BrokerService(PulsarService pulsar) throws Exception {
         this.backlogQuotaManager = new BacklogQuotaManager(pulsar);
         this.backlogQuotaChecker = Executors
                 .newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-backlog-quota-checker"));
+        this.pulsar.addPrometheusRawMetricsProvider(stream -> {

Review comment:
       I think you can follow the current broker metrics(topics metrics, namespace metrics) generate way.

##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java
##########
@@ -694,6 +716,88 @@ public void testManagedLedgerBookieClientStats() throws Exception {
         p2.close();
     }
 
+    @Test
+    public void testAuthMetrics() throws IOException, AuthenticationException {
+        SecretKey secretKey = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256);
+
+        AuthenticationProviderToken provider = new AuthenticationProviderToken();
+
+        Properties properties = new Properties();
+        properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(secretKey));
+
+        ServiceConfiguration conf = new ServiceConfiguration();
+        conf.setProperties(properties);
+        provider.initialize(conf);
+
+        String authExceptionMessage = "";
+
+        try {
+            provider.authenticate(new AuthenticationDataSource() {
+            });
+            fail("Should have failed");
+        } catch (AuthenticationException e) {
+            // expected, no credential passed
+            authExceptionMessage = e.getMessage();
+        }
+
+        String token = AuthTokenUtils.createToken(secretKey, "subject", Optional.empty());
+
+        // Pulsar protocol auth
+        String subject = provider.authenticate(new AuthenticationDataSource() {
+            @Override
+            public boolean hasDataFromCommand() {
+                return true;
+            }
+
+            @Override
+            public String getCommandData() {
+                return token;
+            }
+        });
+
+        getPulsar().addPrometheusRawMetricsProvider(stream -> {
+            try {
+                StringWriter writer = new StringWriter();
+                Set<String> metricsName = new HashSet<>(Arrays.asList("pulsar_auth_success", "pulsar_auth_failures"));
+                TextFormat.write004(writer, CollectorRegistry.defaultRegistry.filteredMetricFamilySamples(metricsName));
+                stream.write(writer.toString());
+            } catch (IOException e) {
+                throw new RuntimeException(e.getMessage(), e.getCause());
+            }
+        });

Review comment:
       It's does not make sense here to copy the implementation code to the test. If the implementation changed, the test also can passed. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org