You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "yashmayya (via GitHub)" <gi...@apache.org> on 2023/03/28 11:22:16 UTC

[GitHub] [kafka] yashmayya opened a new pull request, #13465: KAFKA-14368: WIP: Connect offset write REST API

yashmayya opened a new pull request, #13465:
URL: https://github.com/apache/kafka/pull/13465

   - https://issues.apache.org/jira/browse/KAFKA-14368
   - [KIP-875: First-class offsets support in Kafka Connect](https://cwiki.apache.org/confluence/display/KAFKA/KIP-875%3A+First-class+offsets+support+in+Kafka+Connect)
   - Implements the new `PATCH /connectors/{connector}/offsets` REST API
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1202678391


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java:
##########
@@ -1528,6 +1539,80 @@ public void connectorOffsets(String connName, Callback<ConnectorOffsets> cb) {
         );
     }
 
+    @Override
+    public void alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> callback) {
+        log.trace("Submitting alter offsets request for connector '{}'", connName);
+
+        addRequest(() -> {
+            refreshConfigSnapshot(workerSyncTimeoutMs);
+            if (!alterConnectorOffsetsChecks(connName, callback)) {
+                return null;
+            }
+            // At this point, we should be the leader (the call to alterConnectorOffsetsChecks makes sure of that) and can safely run
+            // a zombie fencing request
+            if (isSourceConnector(connName) && config.exactlyOnceSourceEnabled()) {
+                log.debug("Performing a round of zombie fencing before altering offsets for source connector {} with exactly-once semantics enabled.", connName);
+                getFenceZombieSourceTasksCallable(connName, (error, ignored) -> {
+                    if (error != null) {
+                        log.error("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets", error);
+                        callback.onCompletion(new ConnectException("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets",
+                                error), null);
+                    } else {
+                        log.debug("Successfully completed zombie fencing for source connector {}; proceeding to alter offsets.", connName);
+                        // We need to ensure that we perform the necessary checks again inside alterConnectorOffsetsHerderRequest
+                        // since it is being run in a separate herder request and the conditions could have changed since the
+                        // previous check
+                        addRequest(getAlterConnectorOffsetsCallable(connName, offsets, callback), forwardErrorCallback(callback));
+                    }
+                }).call();
+            } else {
+                getAlterConnectorOffsetsCallable(connName, offsets, callback).call();

Review Comment:
   One convention I've seen used is `foo` (public) and `doFoo` (internal). I'm not a huge fan of it but in some cases it can be useful.
   
   Also, are we issuing unnecessary calls to `alterConnectorOffsetsChecks`? We do a check in the beginning of `alterConnectorOffsets` and then, if the connector is a sink (or exactly-once source support is disabled), we immediately invoke the `Callable` returned by `getAlterConnectorOffsetsCallable`, which calls `alterConnectorOffsetsChecks` a second time.
   
   We might just get rid of `getAlterConnectorOffsetsCallable` altogether and inline the logic for it?



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


[GitHub] [kafka] yashmayya commented on pull request #13465: KAFKA-14368: WIP: Connect offset write REST API

Posted by "yashmayya (via GitHub)" <gi...@apache.org>.
yashmayya commented on PR #13465:
URL: https://github.com/apache/kafka/pull/13465#issuecomment-1486718538

   Hi @C0urante, this will need to be rebased once https://github.com/apache/kafka/pull/13424 and https://github.com/apache/kafka/pull/13434 are merged but I raised this draft PR to solicit some early feedback if possible - particularly around this statement from the KIP:
   
   > Offsets will be reset transactionally for each topic that they exist in: a single transaction will be used to emit all tombstone records for the connector's dedicated offsets topic (if one is used) and another transaction will be used to emit all tombstone records for the worker's global offsets topic.
   
   We don't currently use a transactional producer for writing offsets to the worker's global offset backing store. Wouldn't doing so be a breaking change due to the requirement of additional ACLs for the worker's producer principal? For workers where exactly-once source support is enabled, the only way we could do so would be if the connector's configured offsets topic is the same as the worker's global offset topic (in which case, a connector specific offset store is used with a transactional producer). If the connector's configured offsets topic is a different one or if exactly-once source support is not enabled, I don't think we'll be able to write all the provided offsets in a single transaction for the worker's global offset backing store right? 
   
   Also one minor comment on the KIP section - the "tombstone records" bit that was copied from the resetting offsets section needs to be updated.


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


[GitHub] [kafka] yashmayya commented on pull request #13465: KAFKA-14368: WIP: Connect offset write REST API

Posted by "yashmayya (via GitHub)" <gi...@apache.org>.
yashmayya commented on PR #13465:
URL: https://github.com/apache/kafka/pull/13465#issuecomment-1487350628

   Thanks for the swift reply Chris and no rush on the review, I mainly wanted to get the clarifications regarding transactions on the offset topic(s).
   
   > Since we're introducing a new opt-in API, it's not a breaking change. That said, it's probably worth calling out in the KIP and on the discussion thread. And now that I think about it, maybe a transaction on the global offsets topic isn't really necessary if we've already done a round of zombie fencing. Thoughts? We should notify the discussion thread for the KIP no matter what, just wanted to bounce the idea off you first.
   
   Ah yeah that's a good point about the opt-in API, we wouldn't require the use of a transactional producer for regular connector task initiated offset writes to the worker's global offset backing store. However, I do agree that it doesn't seem necessary to use transactions on the global offsets topic if exactly-once source support is enabled and the connector is using a custom offsets topic (if it isn't, then we can write to the global offsets topic transactionally using a transactional producer corresponding to the connector). This will be in line with how regular offset writes are handled where not only do we not write offsets transactionally to the secondary store of a connector (the worker's global offset backing store assuming the connector has a custom offset topic configured as the primary store), but we also essentially ignore any errors arising from writes to the secondary store.
   
   > The use of transactions is only necessary if exactly-once source support is enabled for source connectors (both paragraphs that mention the use of transactions begin with "If exactly-once source support is enabled").
   
   Ah okay, I misunderstood then. I thought the "If exactly-once source support is enabled" bit was only applicable to the zombie fencing.


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


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

Posted by "yashmayya (via GitHub)" <gi...@apache.org>.
yashmayya commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1203576413


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java:
##########
@@ -1528,6 +1539,80 @@ public void connectorOffsets(String connName, Callback<ConnectorOffsets> cb) {
         );
     }
 
+    @Override
+    public void alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> callback) {
+        log.trace("Submitting alter offsets request for connector '{}'", connName);
+
+        addRequest(() -> {
+            refreshConfigSnapshot(workerSyncTimeoutMs);
+            if (!alterConnectorOffsetsChecks(connName, callback)) {
+                return null;
+            }
+            // At this point, we should be the leader (the call to alterConnectorOffsetsChecks makes sure of that) and can safely run
+            // a zombie fencing request
+            if (isSourceConnector(connName) && config.exactlyOnceSourceEnabled()) {
+                log.debug("Performing a round of zombie fencing before altering offsets for source connector {} with exactly-once semantics enabled.", connName);
+                getFenceZombieSourceTasksCallable(connName, (error, ignored) -> {
+                    if (error != null) {
+                        log.error("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets", error);
+                        callback.onCompletion(new ConnectException("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets",
+                                error), null);
+                    } else {
+                        log.debug("Successfully completed zombie fencing for source connector {}; proceeding to alter offsets.", connName);
+                        // We need to ensure that we perform the necessary checks again inside alterConnectorOffsetsHerderRequest
+                        // since it is being run in a separate herder request and the conditions could have changed since the
+                        // previous check
+                        addRequest(getAlterConnectorOffsetsCallable(connName, offsets, callback), forwardErrorCallback(callback));
+                    }
+                }).call();
+            } else {
+                getAlterConnectorOffsetsCallable(connName, offsets, callback).call();

Review Comment:
   Wow thanks, good catch! I've removed the `getAlterConnectorOffsetsCallable` method and refactored the zombie fencing one to be synchronous and named it `doFenceZombieSourceTasks` (it definitely sounds a lot better 🙂).



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1235,6 +1244,246 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets a mapping from partitions (either source partitions for source connectors, or Kafka topic
+     *                partitions for sink connectors) to offsets that need to be written; may not be null or empty
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+
+        if (offsets == null || offsets.isEmpty()) {
+            throw new ConnectException("The offsets to be altered may not be null or empty");
+        }
+
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be written; may not be null or empty
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.parseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        sinkConnectorConfig,
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                    Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                            .stream()
+                            .filter(entry -> entry.getValue() != null)
+                            .collect(Collectors.toMap(Map.Entry::getKey, e -> new OffsetAndMetadata(e.getValue())));
+
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                            .stream()
+                            .filter(entry -> entry.getValue() == null)
+                            .map(Map.Entry::getKey)
+                            .collect(Collectors.toSet());
+
+                    if (!partitionsToReset.isEmpty()) {
+                        log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                connName, partitionsToReset);
+                        DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                deleteConsumerGroupOffsetsOptions);
+
+                        adminFuture = KafkaFuture.allOf(adminFuture, deleteConsumerGroupOffsetsResult.all());
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            // When a consumer group is non-empty, only group members can commit offsets. An attempt to alter offsets via the admin client
+                            // will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed). Similarly, an attempt to
+                            // delete consumer group offsets for a non-empty consumer group will result in a GroupSubscribedToTopicException
+                            if (error instanceof UnknownMemberIdException || error instanceof GroupSubscribedToTopicException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName, error), null);
+                            }
+                        } else {
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                        Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);

Review Comment:
   Hm I don't really see how the callback logic could bail out in its current shape but I guess using a chained `whenComplete` for relinquishing resources is a better pattern overall.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    private void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                             Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId)
+                : baseProducerConfigs(connName, "connector-offset-producer-" + connName, config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId);
+        KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+        ConnectorOffsetBackingStore offsetStore = config.exactlyOnceSourceEnabled()
+                ? offsetStoreForExactlyOnceSourceConnector(sourceConfig, connName, connector, producer)
+                : offsetStoreForRegularSourceConnector(sourceConfig, connName, connector, producer);
+
+        OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, connName, internalKeyConverter, internalValueConverter);
+        alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, offsetStore, producer, offsetWriter, connectorLoader, cb);
+    }
+
+    // Visible for testing
+    void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                     Map<Map<String, ?>, Map<String, ?>> offsets, ConnectorOffsetBackingStore offsetStore,
+                                     KafkaProducer<byte[], byte[]> producer, OffsetStorageWriter offsetWriter,
+                                     ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SourceConnector) connector).alterOffsets(connectorConfig, offsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                offsetStore.configure(config);
+                // This reads to the end of the offsets topic and can be a potentially time-consuming operation
+                offsetStore.start();
+
+                // The alterSourceConnectorOffsets method should only be called after all the connector's tasks have been stopped, and it's
+                // safe to write offsets via an offset writer
+                offsets.forEach(offsetWriter::offset);
+
+                // We can call begin flush without a timeout because this newly created single-purpose offset writer can't do concurrent
+                // offset writes. We can also ignore the return value since it returns false if and only if there is no data to be flushed,
+                // and we've just put some data in the previous statement
+                offsetWriter.beginFlush();
+
+                try {
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.initTransactions();
+                        producer.beginTransaction();
+                    }
+                    log.debug("Committing the following partition offsets for source connector {}: {}", connName, offsets);
+                    FutureCallback<Void> offsetWriterCallback = new FutureCallback<>();
+                    offsetWriter.doFlush(offsetWriterCallback);
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.commitTransaction();
+                    }
+                    offsetWriterCallback.get(ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS, TimeUnit.MILLISECONDS);

Review Comment:
   Ahh yes, that totally makes sense. Thanks, I hadn't considered this tombstones edge case. I agree that we should attempt to fix it for both the scenarios (REST API offset alterations and task offset commits) together, since designing a specific solution here for the REST API, then fixing it in the `ConnectorOffsetBackingStore` and then reverting the REST API specific solution doesn't seem ideal. However, I do agree that it'd be good if we could fix [KAFKA-15018](https://issues.apache.org/jira/browse/KAFKA-15018) in the same release that this patch goes out (hopefully `3.6`). I'll take a look and I might be able to pick that one up as well!



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    private void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                             Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId)
+                : baseProducerConfigs(connName, "connector-offset-producer-" + connName, config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId);
+        KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+        ConnectorOffsetBackingStore offsetStore = config.exactlyOnceSourceEnabled()
+                ? offsetStoreForExactlyOnceSourceConnector(sourceConfig, connName, connector, producer)
+                : offsetStoreForRegularSourceConnector(sourceConfig, connName, connector, producer);
+
+        OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, connName, internalKeyConverter, internalValueConverter);
+        alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, offsetStore, producer, offsetWriter, connectorLoader, cb);
+    }
+
+    // Visible for testing
+    void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                     Map<Map<String, ?>, Map<String, ?>> offsets, ConnectorOffsetBackingStore offsetStore,
+                                     KafkaProducer<byte[], byte[]> producer, OffsetStorageWriter offsetWriter,
+                                     ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SourceConnector) connector).alterOffsets(connectorConfig, offsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                offsetStore.configure(config);
+                // This reads to the end of the offsets topic and can be a potentially time-consuming operation
+                offsetStore.start();

Review Comment:
   I'm not sure I follow? Everything after the `start()` invocation is wrapped in a try-finally which ensures that the offset store is stopped.  



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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
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


[GitHub] [kafka] yashmayya commented on pull request #13465: KAFKA-14368: Connect offset write REST API

Posted by "yashmayya (via GitHub)" <gi...@apache.org>.
yashmayya commented on PR #13465:
URL: https://github.com/apache/kafka/pull/13465#issuecomment-1564677170

   Thanks Chris! 


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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1188741900


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java:
##########
@@ -1528,6 +1539,80 @@ public void connectorOffsets(String connName, Callback<ConnectorOffsets> cb) {
         );
     }
 
+    @Override
+    public void alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> callback) {
+        log.trace("Submitting alter offsets request for connector '{}'", connName);
+
+        addRequest(() -> {
+            refreshConfigSnapshot(workerSyncTimeoutMs);
+            if (!alterConnectorOffsetsChecks(connName, callback)) {
+                return null;
+            }
+            // At this point, we should be the leader (the call to alterConnectorOffsetsChecks makes sure of that) and can safely run
+            // a zombie fencing request
+            if (isSourceConnector(connName) && config.exactlyOnceSourceEnabled()) {
+                log.debug("Performing a round of zombie fencing before altering offsets for source connector {} with exactly-once semantics enabled.", connName);
+                getFenceZombieSourceTasksCallable(connName, (error, ignored) -> {
+                    if (error != null) {
+                        log.error("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets", error);

Review Comment:
   Similar concerns about wording:
   ```suggestion
                           log.error("Failed to perform zombie fencing for source connector prior to altering offsets", error);
   ```
   
   We could also say "source connector with exactly-once support enabled" but that seems unnecessarily verbose.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java:
##########
@@ -1528,6 +1539,80 @@ public void connectorOffsets(String connName, Callback<ConnectorOffsets> cb) {
         );
     }
 
+    @Override
+    public void alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> callback) {
+        log.trace("Submitting alter offsets request for connector '{}'", connName);
+
+        addRequest(() -> {
+            refreshConfigSnapshot(workerSyncTimeoutMs);
+            if (!alterConnectorOffsetsChecks(connName, callback)) {
+                return null;
+            }
+            // At this point, we should be the leader (the call to alterConnectorOffsetsChecks makes sure of that) and can safely run
+            // a zombie fencing request
+            if (isSourceConnector(connName) && config.exactlyOnceSourceEnabled()) {
+                log.debug("Performing a round of zombie fencing before altering offsets for source connector {} with exactly-once semantics enabled.", connName);
+                getFenceZombieSourceTasksCallable(connName, (error, ignored) -> {
+                    if (error != null) {
+                        log.error("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets", error);
+                        callback.onCompletion(new ConnectException("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets",
+                                error), null);
+                    } else {
+                        log.debug("Successfully completed zombie fencing for source connector {}; proceeding to alter offsets.", connName);
+                        // We need to ensure that we perform the necessary checks again inside alterConnectorOffsetsHerderRequest
+                        // since it is being run in a separate herder request and the conditions could have changed since the
+                        // previous check
+                        addRequest(getAlterConnectorOffsetsCallable(connName, offsets, callback), forwardErrorCallback(callback));
+                    }
+                }).call();
+            } else {
+                getAlterConnectorOffsetsCallable(connName, offsets, callback).call();

Review Comment:
   Same thought here; it's counterintuitive to have a method return a `Callable` just to immediately call it. We can (should) just have synchronous variants instead, and if we still need a way to wrap those in a `Callable`, we can piggyback off of the synchronous one.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java:
##########
@@ -1528,6 +1539,80 @@ public void connectorOffsets(String connName, Callback<ConnectorOffsets> cb) {
         );
     }
 
+    @Override
+    public void alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> callback) {
+        log.trace("Submitting alter offsets request for connector '{}'", connName);
+
+        addRequest(() -> {
+            refreshConfigSnapshot(workerSyncTimeoutMs);

Review Comment:
   Shouldn't we handle the case where this method returns `false`?



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    private void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                             Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId)
+                : baseProducerConfigs(connName, "connector-offset-producer-" + connName, config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId);
+        KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+        ConnectorOffsetBackingStore offsetStore = config.exactlyOnceSourceEnabled()
+                ? offsetStoreForExactlyOnceSourceConnector(sourceConfig, connName, connector, producer)
+                : offsetStoreForRegularSourceConnector(sourceConfig, connName, connector, producer);
+
+        OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, connName, internalKeyConverter, internalValueConverter);
+        alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, offsetStore, producer, offsetWriter, connectorLoader, cb);
+    }
+
+    // Visible for testing
+    void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                     Map<Map<String, ?>, Map<String, ?>> offsets, ConnectorOffsetBackingStore offsetStore,
+                                     KafkaProducer<byte[], byte[]> producer, OffsetStorageWriter offsetWriter,
+                                     ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SourceConnector) connector).alterOffsets(connectorConfig, offsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                offsetStore.configure(config);
+                // This reads to the end of the offsets topic and can be a potentially time-consuming operation
+                offsetStore.start();
+
+                // The alterSourceConnectorOffsets method should only be called after all the connector's tasks have been stopped, and it's
+                // safe to write offsets via an offset writer
+                offsets.forEach(offsetWriter::offset);
+
+                // We can call begin flush without a timeout because this newly created single-purpose offset writer can't do concurrent
+                // offset writes. We can also ignore the return value since it returns false if and only if there is no data to be flushed,
+                // and we've just put some data in the previous statement

Review Comment:
   Are we guaranteed that `offsets` is non-empty here?
   
   I guess we do that check in `ConnectorsResource` but it might be nice to reiterate that expectation in the Javadocs for the various offset-alter methods that `offsets` may be neither null nor empty, and possibly even add an explicit check for that.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    private void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                             Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId)
+                : baseProducerConfigs(connName, "connector-offset-producer-" + connName, config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId);
+        KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+        ConnectorOffsetBackingStore offsetStore = config.exactlyOnceSourceEnabled()
+                ? offsetStoreForExactlyOnceSourceConnector(sourceConfig, connName, connector, producer)
+                : offsetStoreForRegularSourceConnector(sourceConfig, connName, connector, producer);
+
+        OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, connName, internalKeyConverter, internalValueConverter);
+        alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, offsetStore, producer, offsetWriter, connectorLoader, cb);
+    }
+
+    // Visible for testing
+    void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                     Map<Map<String, ?>, Map<String, ?>> offsets, ConnectorOffsetBackingStore offsetStore,
+                                     KafkaProducer<byte[], byte[]> producer, OffsetStorageWriter offsetWriter,
+                                     ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SourceConnector) connector).alterOffsets(connectorConfig, offsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                offsetStore.configure(config);
+                // This reads to the end of the offsets topic and can be a potentially time-consuming operation
+                offsetStore.start();
+
+                // The alterSourceConnectorOffsets method should only be called after all the connector's tasks have been stopped, and it's
+                // safe to write offsets via an offset writer
+                offsets.forEach(offsetWriter::offset);
+
+                // We can call begin flush without a timeout because this newly created single-purpose offset writer can't do concurrent
+                // offset writes. We can also ignore the return value since it returns false if and only if there is no data to be flushed,
+                // and we've just put some data in the previous statement
+                offsetWriter.beginFlush();
+
+                try {
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.initTransactions();
+                        producer.beginTransaction();
+                    }
+                    log.debug("Committing the following partition offsets for source connector {}: {}", connName, offsets);

Review Comment:
   Nit: "partition" doesn't really add anything here. I was considering "source" instead, but that's redundant since we already clarify that it's a source connector.
   ```suggestion
                       log.debug("Committing the following offsets for source connector {}: {}", connName, offsets);
   ```



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    private void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                             Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId)
+                : baseProducerConfigs(connName, "connector-offset-producer-" + connName, config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId);
+        KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+        ConnectorOffsetBackingStore offsetStore = config.exactlyOnceSourceEnabled()
+                ? offsetStoreForExactlyOnceSourceConnector(sourceConfig, connName, connector, producer)
+                : offsetStoreForRegularSourceConnector(sourceConfig, connName, connector, producer);
+
+        OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, connName, internalKeyConverter, internalValueConverter);
+        alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, offsetStore, producer, offsetWriter, connectorLoader, cb);
+    }
+
+    // Visible for testing
+    void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                     Map<Map<String, ?>, Map<String, ?>> offsets, ConnectorOffsetBackingStore offsetStore,
+                                     KafkaProducer<byte[], byte[]> producer, OffsetStorageWriter offsetWriter,
+                                     ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SourceConnector) connector).alterOffsets(connectorConfig, offsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                offsetStore.configure(config);
+                // This reads to the end of the offsets topic and can be a potentially time-consuming operation
+                offsetStore.start();
+
+                // The alterSourceConnectorOffsets method should only be called after all the connector's tasks have been stopped, and it's
+                // safe to write offsets via an offset writer
+                offsets.forEach(offsetWriter::offset);
+
+                // We can call begin flush without a timeout because this newly created single-purpose offset writer can't do concurrent
+                // offset writes. We can also ignore the return value since it returns false if and only if there is no data to be flushed,
+                // and we've just put some data in the previous statement
+                offsetWriter.beginFlush();
+
+                try {
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.initTransactions();
+                        producer.beginTransaction();
+                    }
+                    log.debug("Committing the following partition offsets for source connector {}: {}", connName, offsets);
+                    FutureCallback<Void> offsetWriterCallback = new FutureCallback<>();
+                    offsetWriter.doFlush(offsetWriterCallback);
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.commitTransaction();
+                    }
+                    offsetWriterCallback.get(ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS, TimeUnit.MILLISECONDS);
+                } catch (ExecutionException e) {
+                    throw new ConnectException("Failed to alter offsets for source connector " + connName, e.getCause());

Review Comment:
   We include the set of to-be-altered partitions in the error messages for sink connectors. Can/should we do that for source connectors as well?



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java:
##########
@@ -1528,6 +1539,80 @@ public void connectorOffsets(String connName, Callback<ConnectorOffsets> cb) {
         );
     }
 
