-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-9422: Track the set of topics a connector is using (KIP-558) #8017
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
49dfb95
ac9e125
228f634
ec119b9
7c36f7b
9529294
7328b17
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,110 @@ | ||
| /* | ||
| * 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.util.ConnectorTaskId; | ||
|
|
||
| import java.util.Objects; | ||
|
|
||
| /** | ||
| * Represents the metadata that is stored as the value of the record that is stored in the | ||
| * {@link org.apache.kafka.connect.storage.StatusBackingStore#put(TopicStatus)}, | ||
| */ | ||
| public class TopicStatus { | ||
| private final String topic; | ||
| private final String connector; | ||
| private final int task; | ||
| private final long discoverTimestamp; | ||
|
|
||
| public TopicStatus(String topic, ConnectorTaskId task, long discoverTimestamp) { | ||
| this(topic, task.connector(), task.task(), discoverTimestamp); | ||
| } | ||
|
|
||
| public TopicStatus(String topic, String connector, int task, long discoverTimestamp) { | ||
| this.topic = Objects.requireNonNull(topic); | ||
| this.connector = Objects.requireNonNull(connector); | ||
| this.task = task; | ||
| this.discoverTimestamp = discoverTimestamp; | ||
| } | ||
|
|
||
| /** | ||
| * Get the name of the topic. | ||
| * | ||
| * @return the topic name; never null | ||
| */ | ||
| public String topic() { | ||
| return topic; | ||
| } | ||
|
|
||
| /** | ||
| * Get the name of the connector. | ||
| * | ||
| * @return the connector name; never null | ||
| */ | ||
| public String connector() { | ||
| return connector; | ||
| } | ||
|
|
||
| /** | ||
| * Get the ID of the task that stored the topic status. | ||
| * | ||
| * @return the task ID | ||
| */ | ||
| public int task() { | ||
| return task; | ||
| } | ||
|
|
||
| /** | ||
| * Get a timestamp that represents when this topic was discovered as being actively used by | ||
| * this connector. | ||
| * | ||
| * @return the discovery timestamp | ||
| */ | ||
| public long discoverTimestamp() { | ||
| return discoverTimestamp; | ||
| } | ||
|
rhauch marked this conversation as resolved.
|
||
|
|
||
| @Override | ||
| public String toString() { | ||
| return "TopicStatus{" + | ||
| "topic='" + topic + '\'' + | ||
| ", connector='" + connector + '\'' + | ||
| ", task=" + task + | ||
| ", discoverTimestamp=" + discoverTimestamp + | ||
| '}'; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean equals(Object o) { | ||
| if (this == o) { | ||
| return true; | ||
| } | ||
| if (!(o instanceof TopicStatus)) { | ||
| return false; | ||
| } | ||
| TopicStatus that = (TopicStatus) o; | ||
| return task == that.task && | ||
| discoverTimestamp == that.discoverTimestamp && | ||
| topic.equals(that.topic) && | ||
| connector.equals(that.connector); | ||
| } | ||
|
|
||
| @Override | ||
| public int hashCode() { | ||
| return Objects.hash(topic, connector, task, discoverTimestamp); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -30,6 +30,7 @@ | |
| import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; | ||
| import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; | ||
| import org.apache.kafka.connect.runtime.isolation.Plugins; | ||
| import org.apache.kafka.connect.storage.StatusBackingStore; | ||
| import org.apache.kafka.connect.util.ConnectorTaskId; | ||
| import org.apache.kafka.connect.util.LoggingContext; | ||
| import org.slf4j.Logger; | ||
|
|
@@ -57,6 +58,8 @@ abstract class WorkerTask implements Runnable { | |
| protected final ConnectorTaskId id; | ||
| private final TaskStatus.Listener statusListener; | ||
| protected final ClassLoader loader; | ||
| protected final StatusBackingStore statusBackingStore; | ||
| protected final Time time; | ||
| private final CountDownLatch shutdownLatch = new CountDownLatch(1); | ||
| private final TaskMetricsGroup taskMetricsGroup; | ||
| private volatile TargetState targetState; | ||
|
|
@@ -70,7 +73,9 @@ public WorkerTask(ConnectorTaskId id, | |
| TargetState initialState, | ||
| ClassLoader loader, | ||
| ConnectMetrics connectMetrics, | ||
| RetryWithToleranceOperator retryWithToleranceOperator) { | ||
| RetryWithToleranceOperator retryWithToleranceOperator, | ||
| Time time, | ||
| StatusBackingStore statusBackingStore) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm wondering if we really want to pass in the StatusBackingStore interface, or whether we should define a new interface? It'd be nice to constrain future developers to only use the methods related to active topics. It's a bit more complicated, but better captures the current intent to just allow the worker tasks to check whether a topic is considered active for a connector and if not to record that a topic has been used.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. My main issue is that the new methods seem to belong to the
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Neither approach is terribly clean. I think the current approach certainly works, but as you point out if we add something like Let's keep things simple. |
||
| this.id = id; | ||
| this.taskMetricsGroup = new TaskMetricsGroup(this.id, connectMetrics, statusListener); | ||
| this.statusListener = taskMetricsGroup; | ||
|
|
@@ -80,6 +85,8 @@ public WorkerTask(ConnectorTaskId id, | |
| this.cancelled = false; | ||
| this.taskMetricsGroup.recordState(this.targetState); | ||
| this.retryWithToleranceOperator = retryWithToleranceOperator; | ||
| this.time = time; | ||
| this.statusBackingStore = statusBackingStore; | ||
| } | ||
|
|
||
| public ConnectorTaskId id() { | ||
|
|
@@ -279,6 +286,20 @@ public void transitionTo(TargetState state) { | |
| } | ||
| } | ||
|
|
||
| /** | ||
| * Include this topic to the set of active topics for the connector that this worker task | ||
| * is running. This information is persisted in the status backing store used by this worker. | ||
| * | ||
| * @param topic the topic to mark as active for this connector | ||
| */ | ||
| protected void recordActiveTopic(String topic) { | ||
| if (statusBackingStore.getTopic(id.connector(), topic) != null) { | ||
| // The topic is already recorded as active. No further action is required. | ||
| return; | ||
| } | ||
| statusBackingStore.put(new TopicStatus(topic, id, time.milliseconds())); | ||
| } | ||
|
|
||
| /** | ||
| * Record that offsets have been committed. | ||
| * | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.