C0urante commented on a change in pull request #8844:
URL: https://github.com/apache/kafka/pull/8844#discussion_r443299808



##########
File path: 
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerMetricsGroup.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.kafka.connect.runtime;
+
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.metrics.stats.CumulativeSum;
+import org.apache.kafka.common.metrics.stats.Frequencies;
+import org.apache.kafka.connect.util.ConnectorTaskId;
+
+import java.util.Map;
+
+class WorkerMetricsGroup {
+    private final ConnectMetrics.MetricGroup metricGroup;
+    private final Sensor connectorStartupAttempts;
+    private final Sensor connectorStartupSuccesses;
+    private final Sensor connectorStartupFailures;
+    private final Sensor connectorStartupResults;
+    private final Sensor taskStartupAttempts;
+    private final Sensor taskStartupSuccesses;
+    private final Sensor taskStartupFailures;
+    private final Sensor taskStartupResults;
+
+    public WorkerMetricsGroup(final Map<String, WorkerConnector> connectors, 
Map<ConnectorTaskId, WorkerTask> tasks, ConnectMetrics connectMetrics) {
+        ConnectMetricsRegistry registry = connectMetrics.registry();
+        metricGroup = connectMetrics.group(registry.workerGroupName());
+
+        metricGroup.addValueMetric(registry.connectorCount, now -> (double) 
connectors.size());
+        metricGroup.addValueMetric(registry.taskCount, now -> (double) 
tasks.size());
+
+        MetricName connectorFailurePct = 
metricGroup.metricName(registry.connectorStartupFailurePercentage);
+        MetricName connectorSuccessPct = 
metricGroup.metricName(registry.connectorStartupSuccessPercentage);
+        Frequencies connectorStartupResultFrequencies = 
Frequencies.forBooleanValues(connectorFailurePct, connectorSuccessPct);
+        connectorStartupResults = 
metricGroup.sensor("connector-startup-results");
+        connectorStartupResults.add(connectorStartupResultFrequencies);
+
+        connectorStartupAttempts = 
metricGroup.sensor("connector-startup-attempts");
+        
connectorStartupAttempts.add(metricGroup.metricName(registry.connectorStartupAttemptsTotal),
 new CumulativeSum());
+
+        connectorStartupSuccesses = 
metricGroup.sensor("connector-startup-successes");
+        
connectorStartupSuccesses.add(metricGroup.metricName(registry.connectorStartupSuccessTotal),
 new CumulativeSum());
+
+        connectorStartupFailures = 
metricGroup.sensor("connector-startup-failures");
+        
connectorStartupFailures.add(metricGroup.metricName(registry.connectorStartupFailureTotal),
 new CumulativeSum());
+
+        MetricName taskFailurePct = 
metricGroup.metricName(registry.taskStartupFailurePercentage);
+        MetricName taskSuccessPct = 
metricGroup.metricName(registry.taskStartupSuccessPercentage);
+        Frequencies taskStartupResultFrequencies = 
Frequencies.forBooleanValues(taskFailurePct, taskSuccessPct);
+        taskStartupResults = metricGroup.sensor("task-startup-results");
+        taskStartupResults.add(taskStartupResultFrequencies);
+
+        taskStartupAttempts = metricGroup.sensor("task-startup-attempts");
+        
taskStartupAttempts.add(metricGroup.metricName(registry.taskStartupAttemptsTotal),
 new CumulativeSum());
+
+        taskStartupSuccesses = metricGroup.sensor("task-startup-successes");
+        
taskStartupSuccesses.add(metricGroup.metricName(registry.taskStartupSuccessTotal),
 new CumulativeSum());
+
+        taskStartupFailures = metricGroup.sensor("task-startup-failures");
+        
taskStartupFailures.add(metricGroup.metricName(registry.taskStartupFailureTotal),
 new CumulativeSum());
+    }
+
+    void close() {
+        metricGroup.close();
+    }
+
+    void recordConnectorStartupFailure() {
+        connectorStartupAttempts.record(1.0);
+        connectorStartupFailures.record(1.0);
+        connectorStartupResults.record(0.0);
+    }
+
+    void recordConnectorStartupSuccess() {
+        connectorStartupAttempts.record(1.0);
+        connectorStartupSuccesses.record(1.0);
+        connectorStartupResults.record(1.0);
+    }
+
+    void recordTaskFailure() {
+        taskStartupAttempts.record(1.0);
+        taskStartupFailures.record(1.0);
+        taskStartupResults.record(0.0);
+    }
+
+    void recordTaskSuccess() {
+        taskStartupAttempts.record(1.0);
+        taskStartupSuccesses.record(1.0);
+        taskStartupResults.record(1.0);
+    }
+
+    protected ConnectMetrics.MetricGroup metricGroup() {
+        return metricGroup;
+    }
+
+    ConnectorStatus.Listener 
wrapConnectorStatusListener(ConnectorStatus.Listener delegateListener) {

Review comment:
       Nit: the name here is a bit verbose. The type signature of the parameter 
here already tells us that this is for a connector status listener; do you 
think `wrapStatusListener` or even just `statusListener` might convey the 
necessary information?

##########
File path: 
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerMetricsGroup.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.kafka.connect.runtime;
+
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.metrics.stats.CumulativeSum;
+import org.apache.kafka.common.metrics.stats.Frequencies;
+import org.apache.kafka.connect.util.ConnectorTaskId;
+
+import java.util.Map;
+
+class WorkerMetricsGroup {
+    private final ConnectMetrics.MetricGroup metricGroup;
+    private final Sensor connectorStartupAttempts;
+    private final Sensor connectorStartupSuccesses;
+    private final Sensor connectorStartupFailures;
+    private final Sensor connectorStartupResults;
+    private final Sensor taskStartupAttempts;
+    private final Sensor taskStartupSuccesses;
+    private final Sensor taskStartupFailures;
+    private final Sensor taskStartupResults;
+
+    public WorkerMetricsGroup(final Map<String, WorkerConnector> connectors, 
Map<ConnectorTaskId, WorkerTask> tasks, ConnectMetrics connectMetrics) {
+        ConnectMetricsRegistry registry = connectMetrics.registry();
+        metricGroup = connectMetrics.group(registry.workerGroupName());
+
+        metricGroup.addValueMetric(registry.connectorCount, now -> (double) 
connectors.size());
+        metricGroup.addValueMetric(registry.taskCount, now -> (double) 
tasks.size());
+
+        MetricName connectorFailurePct = 
metricGroup.metricName(registry.connectorStartupFailurePercentage);
+        MetricName connectorSuccessPct = 
metricGroup.metricName(registry.connectorStartupSuccessPercentage);
+        Frequencies connectorStartupResultFrequencies = 
Frequencies.forBooleanValues(connectorFailurePct, connectorSuccessPct);
+        connectorStartupResults = 
metricGroup.sensor("connector-startup-results");
+        connectorStartupResults.add(connectorStartupResultFrequencies);
+
+        connectorStartupAttempts = 
metricGroup.sensor("connector-startup-attempts");
+        
connectorStartupAttempts.add(metricGroup.metricName(registry.connectorStartupAttemptsTotal),
 new CumulativeSum());
+
+        connectorStartupSuccesses = 
metricGroup.sensor("connector-startup-successes");
+        
connectorStartupSuccesses.add(metricGroup.metricName(registry.connectorStartupSuccessTotal),
 new CumulativeSum());
+
+        connectorStartupFailures = 
metricGroup.sensor("connector-startup-failures");
+        
connectorStartupFailures.add(metricGroup.metricName(registry.connectorStartupFailureTotal),
 new CumulativeSum());
+
+        MetricName taskFailurePct = 
metricGroup.metricName(registry.taskStartupFailurePercentage);
+        MetricName taskSuccessPct = 
metricGroup.metricName(registry.taskStartupSuccessPercentage);
+        Frequencies taskStartupResultFrequencies = 
Frequencies.forBooleanValues(taskFailurePct, taskSuccessPct);
+        taskStartupResults = metricGroup.sensor("task-startup-results");
+        taskStartupResults.add(taskStartupResultFrequencies);
+
+        taskStartupAttempts = metricGroup.sensor("task-startup-attempts");
+        
taskStartupAttempts.add(metricGroup.metricName(registry.taskStartupAttemptsTotal),
 new CumulativeSum());
+
+        taskStartupSuccesses = metricGroup.sensor("task-startup-successes");
+        
taskStartupSuccesses.add(metricGroup.metricName(registry.taskStartupSuccessTotal),
 new CumulativeSum());
+
+        taskStartupFailures = metricGroup.sensor("task-startup-failures");
+        
taskStartupFailures.add(metricGroup.metricName(registry.taskStartupFailureTotal),
 new CumulativeSum());
+    }
+
+    void close() {
+        metricGroup.close();
+    }
+
+    void recordConnectorStartupFailure() {
+        connectorStartupAttempts.record(1.0);
+        connectorStartupFailures.record(1.0);
+        connectorStartupResults.record(0.0);
+    }
+
+    void recordConnectorStartupSuccess() {
+        connectorStartupAttempts.record(1.0);
+        connectorStartupSuccesses.record(1.0);
+        connectorStartupResults.record(1.0);
+    }
+
+    void recordTaskFailure() {
+        taskStartupAttempts.record(1.0);
+        taskStartupFailures.record(1.0);
+        taskStartupResults.record(0.0);
+    }
+
+    void recordTaskSuccess() {
+        taskStartupAttempts.record(1.0);
+        taskStartupSuccesses.record(1.0);
+        taskStartupResults.record(1.0);
+    }
+
+    protected ConnectMetrics.MetricGroup metricGroup() {
+        return metricGroup;
+    }
+
+    ConnectorStatus.Listener 
wrapConnectorStatusListener(ConnectorStatus.Listener delegateListener) {
+        return new WorkerMetricsGroupConnectorStatusListener(delegateListener);
+    }
+
+    TaskStatus.Listener wrapTaskStatusListener(TaskStatus.Listener 
delegateListener) {
+        return new WorkerMetricsGroupTaskStatusListener(delegateListener);
+    }
+
+    class WorkerMetricsGroupConnectorStatusListener implements 
ConnectorStatus.Listener {
+        private final ConnectorStatus.Listener delegateListener;
+        private boolean startupSucceeded = false;
+
+        WorkerMetricsGroupConnectorStatusListener(ConnectorStatus.Listener 
delegateListener) {
+            this.delegateListener = delegateListener;
+        }
+
+        @Override
+        public void onShutdown(final String connector) {
+            delegateListener.onShutdown(connector);
+        }
+
+        @Override
+        public void onFailure(final String connector, final Throwable cause) {
+            if (!startupSucceeded) {
+                recordConnectorStartupFailure();
+            }
+            delegateListener.onFailure(connector, cause);
+        }
+
+        @Override
+        public void onPause(final String connector) {
+            delegateListener.onPause(connector);
+        }
+
+        @Override
+        public void onResume(final String connector) {
+            delegateListener.onResume(connector);
+        }
+
+        @Override
+        public void onStartup(final String connector) {
+            delegateListener.onStartup(connector);
+            startupSucceeded = true;
+            recordConnectorStartupSuccess();
+        }
+
+        @Override
+        public void onDeletion(final String connector) {
+            delegateListener.onDeletion(connector);
+        }
+    }
+
+    class WorkerMetricsGroupTaskStatusListener implements TaskStatus.Listener {

Review comment:
       Same (nitty) comment here: maybe just `TaskStatusListener`?

##########
File path: 
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerMetricsGroup.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.kafka.connect.runtime;
+
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.metrics.stats.CumulativeSum;
+import org.apache.kafka.common.metrics.stats.Frequencies;
+import org.apache.kafka.connect.util.ConnectorTaskId;
+
+import java.util.Map;
+
+class WorkerMetricsGroup {
+    private final ConnectMetrics.MetricGroup metricGroup;
+    private final Sensor connectorStartupAttempts;
+    private final Sensor connectorStartupSuccesses;
+    private final Sensor connectorStartupFailures;
+    private final Sensor connectorStartupResults;
+    private final Sensor taskStartupAttempts;
+    private final Sensor taskStartupSuccesses;
+    private final Sensor taskStartupFailures;
+    private final Sensor taskStartupResults;
+
+    public WorkerMetricsGroup(final Map<String, WorkerConnector> connectors, 
Map<ConnectorTaskId, WorkerTask> tasks, ConnectMetrics connectMetrics) {
+        ConnectMetricsRegistry registry = connectMetrics.registry();
+        metricGroup = connectMetrics.group(registry.workerGroupName());
+
+        metricGroup.addValueMetric(registry.connectorCount, now -> (double) 
connectors.size());
+        metricGroup.addValueMetric(registry.taskCount, now -> (double) 
tasks.size());
+
+        MetricName connectorFailurePct = 
metricGroup.metricName(registry.connectorStartupFailurePercentage);
+        MetricName connectorSuccessPct = 
metricGroup.metricName(registry.connectorStartupSuccessPercentage);
+        Frequencies connectorStartupResultFrequencies = 
Frequencies.forBooleanValues(connectorFailurePct, connectorSuccessPct);
+        connectorStartupResults = 
metricGroup.sensor("connector-startup-results");
+        connectorStartupResults.add(connectorStartupResultFrequencies);
+
+        connectorStartupAttempts = 
metricGroup.sensor("connector-startup-attempts");
+        
connectorStartupAttempts.add(metricGroup.metricName(registry.connectorStartupAttemptsTotal),
 new CumulativeSum());
+
+        connectorStartupSuccesses = 
metricGroup.sensor("connector-startup-successes");
+        
connectorStartupSuccesses.add(metricGroup.metricName(registry.connectorStartupSuccessTotal),
 new CumulativeSum());
+
+        connectorStartupFailures = 
metricGroup.sensor("connector-startup-failures");
+        
connectorStartupFailures.add(metricGroup.metricName(registry.connectorStartupFailureTotal),
 new CumulativeSum());
+
+        MetricName taskFailurePct = 
metricGroup.metricName(registry.taskStartupFailurePercentage);
+        MetricName taskSuccessPct = 
metricGroup.metricName(registry.taskStartupSuccessPercentage);
+        Frequencies taskStartupResultFrequencies = 
Frequencies.forBooleanValues(taskFailurePct, taskSuccessPct);
+        taskStartupResults = metricGroup.sensor("task-startup-results");
+        taskStartupResults.add(taskStartupResultFrequencies);
+
+        taskStartupAttempts = metricGroup.sensor("task-startup-attempts");
+        
taskStartupAttempts.add(metricGroup.metricName(registry.taskStartupAttemptsTotal),
 new CumulativeSum());
+
+        taskStartupSuccesses = metricGroup.sensor("task-startup-successes");
+        
taskStartupSuccesses.add(metricGroup.metricName(registry.taskStartupSuccessTotal),
 new CumulativeSum());
+
+        taskStartupFailures = metricGroup.sensor("task-startup-failures");
+        
taskStartupFailures.add(metricGroup.metricName(registry.taskStartupFailureTotal),
 new CumulativeSum());
+    }
+
+    void close() {
+        metricGroup.close();
+    }
+
+    void recordConnectorStartupFailure() {
+        connectorStartupAttempts.record(1.0);
+        connectorStartupFailures.record(1.0);
+        connectorStartupResults.record(0.0);
+    }
+
+    void recordConnectorStartupSuccess() {
+        connectorStartupAttempts.record(1.0);
+        connectorStartupSuccesses.record(1.0);
+        connectorStartupResults.record(1.0);
+    }
+
+    void recordTaskFailure() {
+        taskStartupAttempts.record(1.0);
+        taskStartupFailures.record(1.0);
+        taskStartupResults.record(0.0);
+    }
+
+    void recordTaskSuccess() {
+        taskStartupAttempts.record(1.0);
+        taskStartupSuccesses.record(1.0);
+        taskStartupResults.record(1.0);
+    }
+
+    protected ConnectMetrics.MetricGroup metricGroup() {
+        return metricGroup;
+    }
+
+    ConnectorStatus.Listener 
wrapConnectorStatusListener(ConnectorStatus.Listener delegateListener) {
+        return new WorkerMetricsGroupConnectorStatusListener(delegateListener);
+    }
+
+    TaskStatus.Listener wrapTaskStatusListener(TaskStatus.Listener 
delegateListener) {
+        return new WorkerMetricsGroupTaskStatusListener(delegateListener);
+    }
+
+    class WorkerMetricsGroupConnectorStatusListener implements 
ConnectorStatus.Listener {

Review comment:
       Another nit (sorry!): given that this is already an inner class for the 
`WorkerMetricsGroup` class, the `WorkerMetricsGroup` prefix seems redundant. 
What do you think about just `ConnectoStatusListener`?

##########
File path: 
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerMetricsGroup.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.kafka.connect.runtime;
+
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.metrics.stats.CumulativeSum;
+import org.apache.kafka.common.metrics.stats.Frequencies;
+import org.apache.kafka.connect.util.ConnectorTaskId;
+
+import java.util.Map;
+
+class WorkerMetricsGroup {
+    private final ConnectMetrics.MetricGroup metricGroup;
+    private final Sensor connectorStartupAttempts;
+    private final Sensor connectorStartupSuccesses;
+    private final Sensor connectorStartupFailures;
+    private final Sensor connectorStartupResults;
+    private final Sensor taskStartupAttempts;
+    private final Sensor taskStartupSuccesses;
+    private final Sensor taskStartupFailures;
+    private final Sensor taskStartupResults;
+
+    public WorkerMetricsGroup(final Map<String, WorkerConnector> connectors, 
Map<ConnectorTaskId, WorkerTask> tasks, ConnectMetrics connectMetrics) {
+        ConnectMetricsRegistry registry = connectMetrics.registry();
+        metricGroup = connectMetrics.group(registry.workerGroupName());
+
+        metricGroup.addValueMetric(registry.connectorCount, now -> (double) 
connectors.size());
+        metricGroup.addValueMetric(registry.taskCount, now -> (double) 
tasks.size());
+
+        MetricName connectorFailurePct = 
metricGroup.metricName(registry.connectorStartupFailurePercentage);
+        MetricName connectorSuccessPct = 
metricGroup.metricName(registry.connectorStartupSuccessPercentage);
+        Frequencies connectorStartupResultFrequencies = 
Frequencies.forBooleanValues(connectorFailurePct, connectorSuccessPct);
+        connectorStartupResults = 
metricGroup.sensor("connector-startup-results");
+        connectorStartupResults.add(connectorStartupResultFrequencies);
+
+        connectorStartupAttempts = 
metricGroup.sensor("connector-startup-attempts");
+        
connectorStartupAttempts.add(metricGroup.metricName(registry.connectorStartupAttemptsTotal),
 new CumulativeSum());
+
+        connectorStartupSuccesses = 
metricGroup.sensor("connector-startup-successes");
+        
connectorStartupSuccesses.add(metricGroup.metricName(registry.connectorStartupSuccessTotal),
 new CumulativeSum());
+
+        connectorStartupFailures = 
metricGroup.sensor("connector-startup-failures");
+        
connectorStartupFailures.add(metricGroup.metricName(registry.connectorStartupFailureTotal),
 new CumulativeSum());
+
+        MetricName taskFailurePct = 
metricGroup.metricName(registry.taskStartupFailurePercentage);
+        MetricName taskSuccessPct = 
metricGroup.metricName(registry.taskStartupSuccessPercentage);
+        Frequencies taskStartupResultFrequencies = 
Frequencies.forBooleanValues(taskFailurePct, taskSuccessPct);
+        taskStartupResults = metricGroup.sensor("task-startup-results");
+        taskStartupResults.add(taskStartupResultFrequencies);
+
+        taskStartupAttempts = metricGroup.sensor("task-startup-attempts");
+        
taskStartupAttempts.add(metricGroup.metricName(registry.taskStartupAttemptsTotal),
 new CumulativeSum());
+
+        taskStartupSuccesses = metricGroup.sensor("task-startup-successes");
+        
taskStartupSuccesses.add(metricGroup.metricName(registry.taskStartupSuccessTotal),
 new CumulativeSum());
+
+        taskStartupFailures = metricGroup.sensor("task-startup-failures");
+        
taskStartupFailures.add(metricGroup.metricName(registry.taskStartupFailureTotal),
 new CumulativeSum());
+    }
+
+    void close() {
+        metricGroup.close();
+    }
+
+    void recordConnectorStartupFailure() {
+        connectorStartupAttempts.record(1.0);
+        connectorStartupFailures.record(1.0);
+        connectorStartupResults.record(0.0);
+    }
+
+    void recordConnectorStartupSuccess() {
+        connectorStartupAttempts.record(1.0);
+        connectorStartupSuccesses.record(1.0);
+        connectorStartupResults.record(1.0);
+    }
+
+    void recordTaskFailure() {
+        taskStartupAttempts.record(1.0);
+        taskStartupFailures.record(1.0);
+        taskStartupResults.record(0.0);
+    }
+
+    void recordTaskSuccess() {
+        taskStartupAttempts.record(1.0);
+        taskStartupSuccesses.record(1.0);
+        taskStartupResults.record(1.0);
+    }
+
+    protected ConnectMetrics.MetricGroup metricGroup() {
+        return metricGroup;
+    }
+
+    ConnectorStatus.Listener 
wrapConnectorStatusListener(ConnectorStatus.Listener delegateListener) {
+        return new WorkerMetricsGroupConnectorStatusListener(delegateListener);
+    }
+
+    TaskStatus.Listener wrapTaskStatusListener(TaskStatus.Listener 
delegateListener) {

Review comment:
       Same comment here w/r/t naming; I think `wrapStatusListener` or 
`statusListener` may be warranted.

##########
File path: 
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -289,7 +285,6 @@ public void startConnector(
                 // Can't be put in a finally block because it needs to be 
swapped before the call on
                 // statusListener
                 Plugins.compareAndSwapLoaders(savedLoader);
-                workerMetricsGroup.recordConnectorStartupFailure();

Review comment:
       I think we might want to keep this line here in case we fail somehow 
before even creating the `WorkerConnector` instance.

##########
File path: 
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java
##########
@@ -56,7 +56,7 @@
     private static final String THREAD_NAME_PREFIX = "task-thread-";
 
     protected final ConnectorTaskId id;
-    private final TaskStatus.Listener statusListener;
+    protected final TaskStatus.Listener statusListener;

Review comment:
       It seems a little unclean to start calling the `statusListener` from the 
subclass when it's been used exclusively by the `WorkerTask` abstract class up 
to this point. Not the end of the world but I think we might be able to do this 
more cleanly by decomposing the existing `execute` method into separate 
`initializeAndStart` (name obviously subject to change) and `execute` methods, 
with the call to `statusListener::onStartup` in between them. This way, we can 
avoid worrying about the status listener in the subclasses and can  encapsulate 
some shared logic in the abstract superclass. What are your thoughts?

##########
File path: 
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -562,7 +556,6 @@ public boolean startTask(
                 // statusListener
                 Plugins.compareAndSwapLoaders(savedLoader);
                 connectorStatusMetricsGroup.recordTaskRemoved(id);
-                workerMetricsGroup.recordTaskFailure();

Review comment:
       I think we might want to keep this line here in case we fail somehow 
before even creating the `WorkerTask` instance. This can happen if a 
`Converter`, `Transformation`, etc. throws an exception during startup.




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to