+    @Override
+    public void alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> callback) {
+        log.trace("Submitting alter offsets request for connector '{}'", connName);
+
+        addRequest(() -> {
+            refreshConfigSnapshot(workerSyncTimeoutMs);

Review Comment:
   Also, is it wise to force a read to the end of the config topic when we may not even be the leader of the cluster? Feels like we should do that check before the read-to-end.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java:
##########
@@ -1528,6 +1539,80 @@ public void connectorOffsets(String connName, Callback<ConnectorOffsets> cb) {
         );
     }
 
+    @Override
+    public void alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> callback) {
+        log.trace("Submitting alter offsets request for connector '{}'", connName);
+
+        addRequest(() -> {
+            refreshConfigSnapshot(workerSyncTimeoutMs);
+            if (!alterConnectorOffsetsChecks(connName, callback)) {
+                return null;
+            }
+            // At this point, we should be the leader (the call to alterConnectorOffsetsChecks makes sure of that) and can safely run
+            // a zombie fencing request
+            if (isSourceConnector(connName) && config.exactlyOnceSourceEnabled()) {
+                log.debug("Performing a round of zombie fencing before altering offsets for source connector {} with exactly-once semantics enabled.", connName);
+                getFenceZombieSourceTasksCallable(connName, (error, ignored) -> {
+                    if (error != null) {
+                        log.error("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets", error);
+                        callback.onCompletion(new ConnectException("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets",
+                                error), null);
+                    } else {
+                        log.debug("Successfully completed zombie fencing for source connector {}; proceeding to alter offsets.", connName);
+                        // We need to ensure that we perform the necessary checks again inside alterConnectorOffsetsHerderRequest

Review Comment:
   Does `alterConnectorOffsetsHerderRequest` need to be updated? This is the only place in the class where that name is used.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten

Review Comment:
   Nits:
   
   1. Might help to be explicit about what "partitions" refer to here
   2. We aren't necessarily overwriting here (it's possible that no offset exists for one or more of the specified partitions)
   
   ```suggestion
        * @param offsets  a mapping from partitions (either source partitions, or Kafka topic partitions) to offsets that need to be written
   ```



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java:
##########
@@ -1528,6 +1539,80 @@ public void connectorOffsets(String connName, Callback<ConnectorOffsets> cb) {
         );
     }
 
+    @Override
+    public void alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> callback) {
+        log.trace("Submitting alter offsets request for connector '{}'", connName);
+
+        addRequest(() -> {
+            refreshConfigSnapshot(workerSyncTimeoutMs);
+            if (!alterConnectorOffsetsChecks(connName, callback)) {
+                return null;
+            }
+            // At this point, we should be the leader (the call to alterConnectorOffsetsChecks makes sure of that) and can safely run
+            // a zombie fencing request
+            if (isSourceConnector(connName) && config.exactlyOnceSourceEnabled()) {
+                log.debug("Performing a round of zombie fencing before altering offsets for source connector {} with exactly-once semantics enabled.", connName);

Review Comment:
   Nit: we say "support" instead of "semantics" for source connectors in Kafka Connect since it's still possible for connectors to not be exactly-once if they don't define meaningful source offsets.
   ```suggestion
                   log.debug("Performing a round of zombie fencing before altering offsets for source connector {} with exactly-once support enabled.", connName);
   ```



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java:
##########
@@ -1528,6 +1539,80 @@ public void connectorOffsets(String connName, Callback<ConnectorOffsets> cb) {
         );
     }
 
+    @Override
+    public void alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> callback) {
+        log.trace("Submitting alter offsets request for connector '{}'", connName);
+
+        addRequest(() -> {
+            refreshConfigSnapshot(workerSyncTimeoutMs);
+            if (!alterConnectorOffsetsChecks(connName, callback)) {
+                return null;
+            }
+            // At this point, we should be the leader (the call to alterConnectorOffsetsChecks makes sure of that) and can safely run
+            // a zombie fencing request
+            if (isSourceConnector(connName) && config.exactlyOnceSourceEnabled()) {
+                log.debug("Performing a round of zombie fencing before altering offsets for source connector {} with exactly-once semantics enabled.", connName);
+                getFenceZombieSourceTasksCallable(connName, (error, ignored) -> {
+                    if (error != null) {
+                        log.error("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets", error);
+                        callback.onCompletion(new ConnectException("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets",
+                                error), null);
+                    } else {
+                        log.debug("Successfully completed zombie fencing for source connector {}; proceeding to alter offsets.", connName);
+                        // We need to ensure that we perform the necessary checks again inside alterConnectorOffsetsHerderRequest
+                        // since it is being run in a separate herder request and the conditions could have changed since the

Review Comment:
   It's not that it's being run in a separate herder request, it's that zombie fencing takes place asynchronously (i.e., off of the herder tick thread).



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java:
##########
@@ -1528,6 +1539,80 @@ public void connectorOffsets(String connName, Callback<ConnectorOffsets> cb) {
         );
     }
 
+    @Override
+    public void alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> callback) {
+        log.trace("Submitting alter offsets request for connector '{}'", connName);
+
+        addRequest(() -> {
+            refreshConfigSnapshot(workerSyncTimeoutMs);
+            if (!alterConnectorOffsetsChecks(connName, callback)) {
+                return null;
+            }
+            // At this point, we should be the leader (the call to alterConnectorOffsetsChecks makes sure of that) and can safely run
+            // a zombie fencing request
+            if (isSourceConnector(connName) && config.exactlyOnceSourceEnabled()) {
+                log.debug("Performing a round of zombie fencing before altering offsets for source connector {} with exactly-once semantics enabled.", connName);
+                getFenceZombieSourceTasksCallable(connName, (error, ignored) -> {
+                    if (error != null) {
+                        log.error("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets", error);
+                        callback.onCompletion(new ConnectException("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets",
+                                error), null);
+                    } else {
+                        log.debug("Successfully completed zombie fencing for source connector {}; proceeding to alter offsets.", connName);
+                        // We need to ensure that we perform the necessary checks again inside alterConnectorOffsetsHerderRequest
+                        // since it is being run in a separate herder request and the conditions could have changed since the
+                        // previous check
+                        addRequest(getAlterConnectorOffsetsCallable(connName, offsets, callback), forwardErrorCallback(callback));
+                    }
+                }).call();

Review Comment:
   This is a little odd... could we have a method to synchronously do a round of zombie fencing and invoke that directly here (moving the callback logic to after that invocation)?



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);

Review Comment:
   We just created a new instance of this class to pass to `adminConfigs`; can we save that one and use it both there and here?



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java:
##########
@@ -1528,6 +1539,80 @@ public void connectorOffsets(String connName, Callback<ConnectorOffsets> cb) {
         );
     }
 
+    @Override
+    public void alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> callback) {
+        log.trace("Submitting alter offsets request for connector '{}'", connName);
+
+        addRequest(() -> {
+            refreshConfigSnapshot(workerSyncTimeoutMs);
+            if (!alterConnectorOffsetsChecks(connName, callback)) {
+                return null;
+            }
+            // At this point, we should be the leader (the call to alterConnectorOffsetsChecks makes sure of that) and can safely run
+            // a zombie fencing request
+            if (isSourceConnector(connName) && config.exactlyOnceSourceEnabled()) {
+                log.debug("Performing a round of zombie fencing before altering offsets for source connector {} with exactly-once semantics enabled.", connName);
+                getFenceZombieSourceTasksCallable(connName, (error, ignored) -> {
+                    if (error != null) {
+                        log.error("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets", error);
+                        callback.onCompletion(new ConnectException("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets",
+                                error), null);
+                    } else {
+                        log.debug("Successfully completed zombie fencing for source connector {}; proceeding to alter offsets.", connName);
+                        // We need to ensure that we perform the necessary checks again inside alterConnectorOffsetsHerderRequest
+                        // since it is being run in a separate herder request and the conditions could have changed since the
+                        // previous check
+                        addRequest(getAlterConnectorOffsetsCallable(connName, offsets, callback), forwardErrorCallback(callback));
+                    }
+                }).call();
+            } else {
+                getAlterConnectorOffsetsCallable(connName, offsets, callback).call();
+            }
+            return null;
+        }, forwardErrorCallback(callback));
+    }
+
+    private Callable<Void> getAlterConnectorOffsetsCallable(String connName, Map<Map<String, ?>, Map<String, ?>> offsets,
+                                                            Callback<Message> callback) {
+        return () -> {
+            if (!alterConnectorOffsetsChecks(connName, callback)) {
+                return null;
+            }
+            worker.alterConnectorOffsets(connName, configState.connectorConfig(connName), offsets, callback);
+            return null;
+        };
+    }
+
+    /**
+     * This method performs a few checks for alter connector offsets request and completes the callback exceptionally
+     * if any check fails.
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param callback callback to invoke upon completion
+     * @return true if all the checks passed, false otherwise
+     */
+    private boolean alterConnectorOffsetsChecks(String connName, Callback<Message> callback) {
+        if (checkRebalanceNeeded(callback)) {
+            return false;
+        }
+        if (!configState.contains(connName)) {
+            callback.onCompletion(new NotFoundException("Connector " + connName + " not found", null), null);
+            return false;
+        }
+        // If the target state for the connector is stopped, its task count is 0, and there is no rebalance pending (checked above),
+        // we can be sure that the tasks have at least been attempted to be stopped (or cancelled if they took too long to stop).
+        // Zombie tasks are handled by a round of zombie fencing for exactly once source connectors. Zombie sink tasks are handled
+        // naturally because requests to alter consumer group offsets will fail if there are still active members in the group.
+        if (configState.targetState(connName) != TargetState.STOPPED || configState.taskCount(connName) != 0) {
+            callback.onCompletion(new BadRequestException("The connector needs to be stopped before its offsets can be altered"), null);
+            return false;
+        }
+        if (!isLeader()) {
+            callback.onCompletion(new NotLeaderException("Only the leader can process alter offsets requests", leaderUrl()), null);
+            return false;
+        }

Review Comment:
   Can we move this to the beginning of the method (or maybe just after the `checkRebalanceNeeded` call)? If we're going to forward to the leader, best to do it sooner rather than later to reduce duplicated work.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/util/SinkUtils.java:
##########
@@ -52,4 +53,82 @@ public static ConnectorOffsets consumerGroupOffsetsToConnectorOffsets(Map<TopicP
 
         return new ConnectorOffsets(connectorOffsets);
     }
+
+    /**
+     * Validate that the provided partitions (keys in the {@code partitionOffsets} map) look like:
+     * <pre>
+     *     {
+     *       "kafka_topic": "topic"
+     *       "kafka_partition": 3
+     *     }
+     * </pre>
+     *
+     * and that the provided offsets (values in the {@code partitionOffsets} map) look like:
+     * <pre>
+     *     {
+     *       "kafka_offset": 1000
+     *     }
+     * </pre>
+     *
+     * This method then parses them into a mapping from {@link TopicPartition}s to their corresponding {@link Long}
+     * valued offsets.
+     *
+     * @param partitionOffsets the partitions to offset map that needs to be validated and parsed.
+     * @return the parsed mapping from {@link TopicPartition} to its corresponding {@link Long} valued offset.
+     *
+     * @throws BadRequestException if the provided offsets aren't in the expected format
+     */
+    public static Map<TopicPartition, Long> validateAndParseSinkConnectorOffsets(Map<Map<String, ?>, Map<String, ?>> partitionOffsets) {

Review Comment:
   Validation is an implied part of parsing (think `Long::parseLong`, `Integer::parseInt`, etc.).
   
   I think we can just name this `parseSinkConnectorOffsets` and leave out mention of validation from the Javadoc except for the `@throws` clause.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));

Review Comment:
   This can be simplified:
   
   ```suggestion
                           .collect(Collectors.toMap(Map.Entry::getKey, e -> new OffsetAndMetadata(e.getValue())));
   ```



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);

Review Comment:
   Nit: This can be inside the scope of the `try` block starting at line 1343.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",

Review Comment:
   Nit: "topic partition" is redundant if we're already talking about "consumer offset"
   ```suggestion
                           log.debug("Committing the following consumer group offsets using an admin client for sink connector {}: {}.",
   ```



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    private void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                             Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId)
+                : baseProducerConfigs(connName, "connector-offset-producer-" + connName, config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId);
+        KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+        ConnectorOffsetBackingStore offsetStore = config.exactlyOnceSourceEnabled()
+                ? offsetStoreForExactlyOnceSourceConnector(sourceConfig, connName, connector, producer)
+                : offsetStoreForRegularSourceConnector(sourceConfig, connName, connector, producer);
+
+        OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, connName, internalKeyConverter, internalValueConverter);
+        alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, offsetStore, producer, offsetWriter, connectorLoader, cb);
+    }
+
+    // Visible for testing
+    void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                     Map<Map<String, ?>, Map<String, ?>> offsets, ConnectorOffsetBackingStore offsetStore,
+                                     KafkaProducer<byte[], byte[]> producer, OffsetStorageWriter offsetWriter,
+                                     ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SourceConnector) connector).alterOffsets(connectorConfig, offsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                offsetStore.configure(config);
+                // This reads to the end of the offsets topic and can be a potentially time-consuming operation
+                offsetStore.start();

Review Comment:
   Doesn't this potentially leak the offset store if an exception occurs before we reach the `try` (and accompanying `finally`) block at line 1471?



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);

Review Comment:
   We close the admin client in three different `if/else if/else` branches in this lambda body, which is verbose (not a huge deal) and brittle (bigger deal).
   
   If we can combine everything into a single future using the `KafkaFuture::allOf` approach, then we can put the admin cleanup into a `whenComplete` call for that future, which will be a bit cleaner.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten

Review Comment:
   Same nit RE "overwritten":
   ```suggestion
        * @param offsets a mapping from partitions to offsets that need to be written
   ```



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    private void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                             Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId)
+                : baseProducerConfigs(connName, "connector-offset-producer-" + connName, config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId);
+        KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+        ConnectorOffsetBackingStore offsetStore = config.exactlyOnceSourceEnabled()
+                ? offsetStoreForExactlyOnceSourceConnector(sourceConfig, connName, connector, producer)
+                : offsetStoreForRegularSourceConnector(sourceConfig, connName, connector, producer);
+
+        OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, connName, internalKeyConverter, internalValueConverter);
+        alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, offsetStore, producer, offsetWriter, connectorLoader, cb);
+    }
+
+    // Visible for testing
+    void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                     Map<Map<String, ?>, Map<String, ?>> offsets, ConnectorOffsetBackingStore offsetStore,
+                                     KafkaProducer<byte[], byte[]> producer, OffsetStorageWriter offsetWriter,
+                                     ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SourceConnector) connector).alterOffsets(connectorConfig, offsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                offsetStore.configure(config);
+                // This reads to the end of the offsets topic and can be a potentially time-consuming operation
+                offsetStore.start();
+
+                // The alterSourceConnectorOffsets method should only be called after all the connector's tasks have been stopped, and it's
+                // safe to write offsets via an offset writer
+                offsets.forEach(offsetWriter::offset);
+
+                // We can call begin flush without a timeout because this newly created single-purpose offset writer can't do concurrent
+                // offset writes. We can also ignore the return value since it returns false if and only if there is no data to be flushed,
+                // and we've just put some data in the previous statement
+                offsetWriter.beginFlush();
+
+                try {
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.initTransactions();
+                        producer.beginTransaction();
+                    }
+                    log.debug("Committing the following partition offsets for source connector {}: {}", connName, offsets);
+                    FutureCallback<Void> offsetWriterCallback = new FutureCallback<>();
+                    offsetWriter.doFlush(offsetWriterCallback);
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.commitTransaction();
+                    }
+                    offsetWriterCallback.get(ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS, TimeUnit.MILLISECONDS);
+                } catch (ExecutionException e) {
+                    throw new ConnectException("Failed to alter offsets for source connector " + connName, e.getCause());
+                } catch (TimeoutException e) {
+                    throw new ConnectException("Timed out while attempting to alter offsets for source connector " + connName, e);
+                } catch (InterruptedException e) {
+                    throw new ConnectException("Unexpectedly interrupted while attempting to alter offsets for source connector " + connName, e);
+                } finally {
+                    offsetStore.stop();
+                }
+
+                completeAlterOffsetsCallback(alterOffsetsResult, cb);
+            } catch (Throwable t) {
+                log.error("Failed to alter offsets for source connector {}", connName, t);
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for source connector " + connName), null);
+            }
+        }));
+    }
+
+    private void completeAlterOffsetsCallback(boolean alterOffsetsResult, Callback<Message> cb) {
+        if (alterOffsetsResult) {
+            cb.onCompletion(null, new Message("The offsets for this connector have been altered successfully"));
+        } else {
+            cb.onCompletion(null, new Message("The Connect framework managed offsets for this connector have been " +

Review Comment:
   Nit: hyphen (like the KIP uses)
   ```suggestion
               cb.onCompletion(null, new Message("The Connect framework-managed offsets for this connector have been " +
   ```



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    private void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                             Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId)
+                : baseProducerConfigs(connName, "connector-offset-producer-" + connName, config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId);
+        KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+        ConnectorOffsetBackingStore offsetStore = config.exactlyOnceSourceEnabled()
+                ? offsetStoreForExactlyOnceSourceConnector(sourceConfig, connName, connector, producer)
+                : offsetStoreForRegularSourceConnector(sourceConfig, connName, connector, producer);
+
+        OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, connName, internalKeyConverter, internalValueConverter);
+        alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, offsetStore, producer, offsetWriter, connectorLoader, cb);
+    }
+
+    // Visible for testing
+    void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                     Map<Map<String, ?>, Map<String, ?>> offsets, ConnectorOffsetBackingStore offsetStore,
+                                     KafkaProducer<byte[], byte[]> producer, OffsetStorageWriter offsetWriter,
+                                     ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SourceConnector) connector).alterOffsets(connectorConfig, offsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                offsetStore.configure(config);
+                // This reads to the end of the offsets topic and can be a potentially time-consuming operation
+                offsetStore.start();
+
+                // The alterSourceConnectorOffsets method should only be called after all the connector's tasks have been stopped, and it's
+                // safe to write offsets via an offset writer
+                offsets.forEach(offsetWriter::offset);
+
+                // We can call begin flush without a timeout because this newly created single-purpose offset writer can't do concurrent
+                // offset writes. We can also ignore the return value since it returns false if and only if there is no data to be flushed,
+                // and we've just put some data in the previous statement
+                offsetWriter.beginFlush();
+
+                try {
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.initTransactions();
+                        producer.beginTransaction();
+                    }
+                    log.debug("Committing the following partition offsets for source connector {}: {}", connName, offsets);
+                    FutureCallback<Void> offsetWriterCallback = new FutureCallback<>();
+                    offsetWriter.doFlush(offsetWriterCallback);
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.commitTransaction();
+                    }
+                    offsetWriterCallback.get(ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS, TimeUnit.MILLISECONDS);

