Skip to content
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 @@ -2394,6 +2394,10 @@ protected synchronized void recordTaskAdded(ConnectorTaskId connectorTaskId) {
}

protected synchronized void recordTaskRemoved(ConnectorTaskId connectorTaskId) {
if (!connectorStatusMetrics.containsKey(connectorTaskId.connector())) {
return;
}

// Unregister connector task count metric if we remove the last task of the connector
if (tasks.keySet().stream().noneMatch(id -> id.connector().equals(connectorTaskId.connector()))) {
connectorStatusMetrics.get(connectorTaskId.connector()).close();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -146,6 +146,7 @@
import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG;
import static org.apache.kafka.connect.sink.SinkTask.TOPICS_CONFIG;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
Expand Down Expand Up @@ -898,6 +899,35 @@ public void testConnectorStatusMetricsGroup_taskStatusCounter(boolean enableTopi
verifyKafkaClusterId();
}

@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testConnectorStatusMetricsGroup_tasksFailedToStart(boolean enableTopicCreation) {
setup(enableTopicCreation);
mockKafkaClusterId();
mockInternalConverters();
mockFileConfigProvider();

worker = new Worker(WORKER_ID,
new MockTime(),
plugins,
config,
offsetBackingStore,
noneConnectorClientConfigOverridePolicy);
worker.herder = herder;

// Pass an empty tasks map to simulate all tasks failing to start
Worker.ConnectorStatusMetricsGroup metricsGroup = new Worker.ConnectorStatusMetricsGroup(
worker.metrics(), new ConcurrentHashMap<>(), herder
);

ConnectorTaskId taskId1 = new ConnectorTaskId("c1", 0);
ConnectorTaskId taskId2 = new ConnectorTaskId("c1", 1);
metricsGroup.recordTaskAdded(taskId1);
metricsGroup.recordTaskAdded(taskId2);
metricsGroup.recordTaskRemoved(taskId1);
assertDoesNotThrow(() -> metricsGroup.recordTaskRemoved(taskId2), "should not throw NPE");
}

@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testStartTaskFailure(boolean enableTopicCreation) {
Expand Down