You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "C0urante (via GitHub)" <gi...@apache.org> on 2023/04/12 17:54:25 UTC

[GitHub] [kafka] C0urante commented on a diff in pull request #13465: KAFKA-14368: Connect offset write REST API

C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1164465874


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -114,6 +127,7 @@ public class Worker {
 
     public static final long CONNECTOR_GRACEFUL_SHUTDOWN_TIMEOUT_MS = TimeUnit.SECONDS.toMillis(5);
     public static final long EXECUTOR_SHUTDOWN_TERMINATION_TIMEOUT_MS = TimeUnit.SECONDS.toMillis(1);
+    public static final long ALTER_OFFSETS_TIMEOUT_MS = TimeUnit.SECONDS.toMillis(5);

Review Comment:
   Hmm... I think this may be too short. With sink connectors it's fairly straightforward to alter consumer group offsets, but for source connectors we have to start and complete a read-to-end of the offsets topic, then write the new offsets to it. And in both cases, we have the `alterOffsets` connector method to worry about as well.
   
   Can we make the `Worker` API for altering offsets asynchronous, similar to what we do for reading offsets?
   
   I know that there's concern about tasks being brought up for the connector while the request is being handled, but I think this might be alright.
   
   If the connector is a sink connector, the requests to alter its consumer group's offsets will be rejected by the broker if any tasks are active.
   
   If the connector is a source connector and exactly-once support is enabled, zombie fencing will take place and we won't be able to complete our write to the offsets topic.
   
   Unless I'm mistaken, the only case that's left is non-exactly-once source connectors, which IMO it's acceptable for us to ignore since we can't guarantee that there aren't zombie tasks running around writing their own offsets anyways.



-- 
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-unsubscribe@kafka.apache.org

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