Review Comment:
   If the connector uses a separate offsets topic, this doesn't guarantee that the request to alter offsets has successfully completed on the worker's global offsets topic (see [here](https://github.com/apache/kafka/blob/228434d23583189cdcaa7f4a90ebb178ccc17c73/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConnectorOffsetBackingStore.java#L288-L289)).
   
   I think we may have to:
   - Detect when a connector-specific offsets store is necessary, and if it is:
   - - Construct just that store (without the secondary global offsets store)
   - - Alter the offsets for the connector using that store
   - Then, regardless of whether a connector-specific store was necessary, alter the offsets for the connector using the worker's global offsets store



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1512,11 +1773,25 @@ public WorkerTask doBuild(Task task,
         }
     }
 
-    // Visible for testing
+    /**
+     * Builds and returns an offset backing store for a regular source connector (i.e. when exactly-once support for source connectors is disabled).
+     * The offset backing store will either be just the worker's global offset backing store (if the connector doesn't define a connector-specific
+     * offset topic via its configs), just the connector-specific offset backing store (if the connector defines a connector-specific offsets
+     * topic which happens to be the same as the worker's global offset topic) or a combination of both the worker's global offset backing store

Review Comment:
   Nit: the way we determine this is susceptible to edge cases where the topics aren't the same
   ```suggestion
        * topic which appears to be the same as the worker's global offset topic) or a combination of both the worker's global offset backing store
   ```



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1576,11 +1854,24 @@ ConnectorOffsetBackingStore offsetStoreForRegularSourceConnector(
         }
     }
 
-    // Visible for testing
+    /**
+     * Builds and returns an offset backing store for an exactly-once source connector. The offset backing store will either be just
+     * the connector-specific offset backing store (if the connector defines a connector-specific offsets topic which happens to be

Review Comment:
   The connector doesn't have to define a custom offsets topic, in which case, we also use just a connector-specific store.
   ```suggestion
        * the connector-specific offset backing store (if the connector's offsets topic is
   ```



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {

Review Comment:
   I like the idea of performing both operations simultaneously and combining them with `KafkaFuture::allOf`. This optimizes for the happy-path case and the tradeoff if one of the operations fails isn't significant; the user knows that failed calls to the offset reset API can leaves the offsets for a connector in an inconsistent state, and can always check on the latest ones via the offset read API.



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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1166062323


##########
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:
   > users could get successful responses from the alter offsets API but the connector could completely ignore the overwritten offsets (if the user resumes the connector in the interim).
   
   That's a fair point. I was thinking that with the `STOPPED` state and a possible additional guard to prevent cancelled source tasks from committing offsets, we would have reasonable protection against zombie tasks from overwriting recently-altered offsets. However, I wasn't thinking of the other scenario, where the offset alter request is initiated and left ongoing while the connector is resumed.
   
   I think the risk of blocking the herder thread that you've brought up is perhaps the most convincing argument still in favor of making this operation asynchronous. There's risks not just with calls to the `alterOffsets` method, but also with reading to the end of offsets topics and contacting the transaction coordinator (if altering offsets for an exactly-once source connector), to name a few.
   
   I think if we really want to get fancy, one way we could try to decrease the risks of an asynchronous API for non-exactly-once source connectors could be to refuse to assign tasks for connectors with ongoing offset alterations during rebalance, even if the connector is resumed. The task configs for that connector could continue to live in the config topic, we'd just hold off on assigning them until the operation succeeds. Of course, this doesn't work if the leader of the cluster changes while an offset alter request is being serviced, but then (correct me if I'm wrong?) the same risks apply even with a synchronous API (although they're probably less likely). We could also try to add interruption logic that cancels any in-progress offset alter/reset requests when a rebalance starts. Either of these would be fine as a follow-up ticket, if they sound reasonable at all.



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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1166062323


##########
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:
   > users could get successful responses from the alter offsets API but the connector could completely ignore the overwritten offsets (if the user resumes the connector in the interim).
   
   That's a fair point. I was thinking that with the `STOPPED` state and a possible additional guard to prevent cancelled source tasks from committing offsets, we would have reasonable protection against zombie tasks from overwriting recently-altered offsets. However, I wasn't thinking of the other scenario, where the offset alter request is initiated and left ongoing while the connector is resumed.
   
   I think the risk of blocking the herder thread that you've brought up is perhaps the most convincing argument still in favor of making this operation asynchronous. There's risks not just with calls to the `alterOffsets` method, but also with reading to the end of offsets topics and contacting the transaction coordinator (if altering offsets for an exactly-once source connector), to name a few.
   
   If we really want to get fancy, one way we could try to decrease the risks of an asynchronous API for non-exactly-once source connectors could be to refuse to assign tasks for connectors with ongoing offset alterations during rebalance, even if the connector is resumed. The task configs for that connector could continue to live in the config topic, we'd just hold off on assigning them until the operation succeeds. Of course, this doesn't work if the leader of the cluster changes while an offset alter request is being serviced, but then (correct me if I'm wrong?) the same risks apply even with a synchronous API (although they're probably less likely). We could also try to add interruption logic that cancels any in-progress offset alter/reset requests when a rebalance starts. Either of these would be fine as a follow-up ticket, if they sound reasonable at all.



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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1204612735


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    private void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                             Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId)
+                : baseProducerConfigs(connName, "connector-offset-producer-" + connName, config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId);
+        KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+        ConnectorOffsetBackingStore offsetStore = config.exactlyOnceSourceEnabled()
+                ? offsetStoreForExactlyOnceSourceConnector(sourceConfig, connName, connector, producer)
+                : offsetStoreForRegularSourceConnector(sourceConfig, connName, connector, producer);
+
+        OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, connName, internalKeyConverter, internalValueConverter);
+        alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, offsetStore, producer, offsetWriter, connectorLoader, cb);
+    }
+
+    // Visible for testing
+    void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                     Map<Map<String, ?>, Map<String, ?>> offsets, ConnectorOffsetBackingStore offsetStore,
+                                     KafkaProducer<byte[], byte[]> producer, OffsetStorageWriter offsetWriter,
+                                     ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SourceConnector) connector).alterOffsets(connectorConfig, offsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                offsetStore.configure(config);
+                // This reads to the end of the offsets topic and can be a potentially time-consuming operation
+                offsetStore.start();
+
+                // The alterSourceConnectorOffsets method should only be called after all the connector's tasks have been stopped, and it's
+                // safe to write offsets via an offset writer
+                offsets.forEach(offsetWriter::offset);
+
+                // We can call begin flush without a timeout because this newly created single-purpose offset writer can't do concurrent
+                // offset writes. We can also ignore the return value since it returns false if and only if there is no data to be flushed,
+                // and we've just put some data in the previous statement
+                offsetWriter.beginFlush();
+
+                try {
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.initTransactions();
+                        producer.beginTransaction();
+                    }
+                    log.debug("Committing the following partition offsets for source connector {}: {}", connName, offsets);
+                    FutureCallback<Void> offsetWriterCallback = new FutureCallback<>();
+                    offsetWriter.doFlush(offsetWriterCallback);
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.commitTransaction();
+                    }
+                    offsetWriterCallback.get(ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS, TimeUnit.MILLISECONDS);

Review Comment:
   💯 Thanks, hoping we can fix KAFKA-15018 alongside releasing this feature as well!



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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1202684769


##########
connect/runtime/src/main/java/org/apache/kafka/connect/util/SinkUtils.java:
##########
@@ -52,4 +53,82 @@ public static ConnectorOffsets consumerGroupOffsetsToConnectorOffsets(Map<TopicP
 
         return new ConnectorOffsets(connectorOffsets);
     }
+
+    /**
+     * Validate that the provided partitions (keys in the {@code partitionOffsets} map) look like:
+     * <pre>
+     *     {
+     *       "kafka_topic": "topic"
+     *       "kafka_partition": 3
+     *     }
+     * </pre>
+     *
+     * and that the provided offsets (values in the {@code partitionOffsets} map) look like:
+     * <pre>
+     *     {
+     *       "kafka_offset": 1000
+     *     }
+     * </pre>
+     *
+     * This method then parses them into a mapping from {@link TopicPartition}s to their corresponding {@link Long}
+     * valued offsets.
+     *
+     * @param partitionOffsets the partitions to offset map that needs to be validated and parsed.
+     * @return the parsed mapping from {@link TopicPartition} to its corresponding {@link Long} valued offset.
+     *
+     * @throws BadRequestException if the provided offsets aren't in the expected format
+     */
+    public static Map<TopicPartition, Long> validateAndParseSinkConnectorOffsets(Map<Map<String, ?>, Map<String, ?>> partitionOffsets) {

Review Comment:
   Well done, looks great 👍



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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1162903146


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1259,217 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorConfig the connector's configurations
+     *
+     * @return true if the connector plugin has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * / {@link org.apache.kafka.connect.source.SourceConnector#alterOffsets(Map, Map)} and it returns true for the provided offsets,
+     * false otherwise
+     *
+     */
+    public boolean alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets,
+                                         Map<String, String> connectorConfig) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                return alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                return alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @return true if the sink connector has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * and it returns true for the provided offsets, false otherwise
+     */
+    private boolean alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                              Map<Map<String, ?>, Map<String, ?>> offsets) {
+        return alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, Admin::create);
+    }
+
+    // Visible for testing; allows mocking the admin client for testing
+    boolean alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Function<Map<String, Object>, Admin> adminFactory) {
+
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+        Timer timer = time.timer(ALTER_OFFSETS_TIMEOUT_MS);
+        boolean alterOffsetsResult;
+        try {
+            alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+        } catch (UnsupportedOperationException e) {
+            throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                    "modification of offsets", e);
+        }
+        timer.update();
+
+        Class<? extends Connector> sinkConnectorClass = connector.getClass();
+        Map<String, Object> adminConfig = adminConfigs(
+                connName,
+                "connector-worker-adminclient-" + connName,
+                config,
+                new SinkConnectorConfig(plugins, connectorConfig),
+                sinkConnectorClass,
+                connectorClientConfigOverridePolicy,
+                kafkaClusterId,
+                ConnectorType.SINK);
+
+        try (Admin admin = adminFactory.apply(adminConfig)) {
+            SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+            String groupId = (String) baseConsumerConfigs(
+                    connName, "connector-consumer-", config, sinkConnectorConfig,
+                    sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+            Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                    .stream()
+                    .filter(entry -> entry.getValue() != null)
+                    .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                    .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+            if (!offsetsToAlter.isEmpty()) {
+                AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter);
+                try {
+                    log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                            connName, offsetsToAlter);
+                    alterConsumerGroupOffsetsResult.all().get(timer.remainingMs(), TimeUnit.MILLISECONDS);
+                } catch (ExecutionException e) {
+                    // TODO: Handle different exception cause types to surface more fine-grained errors here?
+                    throw new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for connector "
+                            + connName, e.getCause());
+                } catch (TimeoutException e) {
+                    throw new ConnectException("Timed out while attempting to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet()
+                            + " for connector" + connName, e);
+                } catch (InterruptedException e) {
+                    throw new ConnectException("Unexpectedly interrupted while attempting to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet()
+                            + " for connector" + connName, e);
+                }
+                timer.update();
+            }
+
+            Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                    .stream()
+                    .filter(entry -> entry.getValue() == null)
+                    .map(Map.Entry::getKey)
+                    .collect(Collectors.toSet());
+
+            if (!partitionsToReset.isEmpty()) {
+                DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset);
+                try {
+                    log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                            connName, partitionsToReset);
+                    deleteConsumerGroupOffsetsResult.all().get(timer.remainingMs(), TimeUnit.MILLISECONDS);
+                } catch (ExecutionException e) {
+                    // TODO: Handle different exception cause types to surface more fine-grained errors here?
+                    throw new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                            + connName, e.getCause());
+                } catch (TimeoutException e) {
+                    throw new ConnectException("Timed out while attempting to delete consumer group offsets for topic partitions " + partitionsToReset
+                            + " for connector" + connName, e);
+                } catch (InterruptedException e) {
+                    throw new ConnectException("Unexpectedly interrupted while attempting to delete consumer group offsets for topic partitions " + partitionsToReset
+                            + " for connector" + connName, e);
+                }
+            }
+            return alterOffsetsResult;
+        }
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @return true if the source connector has implemented {@link org.apache.kafka.connect.source.SourceConnector#alterOffsets(Map, Map)}
+     * and it returns true for the provided offsets, false otherwise
+     */
+    private boolean alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                                Map<Map<String, ?>, Map<String, ?>> offsets) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,

Review Comment:
   We could always use the transactional ID for task 0, but that feels a little unclean. I'll ping the vote thread with this idea (using a transactional ID that corresponds to the connector name) and the other updates previously discussed.



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


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

Posted by "yashmayya (via GitHub)" <gi...@apache.org>.
yashmayya commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1159694956


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1259,217 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorConfig the connector's configurations
+     *
+     * @return true if the connector plugin has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * / {@link org.apache.kafka.connect.source.SourceConnector#alterOffsets(Map, Map)} and it returns true for the provided offsets,
+     * false otherwise
+     *
+     */
+    public boolean alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets,
+                                         Map<String, String> connectorConfig) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                return alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                return alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @return true if the sink connector has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * and it returns true for the provided offsets, false otherwise
+     */
+    private boolean alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                              Map<Map<String, ?>, Map<String, ?>> offsets) {
+        return alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, Admin::create);
+    }
+
+    // Visible for testing; allows mocking the admin client for testing
+    boolean alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Function<Map<String, Object>, Admin> adminFactory) {
+
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+        Timer timer = time.timer(ALTER_OFFSETS_TIMEOUT_MS);
+        boolean alterOffsetsResult;
+        try {
+            alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+        } catch (UnsupportedOperationException e) {
+            throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                    "modification of offsets", e);
+        }
+        timer.update();
+
+        Class<? extends Connector> sinkConnectorClass = connector.getClass();
+        Map<String, Object> adminConfig = adminConfigs(
+                connName,
+                "connector-worker-adminclient-" + connName,
+                config,
+                new SinkConnectorConfig(plugins, connectorConfig),
+                sinkConnectorClass,
+                connectorClientConfigOverridePolicy,
+                kafkaClusterId,
+                ConnectorType.SINK);
+
+        try (Admin admin = adminFactory.apply(adminConfig)) {
+            SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+            String groupId = (String) baseConsumerConfigs(
+                    connName, "connector-consumer-", config, sinkConnectorConfig,
+                    sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+            Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                    .stream()
+                    .filter(entry -> entry.getValue() != null)
+                    .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                    .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+            if (!offsetsToAlter.isEmpty()) {
+                AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter);
+                try {
+                    log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                            connName, offsetsToAlter);
+                    alterConsumerGroupOffsetsResult.all().get(timer.remainingMs(), TimeUnit.MILLISECONDS);
+                } catch (ExecutionException e) {
+                    // TODO: Handle different exception cause types to surface more fine-grained errors here?
+                    throw new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for connector "
+                            + connName, e.getCause());
+                } catch (TimeoutException e) {
+                    throw new ConnectException("Timed out while attempting to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet()
+                            + " for connector" + connName, e);
+                } catch (InterruptedException e) {
+                    throw new ConnectException("Unexpectedly interrupted while attempting to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet()
+                            + " for connector" + connName, e);
+                }
+                timer.update();
+            }
+
+            Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                    .stream()
+                    .filter(entry -> entry.getValue() == null)
+                    .map(Map.Entry::getKey)
+                    .collect(Collectors.toSet());
+
+            if (!partitionsToReset.isEmpty()) {
+                DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset);
+                try {
+                    log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                            connName, partitionsToReset);
+                    deleteConsumerGroupOffsetsResult.all().get(timer.remainingMs(), TimeUnit.MILLISECONDS);
+                } catch (ExecutionException e) {
+                    // TODO: Handle different exception cause types to surface more fine-grained errors here?
+                    throw new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                            + connName, e.getCause());
+                } catch (TimeoutException e) {
+                    throw new ConnectException("Timed out while attempting to delete consumer group offsets for topic partitions " + partitionsToReset
+                            + " for connector" + connName, e);
+                } catch (InterruptedException e) {
+                    throw new ConnectException("Unexpectedly interrupted while attempting to delete consumer group offsets for topic partitions " + partitionsToReset
+                            + " for connector" + connName, e);
+                }
+            }
+            return alterOffsetsResult;
+        }
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @return true if the source connector has implemented {@link org.apache.kafka.connect.source.SourceConnector#alterOffsets(Map, Map)}
+     * and it returns true for the provided offsets, false otherwise
+     */
+    private boolean alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                                Map<Map<String, ?>, Map<String, ?>> offsets) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,

Review Comment:
   It would be cleaner to use a connector specific producer transactional ID here but that would require additional ACLs over what is described in [KIP-618](https://cwiki.apache.org/confluence/display/KAFKA/KIP-618%3A+Exactly-Once+Support+for+Source+Connectors#KIP618:ExactlyOnceSupportforSourceConnectors-Producer) (unless a wildcard prefix is used). @C0urante would it be possible to amend [KIP-875](https://cwiki.apache.org/confluence/display/KAFKA/KIP-875%3A+First-class+offsets+support+in+Kafka+Connect) and document this additional required ACL for altering / resetting offsets for a source connector when exactly once support is enabled on the worker?



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


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

Posted by "yashmayya (via GitHub)" <gi...@apache.org>.
yashmayya commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1167415436


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {

Review Comment:
   While this whole bit does seem a bit convoluted, it offers the advantage of not trying to delete partition offsets when the alter offsets attempt fails. The alternative could be to simply do both the operations concurrently (`alterConsumerGroupOffsets` and `deleteConsumerGroupOffsets`), and then react to the combined response (combining the returned futures using `KafkaFuture::allOf`)



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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1202694716


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1235,6 +1244,246 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets a mapping from partitions (either source partitions for source connectors, or Kafka topic
+     *                partitions for sink connectors) to offsets that need to be written; may not be null or empty
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+
+        if (offsets == null || offsets.isEmpty()) {
+            throw new ConnectException("The offsets to be altered may not be null or empty");
+        }
+
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be written; may not be null or empty
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.parseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        sinkConnectorConfig,
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);

Review Comment:
   Can we construct a `List<KafkaFuture<Void>>` here and add elements to that as necessary (i.e., depending on whether the sets of to-be-removed and to-be-altered partitions are empty), then combine then with `KafkaFuture<Void> adminFuture = KafkaFuture.allOf(futures.toArray(new KafkaFuture[0]));`?
   
   The use of a no-op future is a little hard to read.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1235,6 +1244,246 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets a mapping from partitions (either source partitions for source connectors, or Kafka topic
+     *                partitions for sink connectors) to offsets that need to be written; may not be null or empty
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+
+        if (offsets == null || offsets.isEmpty()) {
+            throw new ConnectException("The offsets to be altered may not be null or empty");
+        }
+
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be written; may not be null or empty
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.parseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        sinkConnectorConfig,
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                    Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                            .stream()
+                            .filter(entry -> entry.getValue() != null)
+                            .collect(Collectors.toMap(Map.Entry::getKey, e -> new OffsetAndMetadata(e.getValue())));
+
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                            .stream()
+                            .filter(entry -> entry.getValue() == null)
+                            .map(Map.Entry::getKey)
+                            .collect(Collectors.toSet());
+
+                    if (!partitionsToReset.isEmpty()) {
+                        log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                connName, partitionsToReset);
+                        DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                deleteConsumerGroupOffsetsOptions);
+
+                        adminFuture = KafkaFuture.allOf(adminFuture, deleteConsumerGroupOffsetsResult.all());
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            // When a consumer group is non-empty, only group members can commit offsets. An attempt to alter offsets via the admin client
+                            // will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed). Similarly, an attempt to
+                            // delete consumer group offsets for a non-empty consumer group will result in a GroupSubscribedToTopicException
+                            if (error instanceof UnknownMemberIdException || error instanceof GroupSubscribedToTopicException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName, error), null);
+                            }
+                        } else {
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                        Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);

Review Comment:
   This will leak the client if any of the preceding callback logic fails.
   
   Can we perform this separately with another chained `whenComplete` invocation?



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    private void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                             Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId)
+                : baseProducerConfigs(connName, "connector-offset-producer-" + connName, config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId);
+        KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+        ConnectorOffsetBackingStore offsetStore = config.exactlyOnceSourceEnabled()
+                ? offsetStoreForExactlyOnceSourceConnector(sourceConfig, connName, connector, producer)
+                : offsetStoreForRegularSourceConnector(sourceConfig, connName, connector, producer);
+
+        OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, connName, internalKeyConverter, internalValueConverter);
+        alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, offsetStore, producer, offsetWriter, connectorLoader, cb);
+    }
+
+    // Visible for testing
+    void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                     Map<Map<String, ?>, Map<String, ?>> offsets, ConnectorOffsetBackingStore offsetStore,
+                                     KafkaProducer<byte[], byte[]> producer, OffsetStorageWriter offsetWriter,
+                                     ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SourceConnector) connector).alterOffsets(connectorConfig, offsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                offsetStore.configure(config);
+                // This reads to the end of the offsets topic and can be a potentially time-consuming operation
+                offsetStore.start();

Review Comment:
   Believe this still needs to be addressed?



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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on PR #13465:
URL: https://github.com/apache/kafka/pull/13465#issuecomment-1562958860

   Ah, spoke too soon! Looks like there are some CI test failures. Can you look into these?
   
   org.apache.kafka.connect.runtime.WorkerTest.testAlterOffsetsSourceConnectorError:
   
   > Wanted but not invoked:
   connectorOffsetBackingStore.stop();
   -> at org.apache.kafka.connect.storage.ConnectorOffsetBackingStore.stop(ConnectorOffsetBackingStore.java:188)
   However, there was exactly 1 interaction with this mock:
   connectorOffsetBackingStore.start();
   -> at org.apache.kafka.connect.runtime.Worker.lambda$alterSourceConnectorOffsets$16(Worker.java:1438)
   at app//org.apache.kafka.connect.storage.ConnectorOffsetBackingStore.stop(ConnectorOffsetBackingStore.java:188)
   at app//org.apache.kafka.connect.runtime.WorkerTest.testAlterOffsetsSourceConnectorError(WorkerTest.java:2000)
   
   
   org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testAlterSinkConnectorOffsetsDifferentKafkaClusterTargeted:
   
   > org.opentest4j.AssertionFailedError: Condition not met within timeout 15000. Sink connector consumer group offsets should catch up to the topic end offsets ==> expected: &lt;true&gt; but was: &lt;false&gt;
            at org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:151)
            at org.junit.jupiter.api.AssertionFailureBuilder.buildAndThrow(AssertionFailureBuilder.java:132)
            at org.junit.jupiter.api.AssertTrue.failNotTrue(AssertTrue.java:63)
            at org.junit.jupiter.api.AssertTrue.assertTrue(AssertTrue.java:36)
            at org.junit.jupiter.api.Assertions.assertTrue(Assertions.java:211)
            at org.apache.kafka.test.TestUtils.lambda$waitForCondition$3(TestUtils.java:331)
            at org.apache.kafka.test.TestUtils.retryOnExceptionWithTimeout(TestUtils.java:379)
            at org.apache.kafka.test.TestUtils.waitForCondition(TestUtils.java:328)
            at org.apache.kafka.test.TestUtils.waitForCondition(TestUtils.java:312)
            at org.apache.kafka.test.TestUtils.waitForCondition(TestUtils.java:285)
            at org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.waitForExpectedSinkConnectorOffsets(OffsetsApiIntegrationTest.java:602)
            at org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.alterAndVerifySinkConnectorOffsets(OffsetsApiIntegrationTest.java:312)
            at org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testAlterSinkConnectorOffsetsDifferentKafkaClusterTargeted(OffsetsApiIntegrationTest.java:292)


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


[GitHub] [kafka] yashmayya commented on pull request #13465: KAFKA-14368: Connect offset write REST API

Posted by "yashmayya (via GitHub)" <gi...@apache.org>.
yashmayya commented on PR #13465:
URL: https://github.com/apache/kafka/pull/13465#issuecomment-1562952794

   I just noticed that `testAlterSinkConnectorOffsetsDifferentKafkaClusterTargeted` is failing in the CI run. There's also https://issues.apache.org/jira/browse/KAFKA-14956 where `testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted ` has been failing on CI for a while. Interestingly, neither of them have failed for me locally in over a 100 runs under various loads. The point at which both of them are failing is also interesting:
   
   1. We create an embedded Connect cluster with its own embedded backing Kafka cluster
   2. We create a second embedded Kafka cluster
   3. We configure a sink connector in the embedded Connect cluster which consumes from a topic on the second embedded Kafka cluster
   4. We produce 10 messages each to 5 different partitions of a topic on the second Kafka cluster (which the connector is configured to consume from)
   5. We use the offsets read REST API to get the consumer group offsets for the sink connector and wait until it "catches up" to the expected offsets. This operation is retried up to 15 seconds and if the consumer group offsets (obtained via an admin client in the worker) don't match the expected offsets, the test fails. 
   
   Both the tests are failing at this point. Since they consistently pass locally, it doesn't seem to be a correctness issue with connectors that target different Kafka clusters. I'm wondering if we need to up the timeout although 15 seconds should be enough to consume just 50 messages 😕 


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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on PR #13465:
URL: https://github.com/apache/kafka/pull/13465#issuecomment-1562965406

   Ah, spoke too soon!
   
   I'd be open to bumping timeouts. If this does turn out to be a correctness issue (which is still possible since the timing on CI may be different and therefore more likely to unearth certain kinds of concurrency bugs), we can investigate further.
   
   Also worth noting that `WorkerTest::testAlterOffsetsSourceConnectorError` is also failing right now because `offsetStore::stop` hasn't been invoked by the time we check for it. I think you handle this kind of issue elsewhere by adding a second `timeout(1000)` argument when making calls to `Mockito::verify`; hopefully that's sufficient for this test as well?


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


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

Posted by "yashmayya (via GitHub)" <gi...@apache.org>.
yashmayya commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1163225504


##########
connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java:
##########
@@ -19,24 +19,25 @@
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.connect.errors.ConnectException;
 import org.apache.kafka.connect.runtime.isolation.Plugins;
 import org.apache.kafka.connect.runtime.rest.entities.ActiveTopicsInfo;
 import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos;
-import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets;
 import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo;
+import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets;
 import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo;
 import org.apache.kafka.connect.runtime.rest.entities.ServerInfo;
 import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException;
+import org.eclipse.jetty.client.HttpClient;

Review Comment:
   Yeah, I did want to avoid doing this, but it was essentially a last resort. The `HttpURLConnection` API doesn't support the `PATCH` HTTP method and there is a won't fix bug ticket in OpenJDK for this - https://bugs.openjdk.org/browse/JDK-7016595. The new `HttpClient` (incubated in Java 9 and made official in Java 11) which supersedes `HttpURLConnection` does fix this issue, but since we still support Java 8, that wasn't an option either. Since the Jetty HTTP client is already being used by the main Connect codebase, I figured it'd be the best option here.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1254,224 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorConfig the connector's configurations
+     *
+     * @return true if the connector plugin has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * / {@link org.apache.kafka.connect.source.SourceConnector#alterOffsets(Map, Map)} and it returns true for the provided offsets,
+     * false otherwise
+     *
+     */
+    public boolean alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets,
+                                         Map<String, String> connectorConfig) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                return alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                return alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @return true if the sink connector has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * and it returns true for the provided offsets, false otherwise
+     */
+    boolean alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets) {
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+        Timer timer = time.timer(ALTER_OFFSETS_TIMEOUT_MS);
+        boolean alterOffsetsResult;
+        try {
+            alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+        } catch (UnsupportedOperationException e) {
+            throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                    "modification of offsets", e);
+        }
+        timer.update();

Review Comment:
   Good point, I figured that in this case the next admin client operation would just use a timeout of 0 and timeout immediately. But now that I think about it, this'll lead to a misleading exception being thrown (indicating an issue with the admin client operation when the actual issue was the call to the plugin's `alterOffsets` method). We have a number of these timer updates after various operations in both the sink and source alter offsets implementations. While we can't use timeouts for these operations, maybe we can use a utility method which updates the timer and throws a `ConnectException` with a relevant message if the timer expires? WDYT?



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


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

Posted by "gharris1727 (via GitHub)" <gi...@apache.org>.
gharris1727 commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1163320472


##########
connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java:
##########
@@ -19,24 +19,25 @@
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.connect.errors.ConnectException;
 import org.apache.kafka.connect.runtime.isolation.Plugins;
 import org.apache.kafka.connect.runtime.rest.entities.ActiveTopicsInfo;
 import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos;
-import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets;
 import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo;
+import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets;
 import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo;
 import org.apache.kafka.connect.runtime.rest.entities.ServerInfo;
 import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException;
+import org.eclipse.jetty.client.HttpClient;

