gharris1727 commented on code in PR #17988:
URL: https://github.com/apache/kafka/pull/17988#discussion_r1987931269


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -562,6 +562,20 @@ public boolean isRunning(String connName) {
         return workerConnector != null && workerConnector.isRunning();
     }
 
+    public String connectorVersion(String connName) {
+        if (connectors.get(connName) == null) {
+            return null;
+        }
+        return connectors.get(connName).connectorVersion();
+    }
+
+    public String taskVersion(ConnectorTaskId taskId) {
+        if (tasks.get(taskId) == null) {
+            return null;
+        }
+        return tasks.get(taskId).taskVersion();
+    }
+

Review Comment:
   These are thread-unsafe and could cause NullPointerExceptions.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskStatus.java:
##########
@@ -20,6 +20,10 @@
 
 public class TaskStatus extends AbstractStatus<ConnectorTaskId> {
 
+    public TaskStatus(ConnectorTaskId id, State state, String workerUrl, int 
generation, String trace, String version) {
+        super(id, state, workerUrl, generation, trace, version);
+    }
+
     public TaskStatus(ConnectorTaskId id, State state, String workerUrl, int 
generation, String trace) {

Review Comment:
   nit: This method is unused.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractStatus.java:
##########
@@ -34,18 +34,29 @@ public enum State {
     private final State state;
     private final String trace;
     private final String workerId;
+    private final String version;

Review Comment:
   Should this also be part of equals/hashCode?



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java:
##########
@@ -61,15 +61,18 @@ public ConnectorType type() {
         return type;
     }
 
+

Review Comment:
   nit: newline



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java:
##########
@@ -391,6 +391,11 @@ public void execute() {
         finalOffsetCommit(false);
     }
 
+    @Override
+    public String taskVersion() {
+        return task.version();

Review Comment:
   This is a call into the real task object, which requires a context 
classloader swap, can throw arbitary exceptions, and may block indefinitely. We 
should either use Plugins#pluginVersion to compute this, or compute it once 
during instantiation and then return it whenever necessary.
   
   The same comment applies to WorkerConnector, WorkerSinkTask, and 
TaskPluginsMetadata.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java:
##########
@@ -87,14 +90,26 @@ public String workerId() {
         public String trace() {
             return trace;
         }
+
+        @JsonProperty
+        @JsonInclude(JsonInclude.Include.NON_EMPTY)
+        public String version() {
+            return version;
+        }
     }
 
     public static class ConnectorState extends AbstractState {
+
         @JsonCreator
         public ConnectorState(@JsonProperty("state") String state,
                               @JsonProperty("worker_id") String worker,
-                              @JsonProperty("msg") String msg) {
-            super(state, worker, msg);
+                              @JsonProperty("msg") String msg,
+                              @JsonProperty("version") String version) {
+            super(state, worker, msg, version);
+        }
+
+        public ConnectorState(String state, String worker, String trace) {

Review Comment:
   nit: I would eliminate this and change the call-sites.
   
   Same comment for TaskState



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorStatus.java:
##########
@@ -19,10 +19,18 @@
 
 public class ConnectorStatus extends AbstractStatus<String> {
 
+    public ConnectorStatus(String connector, State state, String msg, String 
workerUrl, int generation, String version) {
+        super(connector, state, workerUrl, generation, msg, version);
+    }
+
     public ConnectorStatus(String connector, State state, String msg, String 
workerUrl, int generation) {

Review Comment:
   nit: This constructor is unused.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskPluginsMetadata.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.connect.connector.Connector;
+import org.apache.kafka.connect.connector.Task;
+import org.apache.kafka.connect.health.ConnectorType;
+import org.apache.kafka.connect.runtime.isolation.LoaderSwap;
+import org.apache.kafka.connect.runtime.isolation.PluginType;
+import org.apache.kafka.connect.runtime.isolation.PluginUtils;
+import org.apache.kafka.connect.runtime.isolation.Plugins;
+import org.apache.kafka.connect.sink.SinkConnector;
+import org.apache.kafka.connect.source.SourceConnector;
+import org.apache.kafka.connect.storage.Converter;
+import org.apache.kafka.connect.storage.HeaderConverter;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class TaskPluginsMetadata {
+
+    private final String connectorClass;
+    private final String connectorVersion;
+    private final ConnectorType connectorType;
+    private final String taskClass;
+    private final String taskVersion;
+    private final String keyConverterClass;
+    private final String keyConverterVersion;
+    private final String valueConverterClass;
+    private final String valueConverterVersion;
+    private final String headerConverterClass;
+    private final String headerConverterVersion;
+    private final Set<TransformationStage.AliasedPluginInfo> transformations;
+    private final Set<TransformationStage.AliasedPluginInfo> predicates;
+
+    public TaskPluginsMetadata(
+            Class<? extends Connector> connectorClass,
+            Task task,
+            Converter keyConverter,
+            Converter valueConverter,
+            HeaderConverter headerConverter,
+            List<TransformationStage.StageInfo> transformationStageInfo,
+            Plugins plugins
+    ) {
+
+        assert connectorClass != null;
+        assert task != null;
+        assert keyConverter != null;
+        assert valueConverter != null;
+        assert headerConverter != null;
+        assert transformationStageInfo != null;
+
+        Function<ClassLoader, LoaderSwap> pluginLoaderSwapper = 
plugins.safeLoaderSwapper();
+
+        this.connectorClass = connectorClass.getName();
+        this.connectorVersion = 
plugins.pluginVersion(connectorClass.getName(), 
connectorClass.getClassLoader(), PluginType.SINK, PluginType.SOURCE);
+        this.connectorType = getConnectorType(connectorClass, 
pluginLoaderSwapper);
+        this.taskClass = task.getClass().getName();
+        this.taskVersion = task.version();
+        this.keyConverterClass = keyConverter.getClass().getName();
+        this.keyConverterVersion = 
PluginUtils.getVersionOrUndefined(keyConverter, pluginLoaderSwapper);

Review Comment:
   Why does the connector use Plugins#pluginVersion, but all the sub-plugins 
use PluginUtils.getVersionOrUndefined?
   
   If you only used pluginVersion, you wouldn't need the pluginLoaderSwapper at 
all, because no isolated code would be called. This would also prevent 
exceptions from plugin code from appearing.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java:
##########
@@ -87,14 +90,26 @@ public String workerId() {
         public String trace() {
             return trace;
         }
+
+        @JsonProperty
+        @JsonInclude(JsonInclude.Include.NON_EMPTY)

Review Comment:
   The other place in the REST API that surfaces the version is the PluginInfo 
object, which filters out `"undefined"`. We should probably be consistent here.
   
   That doesn't filter out `null` as far as i can tell, maybe it should be 
modified to filter out both null and undefined.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java:
##########
@@ -442,8 +456,51 @@ private void addRatioMetric(final State matchingState, 
MetricNameTemplate templa
                     taskStateTimer.durationRatio(matchingState, now));
         }
 
+        public void addPluginInfoMetric(TaskPluginsMetadata pluginsMetadata) {
+            ConnectMetricsRegistry registry = connectMetrics.registry();
+            metricGroup.addValueMetric(registry.taskConnectorClass, now -> 
pluginsMetadata.connectorClass());
+            metricGroup.addValueMetric(registry.taskConnectorClassVersion, now 
-> pluginsMetadata.connectorVersion());
+            metricGroup.addValueMetric(registry.taskConnectorType, now -> 
pluginsMetadata.connectorType());
+            metricGroup.addValueMetric(registry.taskClass, now -> 
pluginsMetadata.taskClass());
+            metricGroup.addValueMetric(registry.taskVersion, now -> 
pluginsMetadata.taskVersion());
+            metricGroup.addValueMetric(registry.taskKeyConverterClass, now -> 
pluginsMetadata.keyConverterClass());
+            metricGroup.addValueMetric(registry.taskKeyConverterVersion, now 
-> pluginsMetadata.keyConverterVersion());
+            metricGroup.addValueMetric(registry.taskValueConverterClass, now 
-> pluginsMetadata.valueConverterClass());
+            metricGroup.addValueMetric(registry.taskValueConverterVersion, now 
-> pluginsMetadata.valueConverterVersion());
+            metricGroup.addValueMetric(registry.taskHeaderConverterClass, now 
-> pluginsMetadata.headerConverterClass());
+            metricGroup.addValueMetric(registry.taskHeaderConverterVersion, 
now -> pluginsMetadata.headerConverterVersion());
+
+            if (!pluginsMetadata.transformations().isEmpty()) {
+                for (TransformationStage.AliasedPluginInfo entry : 
pluginsMetadata.transformations()) {
+                    MetricGroup transformationGroup = 
connectMetrics.group(registry.transformsGroupName(),
+                            registry.connectorTagName(), id.connector(),
+                            registry.taskTagName(), 
Integer.toString(id.task()),
+                            registry.transformsTagName(), entry.alias());
+                    transformationGroup.close();

Review Comment:
   Why do the transformationGroup and predicateGroup get closed in 
`addPluginInfoMetric`? Is that supposed to be done only in the 
`TaskMetricsGroup#close()` method?



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1840,10 +1855,16 @@ public WorkerTask<T, R> build() {
             TransformationChain<T, R> transformationChain = new 
TransformationChain<>(connectorConfig.<R>transformationStages(plugins, id, 
metrics), retryWithToleranceOperator);
             log.info("Initializing: {}", transformationChain);
 
-            return doBuild(task, id, configState, statusListener, initialState,
-                    connectorConfig, keyConverterPlugin, valueConverterPlugin, 
headerConverterPlugin, classLoader,
-                    retryWithToleranceOperator, transformationChain,
-                    errorHandlingMetrics, connectorClass);
+            TaskPluginsMetadata taskPluginsMetadata = new TaskPluginsMetadata(
+                    connectorClass, task, keyConverterPlugin.get(), 
valueConverterPlugin.get(), headerConverterPlugin.get(), 
transformationChain.transformationChainInfo(), plugins);
+
+            WorkerTask<T, R> workerTask = doBuild(task, id, configState, 
statusListener, initialState,
+                connectorConfig, keyConverterPlugin, valueConverterPlugin, 
headerConverterPlugin, classLoader,
+                retryWithToleranceOperator, transformationChain,
+                errorHandlingMetrics, connectorClass);
+
+            workerTask.addPluginsMetrics(taskPluginsMetadata);

Review Comment:
   This seems to be unnecessary public mutability of the WorkerTask object. For 
example, what is expected to happen when WorkerTask#addPluginsMetrics is called 
0 or 2 times, or is called after `stop()`, `cancel()`, `removeMetrics`, etc.
   
   Can you pass taskPluginsMetadata into doBuild and the WorkerTask 
constructor?  Or, could WorkerTask construct the TaskPluginsMetadata itself? 
it's passed most of the other arguments already.
   



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationStage.java:
##########
@@ -89,4 +96,69 @@ public String toString() {
                 ", negate=" + negate +
                 '}';
     }
+
+    public static class AliasedPluginInfo {

Review Comment:
   nit: AliasedPluginInfo and StageInfo could be records!



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskPluginsMetadata.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.connect.connector.Connector;
+import org.apache.kafka.connect.connector.Task;
+import org.apache.kafka.connect.health.ConnectorType;
+import org.apache.kafka.connect.runtime.isolation.LoaderSwap;
+import org.apache.kafka.connect.runtime.isolation.PluginType;
+import org.apache.kafka.connect.runtime.isolation.PluginUtils;
+import org.apache.kafka.connect.runtime.isolation.Plugins;
+import org.apache.kafka.connect.sink.SinkConnector;
+import org.apache.kafka.connect.source.SourceConnector;
+import org.apache.kafka.connect.storage.Converter;
+import org.apache.kafka.connect.storage.HeaderConverter;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class TaskPluginsMetadata {
+
+    private final String connectorClass;
+    private final String connectorVersion;
+    private final ConnectorType connectorType;
+    private final String taskClass;
+    private final String taskVersion;
+    private final String keyConverterClass;
+    private final String keyConverterVersion;
+    private final String valueConverterClass;
+    private final String valueConverterVersion;
+    private final String headerConverterClass;
+    private final String headerConverterVersion;
+    private final Set<TransformationStage.AliasedPluginInfo> transformations;
+    private final Set<TransformationStage.AliasedPluginInfo> predicates;
+
+    public TaskPluginsMetadata(
+            Class<? extends Connector> connectorClass,
+            Task task,
+            Converter keyConverter,
+            Converter valueConverter,
+            HeaderConverter headerConverter,
+            List<TransformationStage.StageInfo> transformationStageInfo,
+            Plugins plugins
+    ) {
+
+        assert connectorClass != null;
+        assert task != null;
+        assert keyConverter != null;
+        assert valueConverter != null;
+        assert headerConverter != null;
+        assert transformationStageInfo != null;
+
+        Function<ClassLoader, LoaderSwap> pluginLoaderSwapper = 
plugins.safeLoaderSwapper();
+
+        this.connectorClass = connectorClass.getName();
+        this.connectorVersion = 
plugins.pluginVersion(connectorClass.getName(), 
connectorClass.getClassLoader(), PluginType.SINK, PluginType.SOURCE);
+        this.connectorType = getConnectorType(connectorClass, 
pluginLoaderSwapper);
+        this.taskClass = task.getClass().getName();
+        this.taskVersion = task.version();
+        this.keyConverterClass = keyConverter.getClass().getName();
+        this.keyConverterVersion = 
PluginUtils.getVersionOrUndefined(keyConverter, pluginLoaderSwapper);
+        this.valueConverterClass = valueConverter.getClass().getName();
+        this.valueConverterVersion = 
PluginUtils.getVersionOrUndefined(valueConverter, pluginLoaderSwapper);
+        this.headerConverterClass = headerConverter.getClass().getName();
+        this.headerConverterVersion = 
PluginUtils.getVersionOrUndefined(headerConverter, pluginLoaderSwapper);
+        this.transformations = 
transformationStageInfo.stream().map(TransformationStage.StageInfo::transform).collect(Collectors.toSet());
+        this.predicates = 
transformationStageInfo.stream().map(TransformationStage.StageInfo::predicate).filter(Objects::nonNull).collect(Collectors.toSet());
+    }
+
+
+    public ConnectorType getConnectorType(Class<? extends Connector> 
connectorClass, Function<ClassLoader, LoaderSwap> pluginLoaderSwapper) {

Review Comment:
   Replace this method with `ConnectorType.from`.
   
   Also, checking the class assignability doesn't require a loader swap.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetricsRegistry.java:
##########
@@ -379,6 +435,7 @@ public ConnectMetricsRegistry(Set<String> tags) {
                 "The number of failed writes to the dead letter queue.", 
taskErrorHandlingTags);
         lastErrorTimestamp = createTemplate("last-error-timestamp", 
TASK_ERROR_HANDLING_GROUP_NAME,
                 "The epoch timestamp when this task last encountered an 
error.", taskErrorHandlingTags);
+

Review Comment:
   nit: newline



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginUtils.java:
##########
@@ -510,4 +512,16 @@ public static VersionRange 
connectorVersionRequirement(String version) throws In
         version = "[" + version + "]";
         return VersionRange.createFromVersionSpec(version);
     }
+
+    public static <T> String getVersionOrUndefined(T obj, 
Function<ClassLoader, LoaderSwap> pluginLoaderSwapper) {

Review Comment:
   IMHO this is an unhealthy method to add. Arbitrary code should not be 
deriving the version of a plugin from the object, the version should be known 
at the time the object is being instantiated and should be tracked in parallel.
   
   Essentially, I believe the Versioned#version method should only be called 
during plugin discovery, and everything downstream from that should use 
whatever result was returned then.
   And interestingly enough, this method doesn't return the right answer for 
general Task instances :)



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskPluginsMetadata.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.connect.connector.Connector;
+import org.apache.kafka.connect.connector.Task;
+import org.apache.kafka.connect.health.ConnectorType;
+import org.apache.kafka.connect.runtime.isolation.LoaderSwap;
+import org.apache.kafka.connect.runtime.isolation.PluginType;
+import org.apache.kafka.connect.runtime.isolation.PluginUtils;
+import org.apache.kafka.connect.runtime.isolation.Plugins;
+import org.apache.kafka.connect.sink.SinkConnector;
+import org.apache.kafka.connect.source.SourceConnector;
+import org.apache.kafka.connect.storage.Converter;
+import org.apache.kafka.connect.storage.HeaderConverter;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class TaskPluginsMetadata {
+
+    private final String connectorClass;
+    private final String connectorVersion;
+    private final ConnectorType connectorType;
+    private final String taskClass;
+    private final String taskVersion;
+    private final String keyConverterClass;
+    private final String keyConverterVersion;
+    private final String valueConverterClass;
+    private final String valueConverterVersion;
+    private final String headerConverterClass;
+    private final String headerConverterVersion;
+    private final Set<TransformationStage.AliasedPluginInfo> transformations;
+    private final Set<TransformationStage.AliasedPluginInfo> predicates;
+
+    public TaskPluginsMetadata(
+            Class<? extends Connector> connectorClass,
+            Task task,
+            Converter keyConverter,
+            Converter valueConverter,
+            HeaderConverter headerConverter,
+            List<TransformationStage.StageInfo> transformationStageInfo,
+            Plugins plugins
+    ) {
+
+        assert connectorClass != null;
+        assert task != null;
+        assert keyConverter != null;
+        assert valueConverter != null;
+        assert headerConverter != null;
+        assert transformationStageInfo != null;
+
+        Function<ClassLoader, LoaderSwap> pluginLoaderSwapper = 
plugins.safeLoaderSwapper();
+
+        this.connectorClass = connectorClass.getName();
+        this.connectorVersion = 
plugins.pluginVersion(connectorClass.getName(), 
connectorClass.getClassLoader(), PluginType.SINK, PluginType.SOURCE);
+        this.connectorType = getConnectorType(connectorClass, 
pluginLoaderSwapper);
+        this.taskClass = task.getClass().getName();
+        this.taskVersion = task.version();

Review Comment:
   This is a direct call to the task which would require a loader swap.



-- 
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.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

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

Reply via email to