Review Comment:
   Thanks for the clarification, I think it's reasonable to use jetty in that circumstance.



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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1202709620


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    private void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                             Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId)
+                : baseProducerConfigs(connName, "connector-offset-producer-" + connName, config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId);
+        KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+        ConnectorOffsetBackingStore offsetStore = config.exactlyOnceSourceEnabled()
+                ? offsetStoreForExactlyOnceSourceConnector(sourceConfig, connName, connector, producer)
+                : offsetStoreForRegularSourceConnector(sourceConfig, connName, connector, producer);
+
+        OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, connName, internalKeyConverter, internalValueConverter);
+        alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, offsetStore, producer, offsetWriter, connectorLoader, cb);
+    }
+
+    // Visible for testing
+    void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                     Map<Map<String, ?>, Map<String, ?>> offsets, ConnectorOffsetBackingStore offsetStore,
+                                     KafkaProducer<byte[], byte[]> producer, OffsetStorageWriter offsetWriter,
+                                     ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SourceConnector) connector).alterOffsets(connectorConfig, offsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                offsetStore.configure(config);
+                // This reads to the end of the offsets topic and can be a potentially time-consuming operation
+                offsetStore.start();
+
+                // The alterSourceConnectorOffsets method should only be called after all the connector's tasks have been stopped, and it's
+                // safe to write offsets via an offset writer
+                offsets.forEach(offsetWriter::offset);
+
+                // We can call begin flush without a timeout because this newly created single-purpose offset writer can't do concurrent
+                // offset writes. We can also ignore the return value since it returns false if and only if there is no data to be flushed,
+                // and we've just put some data in the previous statement
+                offsetWriter.beginFlush();
+
+                try {
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.initTransactions();
+                        producer.beginTransaction();
+                    }
+                    log.debug("Committing the following partition offsets for source connector {}: {}", connName, offsets);
+                    FutureCallback<Void> offsetWriterCallback = new FutureCallback<>();
+                    offsetWriter.doFlush(offsetWriterCallback);
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.commitTransaction();
+                    }
+                    offsetWriterCallback.get(ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS, TimeUnit.MILLISECONDS);
+                } catch (ExecutionException e) {
+                    throw new ConnectException("Failed to alter offsets for source connector " + connName, e.getCause());

Review Comment:
   Fair enough, LGTM 👍



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


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

Posted by "yashmayya (via GitHub)" <gi...@apache.org>.
yashmayya commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1205023533


##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java:
##########
@@ -4123,6 +4136,280 @@ public void testConnectorOffsets() throws Exception {
         PowerMock.verifyAll();
     }
 
+    @Test
+    public void testAlterOffsetsConnectorNotFound() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets("connector-does-not-exist", new HashMap<>(), callback);
+        herder.tick();
+        ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertTrue(e.getCause() instanceof NotFoundException);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsConnectorNotInStoppedState() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, new HashMap<>(), callback);
+        herder.tick();
+        ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertTrue(e.getCause() instanceof BadRequestException);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsNotLeader() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("member");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), false);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();

Review Comment:
   Yeah, it does seem like it is intentional - apparently `expectLastCall` (both `EasyMock` and `PowerMock`) is optional for void methods and is mainly used for clarity. FWIW, I was able to get the test to pass even after removing the last one (after the call to `member.poll()`).



##########
connect/runtime/src/test/java/org/apache/kafka/connect/util/SinkUtilsTest.java:
##########
@@ -46,4 +52,124 @@ public void testConsumerGroupOffsetsToConnectorOffsets() {
         expectedPartition.put(SinkUtils.KAFKA_PARTITION_KEY, 0);
         assertEquals(expectedPartition, connectorOffsets.offsets().get(0).partition());
     }
+
+    @Test
+    public void testValidateAndParseEmptyPartitionOffsetMap() {
+        // expect no exception to be thrown
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.parseSinkConnectorOffsets(new HashMap<>());
+        assertTrue(parsedOffsets.isEmpty());
+    }
+
+    @Test
+    public void testValidateAndParseInvalidPartition() {
+        Map<String, Object> partition = new HashMap<>();
+        partition.put(SinkUtils.KAFKA_TOPIC_KEY, "topic");
+        Map<String, Object> offset = new HashMap<>();
+        offset.put(SinkUtils.KAFKA_OFFSET_KEY, 100);
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        partitionOffsets.put(partition, offset);
+
+        // missing partition key
+        ConnectException e = assertThrows(ConnectException.class, () -> SinkUtils.parseSinkConnectorOffsets(partitionOffsets));
+        assertThat(e.getMessage(), containsString("The partition for a sink connector offset must contain the keys 'kafka_topic' and 'kafka_partition'"));
+
+        partition.put(SinkUtils.KAFKA_PARTITION_KEY, "not a number");
+        // bad partition key
+        e = assertThrows(ConnectException.class, () -> SinkUtils.parseSinkConnectorOffsets(partitionOffsets));
+        assertThat(e.getMessage(), containsString("Failed to parse the following Kafka partition value in the provided offsets: 'not a number'"));
+
+        partition.remove(SinkUtils.KAFKA_TOPIC_KEY);
+        partition.put(SinkUtils.KAFKA_PARTITION_KEY, "5");
+        // missing topic key
+        e = assertThrows(ConnectException.class, () -> SinkUtils.parseSinkConnectorOffsets(partitionOffsets));
+        assertThat(e.getMessage(), containsString("The partition for a sink connector offset must contain the keys 'kafka_topic' and 'kafka_partition'"));
+    }
+
+    @Test
+    public void testValidateAndParseInvalidOffset() {
+        Map<String, Object> partition = new HashMap<>();
+        partition.put(SinkUtils.KAFKA_TOPIC_KEY, "topic");
+        partition.put(SinkUtils.KAFKA_PARTITION_KEY, 10);
+        Map<String, Object> offset = new HashMap<>();
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        partitionOffsets.put(partition, offset);
+
+        // missing offset key
+        ConnectException e = assertThrows(ConnectException.class, () -> SinkUtils.parseSinkConnectorOffsets(partitionOffsets));
+        assertThat(e.getMessage(), containsString("The offset for a sink connector should either be null or contain the key 'kafka_offset'"));
+
+        // bad offset key
+        offset.put(SinkUtils.KAFKA_OFFSET_KEY, "not a number");
+        e = assertThrows(ConnectException.class, () -> SinkUtils.parseSinkConnectorOffsets(partitionOffsets));
+        assertThat(e.getMessage(), containsString("Failed to parse the following Kafka offset value in the provided offsets: 'not a number'"));
+    }
+
+    @Test
+    public void testValidateAndParseStringPartitionValue() {
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = createPartitionOffsetMap("topic", "10", "100");
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.parseSinkConnectorOffsets(partitionOffsets);
+        assertEquals(1, parsedOffsets.size());
+        TopicPartition tp = parsedOffsets.keySet().iterator().next();
+        assertEquals(10, tp.partition());
+    }
+
+    @Test
+    public void testValidateAndParseIntegerPartitionValue() {
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = createPartitionOffsetMap("topic", 10, "100");
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.parseSinkConnectorOffsets(partitionOffsets);
+        assertEquals(1, parsedOffsets.size());
+        TopicPartition tp = parsedOffsets.keySet().iterator().next();
+        assertEquals(10, tp.partition());
+    }
+
+    @Test
+    public void testValidateAndParseStringOffsetValue() {
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = createPartitionOffsetMap("topic", "10", "100");
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.parseSinkConnectorOffsets(partitionOffsets);
+        assertEquals(1, parsedOffsets.size());
+        Long offsetValue = parsedOffsets.values().iterator().next();
+        assertEquals(100L, offsetValue.longValue());
+    }
+
+    @Test
+    public void testValidateAndParseIntegerOffsetValue() {
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = createPartitionOffsetMap("topic", "10", 100);
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.parseSinkConnectorOffsets(partitionOffsets);
+        assertEquals(1, parsedOffsets.size());
+        Long offsetValue = parsedOffsets.values().iterator().next();
+        assertEquals(100L, offsetValue.longValue());
+    }
+
+    @Test
+    public void testNullOffset() {
+        Map<String, Object> partitionMap = new HashMap<>();
+        partitionMap.put(SinkUtils.KAFKA_TOPIC_KEY, "topic");
+        partitionMap.put(SinkUtils.KAFKA_PARTITION_KEY, 10);
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        partitionOffsets.put(partitionMap, null);
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.parseSinkConnectorOffsets(partitionOffsets);
+        assertEquals(1, parsedOffsets.size());
+        assertNull(parsedOffsets.values().iterator().next());
+    }
+
+    @Test
+    public void testNullPartition() {

Review Comment:
   Ah, I hadn't noticed that we're treating null topic names as the Kafka topic "null". While that is a valid Kafka topic name, it should be provided as the string value "null" and it makes more sense to invalidate a null topic name. I've updated the `parseSinkConnectorOffsets` method to include this check and added a new test for this case, thanks!



##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java:
##########
@@ -4123,6 +4136,280 @@ public void testConnectorOffsets() throws Exception {
         PowerMock.verifyAll();
     }
 
+    @Test
+    public void testAlterOffsetsConnectorNotFound() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets("connector-does-not-exist", new HashMap<>(), callback);
+        herder.tick();
+        ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertTrue(e.getCause() instanceof NotFoundException);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsConnectorNotInStoppedState() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, new HashMap<>(), callback);
+        herder.tick();
+        ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertTrue(e.getCause() instanceof BadRequestException);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsNotLeader() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("member");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), false);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, new HashMap<>(), callback);
+        herder.tick();
+        ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertTrue(e.getCause() instanceof NotLeaderException);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsSourceConnectorExactlyOnceDisabled() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        Map<Map<String, ?>, Map<String, ?>> offsets = new HashMap<>();
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        Capture<Callback<Message>> workerCallbackCapture = Capture.newInstance();
+        worker.alterConnectorOffsets(EasyMock.eq(CONN1), EasyMock.eq(CONN1_CONFIG), EasyMock.eq(offsets), capture(workerCallbackCapture));
+        Message msg = new Message("The offsets for this connector have been altered successfully");
+        EasyMock.expectLastCall().andAnswer(() -> {
+            workerCallbackCapture.getValue().onCompletion(null, msg);
+            return null;
+        });
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, offsets, callback);
+        herder.tick();
+        assertEquals(msg, callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertEquals("The offsets for this connector have been altered successfully", msg.message());
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsSourceConnectorExactlyOnceEnabled() throws Exception {
+        // Setup herder with exactly-once support for source connectors enabled
+        herder = exactlyOnceHerder();
+        rebalanceListener = herder.new RebalanceListener(time);
+        PowerMock.expectPrivate(herder, "updateDeletedConnectorStatus").andVoid().anyTimes();
+        PowerMock.expectPrivate(herder, "updateDeletedTaskStatus").andVoid().anyTimes();
+
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall().anyTimes();
+
+        // Now handle the alter connector offsets request
+        Map<Map<String, ?>, Map<String, ?>> offsets = new HashMap<>();
+        member.wakeup();
+        PowerMock.expectLastCall().anyTimes();
+        member.ensureActive();
+        PowerMock.expectLastCall().anyTimes();
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        EasyMock.expect(herder.connectorType(EasyMock.anyObject())).andReturn(ConnectorType.SOURCE).anyTimes();
+
+        // Expect a round of zombie fencing to occur
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        KafkaFuture<Void> workerFencingFuture = EasyMock.mock(KafkaFuture.class);
+        KafkaFuture<Void> herderFencingFuture = EasyMock.mock(KafkaFuture.class);
+        EasyMock.expect(worker.fenceZombies(CONN1, SNAPSHOT_STOPPED_CONN1.taskCountRecord(CONN1), CONN1_CONFIG)).andReturn(workerFencingFuture);
+        EasyMock.expect(workerFencingFuture.thenApply(EasyMock.<KafkaFuture.BaseFunction<Void, Void>>anyObject())).andReturn(herderFencingFuture);
+
+        // Two fencing callbacks are added - one is in ZombieFencing::start itself to remove the connector from the active
+        // fencing list. The other is the callback passed from DistributedHerder::alterConnectorOffsets in order to
+        // queue up the actual alter offsets request if the zombie fencing succeeds.
+        for (int i = 0; i < 2; i++) {
+            Capture<KafkaFuture.BiConsumer<Void, Throwable>> herderFencingCallback = EasyMock.newCapture();
+            EasyMock.expect(herderFencingFuture.whenComplete(EasyMock.capture(herderFencingCallback))).andAnswer(() -> {
+                herderFencingCallback.getValue().accept(null, null);
+                return null;
+            });
+        }
+
+        Capture<Callback<Message>> workerCallbackCapture = Capture.newInstance();
+        Message msg = new Message("The offsets for this connector have been altered successfully");
+        worker.alterConnectorOffsets(EasyMock.eq(CONN1), EasyMock.eq(CONN1_CONFIG), EasyMock.eq(offsets), capture(workerCallbackCapture));
+        EasyMock.expectLastCall().andAnswer(() -> {
+            workerCallbackCapture.getValue().onCompletion(null, msg);
+            return null;
+        });
+
+        // Handle the second alter connector offsets request. No zombie fencing request to the worker is expected now since we
+        // already did a round of zombie fencing last time and no new tasks came up in the meanwhile. The config snapshot is
+        // refreshed once at the beginning of the DistributedHerder::alterConnectorOffsets method, once before checking
+        // whether zombie fencing is required, and once before actually proceeding to alter connector offsets.
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1_FENCED);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1_FENCED);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1_FENCED);
+        Capture<Callback<Message>> workerCallbackCapture2 = Capture.newInstance();
+        worker.alterConnectorOffsets(EasyMock.eq(CONN1), EasyMock.eq(CONN1_CONFIG), EasyMock.eq(offsets), capture(workerCallbackCapture2));
+        EasyMock.expectLastCall().andAnswer(() -> {
+            workerCallbackCapture2.getValue().onCompletion(null, msg);
+            return null;
+        });
+
+        PowerMock.replayAll(workerFencingFuture, herderFencingFuture);
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, offsets, callback);
+        // Process the zombie fencing request
+        herder.tick();
+        // Process the alter offsets request
+        herder.tick();
+        assertEquals(msg, callback.get(1000L, TimeUnit.MILLISECONDS));
+
+        FutureCallback<Message> callback2 = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, offsets, callback2);
+        herder.tick();
+        assertEquals(msg, callback.get(1000L, TimeUnit.MILLISECONDS));
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsSourceConnectorExactlyOnceEnabledZombieFencingFailure() throws Exception {

Review Comment:
   Thanks! Yeah, I hadn't added one for sink connectors because in the current shape it's identical to the non-EoS source case. However, additional test cases don't hurt and if the behavior ever diverges in the future for sink connectors versus non-EoS source connectors, it'll be easier if there's already a test case in place to update. The newly added test looks slightly messy (due to the need to reset previous expectations and add new ones in the EasyMock / PowerMock paradigm) but will look cleaner when I migrate it to Mockito along with the other tests here.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    private void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                             Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId)
+                : baseProducerConfigs(connName, "connector-offset-producer-" + connName, config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId);
+        KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+        ConnectorOffsetBackingStore offsetStore = config.exactlyOnceSourceEnabled()
+                ? offsetStoreForExactlyOnceSourceConnector(sourceConfig, connName, connector, producer)
+                : offsetStoreForRegularSourceConnector(sourceConfig, connName, connector, producer);
+
+        OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, connName, internalKeyConverter, internalValueConverter);
+        alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, offsetStore, producer, offsetWriter, connectorLoader, cb);
+    }
+
+    // Visible for testing
+    void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                     Map<Map<String, ?>, Map<String, ?>> offsets, ConnectorOffsetBackingStore offsetStore,
+                                     KafkaProducer<byte[], byte[]> producer, OffsetStorageWriter offsetWriter,
+                                     ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SourceConnector) connector).alterOffsets(connectorConfig, offsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                offsetStore.configure(config);
+                // This reads to the end of the offsets topic and can be a potentially time-consuming operation
+                offsetStore.start();

Review Comment:
   Ah okay, I get it now, thanks. I've moved the `configure` call to the top level `alterSourceConnectorOffsets` method and moved the finally block where cleanup occurs to the top level try catch in the overloaded method (this is because the `KafkaBasedLog` is instantiated in the `configure` method and we need it to ensure resource cleanup in the call to `KafkaOffsetBackingStore::stop`). This also assumes that the call to `KafkaOffsetBackingStore::configure` won't throw an exception because at this point the connector tasks should already have successfully configured and started their own offset stores. 
   
   On a side note, it doesn't look like we have exception handling in place for calls to `KafkaOffsetBackingStore::configure` elsewhere either and could lead to leaking clients created for the store.



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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1162903146


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1259,217 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorConfig the connector's configurations
+     *
+     * @return true if the connector plugin has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * / {@link org.apache.kafka.connect.source.SourceConnector#alterOffsets(Map, Map)} and it returns true for the provided offsets,
+     * false otherwise
+     *
+     */
+    public boolean alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets,
+                                         Map<String, String> connectorConfig) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                return alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                return alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @return true if the sink connector has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * and it returns true for the provided offsets, false otherwise
+     */
+    private boolean alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                              Map<Map<String, ?>, Map<String, ?>> offsets) {
+        return alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, Admin::create);
+    }
+
+    // Visible for testing; allows mocking the admin client for testing
+    boolean alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Function<Map<String, Object>, Admin> adminFactory) {
+
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+        Timer timer = time.timer(ALTER_OFFSETS_TIMEOUT_MS);
+        boolean alterOffsetsResult;
+        try {
+            alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+        } catch (UnsupportedOperationException e) {
+            throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                    "modification of offsets", e);
+        }
+        timer.update();
+
+        Class<? extends Connector> sinkConnectorClass = connector.getClass();
+        Map<String, Object> adminConfig = adminConfigs(
+                connName,
+                "connector-worker-adminclient-" + connName,
+                config,
+                new SinkConnectorConfig(plugins, connectorConfig),
+                sinkConnectorClass,
+                connectorClientConfigOverridePolicy,
+                kafkaClusterId,
+                ConnectorType.SINK);
+
+        try (Admin admin = adminFactory.apply(adminConfig)) {
+            SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+            String groupId = (String) baseConsumerConfigs(
+                    connName, "connector-consumer-", config, sinkConnectorConfig,
+                    sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+            Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                    .stream()
+                    .filter(entry -> entry.getValue() != null)
+                    .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                    .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+            if (!offsetsToAlter.isEmpty()) {
+                AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter);
+                try {
+                    log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                            connName, offsetsToAlter);
+                    alterConsumerGroupOffsetsResult.all().get(timer.remainingMs(), TimeUnit.MILLISECONDS);
+                } catch (ExecutionException e) {
+                    // TODO: Handle different exception cause types to surface more fine-grained errors here?
+                    throw new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for connector "
+                            + connName, e.getCause());
+                } catch (TimeoutException e) {
+                    throw new ConnectException("Timed out while attempting to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet()
+                            + " for connector" + connName, e);
+                } catch (InterruptedException e) {
+                    throw new ConnectException("Unexpectedly interrupted while attempting to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet()
+                            + " for connector" + connName, e);
+                }
+                timer.update();
+            }
+
+            Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                    .stream()
+                    .filter(entry -> entry.getValue() == null)
+                    .map(Map.Entry::getKey)
+                    .collect(Collectors.toSet());
+
+            if (!partitionsToReset.isEmpty()) {
+                DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset);
+                try {
+                    log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                            connName, partitionsToReset);
+                    deleteConsumerGroupOffsetsResult.all().get(timer.remainingMs(), TimeUnit.MILLISECONDS);
+                } catch (ExecutionException e) {
+                    // TODO: Handle different exception cause types to surface more fine-grained errors here?
+                    throw new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                            + connName, e.getCause());
+                } catch (TimeoutException e) {
+                    throw new ConnectException("Timed out while attempting to delete consumer group offsets for topic partitions " + partitionsToReset
+                            + " for connector" + connName, e);
+                } catch (InterruptedException e) {
+                    throw new ConnectException("Unexpectedly interrupted while attempting to delete consumer group offsets for topic partitions " + partitionsToReset
+                            + " for connector" + connName, e);
+                }
+            }
+            return alterOffsetsResult;
+        }
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @return true if the source connector has implemented {@link org.apache.kafka.connect.source.SourceConnector#alterOffsets(Map, Map)}
+     * and it returns true for the provided offsets, false otherwise
+     */
+    private boolean alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                                Map<Map<String, ?>, Map<String, ?>> offsets) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,

Review Comment:
   We could always use the transactional ID for task 0, but that feels a little unclean. I'll ping the vote thread with this and the other updates previously discussed.



##########
checkstyle/import-control.xml:
##########
@@ -640,6 +640,10 @@
       <subpackage name="distributed">
         <allow pkg="javax.ws.rs.core" />
       </subpackage>
+
+      <subpackage name="standalone">
+        <allow pkg="javax.ws.rs.core" />
+      </subpackage>

Review Comment:
   Is this still necessary? Builds fine for me locally without this part.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1261,229 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorConfig the connector's configurations
+     *
+     * @return true if the connector plugin has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * / {@link org.apache.kafka.connect.source.SourceConnector#alterOffsets(Map, Map)} and it returns true for the provided offsets,
+     * false otherwise
+     *
+     */
+    public boolean alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets,
+                                         Map<String, String> connectorConfig) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                return alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                return alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @return true if the sink connector has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * and it returns true for the provided offsets, false otherwise
+     */
+    private boolean alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                              Map<Map<String, ?>, Map<String, ?>> offsets) {
+        return alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, Admin::create);
+    }
+
+    // Visible for testing; allows mocking the admin client for testing

Review Comment:
   I'm a bit flattered that this overloading strategy seems to be coming in handy for more places than just zombie fencing 😄
   
   That said, it's getting a bit verbose with the number of variants we've added to implement it. It'd be more succinct if we added a new `Worker` constructor that allows us to specify an admin factory instead of new methods. Would that be easy to work with for our testing strategy, or is this approach better for that purpose?



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


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

Posted by "gharris1727 (via GitHub)" <gi...@apache.org>.
gharris1727 commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1163322562


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1254,224 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorConfig the connector's configurations
+     *
+     * @return true if the connector plugin has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * / {@link org.apache.kafka.connect.source.SourceConnector#alterOffsets(Map, Map)} and it returns true for the provided offsets,
+     * false otherwise
+     *
+     */
+    public boolean alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets,
+                                         Map<String, String> connectorConfig) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                return alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                return alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @return true if the sink connector has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * and it returns true for the provided offsets, false otherwise
+     */
+    boolean alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets) {
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+        Timer timer = time.timer(ALTER_OFFSETS_TIMEOUT_MS);
+        boolean alterOffsetsResult;
+        try {
+            alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+        } catch (UnsupportedOperationException e) {
+            throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                    "modification of offsets", e);
+        }
+        timer.update();

Review Comment:
   I don't think there's enough call-sites to justify a utility method, but i'll leave that up to you.



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


[GitHub] [kafka] gharris1727 commented on pull request #13465: KAFKA-14368: Connect offset write REST API

Posted by "gharris1727 (via GitHub)" <gi...@apache.org>.
gharris1727 commented on PR #13465:
URL: https://github.com/apache/kafka/pull/13465#issuecomment-1504231021

   So I was manually testing this feature and ran across a serialization problem. Here's the most concise repro case I can think of:
   ```
   $ curl -sSX PATCH -H "Content-Type: application/json" localhost:8083/connectors/test/offsets -d '{
     "offsets": [
       {
         "partition": {
           "float": 1.0
         },
         "offset": {
           "key": "value"
         }
       }
     ]
   }' | jq .
   {
     "message": "The Connect framework managed offsets for this connector have been altered successfully. However, if this connector manages offsets externally, they will need to be manually altered in the system that the connector uses."
   }
   $ curl -X GET localhost:8083/connectors/verifiable-test/offsets | jq .
   {
     "offsets": [
       {
         "partition": {
           "float": 1
         },
         "offset": {
           "key": "value"
         }
       }
     ]
   }
   $ curl -sSX PATCH -H "Content-Type: application/json" localhost:8083/connectors/test/offsets -d '{
     "offsets": [
       {
         "partition": { 
           "float": 1
         },  
         "offset": {
           "key": "value"
         }
       }
     ]
   }' | jq .
   {
     "message": "The Connect framework managed offsets for this connector have been altered successfully. However, if this connector manages offsets externally, they will need to be manually altered in the system that the connector uses."
   }
   $ curl -sSX GET localhost:8083/connectors/test/offsets | jq .
   {
     "offsets": [
       {
         "partition": {
           "float": 1
         },
         "offset": {
           "key": "value"
         }
       },
       {
         "partition": {
           "float": 1
         },
         "offset": {
           "key": "value"
         }
       }
     ]
   }
   ```
   
   The GET portion of the API is mapping decimals `1.0` to the integer-looking `1`, which is distinct when serialized by the JsonConverter in the offsets topic. When you copy-paste the result of GET for a subsequent PATCH, it actually edits a completely different partition, since the equality check (and kafka keying/compaction) is done by the serialized form.
   
   I think the ConnectorOffset serialization needs to be tweaked to force showing the decimals, to be consistent with the JsonConverter.


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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1162903146


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1259,217 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorConfig the connector's configurations
+     *
+     * @return true if the connector plugin has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * / {@link org.apache.kafka.connect.source.SourceConnector#alterOffsets(Map, Map)} and it returns true for the provided offsets,
+     * false otherwise
+     *
+     */
+    public boolean alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets,
+                                         Map<String, String> connectorConfig) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                return alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                return alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @return true if the sink connector has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * and it returns true for the provided offsets, false otherwise
+     */
+    private boolean alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                              Map<Map<String, ?>, Map<String, ?>> offsets) {
+        return alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, Admin::create);
+    }
+
+    // Visible for testing; allows mocking the admin client for testing
+    boolean alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Function<Map<String, Object>, Admin> adminFactory) {
+
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+        Timer timer = time.timer(ALTER_OFFSETS_TIMEOUT_MS);
+        boolean alterOffsetsResult;
+        try {
+            alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+        } catch (UnsupportedOperationException e) {
+            throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                    "modification of offsets", e);
+        }
+        timer.update();
+
+        Class<? extends Connector> sinkConnectorClass = connector.getClass();
+        Map<String, Object> adminConfig = adminConfigs(
+                connName,
+                "connector-worker-adminclient-" + connName,
+                config,
+                new SinkConnectorConfig(plugins, connectorConfig),
+                sinkConnectorClass,
+                connectorClientConfigOverridePolicy,
+                kafkaClusterId,
+                ConnectorType.SINK);
+
+        try (Admin admin = adminFactory.apply(adminConfig)) {
+            SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+            String groupId = (String) baseConsumerConfigs(
+                    connName, "connector-consumer-", config, sinkConnectorConfig,
+                    sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+            Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                    .stream()
+                    .filter(entry -> entry.getValue() != null)
+                    .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                    .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+            if (!offsetsToAlter.isEmpty()) {
+                AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter);
+                try {
+                    log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                            connName, offsetsToAlter);
+                    alterConsumerGroupOffsetsResult.all().get(timer.remainingMs(), TimeUnit.MILLISECONDS);
+                } catch (ExecutionException e) {
+                    // TODO: Handle different exception cause types to surface more fine-grained errors here?
+                    throw new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for connector "
+                            + connName, e.getCause());
+                } catch (TimeoutException e) {
+                    throw new ConnectException("Timed out while attempting to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet()
+                            + " for connector" + connName, e);
+                } catch (InterruptedException e) {
+                    throw new ConnectException("Unexpectedly interrupted while attempting to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet()
+                            + " for connector" + connName, e);
+                }
+                timer.update();
+            }
+
+            Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                    .stream()
+                    .filter(entry -> entry.getValue() == null)
+                    .map(Map.Entry::getKey)
+                    .collect(Collectors.toSet());
+
+            if (!partitionsToReset.isEmpty()) {
+                DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset);
+                try {
+                    log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                            connName, partitionsToReset);
+                    deleteConsumerGroupOffsetsResult.all().get(timer.remainingMs(), TimeUnit.MILLISECONDS);
+                } catch (ExecutionException e) {
+                    // TODO: Handle different exception cause types to surface more fine-grained errors here?
+                    throw new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                            + connName, e.getCause());
+                } catch (TimeoutException e) {
+                    throw new ConnectException("Timed out while attempting to delete consumer group offsets for topic partitions " + partitionsToReset
+                            + " for connector" + connName, e);
+                } catch (InterruptedException e) {
+                    throw new ConnectException("Unexpectedly interrupted while attempting to delete consumer group offsets for topic partitions " + partitionsToReset
+                            + " for connector" + connName, e);
+                }
+            }
+            return alterOffsetsResult;
+        }
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @return true if the source connector has implemented {@link org.apache.kafka.connect.source.SourceConnector#alterOffsets(Map, Map)}
+     * and it returns true for the provided offsets, false otherwise
+     */
+    private boolean alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                                Map<Map<String, ?>, Map<String, ?>> offsets) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,

Review Comment:
   We could always use the transactional ID for task 0, but that feels a little unclean. I'll ping the vote thread with this idea (using a transactional ID that corresponds to the connector name instead of the one for task 0) and the other updates previously discussed.



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


[GitHub] [kafka] gharris1727 commented on pull request #13465: KAFKA-14368: Connect offset write REST API

Posted by "gharris1727 (via GitHub)" <gi...@apache.org>.
gharris1727 commented on PR #13465:
URL: https://github.com/apache/kafka/pull/13465#issuecomment-1504680664

   > jq is automatically converting floats like x.0 into x
   
   Thanks for catching that, you're completely correct. I should have cut jq out of my tests to verify that but I hadn't even considered that jq would change that. 
   
   It sounds like there's nothing to be done on the framework side then. Hopefully offsets containing floating point numbers is rare enough that not too many people end up finding the same footgun.


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


[GitHub] [kafka] C0urante merged pull request #13465: KAFKA-14368: Connect offset write REST API

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante merged PR #13465:
URL: https://github.com/apache/kafka/pull/13465


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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on PR #13465:
URL: https://github.com/apache/kafka/pull/13465#issuecomment-1487310988

   Hi Yash! Thanks for the draft, I'll try to get to it sometime this week but that may not happen.
   
   With regards to your points:
   
   > We don't currently use a transactional producer for writing offsets to the worker's global offset backing store. Wouldn't doing so be a breaking change due to the requirement of additional ACLs for the worker's producer principal?
   
   Since we're introducing a new opt-in API, it's not a breaking change. That said, it's probably worth calling out in the KIP and on the discussion thread. And now that I think about it, maybe a transaction on the global offsets topic isn't really necessary if we've already done a round of zombie fencing. Thoughts? We should notify the discussion thread for the KIP no matter what, just wanted to bounce the idea off you first.
   
   > If the connector's configured offsets topic is a different one or if exactly-once source support is not enabled, I don't think we'll be able to write all the provided offsets in a single transaction for the worker's global offset backing store right?
   
   The use of transactions is only necessary if exactly-once source support is enabled for source connectors (both paragraphs that mention the use of transactions begin with "If exactly-once source support is enabled").
   
   > Also one minor comment on the KIP section - the "tombstone records" bit that was copied from the resetting offsets section needs to be updated.
   
   🤦  (done)


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


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

Posted by "yashmayya (via GitHub)" <gi...@apache.org>.
yashmayya commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1200530069


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java:
##########
@@ -1528,6 +1539,80 @@ public void connectorOffsets(String connName, Callback<ConnectorOffsets> cb) {
         );
     }
 
+    @Override
+    public void alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> callback) {
+        log.trace("Submitting alter offsets request for connector '{}'", connName);
+
+        addRequest(() -> {
+            refreshConfigSnapshot(workerSyncTimeoutMs);
+            if (!alterConnectorOffsetsChecks(connName, callback)) {
+                return null;
+            }
+            // At this point, we should be the leader (the call to alterConnectorOffsetsChecks makes sure of that) and can safely run
+            // a zombie fencing request
+            if (isSourceConnector(connName) && config.exactlyOnceSourceEnabled()) {
+                log.debug("Performing a round of zombie fencing before altering offsets for source connector {} with exactly-once semantics enabled.", connName);
+                getFenceZombieSourceTasksCallable(connName, (error, ignored) -> {
+                    if (error != null) {
+                        log.error("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets", error);
+                        callback.onCompletion(new ConnectException("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets",
+                                error), null);
+                    } else {
+                        log.debug("Successfully completed zombie fencing for source connector {}; proceeding to alter offsets.", connName);
+                        // We need to ensure that we perform the necessary checks again inside alterConnectorOffsetsHerderRequest
+                        // since it is being run in a separate herder request and the conditions could have changed since the
+                        // previous check
+                        addRequest(getAlterConnectorOffsetsCallable(connName, offsets, callback), forwardErrorCallback(callback));
+                    }
+                }).call();
+            } else {
+                getAlterConnectorOffsetsCallable(connName, offsets, callback).call();

Review Comment:
   I think one of the main motivations here (and above) was naming 😅 
   
   We already have `fenceZombieSourceTasks` and `alterConnectorOffsets` interface methods but there was a need to break out some of the logic from both into separate methods for re-use. I'm happy to take any naming suggestions to make them synchronous and wrap them into callables at the relevant call sites.
   
   Maybe something like `fenceZombieSourceTasksSync` / `alterConnectorOffsetsSync` or `fenceZombieSourceTasksInternal` / `alterConnectorOffsetsInternal`? I don't particularly like either though, naming is hard...



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java:
##########
@@ -1528,6 +1539,80 @@ public void connectorOffsets(String connName, Callback<ConnectorOffsets> cb) {
         );
     }
 
+    @Override
+    public void alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> callback) {
+        log.trace("Submitting alter offsets request for connector '{}'", connName);
+
+        addRequest(() -> {
+            refreshConfigSnapshot(workerSyncTimeoutMs);

Review Comment:
   Thanks, both points make sense, done. Btw shouldn't we also handle the read timeout case in the [stopConnector](https://github.com/apache/kafka/blob/e96a463561ca8974fca37562b8675ae8ae4aff29/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java#L1107) method? The `KafkaConfigBackingStore::putTaskConfigs` method does include a read to the end of the config topic at the end but the subsequently called `KafkaConfigBackingStore::putTargetState` method doesn't.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));

Review Comment:
   Wow, thanks 🤦 



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    private void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                             Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId)
+                : baseProducerConfigs(connName, "connector-offset-producer-" + connName, config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId);
+        KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+        ConnectorOffsetBackingStore offsetStore = config.exactlyOnceSourceEnabled()
+                ? offsetStoreForExactlyOnceSourceConnector(sourceConfig, connName, connector, producer)
+                : offsetStoreForRegularSourceConnector(sourceConfig, connName, connector, producer);
+
+        OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, connName, internalKeyConverter, internalValueConverter);
+        alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, offsetStore, producer, offsetWriter, connectorLoader, cb);
+    }
+
+    // Visible for testing
+    void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                     Map<Map<String, ?>, Map<String, ?>> offsets, ConnectorOffsetBackingStore offsetStore,
+                                     KafkaProducer<byte[], byte[]> producer, OffsetStorageWriter offsetWriter,
+                                     ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SourceConnector) connector).alterOffsets(connectorConfig, offsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                offsetStore.configure(config);
+                // This reads to the end of the offsets topic and can be a potentially time-consuming operation
+                offsetStore.start();
+
+                // The alterSourceConnectorOffsets method should only be called after all the connector's tasks have been stopped, and it's
+                // safe to write offsets via an offset writer
+                offsets.forEach(offsetWriter::offset);
+
+                // We can call begin flush without a timeout because this newly created single-purpose offset writer can't do concurrent
+                // offset writes. We can also ignore the return value since it returns false if and only if there is no data to be flushed,
+                // and we've just put some data in the previous statement
+                offsetWriter.beginFlush();
+
+                try {
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.initTransactions();
+                        producer.beginTransaction();
+                    }
+                    log.debug("Committing the following partition offsets for source connector {}: {}", connName, offsets);
+                    FutureCallback<Void> offsetWriterCallback = new FutureCallback<>();
+                    offsetWriter.doFlush(offsetWriterCallback);
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.commitTransaction();
+                    }
+                    offsetWriterCallback.get(ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS, TimeUnit.MILLISECONDS);
+                } catch (ExecutionException e) {
+                    throw new ConnectException("Failed to alter offsets for source connector " + connName, e.getCause());

Review Comment:
   We could, but I'm wondering whether we should (for sink connectors as well) - these exceptions will surface synchronously as REST API responses and I don't think the partitions add much value since the API requests themselves will contain the partitions. I've updated the sink connector error messages.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/util/SinkUtils.java:
##########
@@ -52,4 +53,82 @@ public static ConnectorOffsets consumerGroupOffsetsToConnectorOffsets(Map<TopicP
 
         return new ConnectorOffsets(connectorOffsets);
     }
+
+    /**
+     * Validate that the provided partitions (keys in the {@code partitionOffsets} map) look like:
+     * <pre>
+     *     {
+     *       "kafka_topic": "topic"
+     *       "kafka_partition": 3
+     *     }
+     * </pre>
+     *
+     * and that the provided offsets (values in the {@code partitionOffsets} map) look like:
+     * <pre>
+     *     {
+     *       "kafka_offset": 1000
+     *     }
+     * </pre>
+     *
+     * This method then parses them into a mapping from {@link TopicPartition}s to their corresponding {@link Long}
+     * valued offsets.
+     *
+     * @param partitionOffsets the partitions to offset map that needs to be validated and parsed.
+     * @return the parsed mapping from {@link TopicPartition} to its corresponding {@link Long} valued offset.
+     *
+     * @throws BadRequestException if the provided offsets aren't in the expected format
+     */
+    public static Map<TopicPartition, Long> validateAndParseSinkConnectorOffsets(Map<Map<String, ?>, Map<String, ?>> partitionOffsets) {

Review Comment:
   Thanks, that makes sense, I've renamed the method. I've kept the expected formats for the partitions and offsets in the Javadoc though, as I feel like they'll be handy to reference at the parsing points.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    private void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                             Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId)
+                : baseProducerConfigs(connName, "connector-offset-producer-" + connName, config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId);
+        KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+        ConnectorOffsetBackingStore offsetStore = config.exactlyOnceSourceEnabled()
+                ? offsetStoreForExactlyOnceSourceConnector(sourceConfig, connName, connector, producer)
+                : offsetStoreForRegularSourceConnector(sourceConfig, connName, connector, producer);
+
+        OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, connName, internalKeyConverter, internalValueConverter);
+        alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, offsetStore, producer, offsetWriter, connectorLoader, cb);
+    }
+
+    // Visible for testing
+    void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                     Map<Map<String, ?>, Map<String, ?>> offsets, ConnectorOffsetBackingStore offsetStore,
+                                     KafkaProducer<byte[], byte[]> producer, OffsetStorageWriter offsetWriter,
+                                     ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SourceConnector) connector).alterOffsets(connectorConfig, offsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                offsetStore.configure(config);
+                // This reads to the end of the offsets topic and can be a potentially time-consuming operation
+                offsetStore.start();

Review Comment:
   Ah, good catch, thanks. 



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    private void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                             Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId)
+                : baseProducerConfigs(connName, "connector-offset-producer-" + connName, config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId);
+        KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+        ConnectorOffsetBackingStore offsetStore = config.exactlyOnceSourceEnabled()
+                ? offsetStoreForExactlyOnceSourceConnector(sourceConfig, connName, connector, producer)
+                : offsetStoreForRegularSourceConnector(sourceConfig, connName, connector, producer);
+
+        OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, connName, internalKeyConverter, internalValueConverter);
+        alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, offsetStore, producer, offsetWriter, connectorLoader, cb);
+    }
+
+    // Visible for testing
+    void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                     Map<Map<String, ?>, Map<String, ?>> offsets, ConnectorOffsetBackingStore offsetStore,
+                                     KafkaProducer<byte[], byte[]> producer, OffsetStorageWriter offsetWriter,
+                                     ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SourceConnector) connector).alterOffsets(connectorConfig, offsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                offsetStore.configure(config);
+                // This reads to the end of the offsets topic and can be a potentially time-consuming operation
+                offsetStore.start();
+
+                // The alterSourceConnectorOffsets method should only be called after all the connector's tasks have been stopped, and it's
+                // safe to write offsets via an offset writer
+                offsets.forEach(offsetWriter::offset);
+
+                // We can call begin flush without a timeout because this newly created single-purpose offset writer can't do concurrent
+                // offset writes. We can also ignore the return value since it returns false if and only if there is no data to be flushed,
+                // and we've just put some data in the previous statement
+                offsetWriter.beginFlush();
+
+                try {
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.initTransactions();
+                        producer.beginTransaction();
+                    }
+                    log.debug("Committing the following partition offsets for source connector {}: {}", connName, offsets);
+                    FutureCallback<Void> offsetWriterCallback = new FutureCallback<>();
+                    offsetWriter.doFlush(offsetWriterCallback);
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.commitTransaction();
+                    }
+                    offsetWriterCallback.get(ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS, TimeUnit.MILLISECONDS);

Review Comment:
   Hm this implementation was intentional; I thought it'd make sense to follow the same pattern as regular offset commits for source connectors (both regular and EoS). Any particular reason we'd want stricter guarantees (i.e. ensuring that the write to the global offsets topic is successful) for offset commits initiated via the REST API versus the connector tasks themselves?



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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1202644196


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java:
##########
@@ -1528,6 +1539,80 @@ public void connectorOffsets(String connName, Callback<ConnectorOffsets> cb) {
         );
     }
 
+    @Override
+    public void alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> callback) {
+        log.trace("Submitting alter offsets request for connector '{}'", connName);
+
+        addRequest(() -> {
+            refreshConfigSnapshot(workerSyncTimeoutMs);

Review Comment:
   Good point--yes, we should handle that in `stopConnector` as well (although we don't have to do that in this PR).



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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1205535710


##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java:
##########
@@ -4123,6 +4136,280 @@ public void testConnectorOffsets() throws Exception {
         PowerMock.verifyAll();
     }
 
+    @Test
+    public void testAlterOffsetsConnectorNotFound() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets("connector-does-not-exist", new HashMap<>(), callback);
+        herder.tick();
+        ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertTrue(e.getCause() instanceof NotFoundException);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsConnectorNotInStoppedState() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, new HashMap<>(), callback);
+        herder.tick();
+        ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertTrue(e.getCause() instanceof BadRequestException);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsNotLeader() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("member");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), false);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, new HashMap<>(), callback);
+        herder.tick();
+        ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertTrue(e.getCause() instanceof NotLeaderException);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsSourceConnectorExactlyOnceDisabled() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        Map<Map<String, ?>, Map<String, ?>> offsets = new HashMap<>();
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        Capture<Callback<Message>> workerCallbackCapture = Capture.newInstance();
+        worker.alterConnectorOffsets(EasyMock.eq(CONN1), EasyMock.eq(CONN1_CONFIG), EasyMock.eq(offsets), capture(workerCallbackCapture));
+        Message msg = new Message("The offsets for this connector have been altered successfully");
+        EasyMock.expectLastCall().andAnswer(() -> {
+            workerCallbackCapture.getValue().onCompletion(null, msg);
+            return null;
+        });
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, offsets, callback);
+        herder.tick();
+        assertEquals(msg, callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertEquals("The offsets for this connector have been altered successfully", msg.message());
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsSourceConnectorExactlyOnceEnabled() throws Exception {
+        // Setup herder with exactly-once support for source connectors enabled
+        herder = exactlyOnceHerder();
+        rebalanceListener = herder.new RebalanceListener(time);
+        PowerMock.expectPrivate(herder, "updateDeletedConnectorStatus").andVoid().anyTimes();
+        PowerMock.expectPrivate(herder, "updateDeletedTaskStatus").andVoid().anyTimes();
+
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall().anyTimes();
+
+        // Now handle the alter connector offsets request
+        Map<Map<String, ?>, Map<String, ?>> offsets = new HashMap<>();
+        member.wakeup();
+        PowerMock.expectLastCall().anyTimes();
+        member.ensureActive();
+        PowerMock.expectLastCall().anyTimes();
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        EasyMock.expect(herder.connectorType(EasyMock.anyObject())).andReturn(ConnectorType.SOURCE).anyTimes();
+
+        // Expect a round of zombie fencing to occur
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        KafkaFuture<Void> workerFencingFuture = EasyMock.mock(KafkaFuture.class);
+        KafkaFuture<Void> herderFencingFuture = EasyMock.mock(KafkaFuture.class);
+        EasyMock.expect(worker.fenceZombies(CONN1, SNAPSHOT_STOPPED_CONN1.taskCountRecord(CONN1), CONN1_CONFIG)).andReturn(workerFencingFuture);
+        EasyMock.expect(workerFencingFuture.thenApply(EasyMock.<KafkaFuture.BaseFunction<Void, Void>>anyObject())).andReturn(herderFencingFuture);
+
+        // Two fencing callbacks are added - one is in ZombieFencing::start itself to remove the connector from the active
+        // fencing list. The other is the callback passed from DistributedHerder::alterConnectorOffsets in order to
+        // queue up the actual alter offsets request if the zombie fencing succeeds.
+        for (int i = 0; i < 2; i++) {
+            Capture<KafkaFuture.BiConsumer<Void, Throwable>> herderFencingCallback = EasyMock.newCapture();
+            EasyMock.expect(herderFencingFuture.whenComplete(EasyMock.capture(herderFencingCallback))).andAnswer(() -> {
+                herderFencingCallback.getValue().accept(null, null);
+                return null;
+            });
+        }
+
+        Capture<Callback<Message>> workerCallbackCapture = Capture.newInstance();
+        Message msg = new Message("The offsets for this connector have been altered successfully");
+        worker.alterConnectorOffsets(EasyMock.eq(CONN1), EasyMock.eq(CONN1_CONFIG), EasyMock.eq(offsets), capture(workerCallbackCapture));
+        EasyMock.expectLastCall().andAnswer(() -> {
+            workerCallbackCapture.getValue().onCompletion(null, msg);
+            return null;
+        });
+
+        // Handle the second alter connector offsets request. No zombie fencing request to the worker is expected now since we
+        // already did a round of zombie fencing last time and no new tasks came up in the meanwhile. The config snapshot is
+        // refreshed once at the beginning of the DistributedHerder::alterConnectorOffsets method, once before checking
+        // whether zombie fencing is required, and once before actually proceeding to alter connector offsets.
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1_FENCED);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1_FENCED);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1_FENCED);
+        Capture<Callback<Message>> workerCallbackCapture2 = Capture.newInstance();
+        worker.alterConnectorOffsets(EasyMock.eq(CONN1), EasyMock.eq(CONN1_CONFIG), EasyMock.eq(offsets), capture(workerCallbackCapture2));
+        EasyMock.expectLastCall().andAnswer(() -> {
+            workerCallbackCapture2.getValue().onCompletion(null, msg);
+            return null;
+        });
+
+        PowerMock.replayAll(workerFencingFuture, herderFencingFuture);
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, offsets, callback);
+        // Process the zombie fencing request
+        herder.tick();
+        // Process the alter offsets request
+        herder.tick();
+        assertEquals(msg, callback.get(1000L, TimeUnit.MILLISECONDS));
+
+        FutureCallback<Message> callback2 = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, offsets, callback2);
+        herder.tick();
+        assertEquals(msg, callback.get(1000L, TimeUnit.MILLISECONDS));
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsSourceConnectorExactlyOnceEnabledZombieFencingFailure() throws Exception {

Review Comment:
   Thanks, LGTM



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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1205533269


##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java:
##########
@@ -4123,6 +4136,280 @@ public void testConnectorOffsets() throws Exception {
         PowerMock.verifyAll();
     }
 
+    @Test
+    public void testAlterOffsetsConnectorNotFound() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets("connector-does-not-exist", new HashMap<>(), callback);
+        herder.tick();
+        ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertTrue(e.getCause() instanceof NotFoundException);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsConnectorNotInStoppedState() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, new HashMap<>(), callback);
+        herder.tick();
+        ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertTrue(e.getCause() instanceof BadRequestException);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsNotLeader() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("member");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), false);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();

Review Comment:
   Huh, TIL!



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


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

Posted by "yashmayya (via GitHub)" <gi...@apache.org>.
yashmayya commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1167415541


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {

Review Comment:
   While this whole portion does seem a bit convoluted, it offers the advantage of skipping the attempt to delete partition offsets when the alter offsets attempt fails. The alternative could be to simply do both the operations concurrently (`alterConsumerGroupOffsets` and `deleteConsumerGroupOffsets`), and then react to the combined response (combining the returned futures using `KafkaFuture::allOf`)
   
   Edit: Another option could be to combine the two admin client requests into a single `alterConsumerGroupOffsets` request, using an offset value of 0 for topic partitions whose offsets are to be reset (indicated by a `null` value in the request body). While this would greatly simplify things here, it wouldn't respect the consumer configuration `auto.offset.reset` which is why I don't think it's the right thing to do.



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


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

Posted by "yashmayya (via GitHub)" <gi...@apache.org>.
yashmayya commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1165441821


##########
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:
   Hm but for non-exactly-once source connectors (which is the default mode), this would leave the door open to confusing behavior where users could get successful responses from the alter offsets API but the connector could completely ignore the overwritten offsets (if the user resumes the connector in the interim). I agree that the zombie task case is unhandled for non EoS source connectors, but at least that would only typically occur for misbehaving connector plugins whereas making the alter offsets API async would allow users to shoot themselves in the foot. I don't disagree that the 5 second timeout is quite non-ideal and even more concerning is the fact that if a connector's `alterOffsets` method hangs, it can disable a worker (something that was a big problem with other connector methods until your elegant fix in https://github.com/apache/kafka/pull/8069). I'm just trying to weigh the pros and cons here but it does seem like doing alter offset operations asynchronously in the
  worker has more benefits than drawbacks.



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


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

Posted by "yashmayya (via GitHub)" <gi...@apache.org>.
yashmayya commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1163144352


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1261,229 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorConfig the connector's configurations
+     *
+     * @return true if the connector plugin has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * / {@link org.apache.kafka.connect.source.SourceConnector#alterOffsets(Map, Map)} and it returns true for the provided offsets,
+     * false otherwise
+     *
+     */
+    public boolean alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets,
+                                         Map<String, String> connectorConfig) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                return alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                return alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @return true if the sink connector has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * and it returns true for the provided offsets, false otherwise
+     */
+    private boolean alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                              Map<Map<String, ?>, Map<String, ?>> offsets) {
+        return alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, Admin::create);
+    }
+
+    // Visible for testing; allows mocking the admin client for testing

Review Comment:
   Ha, yeah, I really liked the pattern! 
   
   > It'd be more succinct if we added a new Worker constructor that allows us to specify an admin factory instead of new methods.
   
   Thanks, great suggestion, done. The verbosity and the number of superfluous methods was bothering me too, this seems much better now.



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


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

Posted by "gharris1727 (via GitHub)" <gi...@apache.org>.
gharris1727 commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1163174096


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1254,224 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorConfig the connector's configurations
+     *
+     * @return true if the connector plugin has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * / {@link org.apache.kafka.connect.source.SourceConnector#alterOffsets(Map, Map)} and it returns true for the provided offsets,
+     * false otherwise
+     *
+     */
+    public boolean alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets,

Review Comment:
   nit: swap order of offsets and config to be consistent with the plugin api



##########
connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java:
##########
@@ -19,24 +19,25 @@
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.connect.errors.ConnectException;
 import org.apache.kafka.connect.runtime.isolation.Plugins;
 import org.apache.kafka.connect.runtime.rest.entities.ActiveTopicsInfo;
 import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos;
-import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets;
 import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo;
+import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets;
 import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo;
 import org.apache.kafka.connect.runtime.rest.entities.ServerInfo;
 import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException;
+import org.eclipse.jetty.client.HttpClient;

Review Comment:
   Using raw java io is certainly clunky, but was there some reason that you couldn't make it work like the existing calls?
   
   I think since this is a class used outside of this repo, we should be careful about adding new dependencies.



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1254,224 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorConfig the connector's configurations
+     *
+     * @return true if the connector plugin has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * / {@link org.apache.kafka.connect.source.SourceConnector#alterOffsets(Map, Map)} and it returns true for the provided offsets,
+     * false otherwise
+     *
+     */
+    public boolean alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets,
+                                         Map<String, String> connectorConfig) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                return alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                return alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @return true if the sink connector has implemented {@link org.apache.kafka.connect.sink.SinkConnector#alterOffsets(Map, Map)}
+     * and it returns true for the provided offsets, false otherwise
+     */
+    boolean alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets) {
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+        Timer timer = time.timer(ALTER_OFFSETS_TIMEOUT_MS);
+        boolean alterOffsetsResult;
+        try {
+            alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+        } catch (UnsupportedOperationException e) {
+            throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                    "modification of offsets", e);
+        }
+        timer.update();

Review Comment:
   Should alterOffsets exceeding the timer cause us to bail out here, or should the timeout only apply to the adminclient operation?



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


[GitHub] [kafka] yashmayya commented on pull request #13465: KAFKA-14368: Connect offset write REST API

Posted by "yashmayya (via GitHub)" <gi...@apache.org>.
yashmayya commented on PR #13465:
URL: https://github.com/apache/kafka/pull/13465#issuecomment-1504645043

   I tried really hard to reproduce this via integration tests and I wasn't able to, so I tried doing a repro manually just as you outlined above. Turns out that this isn't actually a bug in either the GET or PATCH offsets API - `jq` is automatically converting floats like `x.0` into `x` (you can try the exact same repro above without using `jq` to confirm).


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


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

Posted by "yashmayya (via GitHub)" <gi...@apache.org>.
yashmayya commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1167415541


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {

Review Comment:
   While this whole portion does seem a bit convoluted, it offers the advantage of skipping the attempt to delete partition offsets when the alter offsets attempt fails. The alternative could be to simply do both the operations concurrently (`alterConsumerGroupOffsets` and `deleteConsumerGroupOffsets`), and then react to the combined response (combining the returned futures using `KafkaFuture::allOf`)



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


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

Posted by "yashmayya (via GitHub)" <gi...@apache.org>.
yashmayya commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1167384274


##########
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:
   Thanks for the detailed response and it sounds like we're on the same page now. I've refactored the alter offsets worker API to be asynchronous.
   
   > possible additional guard to prevent cancelled source tasks from committing offsets
   
   I guess we don't really need to worry too much about cancelled source tasks since during regular task stop, we also remove the periodic offset commit task in the `SourceTaskOffsetCommitter`?
   
   > one way we could try to decrease the risks of an asynchronous API for non-exactly-once source connectors could be to refuse to assign tasks for connectors with ongoing offset alterations during rebalance, even if the connector is resumed
   
   That's an interesting idea but it does seem to be a pretty invasive change w.r.t the current rebalancing logic which is agnostic to all on-going operations in the workers. The limitation is also a valid one and yeah the same risks apply even with the sync API although I'm not sure I follow why you think it's less likely? Isn't it more likely that a synchronous alter offsets request hangs and causes the leader to fall out of the group leading to a new leader being elected?
   
   > We could also try to add interruption logic that cancels any in-progress offset alter/reset requests when a rebalance starts
   
   We would need to be careful about the exact points where we allow interruptions. For instance, we wouldn't want to abandon a request midway through writing offsets (in the non-EoS source connector case where it isn't an atomic operation, or for consumer groups when we're altering offsets for some partitions + resetting offsets for some others). Although, this does seem like a more appealing option overall and I've filed this Jira as a potential follow up item - https://issues.apache.org/jira/browse/KAFKA-14910



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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1202738629


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    private void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                             Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId)
+                : baseProducerConfigs(connName, "connector-offset-producer-" + connName, config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId);
+        KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+        ConnectorOffsetBackingStore offsetStore = config.exactlyOnceSourceEnabled()
+                ? offsetStoreForExactlyOnceSourceConnector(sourceConfig, connName, connector, producer)
+                : offsetStoreForRegularSourceConnector(sourceConfig, connName, connector, producer);
+
+        OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, connName, internalKeyConverter, internalValueConverter);
+        alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, offsetStore, producer, offsetWriter, connectorLoader, cb);
+    }
+
+    // Visible for testing
+    void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                     Map<Map<String, ?>, Map<String, ?>> offsets, ConnectorOffsetBackingStore offsetStore,
+                                     KafkaProducer<byte[], byte[]> producer, OffsetStorageWriter offsetWriter,
+                                     ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SourceConnector) connector).alterOffsets(connectorConfig, offsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                offsetStore.configure(config);
+                // This reads to the end of the offsets topic and can be a potentially time-consuming operation
+                offsetStore.start();
+
+                // The alterSourceConnectorOffsets method should only be called after all the connector's tasks have been stopped, and it's
+                // safe to write offsets via an offset writer
+                offsets.forEach(offsetWriter::offset);
+
+                // We can call begin flush without a timeout because this newly created single-purpose offset writer can't do concurrent
+                // offset writes. We can also ignore the return value since it returns false if and only if there is no data to be flushed,
+                // and we've just put some data in the previous statement
+                offsetWriter.beginFlush();
+
+                try {
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.initTransactions();
+                        producer.beginTransaction();
+                    }
+                    log.debug("Committing the following partition offsets for source connector {}: {}", connName, offsets);
+                    FutureCallback<Void> offsetWriterCallback = new FutureCallback<>();
+                    offsetWriter.doFlush(offsetWriterCallback);
+                    if (config.exactlyOnceSourceEnabled()) {
+                        producer.commitTransaction();
+                    }
+                    offsetWriterCallback.get(ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS, TimeUnit.MILLISECONDS);

Review Comment:
   The issue here is with offsets that are reset. If we emit a tombstone to the per-connector offsets topic but fail to emit one for the global offsets topic, then that offset will still be visible to tasks.
   
   Come to think of it, this is an issue with how we write task-produced offsets as well. I've documented this in KAFKA-15018.
   
   I'd love it if you could leave your thoughts on that ticket, but for now, I suppose we could leave this as-is since it's not really a new bug but just a new opportunity for an existing bug to arise. Although, it does become more likely for that bug to be hit (since it's usual for source connectors to frequently emit tombstone source offsets, but it's actually an intended use case for the new API we're introducing with this KIP).



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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1204681794


##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java:
##########
@@ -4123,6 +4136,280 @@ public void testConnectorOffsets() throws Exception {
         PowerMock.verifyAll();
     }
 
+    @Test
+    public void testAlterOffsetsConnectorNotFound() throws Exception {

Review Comment:
   Nit: it's a little strange that we're using "not found" in the name here but elsewhere in this and the `StandaloneHerderTest` suite we use "unknown"



##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorOffsetsTest.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.rest.entities;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class ConnectorOffsetsTest {
+
+    @Test
+    public void testConnectorOffsetsToMap() {
+        // Using arbitrary partition and offset formats here to demonstrate that source connector offsets don't
+        // follow a standard pattern
+        Map<String, Object> partition1 = new HashMap<>();
+        partition1.put("partitionKey1", "partitionValue");
+        partition1.put("k", 123);
+        Map<String, Object> offset1 = new HashMap<>();
+        offset1.put("offset", 3.14);

Review Comment:
   Yum!



##########
connect/runtime/src/test/java/org/apache/kafka/connect/util/SinkUtilsTest.java:
##########
@@ -46,4 +52,124 @@ public void testConsumerGroupOffsetsToConnectorOffsets() {
         expectedPartition.put(SinkUtils.KAFKA_PARTITION_KEY, 0);
         assertEquals(expectedPartition, connectorOffsets.offsets().get(0).partition());
     }
+
+    @Test
+    public void testValidateAndParseEmptyPartitionOffsetMap() {
+        // expect no exception to be thrown
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.parseSinkConnectorOffsets(new HashMap<>());
+        assertTrue(parsedOffsets.isEmpty());
+    }
+
+    @Test
+    public void testValidateAndParseInvalidPartition() {
+        Map<String, Object> partition = new HashMap<>();
+        partition.put(SinkUtils.KAFKA_TOPIC_KEY, "topic");
+        Map<String, Object> offset = new HashMap<>();
+        offset.put(SinkUtils.KAFKA_OFFSET_KEY, 100);
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        partitionOffsets.put(partition, offset);
+
+        // missing partition key
+        ConnectException e = assertThrows(ConnectException.class, () -> SinkUtils.parseSinkConnectorOffsets(partitionOffsets));
+        assertThat(e.getMessage(), containsString("The partition for a sink connector offset must contain the keys 'kafka_topic' and 'kafka_partition'"));
+
+        partition.put(SinkUtils.KAFKA_PARTITION_KEY, "not a number");
+        // bad partition key
+        e = assertThrows(ConnectException.class, () -> SinkUtils.parseSinkConnectorOffsets(partitionOffsets));
+        assertThat(e.getMessage(), containsString("Failed to parse the following Kafka partition value in the provided offsets: 'not a number'"));
+
+        partition.remove(SinkUtils.KAFKA_TOPIC_KEY);
+        partition.put(SinkUtils.KAFKA_PARTITION_KEY, "5");
+        // missing topic key
+        e = assertThrows(ConnectException.class, () -> SinkUtils.parseSinkConnectorOffsets(partitionOffsets));
+        assertThat(e.getMessage(), containsString("The partition for a sink connector offset must contain the keys 'kafka_topic' and 'kafka_partition'"));
+    }
+
+    @Test
+    public void testValidateAndParseInvalidOffset() {
+        Map<String, Object> partition = new HashMap<>();
+        partition.put(SinkUtils.KAFKA_TOPIC_KEY, "topic");
+        partition.put(SinkUtils.KAFKA_PARTITION_KEY, 10);
+        Map<String, Object> offset = new HashMap<>();
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        partitionOffsets.put(partition, offset);
+
+        // missing offset key
+        ConnectException e = assertThrows(ConnectException.class, () -> SinkUtils.parseSinkConnectorOffsets(partitionOffsets));
+        assertThat(e.getMessage(), containsString("The offset for a sink connector should either be null or contain the key 'kafka_offset'"));
+
+        // bad offset key
+        offset.put(SinkUtils.KAFKA_OFFSET_KEY, "not a number");
+        e = assertThrows(ConnectException.class, () -> SinkUtils.parseSinkConnectorOffsets(partitionOffsets));
+        assertThat(e.getMessage(), containsString("Failed to parse the following Kafka offset value in the provided offsets: 'not a number'"));
+    }
+
+    @Test
+    public void testValidateAndParseStringPartitionValue() {
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = createPartitionOffsetMap("topic", "10", "100");
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.parseSinkConnectorOffsets(partitionOffsets);
+        assertEquals(1, parsedOffsets.size());
+        TopicPartition tp = parsedOffsets.keySet().iterator().next();
+        assertEquals(10, tp.partition());
+    }
+
+    @Test
+    public void testValidateAndParseIntegerPartitionValue() {
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = createPartitionOffsetMap("topic", 10, "100");
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.parseSinkConnectorOffsets(partitionOffsets);
+        assertEquals(1, parsedOffsets.size());
+        TopicPartition tp = parsedOffsets.keySet().iterator().next();
+        assertEquals(10, tp.partition());
+    }
+
+    @Test
+    public void testValidateAndParseStringOffsetValue() {
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = createPartitionOffsetMap("topic", "10", "100");
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.parseSinkConnectorOffsets(partitionOffsets);
+        assertEquals(1, parsedOffsets.size());
+        Long offsetValue = parsedOffsets.values().iterator().next();
+        assertEquals(100L, offsetValue.longValue());
+    }
+
+    @Test
+    public void testValidateAndParseIntegerOffsetValue() {
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = createPartitionOffsetMap("topic", "10", 100);
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.parseSinkConnectorOffsets(partitionOffsets);
+        assertEquals(1, parsedOffsets.size());
+        Long offsetValue = parsedOffsets.values().iterator().next();
+        assertEquals(100L, offsetValue.longValue());
+    }
+
+    @Test
+    public void testNullOffset() {
+        Map<String, Object> partitionMap = new HashMap<>();
+        partitionMap.put(SinkUtils.KAFKA_TOPIC_KEY, "topic");
+        partitionMap.put(SinkUtils.KAFKA_PARTITION_KEY, 10);
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        partitionOffsets.put(partitionMap, null);
+        Map<TopicPartition, Long> parsedOffsets = SinkUtils.parseSinkConnectorOffsets(partitionOffsets);
+        assertEquals(1, parsedOffsets.size());
+        assertNull(parsedOffsets.values().iterator().next());
+    }
+
+    @Test
+    public void testNullPartition() {

Review Comment:
   These tests are fantastic for the most part. For the sake of completeness, should we include a case with a null topic name as well?



##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java:
##########
@@ -1889,6 +1902,352 @@ public void testGetSourceConnectorOffsetsError() {
         verifyKafkaClusterId();
     }
 
+    @Test
+    public void testAlterOffsetsConnectorDoesNotSupportOffsetAlteration() {
+        mockKafkaClusterId();
+
+        worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, Executors.newSingleThreadExecutor(),
+                allConnectorClientConfigOverridePolicy, null);
+        worker.start();
+
+        mockGenericIsolation();
+        when(plugins.newConnector(anyString())).thenReturn(sourceConnector);
+        when(plugins.withClassLoader(any(ClassLoader.class), any(Runnable.class))).thenAnswer(AdditionalAnswers.returnsSecondArg());
+        when(sourceConnector.alterOffsets(eq(connectorProps), anyMap())).thenThrow(new UnsupportedOperationException("This connector doesn't " +
+                "support altering of offsets"));
+
+        FutureCallback<Message> cb = new FutureCallback<>();
+        worker.alterConnectorOffsets(CONNECTOR_ID, connectorProps,
+                Collections.singletonMap(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue")),
+                cb);
+
+        ExecutionException e = assertThrows(ExecutionException.class, () -> cb.get(1000, TimeUnit.MILLISECONDS));
+        assertEquals(ConnectException.class, e.getCause().getClass());
+        assertEquals("Failed to alter offsets for connector " + CONNECTOR_ID + " because it doesn't support external modification of offsets",
+                e.getCause().getMessage());
+
+        verifyGenericIsolation();
+        verifyKafkaClusterId();
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testAlterOffsetsSourceConnector() throws Exception {
+        mockKafkaClusterId();
+        worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, Executors.newSingleThreadExecutor(),
+                allConnectorClientConfigOverridePolicy, null);
+        worker.start();
+
+        when(plugins.withClassLoader(any(ClassLoader.class), any(Runnable.class))).thenAnswer(AdditionalAnswers.returnsSecondArg());
+        when(sourceConnector.alterOffsets(eq(connectorProps), anyMap())).thenReturn(true);
+        ConnectorOffsetBackingStore offsetStore = mock(ConnectorOffsetBackingStore.class);
+        KafkaProducer<byte[], byte[]> producer = mock(KafkaProducer.class);
+        OffsetStorageWriter offsetWriter = mock(OffsetStorageWriter.class);
+
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        partitionOffsets.put(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue"));
+        partitionOffsets.put(Collections.singletonMap("partitionKey", "partitionValue2"), Collections.singletonMap("offsetKey", "offsetValue"));
+
+        when(offsetWriter.doFlush(any())).thenAnswer(invocation -> {
+            invocation.getArgument(0, Callback.class).onCompletion(null, null);
+            return null;
+        });
+
+        FutureCallback<Message> cb = new FutureCallback<>();
+        worker.alterSourceConnectorOffsets(CONNECTOR_ID, sourceConnector, connectorProps, partitionOffsets, offsetStore, producer,
+                offsetWriter, Thread.currentThread().getContextClassLoader(), cb);
+        assertEquals("The offsets for this connector have been altered successfully", cb.get(1000, TimeUnit.MILLISECONDS).message());
+
+        verify(offsetStore).configure(config);
+        verify(offsetStore).start();
+        partitionOffsets.forEach((partition, offset) -> verify(offsetWriter).offset(partition, offset));
+        verify(offsetWriter).beginFlush();
+        verify(offsetWriter).doFlush(any());
+        verify(offsetStore).stop();
+        verifyKafkaClusterId();
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testAlterOffsetsSourceConnectorError() throws Exception {
+        mockKafkaClusterId();
+        worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, Executors.newSingleThreadExecutor(),
+                allConnectorClientConfigOverridePolicy, null);
+        worker.start();
+
+        when(plugins.withClassLoader(any(ClassLoader.class), any(Runnable.class))).thenAnswer(AdditionalAnswers.returnsSecondArg());
+        when(sourceConnector.alterOffsets(eq(connectorProps), anyMap())).thenReturn(true);
+        ConnectorOffsetBackingStore offsetStore = mock(ConnectorOffsetBackingStore.class);
+        KafkaProducer<byte[], byte[]> producer = mock(KafkaProducer.class);
+        OffsetStorageWriter offsetWriter = mock(OffsetStorageWriter.class);
+
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        partitionOffsets.put(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue"));
+        partitionOffsets.put(Collections.singletonMap("partitionKey", "partitionValue2"), Collections.singletonMap("offsetKey", "offsetValue"));
+
+        when(offsetWriter.doFlush(any())).thenAnswer(invocation -> {
+            invocation.getArgument(0, Callback.class).onCompletion(new RuntimeException("Test exception"), null);
+            return null;
+        });
+
+        FutureCallback<Message> cb = new FutureCallback<>();
+        worker.alterSourceConnectorOffsets(CONNECTOR_ID, sourceConnector, connectorProps, partitionOffsets, offsetStore, producer,
+                offsetWriter, Thread.currentThread().getContextClassLoader(), cb);
+        ExecutionException e = assertThrows(ExecutionException.class, () -> cb.get(1000, TimeUnit.MILLISECONDS).message());
+        assertEquals(ConnectException.class, e.getCause().getClass());
+
+        verify(offsetStore).configure(config);
+        verify(offsetStore).start();
+        partitionOffsets.forEach((partition, offset) -> verify(offsetWriter).offset(partition, offset));
+        verify(offsetWriter).beginFlush();
+        verify(offsetWriter).doFlush(any());
+        verify(offsetStore).stop();
+        verifyKafkaClusterId();
+    }
+
+    @Test
+    public void testAlterOffsetsSinkConnectorNoResets() throws Exception {
+        @SuppressWarnings("unchecked")
+        ArgumentCaptor<Map<TopicPartition, OffsetAndMetadata>> alterOffsetsMapCapture = ArgumentCaptor.forClass(Map.class);
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        Map<String, String> partition1 = new HashMap<>();
+        partition1.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
+        partition1.put(SinkUtils.KAFKA_PARTITION_KEY, "10");
+        partitionOffsets.put(partition1, Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, 500));
+        Map<String, String> partition2 = new HashMap<>();
+        partition2.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
+        partition2.put(SinkUtils.KAFKA_PARTITION_KEY, "20");
+        partitionOffsets.put(partition2, Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, 100));
+
+        // A null value for deleteOffsetsSetCapture indicates that we don't expect any call to Admin::deleteConsumerGroupOffsets
+        alterOffsetsSinkConnector(partitionOffsets, alterOffsetsMapCapture, null);
+
+        assertEquals(2, alterOffsetsMapCapture.getValue().size());
+        assertEquals(500, alterOffsetsMapCapture.getValue().get(new TopicPartition("test_topic", 10)).offset());
+        assertEquals(100, alterOffsetsMapCapture.getValue().get(new TopicPartition("test_topic", 20)).offset());
+    }
+
+    @Test
+    public void testAlterOffsetSinkConnectorOnlyResets() throws Exception {
+        @SuppressWarnings("unchecked")
+        ArgumentCaptor<Set<TopicPartition>> deleteOffsetsSetCapture = ArgumentCaptor.forClass(Set.class);
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        Map<String, String> partition1 = new HashMap<>();
+        partition1.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
+        partition1.put(SinkUtils.KAFKA_PARTITION_KEY, "10");
+        partitionOffsets.put(partition1, null);
+        Map<String, String> partition2 = new HashMap<>();
+        partition2.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
+        partition2.put(SinkUtils.KAFKA_PARTITION_KEY, "20");
+        partitionOffsets.put(partition2, null);
+
+        // A null value for alterOffsetsMapCapture indicates that we don't expect any call to Admin::alterConsumerGroupOffsets
+        alterOffsetsSinkConnector(partitionOffsets, null, deleteOffsetsSetCapture);
+
+        Set<TopicPartition> expectedTopicPartitionsForOffsetDelete = new HashSet<>();
+        expectedTopicPartitionsForOffsetDelete.add(new TopicPartition("test_topic", 10));
+        expectedTopicPartitionsForOffsetDelete.add(new TopicPartition("test_topic", 20));
+
+        assertEquals(2, deleteOffsetsSetCapture.getValue().size());
+
+        // Verify that contents are equal without caring about order
+        assertTrue(expectedTopicPartitionsForOffsetDelete.containsAll(deleteOffsetsSetCapture.getValue()));
+        assertTrue(deleteOffsetsSetCapture.getValue().containsAll(expectedTopicPartitionsForOffsetDelete));

Review Comment:
   Can't we simplify this by comparing the two `Set` instances for equality?
   
   ```suggestion
           // Verify that contents are equal without caring about order
           assertEquals(expectedTopicPartitionsForOffsetDelete, deleteOffsetsSetCapture.getValue());
   ```



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java:
##########
@@ -1515,6 +1520,79 @@ public void connectorOffsets(String connName, Callback<ConnectorOffsets> cb) {
         );
     }
 
+    @Override
+    public void alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> callback) {
+        log.trace("Submitting alter offsets request for connector '{}'", connName);
+
+        addRequest(() -> {
+            if (!alterConnectorOffsetsChecks(connName, callback)) {
+                return null;
+            }
+            // At this point, we should be the leader (the call to alterConnectorOffsetsChecks makes sure of that) and can safely run
+            // a zombie fencing request
+            if (isSourceConnector(connName) && config.exactlyOnceSourceEnabled()) {
+                log.debug("Performing a round of zombie fencing before altering offsets for source connector {} with exactly-once support enabled.", connName);
+                doFenceZombieSourceTasks(connName, (error, ignored) -> {
+                    if (error != null) {
+                        log.error("Failed to perform zombie fencing for source connector prior to altering offsets", error);
+                        callback.onCompletion(new ConnectException("Failed to perform zombie fencing for source connector prior to altering offsets",
+                                error), null);
+                    } else {
+                        log.debug("Successfully completed zombie fencing for source connector {}; proceeding to alter offsets.", connName);
+                        // We need to ensure that we perform the necessary checks again before proceeding to actually altering the connector offsets since
+                        // zombie fencing is done asynchronously and the conditions could have changed since the previous check
+                        addRequest(() -> {
+                            if (alterConnectorOffsetsChecks(connName, callback)) {
+                                worker.alterConnectorOffsets(connName, configState.connectorConfig(connName), offsets, callback);
+                            }
+                            return null;
+                        }, forwardErrorCallback(callback));
+                    }
+                });
+            } else {
+                worker.alterConnectorOffsets(connName, configState.connectorConfig(connName), offsets, callback);
+            }
+            return null;
+        }, forwardErrorCallback(callback));
+    }
+
+    /**
+     * This method performs a few checks for alter connector offsets request and completes the callback exceptionally
+     * if any check fails.
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param callback callback to invoke upon completion
+     * @return true if all the checks passed, false otherwise
+     */
+    private boolean alterConnectorOffsetsChecks(String connName, Callback<Message> callback) {
+        if (checkRebalanceNeeded(callback)) {
+            return false;
+        }
+
+        if (!isLeader()) {
+            callback.onCompletion(new NotLeaderException("Only the leader can process alter offsets requests", leaderUrl()), null);
+            return false;
+        }
+
+        if (!refreshConfigSnapshot(workerSyncTimeoutMs)) {
+            throw new ConnectException("Failed to read to end of config topic before altering connector offsets");
+        }
+
+        if (!configState.contains(connName)) {
+            callback.onCompletion(new NotFoundException("Connector " + connName + " not found", null), null);
+            return false;
+        }
+
+        // If the target state for the connector is stopped, its task count is 0, and there is no rebalance pending (checked above),
+        // we can be sure that the tasks have at least been attempted to be stopped (or cancelled if they took too long to stop).
+        // Zombie tasks are handled by a round of zombie fencing for exactly once source connectors. Zombie sink tasks are handled
+        // naturally because requests to alter consumer group offsets will fail if there are still active members in the group.
+        if (configState.targetState(connName) != TargetState.STOPPED || configState.taskCount(connName) != 0) {
+            callback.onCompletion(new BadRequestException("The connector needs to be stopped before its offsets can be altered"), null);

Review Comment:
   We should be really careful about the wording in this message (and the one in `StandaloneHerder`), since "stopping" a connector used to be synonymous with "deleting" one.
   
   A quick sketch of a more-informative error message:
   > "Connectors must be in the STOPPED state before their offsets can be altered. This can be done for the specified connector by issuing a PUT request to the /connectors/" + connName + "/stop endpoint.



##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java:
##########
@@ -1889,6 +1902,352 @@ public void testGetSourceConnectorOffsetsError() {
         verifyKafkaClusterId();
     }
 
+    @Test
+    public void testAlterOffsetsConnectorDoesNotSupportOffsetAlteration() {
+        mockKafkaClusterId();
+
+        worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, Executors.newSingleThreadExecutor(),
+                allConnectorClientConfigOverridePolicy, null);
+        worker.start();
+
+        mockGenericIsolation();
+        when(plugins.newConnector(anyString())).thenReturn(sourceConnector);
+        when(plugins.withClassLoader(any(ClassLoader.class), any(Runnable.class))).thenAnswer(AdditionalAnswers.returnsSecondArg());
+        when(sourceConnector.alterOffsets(eq(connectorProps), anyMap())).thenThrow(new UnsupportedOperationException("This connector doesn't " +
+                "support altering of offsets"));
+
+        FutureCallback<Message> cb = new FutureCallback<>();
+        worker.alterConnectorOffsets(CONNECTOR_ID, connectorProps,
+                Collections.singletonMap(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue")),
+                cb);
+
+        ExecutionException e = assertThrows(ExecutionException.class, () -> cb.get(1000, TimeUnit.MILLISECONDS));
+        assertEquals(ConnectException.class, e.getCause().getClass());
+        assertEquals("Failed to alter offsets for connector " + CONNECTOR_ID + " because it doesn't support external modification of offsets",
+                e.getCause().getMessage());
+
+        verifyGenericIsolation();
+        verifyKafkaClusterId();
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testAlterOffsetsSourceConnector() throws Exception {
+        mockKafkaClusterId();
+        worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, Executors.newSingleThreadExecutor(),
+                allConnectorClientConfigOverridePolicy, null);
+        worker.start();
+
+        when(plugins.withClassLoader(any(ClassLoader.class), any(Runnable.class))).thenAnswer(AdditionalAnswers.returnsSecondArg());
+        when(sourceConnector.alterOffsets(eq(connectorProps), anyMap())).thenReturn(true);
+        ConnectorOffsetBackingStore offsetStore = mock(ConnectorOffsetBackingStore.class);
+        KafkaProducer<byte[], byte[]> producer = mock(KafkaProducer.class);
+        OffsetStorageWriter offsetWriter = mock(OffsetStorageWriter.class);
+
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        partitionOffsets.put(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue"));
+        partitionOffsets.put(Collections.singletonMap("partitionKey", "partitionValue2"), Collections.singletonMap("offsetKey", "offsetValue"));
+
+        when(offsetWriter.doFlush(any())).thenAnswer(invocation -> {
+            invocation.getArgument(0, Callback.class).onCompletion(null, null);
+            return null;
+        });
+
+        FutureCallback<Message> cb = new FutureCallback<>();
+        worker.alterSourceConnectorOffsets(CONNECTOR_ID, sourceConnector, connectorProps, partitionOffsets, offsetStore, producer,
+                offsetWriter, Thread.currentThread().getContextClassLoader(), cb);
+        assertEquals("The offsets for this connector have been altered successfully", cb.get(1000, TimeUnit.MILLISECONDS).message());
+
+        verify(offsetStore).configure(config);
+        verify(offsetStore).start();
+        partitionOffsets.forEach((partition, offset) -> verify(offsetWriter).offset(partition, offset));
+        verify(offsetWriter).beginFlush();
+        verify(offsetWriter).doFlush(any());
+        verify(offsetStore).stop();
+        verifyKafkaClusterId();
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testAlterOffsetsSourceConnectorError() throws Exception {
+        mockKafkaClusterId();
+        worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, Executors.newSingleThreadExecutor(),
+                allConnectorClientConfigOverridePolicy, null);
+        worker.start();
+
+        when(plugins.withClassLoader(any(ClassLoader.class), any(Runnable.class))).thenAnswer(AdditionalAnswers.returnsSecondArg());
+        when(sourceConnector.alterOffsets(eq(connectorProps), anyMap())).thenReturn(true);
+        ConnectorOffsetBackingStore offsetStore = mock(ConnectorOffsetBackingStore.class);
+        KafkaProducer<byte[], byte[]> producer = mock(KafkaProducer.class);
+        OffsetStorageWriter offsetWriter = mock(OffsetStorageWriter.class);
+
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        partitionOffsets.put(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue"));
+        partitionOffsets.put(Collections.singletonMap("partitionKey", "partitionValue2"), Collections.singletonMap("offsetKey", "offsetValue"));
+
+        when(offsetWriter.doFlush(any())).thenAnswer(invocation -> {
+            invocation.getArgument(0, Callback.class).onCompletion(new RuntimeException("Test exception"), null);
+            return null;
+        });
+
+        FutureCallback<Message> cb = new FutureCallback<>();
+        worker.alterSourceConnectorOffsets(CONNECTOR_ID, sourceConnector, connectorProps, partitionOffsets, offsetStore, producer,
+                offsetWriter, Thread.currentThread().getContextClassLoader(), cb);
+        ExecutionException e = assertThrows(ExecutionException.class, () -> cb.get(1000, TimeUnit.MILLISECONDS).message());
+        assertEquals(ConnectException.class, e.getCause().getClass());
+
+        verify(offsetStore).configure(config);
+        verify(offsetStore).start();
+        partitionOffsets.forEach((partition, offset) -> verify(offsetWriter).offset(partition, offset));
+        verify(offsetWriter).beginFlush();
+        verify(offsetWriter).doFlush(any());
+        verify(offsetStore).stop();
+        verifyKafkaClusterId();
+    }
+
+    @Test
+    public void testAlterOffsetsSinkConnectorNoResets() throws Exception {
+        @SuppressWarnings("unchecked")
+        ArgumentCaptor<Map<TopicPartition, OffsetAndMetadata>> alterOffsetsMapCapture = ArgumentCaptor.forClass(Map.class);
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        Map<String, String> partition1 = new HashMap<>();
+        partition1.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
+        partition1.put(SinkUtils.KAFKA_PARTITION_KEY, "10");
+        partitionOffsets.put(partition1, Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, 500));
+        Map<String, String> partition2 = new HashMap<>();
+        partition2.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
+        partition2.put(SinkUtils.KAFKA_PARTITION_KEY, "20");
+        partitionOffsets.put(partition2, Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, 100));
+
+        // A null value for deleteOffsetsSetCapture indicates that we don't expect any call to Admin::deleteConsumerGroupOffsets
+        alterOffsetsSinkConnector(partitionOffsets, alterOffsetsMapCapture, null);
+
+        assertEquals(2, alterOffsetsMapCapture.getValue().size());
+        assertEquals(500, alterOffsetsMapCapture.getValue().get(new TopicPartition("test_topic", 10)).offset());
+        assertEquals(100, alterOffsetsMapCapture.getValue().get(new TopicPartition("test_topic", 20)).offset());
+    }
+
+    @Test
+    public void testAlterOffsetSinkConnectorOnlyResets() throws Exception {
+        @SuppressWarnings("unchecked")
+        ArgumentCaptor<Set<TopicPartition>> deleteOffsetsSetCapture = ArgumentCaptor.forClass(Set.class);
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        Map<String, String> partition1 = new HashMap<>();
+        partition1.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
+        partition1.put(SinkUtils.KAFKA_PARTITION_KEY, "10");
+        partitionOffsets.put(partition1, null);
+        Map<String, String> partition2 = new HashMap<>();
+        partition2.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
+        partition2.put(SinkUtils.KAFKA_PARTITION_KEY, "20");
+        partitionOffsets.put(partition2, null);
+
+        // A null value for alterOffsetsMapCapture indicates that we don't expect any call to Admin::alterConsumerGroupOffsets
+        alterOffsetsSinkConnector(partitionOffsets, null, deleteOffsetsSetCapture);
+
+        Set<TopicPartition> expectedTopicPartitionsForOffsetDelete = new HashSet<>();
+        expectedTopicPartitionsForOffsetDelete.add(new TopicPartition("test_topic", 10));
+        expectedTopicPartitionsForOffsetDelete.add(new TopicPartition("test_topic", 20));
+
+        assertEquals(2, deleteOffsetsSetCapture.getValue().size());
+
+        // Verify that contents are equal without caring about order
+        assertTrue(expectedTopicPartitionsForOffsetDelete.containsAll(deleteOffsetsSetCapture.getValue()));
+        assertTrue(deleteOffsetsSetCapture.getValue().containsAll(expectedTopicPartitionsForOffsetDelete));
+    }
+
+    @Test
+    public void testAlterOffsetsSinkConnectorAltersAndResets() throws Exception {
+        @SuppressWarnings("unchecked")
+        ArgumentCaptor<Map<TopicPartition, OffsetAndMetadata>> alterOffsetsMapCapture = ArgumentCaptor.forClass(Map.class);
+        @SuppressWarnings("unchecked")
+        ArgumentCaptor<Set<TopicPartition>> deleteOffsetsSetCapture = ArgumentCaptor.forClass(Set.class);
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        Map<String, String> partition1 = new HashMap<>();
+        partition1.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
+        partition1.put(SinkUtils.KAFKA_PARTITION_KEY, "10");
+        partitionOffsets.put(partition1, Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, "100"));
+        Map<String, String> partition2 = new HashMap<>();
+        partition2.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
+        partition2.put(SinkUtils.KAFKA_PARTITION_KEY, "20");
+        partitionOffsets.put(partition2, null);
+
+        alterOffsetsSinkConnector(partitionOffsets, alterOffsetsMapCapture, deleteOffsetsSetCapture);
+
+        assertEquals(1, alterOffsetsMapCapture.getValue().size());
+        assertEquals(100, alterOffsetsMapCapture.getValue().get(new TopicPartition("test_topic", 10)).offset());
+
+        assertEquals(1, deleteOffsetsSetCapture.getValue().size());
+        assertEquals(new TopicPartition("test_topic", 20), deleteOffsetsSetCapture.getValue().iterator().next());

Review Comment:
   Same thought RE comparing `Set` instances for equality here



##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java:
##########
@@ -1889,6 +1902,352 @@ public void testGetSourceConnectorOffsetsError() {
         verifyKafkaClusterId();
     }
 
+    @Test
+    public void testAlterOffsetsConnectorDoesNotSupportOffsetAlteration() {
+        mockKafkaClusterId();
+
+        worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, Executors.newSingleThreadExecutor(),
+                allConnectorClientConfigOverridePolicy, null);
+        worker.start();
+
+        mockGenericIsolation();
+        when(plugins.newConnector(anyString())).thenReturn(sourceConnector);
+        when(plugins.withClassLoader(any(ClassLoader.class), any(Runnable.class))).thenAnswer(AdditionalAnswers.returnsSecondArg());
+        when(sourceConnector.alterOffsets(eq(connectorProps), anyMap())).thenThrow(new UnsupportedOperationException("This connector doesn't " +
+                "support altering of offsets"));
+
+        FutureCallback<Message> cb = new FutureCallback<>();
+        worker.alterConnectorOffsets(CONNECTOR_ID, connectorProps,
+                Collections.singletonMap(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue")),
+                cb);
+
+        ExecutionException e = assertThrows(ExecutionException.class, () -> cb.get(1000, TimeUnit.MILLISECONDS));
+        assertEquals(ConnectException.class, e.getCause().getClass());
+        assertEquals("Failed to alter offsets for connector " + CONNECTOR_ID + " because it doesn't support external modification of offsets",
+                e.getCause().getMessage());
+
+        verifyGenericIsolation();
+        verifyKafkaClusterId();
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testAlterOffsetsSourceConnector() throws Exception {
+        mockKafkaClusterId();
+        worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, Executors.newSingleThreadExecutor(),
+                allConnectorClientConfigOverridePolicy, null);
+        worker.start();
+
+        when(plugins.withClassLoader(any(ClassLoader.class), any(Runnable.class))).thenAnswer(AdditionalAnswers.returnsSecondArg());
+        when(sourceConnector.alterOffsets(eq(connectorProps), anyMap())).thenReturn(true);
+        ConnectorOffsetBackingStore offsetStore = mock(ConnectorOffsetBackingStore.class);
+        KafkaProducer<byte[], byte[]> producer = mock(KafkaProducer.class);
+        OffsetStorageWriter offsetWriter = mock(OffsetStorageWriter.class);
+
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        partitionOffsets.put(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue"));
+        partitionOffsets.put(Collections.singletonMap("partitionKey", "partitionValue2"), Collections.singletonMap("offsetKey", "offsetValue"));
+
+        when(offsetWriter.doFlush(any())).thenAnswer(invocation -> {
+            invocation.getArgument(0, Callback.class).onCompletion(null, null);
+            return null;
+        });
+
+        FutureCallback<Message> cb = new FutureCallback<>();
+        worker.alterSourceConnectorOffsets(CONNECTOR_ID, sourceConnector, connectorProps, partitionOffsets, offsetStore, producer,
+                offsetWriter, Thread.currentThread().getContextClassLoader(), cb);
+        assertEquals("The offsets for this connector have been altered successfully", cb.get(1000, TimeUnit.MILLISECONDS).message());
+
+        verify(offsetStore).configure(config);
+        verify(offsetStore).start();
+        partitionOffsets.forEach((partition, offset) -> verify(offsetWriter).offset(partition, offset));
+        verify(offsetWriter).beginFlush();
+        verify(offsetWriter).doFlush(any());
+        verify(offsetStore).stop();
+        verifyKafkaClusterId();
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testAlterOffsetsSourceConnectorError() throws Exception {
+        mockKafkaClusterId();
+        worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, Executors.newSingleThreadExecutor(),
+                allConnectorClientConfigOverridePolicy, null);
+        worker.start();
+
+        when(plugins.withClassLoader(any(ClassLoader.class), any(Runnable.class))).thenAnswer(AdditionalAnswers.returnsSecondArg());
+        when(sourceConnector.alterOffsets(eq(connectorProps), anyMap())).thenReturn(true);
+        ConnectorOffsetBackingStore offsetStore = mock(ConnectorOffsetBackingStore.class);
+        KafkaProducer<byte[], byte[]> producer = mock(KafkaProducer.class);
+        OffsetStorageWriter offsetWriter = mock(OffsetStorageWriter.class);
+
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        partitionOffsets.put(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue"));
+        partitionOffsets.put(Collections.singletonMap("partitionKey", "partitionValue2"), Collections.singletonMap("offsetKey", "offsetValue"));
+
+        when(offsetWriter.doFlush(any())).thenAnswer(invocation -> {
+            invocation.getArgument(0, Callback.class).onCompletion(new RuntimeException("Test exception"), null);
+            return null;
+        });
+
+        FutureCallback<Message> cb = new FutureCallback<>();
+        worker.alterSourceConnectorOffsets(CONNECTOR_ID, sourceConnector, connectorProps, partitionOffsets, offsetStore, producer,
+                offsetWriter, Thread.currentThread().getContextClassLoader(), cb);
+        ExecutionException e = assertThrows(ExecutionException.class, () -> cb.get(1000, TimeUnit.MILLISECONDS).message());
+        assertEquals(ConnectException.class, e.getCause().getClass());
+
+        verify(offsetStore).configure(config);
+        verify(offsetStore).start();
+        partitionOffsets.forEach((partition, offset) -> verify(offsetWriter).offset(partition, offset));
+        verify(offsetWriter).beginFlush();
+        verify(offsetWriter).doFlush(any());
+        verify(offsetStore).stop();
+        verifyKafkaClusterId();
+    }
+
+    @Test
+    public void testAlterOffsetsSinkConnectorNoResets() throws Exception {
+        @SuppressWarnings("unchecked")
+        ArgumentCaptor<Map<TopicPartition, OffsetAndMetadata>> alterOffsetsMapCapture = ArgumentCaptor.forClass(Map.class);
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        Map<String, String> partition1 = new HashMap<>();
+        partition1.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
+        partition1.put(SinkUtils.KAFKA_PARTITION_KEY, "10");
+        partitionOffsets.put(partition1, Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, 500));
+        Map<String, String> partition2 = new HashMap<>();
+        partition2.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
+        partition2.put(SinkUtils.KAFKA_PARTITION_KEY, "20");
+        partitionOffsets.put(partition2, Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, 100));
+
+        // A null value for deleteOffsetsSetCapture indicates that we don't expect any call to Admin::deleteConsumerGroupOffsets
+        alterOffsetsSinkConnector(partitionOffsets, alterOffsetsMapCapture, null);
+
+        assertEquals(2, alterOffsetsMapCapture.getValue().size());
+        assertEquals(500, alterOffsetsMapCapture.getValue().get(new TopicPartition("test_topic", 10)).offset());
+        assertEquals(100, alterOffsetsMapCapture.getValue().get(new TopicPartition("test_topic", 20)).offset());
+    }
+
+    @Test
+    public void testAlterOffsetSinkConnectorOnlyResets() throws Exception {
+        @SuppressWarnings("unchecked")
+        ArgumentCaptor<Set<TopicPartition>> deleteOffsetsSetCapture = ArgumentCaptor.forClass(Set.class);
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        Map<String, String> partition1 = new HashMap<>();
+        partition1.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
+        partition1.put(SinkUtils.KAFKA_PARTITION_KEY, "10");
+        partitionOffsets.put(partition1, null);
+        Map<String, String> partition2 = new HashMap<>();
+        partition2.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
+        partition2.put(SinkUtils.KAFKA_PARTITION_KEY, "20");
+        partitionOffsets.put(partition2, null);
+
+        // A null value for alterOffsetsMapCapture indicates that we don't expect any call to Admin::alterConsumerGroupOffsets
+        alterOffsetsSinkConnector(partitionOffsets, null, deleteOffsetsSetCapture);
+
+        Set<TopicPartition> expectedTopicPartitionsForOffsetDelete = new HashSet<>();
+        expectedTopicPartitionsForOffsetDelete.add(new TopicPartition("test_topic", 10));
+        expectedTopicPartitionsForOffsetDelete.add(new TopicPartition("test_topic", 20));
+
+        assertEquals(2, deleteOffsetsSetCapture.getValue().size());
+
+        // Verify that contents are equal without caring about order
+        assertTrue(expectedTopicPartitionsForOffsetDelete.containsAll(deleteOffsetsSetCapture.getValue()));
+        assertTrue(deleteOffsetsSetCapture.getValue().containsAll(expectedTopicPartitionsForOffsetDelete));
+    }
+
+    @Test
+    public void testAlterOffsetsSinkConnectorAltersAndResets() throws Exception {
+        @SuppressWarnings("unchecked")
+        ArgumentCaptor<Map<TopicPartition, OffsetAndMetadata>> alterOffsetsMapCapture = ArgumentCaptor.forClass(Map.class);
+        @SuppressWarnings("unchecked")
+        ArgumentCaptor<Set<TopicPartition>> deleteOffsetsSetCapture = ArgumentCaptor.forClass(Set.class);
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
+        Map<String, String> partition1 = new HashMap<>();
+        partition1.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
+        partition1.put(SinkUtils.KAFKA_PARTITION_KEY, "10");
+        partitionOffsets.put(partition1, Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, "100"));
+        Map<String, String> partition2 = new HashMap<>();
+        partition2.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
+        partition2.put(SinkUtils.KAFKA_PARTITION_KEY, "20");
+        partitionOffsets.put(partition2, null);
+
+        alterOffsetsSinkConnector(partitionOffsets, alterOffsetsMapCapture, deleteOffsetsSetCapture);
+
+        assertEquals(1, alterOffsetsMapCapture.getValue().size());
+        assertEquals(100, alterOffsetsMapCapture.getValue().get(new TopicPartition("test_topic", 10)).offset());
+
+        assertEquals(1, deleteOffsetsSetCapture.getValue().size());
+        assertEquals(new TopicPartition("test_topic", 20), deleteOffsetsSetCapture.getValue().iterator().next());
+    }
+
+    private void alterOffsetsSinkConnector(Map<Map<String, ?>, Map<String, ?>> partitionOffsets,
+                                           ArgumentCaptor<Map<TopicPartition, OffsetAndMetadata>> alterOffsetsMapCapture,
+                                           ArgumentCaptor<Set<TopicPartition>> deleteOffsetsSetCapture) throws Exception {
+        mockKafkaClusterId();
+        String connectorClass = SampleSinkConnector.class.getName();
+        connectorProps.put(CONNECTOR_CLASS_CONFIG, connectorClass);
+
+        Admin admin = mock(Admin.class);
+        worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, Executors.newCachedThreadPool(),
+                allConnectorClientConfigOverridePolicy, config -> admin);
+        worker.start();
+
+        when(plugins.withClassLoader(any(ClassLoader.class), any(Runnable.class))).thenAnswer(AdditionalAnswers.returnsSecondArg());
+        when(sinkConnector.alterOffsets(eq(connectorProps), anyMap())).thenReturn(true);
+
+        // If alterOffsetsMapCapture is null, then we won't stub any of the following methods resulting in test failures in case
+        // offsets for certain topic partitions are actually attempted to be altered.
+        if (alterOffsetsMapCapture != null) {
+            AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = mock(AlterConsumerGroupOffsetsResult.class);
+            when(admin.alterConsumerGroupOffsets(anyString(), alterOffsetsMapCapture.capture(), any(AlterConsumerGroupOffsetsOptions.class)))
+                    .thenReturn(alterConsumerGroupOffsetsResult);
+            KafkaFuture<Void> alterFuture = KafkaFuture.completedFuture(null);
+            when(alterConsumerGroupOffsetsResult.all()).thenReturn(alterFuture);
+        }
+
+        // If deleteOffsetsSetCapture is null, then we won't stub any of the following methods resulting in test failures in case
+        // offsets for certain topic partitions are actually attempted to be deleted.
+        if (deleteOffsetsSetCapture != null) {
+            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = mock(DeleteConsumerGroupOffsetsResult.class);
+            when(admin.deleteConsumerGroupOffsets(anyString(), deleteOffsetsSetCapture.capture(), any(DeleteConsumerGroupOffsetsOptions.class)))
+                    .thenReturn(deleteConsumerGroupOffsetsResult);
+            KafkaFuture<Void> deleteFuture = KafkaFuture.completedFuture(null);
+            when(deleteConsumerGroupOffsetsResult.all()).thenReturn(deleteFuture);
+        }
+
+        FutureCallback<Message> cb = new FutureCallback<>();
+        worker.alterSinkConnectorOffsets(CONNECTOR_ID, sinkConnector, connectorProps, partitionOffsets,
+                Thread.currentThread().getContextClassLoader(), cb);
+        assertEquals("The offsets for this connector have been altered successfully", cb.get(1000, TimeUnit.MILLISECONDS).message());
+
+        verify(admin, timeout(1000)).close();
+        verifyKafkaClusterId();
+    }
+
+    @Test
+    public void testAlterOffsetsSinkConnectorAlterOffsetsError() throws Exception {
+        mockKafkaClusterId();
+        String connectorClass = SampleSinkConnector.class.getName();
+        connectorProps.put(CONNECTOR_CLASS_CONFIG, connectorClass);
+
+        Admin admin = mock(Admin.class);
+        worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, Executors.newSingleThreadExecutor(),
+                allConnectorClientConfigOverridePolicy, config -> admin);
+        worker.start();
+
+        when(plugins.withClassLoader(any(ClassLoader.class), any(Runnable.class))).thenAnswer(AdditionalAnswers.returnsSecondArg());
+        when(sinkConnector.alterOffsets(eq(connectorProps), anyMap())).thenReturn(true);
+
+        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = mock(AlterConsumerGroupOffsetsResult.class);
+        when(admin.alterConsumerGroupOffsets(anyString(), anyMap(), any(AlterConsumerGroupOffsetsOptions.class)))
+                .thenReturn(alterConsumerGroupOffsetsResult);
+        KafkaFutureImpl<Void> alterFuture = new KafkaFutureImpl<>();
+        alterFuture.completeExceptionally(new ClusterAuthorizationException("Test exception"));
+        when(alterConsumerGroupOffsetsResult.all()).thenReturn(alterFuture);
+
+        Map<String, String> partition1 = new HashMap<>();
+        partition1.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
+        partition1.put(SinkUtils.KAFKA_PARTITION_KEY, "10");
+        Map<Map<String, ?>, Map<String, ?>> partitionOffsets = Collections.singletonMap(partition1,
+                Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, "100"));
+
+        FutureCallback<Message> cb = new FutureCallback<>();
+        worker.alterSinkConnectorOffsets(CONNECTOR_ID, sinkConnector, connectorProps, partitionOffsets,
+                Thread.currentThread().getContextClassLoader(), cb);
+
+        ExecutionException e = assertThrows(ExecutionException.class, () -> cb.get(1000, TimeUnit.MILLISECONDS));
+        assertEquals(ConnectException.class, e.getCause().getClass());
+
+        verify(admin, timeout(1000)).close();
+        verifyNoMoreInteractions(admin);
+        verifyKafkaClusterId();
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")

Review Comment:
   Nit: unnecessary suppression
   ```suggestion
   ```



##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java:
##########
@@ -4123,6 +4136,280 @@ public void testConnectorOffsets() throws Exception {
         PowerMock.verifyAll();
     }
 
+    @Test
+    public void testAlterOffsetsConnectorNotFound() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets("connector-does-not-exist", new HashMap<>(), callback);
+        herder.tick();
+        ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertTrue(e.getCause() instanceof NotFoundException);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsConnectorNotInStoppedState() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, new HashMap<>(), callback);
+        herder.tick();
+        ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertTrue(e.getCause() instanceof BadRequestException);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsNotLeader() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("member");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), false);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();

Review Comment:
   So this part is missing a succeeding call to `PowerMock::expectLastCall`, but the test case passes anyways (and if we comment out this line, it fails).
   
   I did some experimenting and we can comment out every call to `PowerMock::expectLastCall` except for the last one on line 4216 and the test will still pass.
   
   Probably best to just add a call after this line for consistency's sake, but interesting behavior from PowerMock. If we weren't migrating to Mockito it might be worth looking into to see if that behavior is intentional and, if so, if we could leverage it to reduce noise in our test cases.



##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java:
##########
@@ -4123,6 +4136,280 @@ public void testConnectorOffsets() throws Exception {
         PowerMock.verifyAll();
     }
 
+    @Test
+    public void testAlterOffsetsConnectorNotFound() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets("connector-does-not-exist", new HashMap<>(), callback);
+        herder.tick();
+        ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertTrue(e.getCause() instanceof NotFoundException);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsConnectorNotInStoppedState() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        expectConfigRefreshAndSnapshot(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, new HashMap<>(), callback);
+        herder.tick();
+        ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertTrue(e.getCause() instanceof BadRequestException);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsNotLeader() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("member");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), false);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, new HashMap<>(), callback);
+        herder.tick();
+        ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertTrue(e.getCause() instanceof NotLeaderException);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsSourceConnectorExactlyOnceDisabled() throws Exception {
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // Now handle the alter connector offsets request
+        Map<Map<String, ?>, Map<String, ?>> offsets = new HashMap<>();
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        Capture<Callback<Message>> workerCallbackCapture = Capture.newInstance();
+        worker.alterConnectorOffsets(EasyMock.eq(CONN1), EasyMock.eq(CONN1_CONFIG), EasyMock.eq(offsets), capture(workerCallbackCapture));
+        Message msg = new Message("The offsets for this connector have been altered successfully");
+        EasyMock.expectLastCall().andAnswer(() -> {
+            workerCallbackCapture.getValue().onCompletion(null, msg);
+            return null;
+        });
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, offsets, callback);
+        herder.tick();
+        assertEquals(msg, callback.get(1000L, TimeUnit.MILLISECONDS));
+        assertEquals("The offsets for this connector have been altered successfully", msg.message());
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsSourceConnectorExactlyOnceEnabled() throws Exception {
+        // Setup herder with exactly-once support for source connectors enabled
+        herder = exactlyOnceHerder();
+        rebalanceListener = herder.new RebalanceListener(time);
+        PowerMock.expectPrivate(herder, "updateDeletedConnectorStatus").andVoid().anyTimes();
+        PowerMock.expectPrivate(herder, "updateDeletedTaskStatus").andVoid().anyTimes();
+
+        // Get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0);
+        expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall().anyTimes();
+
+        // Now handle the alter connector offsets request
+        Map<Map<String, ?>, Map<String, ?>> offsets = new HashMap<>();
+        member.wakeup();
+        PowerMock.expectLastCall().anyTimes();
+        member.ensureActive();
+        PowerMock.expectLastCall().anyTimes();
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        EasyMock.expect(herder.connectorType(EasyMock.anyObject())).andReturn(ConnectorType.SOURCE).anyTimes();
+
+        // Expect a round of zombie fencing to occur
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1);
+        KafkaFuture<Void> workerFencingFuture = EasyMock.mock(KafkaFuture.class);
+        KafkaFuture<Void> herderFencingFuture = EasyMock.mock(KafkaFuture.class);
+        EasyMock.expect(worker.fenceZombies(CONN1, SNAPSHOT_STOPPED_CONN1.taskCountRecord(CONN1), CONN1_CONFIG)).andReturn(workerFencingFuture);
+        EasyMock.expect(workerFencingFuture.thenApply(EasyMock.<KafkaFuture.BaseFunction<Void, Void>>anyObject())).andReturn(herderFencingFuture);
+
+        // Two fencing callbacks are added - one is in ZombieFencing::start itself to remove the connector from the active
+        // fencing list. The other is the callback passed from DistributedHerder::alterConnectorOffsets in order to
+        // queue up the actual alter offsets request if the zombie fencing succeeds.
+        for (int i = 0; i < 2; i++) {
+            Capture<KafkaFuture.BiConsumer<Void, Throwable>> herderFencingCallback = EasyMock.newCapture();
+            EasyMock.expect(herderFencingFuture.whenComplete(EasyMock.capture(herderFencingCallback))).andAnswer(() -> {
+                herderFencingCallback.getValue().accept(null, null);
+                return null;
+            });
+        }
+
+        Capture<Callback<Message>> workerCallbackCapture = Capture.newInstance();
+        Message msg = new Message("The offsets for this connector have been altered successfully");
+        worker.alterConnectorOffsets(EasyMock.eq(CONN1), EasyMock.eq(CONN1_CONFIG), EasyMock.eq(offsets), capture(workerCallbackCapture));
+        EasyMock.expectLastCall().andAnswer(() -> {
+            workerCallbackCapture.getValue().onCompletion(null, msg);
+            return null;
+        });
+
+        // Handle the second alter connector offsets request. No zombie fencing request to the worker is expected now since we
+        // already did a round of zombie fencing last time and no new tasks came up in the meanwhile. The config snapshot is
+        // refreshed once at the beginning of the DistributedHerder::alterConnectorOffsets method, once before checking
+        // whether zombie fencing is required, and once before actually proceeding to alter connector offsets.
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1_FENCED);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1_FENCED);
+        expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1_FENCED);
+        Capture<Callback<Message>> workerCallbackCapture2 = Capture.newInstance();
+        worker.alterConnectorOffsets(EasyMock.eq(CONN1), EasyMock.eq(CONN1_CONFIG), EasyMock.eq(offsets), capture(workerCallbackCapture2));
+        EasyMock.expectLastCall().andAnswer(() -> {
+            workerCallbackCapture2.getValue().onCompletion(null, msg);
+            return null;
+        });
+
+        PowerMock.replayAll(workerFencingFuture, herderFencingFuture);
+
+        herder.tick();
+        FutureCallback<Message> callback = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, offsets, callback);
+        // Process the zombie fencing request
+        herder.tick();
+        // Process the alter offsets request
+        herder.tick();
+        assertEquals(msg, callback.get(1000L, TimeUnit.MILLISECONDS));
+
+        FutureCallback<Message> callback2 = new FutureCallback<>();
+        herder.alterConnectorOffsets(CONN1, offsets, callback2);
+        herder.tick();
+        assertEquals(msg, callback.get(1000L, TimeUnit.MILLISECONDS));
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testAlterOffsetsSourceConnectorExactlyOnceEnabledZombieFencingFailure() throws Exception {

Review Comment:
   This goes above and beyond what I was expecting for coverage here, great stuff 👍
   
   Do you think it's also worth adding a case for a sink connector, or is that too similar to the non-exactly-once source case?



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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1204612162


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java:
##########
@@ -1247,6 +1257,257 @@ void sourceConnectorOffsets(String connName, ConnectorOffsetBackingStore offsetS
         });
     }
 
+    /**
+     * Alter a connector's offsets.
+     *
+     * @param connName the name of the connector whose offsets are to be altered
+     * @param connectorConfig the connector's configurations
+     * @param offsets  a mapping from partitions to offsets that need to be overwritten
+     * @param cb callback to invoke upon completion
+     */
+    public void alterConnectorOffsets(String connName, Map<String, String> connectorConfig,
+                                      Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb) {
+        String connectorClassOrAlias = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
+        ClassLoader connectorLoader = plugins.connectorLoader(connectorClassOrAlias);
+        Connector connector;
+
+        try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) {
+            connector = plugins.newConnector(connectorClassOrAlias);
+            if (ConnectUtils.isSinkConnector(connector)) {
+                log.debug("Altering consumer group offsets for sink connector: {}", connName);
+                alterSinkConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            } else {
+                log.debug("Altering offsets for source connector: {}", connName);
+                alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, connectorLoader, cb);
+            }
+        }
+    }
+
+    /**
+     * Alter a sink connector's consumer group offsets.
+     * <p>
+     * Visible for testing.
+     *
+     * @param connName the name of the sink connector whose offsets are to be altered
+     * @param connector an instance of the sink connector
+     * @param connectorConfig the sink connector's configuration
+     * @param offsets a mapping from topic partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    void alterSinkConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                   Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                Map<TopicPartition, Long> parsedOffsets = SinkUtils.validateAndParseSinkConnectorOffsets(offsets);
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SinkConnector) connector).alterOffsets(connectorConfig, parsedOffsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                Class<? extends Connector> sinkConnectorClass = connector.getClass();
+                Map<String, Object> adminConfig = adminConfigs(
+                        connName,
+                        "connector-worker-adminclient-" + connName,
+                        config,
+                        new SinkConnectorConfig(plugins, connectorConfig),
+                        sinkConnectorClass,
+                        connectorClientConfigOverridePolicy,
+                        kafkaClusterId,
+                        ConnectorType.SINK);
+
+                SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, connectorConfig);
+                String groupId = (String) baseConsumerConfigs(
+                        connName, "connector-consumer-", config, sinkConnectorConfig,
+                        sinkConnectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK).get(ConsumerConfig.GROUP_ID_CONFIG);
+
+                Map<TopicPartition, OffsetAndMetadata> offsetsToAlter = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() != null)
+                        .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new OffsetAndMetadata(entry.getValue())))
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+                Set<TopicPartition> partitionsToReset = parsedOffsets.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getValue() == null)
+                        .map(Map.Entry::getKey)
+                        .collect(Collectors.toSet());
+
+                KafkaFuture<Void> adminFuture = KafkaFuture.completedFuture(null);
+
+                Admin admin = adminFactory.apply(adminConfig);
+
+                try {
+                    if (!offsetsToAlter.isEmpty()) {
+                        log.debug("Committing the following consumer group topic partition offsets using an admin client for sink connector {}: {}.",
+                                connName, offsetsToAlter);
+                        AlterConsumerGroupOffsetsOptions alterConsumerGroupOffsetsOptions = new AlterConsumerGroupOffsetsOptions().timeoutMs(
+                                (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                        AlterConsumerGroupOffsetsResult alterConsumerGroupOffsetsResult = admin.alterConsumerGroupOffsets(groupId, offsetsToAlter,
+                                alterConsumerGroupOffsetsOptions);
+
+                        adminFuture = alterConsumerGroupOffsetsResult.all();
+                    }
+
+                    adminFuture.whenComplete((ignored, error) -> {
+                        if (error != null) {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            // When a consumer group is non-empty, only group members can commit offsets. The above attempt to alter offsets via the admin
+                            // client will result in an UnknownMemberIdException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped
+                            // completely or if the connector is resumed while the alter offsets request is being processed)
+                            if (error instanceof UnknownMemberIdException) {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                        null);
+                            } else {
+                                cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for topic partitions " + offsetsToAlter.keySet() + " for " +
+                                                "connector " + connName, error),
+                                        null);
+                            }
+                        } else if (!partitionsToReset.isEmpty()) {
+                            log.debug("Deleting the consumer group offsets for the following topic partitions using an admin client for sink connector {}: {}.",
+                                    connName, partitionsToReset);
+                            DeleteConsumerGroupOffsetsOptions deleteConsumerGroupOffsetsOptions = new DeleteConsumerGroupOffsetsOptions().timeoutMs(
+                                    (int) ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS);
+                            DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsetsResult = admin.deleteConsumerGroupOffsets(groupId, partitionsToReset,
+                                    deleteConsumerGroupOffsetsOptions);
+                            deleteConsumerGroupOffsetsResult.all().whenComplete((ignored2, error2) -> {
+                                Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                                if (error2 != null) {
+                                    // The attempt to delete offsets for certain topic partitions via the admin client will result in a
+                                    // GroupSubscribedToTopicException if the consumer group is non-empty (i.e. if the sink tasks haven't stopped completely
+                                    // or if the connector is resumed while the alter offsets request is being processed).
+                                    if (error2 instanceof GroupSubscribedToTopicException) {
+                                        cb.onCompletion(new ConnectException("Failed to alter consumer group offsets for connector " + connName + " either because its tasks " +
+                                                        "haven't stopped completely yet or the connector was resumed before the request to alter its offsets could be successfully " +
+                                                        "completed. If the connector is in a stopped state, this operation can be safely retried. If it doesn't eventually succeed, the " +
+                                                        "Connect cluster may need to be restarted to get rid of the zombie sink tasks."),
+                                                null);
+                                    } else {
+                                        cb.onCompletion(new ConnectException("Failed to delete consumer group offsets for topic partitions " + partitionsToReset + " for connector "
+                                                        + connName, error2),
+                                                null);
+                                    }
+                                } else {
+                                    completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                                }
+                            });
+                        } else {
+                            Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                            completeAlterOffsetsCallback(alterOffsetsResult, cb);
+                        }
+                    });
+                } catch (Throwable t) {
+                    Utils.closeQuietly(admin, "Offset alter admin for sink connector " + connName);
+                    throw t;
+                }
+            } catch (Throwable t) {
+                cb.onCompletion(ConnectUtils.maybeWrap(t, "Failed to alter offsets for sink connector " + connName), null);
+            }
+        }));
+    }
+
+    /**
+     * Alter a source connector's offsets.
+     *
+     * @param connName the name of the source connector whose offsets are to be altered
+     * @param connector an instance of the source connector
+     * @param connectorConfig the source connector's configuration
+     * @param offsets a mapping from partitions to offsets that need to be overwritten
+     * @param connectorLoader the connector plugin's classloader to be used as the thread context classloader
+     * @param cb callback to invoke upon completion
+     */
+    private void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                             Map<Map<String, ?>, Map<String, ?>> offsets, ClassLoader connectorLoader, Callback<Message> cb) {
+        SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig, config.topicCreationEnable());
+        Map<String, Object> producerProps = config.exactlyOnceSourceEnabled()
+                ? exactlyOnceSourceTaskProducerConfigs(new ConnectorTaskId(connName, 0), config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId)
+                : baseProducerConfigs(connName, "connector-offset-producer-" + connName, config, sourceConfig,
+                connector.getClass(), connectorClientConfigOverridePolicy, kafkaClusterId);
+        KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+        ConnectorOffsetBackingStore offsetStore = config.exactlyOnceSourceEnabled()
+                ? offsetStoreForExactlyOnceSourceConnector(sourceConfig, connName, connector, producer)
+                : offsetStoreForRegularSourceConnector(sourceConfig, connName, connector, producer);
+
+        OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, connName, internalKeyConverter, internalValueConverter);
+        alterSourceConnectorOffsets(connName, connector, connectorConfig, offsets, offsetStore, producer, offsetWriter, connectorLoader, cb);
+    }
+
+    // Visible for testing
+    void alterSourceConnectorOffsets(String connName, Connector connector, Map<String, String> connectorConfig,
+                                     Map<Map<String, ?>, Map<String, ?>> offsets, ConnectorOffsetBackingStore offsetStore,
+                                     KafkaProducer<byte[], byte[]> producer, OffsetStorageWriter offsetWriter,
+                                     ClassLoader connectorLoader, Callback<Message> cb) {
+        executor.submit(plugins.withClassLoader(connectorLoader, () -> {
+            try {
+                boolean alterOffsetsResult;
+                try {
+                    alterOffsetsResult = ((SourceConnector) connector).alterOffsets(connectorConfig, offsets);
+                } catch (UnsupportedOperationException e) {
+                    throw new ConnectException("Failed to alter offsets for connector " + connName + " because it doesn't support external " +
+                            "modification of offsets", e);
+                }
+
+                offsetStore.configure(config);
+                // This reads to the end of the offsets topic and can be a potentially time-consuming operation
+                offsetStore.start();

Review Comment:
   If we initialize resources in `start` and then fail later on in that call, those resources will be leaked right now. This also applies to resources that were pre-allocated for the offset store but initialized before calls to `start` or `configure`.



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


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

Posted by "C0urante (via GitHub)" <gi...@apache.org>.
C0urante commented on code in PR #13465:
URL: https://github.com/apache/kafka/pull/13465#discussion_r1204604996


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java:
##########
@@ -1528,6 +1539,80 @@ public void connectorOffsets(String connName, Callback<ConnectorOffsets> cb) {
         );
     }
 
+    @Override
+    public void alterConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> callback) {
+        log.trace("Submitting alter offsets request for connector '{}'", connName);
+
+        addRequest(() -> {
+            refreshConfigSnapshot(workerSyncTimeoutMs);
+            if (!alterConnectorOffsetsChecks(connName, callback)) {
+                return null;
+            }
+            // At this point, we should be the leader (the call to alterConnectorOffsetsChecks makes sure of that) and can safely run
+            // a zombie fencing request
+            if (isSourceConnector(connName) && config.exactlyOnceSourceEnabled()) {
+                log.debug("Performing a round of zombie fencing before altering offsets for source connector {} with exactly-once semantics enabled.", connName);
+                getFenceZombieSourceTasksCallable(connName, (error, ignored) -> {
+                    if (error != null) {
+                        log.error("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets", error);
+                        callback.onCompletion(new ConnectException("Failed to perform zombie fencing for exactly-once source connector prior to altering offsets",
+                                error), null);
+                    } else {
+                        log.debug("Successfully completed zombie fencing for source connector {}; proceeding to alter offsets.", connName);
+                        // We need to ensure that we perform the necessary checks again inside alterConnectorOffsetsHerderRequest
+                        // since it is being run in a separate herder request and the conditions could have changed since the
+                        // previous check
+                        addRequest(getAlterConnectorOffsetsCallable(connName, offsets, callback), forwardErrorCallback(callback));
+                    }
+                }).call();
+            } else {
+                getAlterConnectorOffsetsCallable(connName, offsets, callback).call();

Review Comment:
   Nice, LGTM 👍



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