You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/06/19 09:16:38 UTC

[GitHub] [kafka] C0urante opened a new pull request #10907: KAFKA-10000: Exactly-once support for source connectors

C0urante opened a new pull request #10907:
URL: https://github.com/apache/kafka/pull/10907


   Implements [KIP-618](https://cwiki.apache.org/confluence/display/KAFKA/KIP-618%3A+Exactly-Once+Support+for+Source+Connectors#KIP618:ExactlyOnceSupportforSourceConnectors-Newmetrics).
   
   There are several changes here that can be reviewed fairly independently of each other:
   - Support for transactional source tasks, which is largely implemented in the `ExactlyOnceWorkerSourceTask` class, its newly-introduced `AbstractWorkerSourceTask` superclass, the `Worker` class (whose API for task starts has been split up from the existing `startTask` method into separate `startSinkTask`, `startSourceTask`, and `startExactlyOnceSourceTask` methods), and the `WorkerTransactionContext` class (which is used to allow connectors to define their own transaction boundaries)
   - Zombie fencing logic and the use of a transactional producer for some writes to the config topic, which are done by the leader of the cluster and are largely implemented in the `DistributedHerder`, `ConfigBackingStore`, `KafkaConfigBackingStore`, and `ClusterConfigState` classes
   - A new method in the `Admin` API for fencing out transactional producers by ID, which is done with changes to the `Admin` interface (unsurprisingly) and the `KafkaAdminClient` class
   - Support for per-connector offsets topics, which touches on the `Worker`, `OffsetStorageReaderImpl`, and `OffsetStorageWriter` classes
   - A few new `SourceConnector` methods for communicating support for exactly-once guarantees and connector-defined transactions; these take place in the `SourceConnector` class (also unsurprisingly) and the `AbstractHerder` class
   
   Existing unit tests are expanded where applicable, and new ones have been introduced where necessary.
   
   Eight new integration tests are added, which cover scenarios including preflight validation checks, all three types of transaction boundary, graceful recovery of the leader when fenced out from the config topic by another worker, ensuring that the correct number of task producers are fenced out across generations, accurate reporting of failure to bring up tasks when fencing does not succeed (includes an ACL-secured embedded Kafka cluster to simulate one of the most likely potential causes of this issue--insufficient permissions on the targeted Kafka cluster), and the use of a custom offsets topic.
   
   Many but not all existing system tests are modified to add cases involving exactly-once source support, which helps give us reasonable confidence that the feature is agnostic with regards to rebalance protocol. A new test is added that is based on the existing bounce test, but with no sink connector and with stricter expectations for delivery guarantees (no duplicates are permitted).
   
   ### 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.

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



[GitHub] [kafka] C0urante commented on pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-995213197


   @rhauch gentle reminder, it's been another month and we're still waiting on that next pass.


-- 
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 #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-937394815


   @rhauch Is that second pass coming any time soon? It'd be nice if we could get this merged in time for the upcoming 3.1 release.
   
   I plan to address the existing comments next week.


-- 
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 change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r665526680



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java
##########
@@ -337,15 +353,29 @@ private void readToLogEnd() {
                 } else {
                     log.trace("Behind end offset {} for {}; last-read offset is {}",
                             endOffset, topicPartition, lastConsumedOffset);
-                    poll(Integer.MAX_VALUE);
+                    if (topicContainsTransactions) {
+                        // The consumer won't return from its poll method if a transaction is aborted, even though
+                        // its position will advance. So, we poll for at most one second, then give ourselves another
+                        // chance to check whether we've reached the end of the topic.
+                        poll(1000);

Review comment:
       I've filed https://issues.apache.org/jira/browse/KAFKA-12980 and plan to begin work on it. Hopefully we can leverage that improvement here to avoid the `poll(1000)` hack.




-- 
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 edited a comment on pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante edited a comment on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-1042573696


   @mimaison @rhauch I've broken things down into smaller PRs.
   
   These can all be reviewed immediately:
   - https://github.com/apache/kafka/pull/11777 (cc @hachikuji in case you want to take a look)
   - https://github.com/apache/kafka/pull/11772
   - https://github.com/apache/kafka/pull/11773
   - https://github.com/apache/kafka/pull/11774
   
   These have requirements on other PRs (either ones outlined above, or ones also in this list):
   - https://github.com/apache/kafka/pull/11775
   - https://github.com/apache/kafka/pull/11776
   - https://github.com/apache/kafka/pull/11778
   - https://github.com/apache/kafka/pull/11779
   - https://github.com/apache/kafka/pull/11780
   - https://github.com/apache/kafka/pull/11781
   
   And these should be reviewed last as they bring all of the above changes together and add integration and system tests for them, respectively:
   - https://github.com/apache/kafka/pull/11782
   - https://github.com/apache/kafka/pull/11783
   
   It was less work than I expected to get them all to be buildable, so (with a few placeholder measures here and there), it should be possible to get a green Jenkins build on all of them.
   
   If we don't feel comfortable merging these directly to trunk (in case we might have to revert before the upcoming 3.2 release... sigh...), could someone set up a `kafka-10000` branch on this repository so that we can get CI builds running against 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] galeaspablo commented on pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
galeaspablo commented on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-1003836408


   Hey @rhauch, @C0urante, I've kept an eye on this PR, its associated KIP, for a while. Is there anything I can do to help? Happy to take guidance to split the PR if that would make reviewing easier?
   
   P.S. I appreciate I haven't committed to this repository, but all my Kafka work was internal while I was at AWS. 


-- 
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 change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r786341813



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java
##########
@@ -0,0 +1,643 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.connect.runtime;
+
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.metrics.stats.Avg;
+import org.apache.kafka.common.metrics.stats.CumulativeSum;
+import org.apache.kafka.common.metrics.stats.Max;
+import org.apache.kafka.common.metrics.stats.Rate;
+import org.apache.kafka.common.metrics.stats.Value;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.header.Headers;
+import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator;
+import org.apache.kafka.connect.runtime.errors.Stage;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.kafka.connect.source.SourceTaskContext;
+import org.apache.kafka.connect.storage.CloseableOffsetStorageReader;
+import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore;
+import org.apache.kafka.connect.storage.Converter;
+import org.apache.kafka.connect.storage.HeaderConverter;
+import org.apache.kafka.connect.storage.OffsetStorageWriter;
+import org.apache.kafka.connect.storage.StatusBackingStore;
+import org.apache.kafka.connect.util.ConnectUtils;
+import org.apache.kafka.connect.util.ConnectorTaskId;
+import org.apache.kafka.connect.util.TopicAdmin;
+import org.apache.kafka.connect.util.TopicCreation;
+import org.apache.kafka.connect.util.TopicCreationGroup;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_TRACKING_ENABLE_CONFIG;
+
+/**
+ * WorkerTask that contains shared logic for running source tasks with either standard or exactly-once delivery guarantees.
+ */
+public abstract class AbstractWorkerSourceTask extends WorkerTask {
+    private static final Logger log = LoggerFactory.getLogger(AbstractWorkerSourceTask.class);
+
+    private static final long SEND_FAILED_BACKOFF_MS = 100;
+
+    /**
+     * Hook to define custom startup behavior before the calling {@link SourceTask#initialize(SourceTaskContext)}
+     * and {@link SourceTask#start(Map)}.
+     */
+    protected abstract void prepareToInitializeTask();
+
+    /**
+     * Hook to define custom initialization behavior when preparing to begin the poll-convert-send loop for the first time,
+     * or when re-entering the loop after being paused.
+     */
+    protected abstract void prepareToEnterSendLoop();
+
+    /**
+     * Hook to define custom periodic checks for health, metrics, etc. Called whenever {@link SourceTask#poll()} is about to be invoked.
+     */
+    protected abstract void prepareToPollTask();
+
+    /**
+     * Invoked when a record provided by the task has been filtered out by a transform or the converter, will be
+     * discarded due to failures during transformation or conversion.
+     * @param record the pre-transform record that has been dropped; never null.
+     */
+    protected abstract void recordDropped(SourceRecord record);
+
+    /**
+     * Invoked when a record is about to be dispatched to the producer. May be invoked multiple times for the same
+     * record if retriable errors are encountered.
+     * @param sourceRecord the pre-transform {@link SourceRecord} provided by the source task; never null.
+     * @param producerRecord the {@link ProducerRecord} produced by transforming and converting the
+     * {@code sourceRecord}; never null;
+     * @param newRecord whether this is the first time the record is being dispatched to the producer.
+     */
+    protected abstract void prepareToSendRecord(SourceRecord sourceRecord, ProducerRecord<byte[], byte[]> producerRecord, boolean newRecord);
+
+    /**
+     * Invoked when a record has been transformed, converted, and dispatched to the producer successfully via
+     * {@link Producer#send}. Does not guarantee that the record has been sent to Kafka or ack'd by the required number
+     * of brokers, but does guarantee that it will never be re-processed.
+     * @param record the pre-transform {@link SourceRecord} that was successfully dispatched to the producer; never null.
+     */
+    protected abstract void recordDispatched(SourceRecord record);
+
+    /**
+     * Invoked when an entire batch of records returned from {@link SourceTask#poll} has been transformed, converted,
+     * and either discarded due to transform/conversion errors, filtered by a transform, or dispatched to the producer
+     * successfully via {@link Producer#send}. Does not guarantee that the records have been sent to Kafka or ack'd by the
+     * required number of brokers, but does guarantee that none of the records in the batch will ever be re-processed during
+     * the lifetime of this task. At most one record batch is polled from the task in between calls to this method.
+     */
+    protected abstract void batchDispatched();
+
+    /**
+     * Invoked when a record has been sent and ack'd by the Kafka cluster. Note that this method may be invoked
+     *  concurrently and should therefore be made thread-safe.
+     * @param sourceRecord  the pre-transform {@link SourceRecord} that was successfully sent to Kafka; never null.
+     * @param producerRecord the {@link ProducerRecord} produced by transforming and converting the
+     * {@code sourceRecord}; never null;
+     * @param recordMetadata the {@link RecordMetadata} for the corresponding producer record; never null.
+     */
+    protected abstract void recordSent(SourceRecord sourceRecord, ProducerRecord<byte[], byte[]> producerRecord, RecordMetadata recordMetadata);
+
+    /**
+     * Invoked when no more records will be polled from the task or dispatched to the producer. Should attempt to
+     * commit the offsets for any outstanding records when possible.
+     * @param failed whether the task is undergoing a healthy or an unhealthy shutdown
+     */
+    protected abstract void finalOffsetCommit(boolean failed);
+
+
+    protected final WorkerConfig workerConfig;
+    protected final WorkerSourceTaskContext sourceTaskContext;
+    protected final OffsetStorageWriter offsetWriter;
+    protected final Producer<byte[], byte[]> producer;
+
+    private final SourceTask task;
+    private final Converter keyConverter;
+    private final Converter valueConverter;
+    private final HeaderConverter headerConverter;
+    private final TransformationChain<SourceRecord> transformationChain;
+    private final TopicAdmin admin;
+    private final CloseableOffsetStorageReader offsetReader;
+    private final ConnectorOffsetBackingStore offsetStore;
+    private final AtomicReference<Exception> producerSendException;
+    private final SourceTaskMetricsGroup sourceTaskMetricsGroup;
+    private final CountDownLatch stopRequestedLatch;
+    private final boolean topicTrackingEnabled;
+    private final TopicCreation topicCreation;
+    private final Executor closeExecutor;
+
+    // Visible for testing
+    boolean lastSendFailed;
+    // Visible for testing
+    List<SourceRecord> toSend;
+    protected Map<String, String> taskConfig;
+    protected boolean started = false;
+
+    protected AbstractWorkerSourceTask(ConnectorTaskId id,
+                                       SourceTask task,
+                                       TaskStatus.Listener statusListener,
+                                       TargetState initialState,
+                                       Converter keyConverter,
+                                       Converter valueConverter,
+                                       HeaderConverter headerConverter,
+                                       TransformationChain<SourceRecord> transformationChain,
+                                       WorkerSourceTaskContext sourceTaskContext,
+                                       Producer<byte[], byte[]> producer,
+                                       TopicAdmin admin,
+                                       Map<String, TopicCreationGroup> topicGroups,
+                                       CloseableOffsetStorageReader offsetReader,
+                                       OffsetStorageWriter offsetWriter,
+                                       ConnectorOffsetBackingStore offsetStore,
+                                       WorkerConfig workerConfig,
+                                       ConnectMetrics connectMetrics,
+                                       ClassLoader loader,
+                                       Time time,
+                                       RetryWithToleranceOperator retryWithToleranceOperator,
+                                       StatusBackingStore statusBackingStore,
+                                       Executor closeExecutor) {
+
+        super(id, statusListener, initialState, loader, connectMetrics,
+                retryWithToleranceOperator, time, statusBackingStore);
+
+        this.workerConfig = workerConfig;
+        this.task = task;
+        this.keyConverter = keyConverter;
+        this.valueConverter = valueConverter;
+        this.headerConverter = headerConverter;
+        this.transformationChain = transformationChain;
+        this.producer = producer;
+        this.admin = admin;
+        this.offsetReader = offsetReader;
+        this.offsetWriter = offsetWriter;
+        this.offsetStore = offsetStore;
+        this.closeExecutor = closeExecutor;
+        this.sourceTaskContext = sourceTaskContext;
+
+        this.producerSendException = new AtomicReference<>();
+        this.stopRequestedLatch = new CountDownLatch(1);
+        this.sourceTaskMetricsGroup = new SourceTaskMetricsGroup(id, connectMetrics);
+        this.topicTrackingEnabled = workerConfig.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG);
+        this.topicCreation = TopicCreation.newTopicCreation(workerConfig, topicGroups);
+    }
+
+    @Override
+    public void initialize(TaskConfig taskConfig) {
+        try {
+            this.taskConfig = taskConfig.originalsStrings();
+        } catch (Throwable t) {
+            log.error("{} Task failed initialization and will not be started.", this, t);
+            onFailure(t);
+        }
+    }
+
+    @Override
+    protected void initializeAndStart() {
+        prepareToInitializeTask();
+        // If we try to start the task at all by invoking initialize, then count this as
+        // "started" and expect a subsequent call to the task's stop() method
+        // to properly clean up any resources allocated by its initialize() or
+        // start() methods. If the task throws an exception during stop(),
+        // the worst thing that happens is another exception gets logged for an already-
+        // failed task
+        started = true;
+        task.initialize(sourceTaskContext);
+        task.start(taskConfig);
+        log.info("{} Source task finished initialization and start", this);
+    }
+
+    @Override
+    public void cancel() {
+        super.cancel();
+        // Preemptively close the offset reader in case the task is blocked on an offset read.
+        offsetReader.close();
+        // We proactively close the producer here as the main work thread for the task may
+        // be blocked indefinitely in a call to Producer::send if automatic topic creation is
+        // not enabled on either the connector or the Kafka cluster. Closing the producer should
+        // unblock it in that case and allow shutdown to proceed normally.
+        // With a duration of 0, the producer's own shutdown logic should be fairly quick,
+        // but closing user-pluggable classes like interceptors may lag indefinitely. So, we
+        // call close on a separate thread in order to avoid blocking the herder's tick thread.
+        closeExecutor.execute(() -> closeProducer(Duration.ZERO));
+    }
+
+    @Override
+    public void stop() {
+        super.stop();
+        stopRequestedLatch.countDown();
+    }
+
+    @Override
+    public void removeMetrics() {
+        Utils.closeQuietly(sourceTaskMetricsGroup::close, "source task metrics tracker");
+        super.removeMetrics();
+    }
+
+    @Override
+    protected void close() {
+        if (started) {
+            Utils.closeQuietly(task::stop, "source task");
+        }
+
+        closeProducer(Duration.ofSeconds(30));
+
+        if (admin != null) {
+            Utils.closeQuietly(() -> admin.close(Duration.ofSeconds(30)), "source task admin");
+        }
+        Utils.closeQuietly(transformationChain, "transformation chain");
+        Utils.closeQuietly(retryWithToleranceOperator, "retry operator");
+        Utils.closeQuietly(offsetReader, "offset reader");
+        Utils.closeQuietly(offsetStore::stop, "offset backing store");
+    }
+
+    private void closeProducer(Duration duration) {
+        if (producer != null) {
+            Utils.closeQuietly(() -> producer.close(duration), "source task producer");
+        }
+    }
+
+    @Override
+    public void execute() {
+        try {
+            prepareToEnterSendLoop();
+            while (!isStopping()) {
+                if (shouldPause()) {
+                    onPause();
+                    if (awaitUnpause()) {
+                        onResume();
+                        prepareToEnterSendLoop();
+                    }
+                    continue;
+                }
+
+                if (toSend == null) {
+                    prepareToPollTask();
+
+                    log.trace("{} Nothing to send to Kafka. Polling source for additional records", this);
+                    long start = time.milliseconds();
+                    toSend = poll();
+                    if (toSend != null) {
+                        recordPollReturned(toSend.size(), time.milliseconds() - start);
+                    }
+                }
+                if (toSend == null)
+                    continue;
+                log.trace("{} About to send {} records to Kafka", this, toSend.size());
+                if (sendRecords()) {
+                    batchDispatched();
+                } else {
+                    stopRequestedLatch.await(SEND_FAILED_BACKOFF_MS, TimeUnit.MILLISECONDS);
+                }
+            }
+        } catch (InterruptedException e) {
+            // Ignore and allow to exit.
+        } catch (RuntimeException e) {
+            try {
+                finalOffsetCommit(true);
+            } catch (Exception offsetException) {
+                log.error("Failed to commit offsets for already-failing task", offsetException);
+            }
+            throw e;
+        }
+        finalOffsetCommit(false);
+    }
+
+    /**
+     * Try to send a batch of records. If a send fails and is retriable, this saves the remainder of the batch so it can
+     * be retried after backing off. If a send fails and is not retriable, this will throw a ConnectException.
+     * @return true if all messages were sent, false if some need to be retried
+     */
+    // Visible for testing
+    boolean sendRecords() {
+        int processed = 0;
+        recordBatch(toSend.size());
+        final SourceRecordWriteCounter counter =
+                toSend.size() > 0 ? new SourceRecordWriteCounter(toSend.size(), sourceTaskMetricsGroup) : null;
+        for (final SourceRecord preTransformRecord : toSend) {
+            retryWithToleranceOperator.sourceRecord(preTransformRecord);
+            final SourceRecord record = transformationChain.apply(preTransformRecord);
+            final ProducerRecord<byte[], byte[]> producerRecord = convertTransformedRecord(record);
+            if (producerRecord == null || retryWithToleranceOperator.failed()) {
+                counter.skipRecord();
+                recordDropped(preTransformRecord);
+                continue;
+            }
+
+            log.trace("{} Appending record to the topic {} with key {}, value {}", this, record.topic(), record.key(), record.value());
+            prepareToSendRecord(preTransformRecord, producerRecord, !lastSendFailed);
+            try {
+                final String topic = producerRecord.topic();
+                maybeCreateTopic(topic);
+                producer.send(
+                    producerRecord,
+                    (recordMetadata, e) -> {
+                        if (e != null) {
+                            log.debug("{} failed to send record to {}: ", AbstractWorkerSourceTask.this, topic, e);
+                            log.trace("{} Failed record: {}", AbstractWorkerSourceTask.this, preTransformRecord);
+                            if (isFatalProducerSendException(e)) {
+                                producerSendException.compareAndSet(null, e);
+                            }

Review comment:
       This will have to be tweaked slightly in order to invoke `SourceTask::commitRecord` and correctly track the number of in-flight records for metrics.




-- 
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 change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r786311430



##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java
##########
@@ -28,4 +30,31 @@
     protected SourceConnectorContext context() {
         return (SourceConnectorContext) context;
     }
+
+    /**
+     * Signals whether the connector supports exactly-once delivery guarantees with a proposed configuration.
+     * Developers can assume that worker-level exactly-once support is enabled when this method is invoked.
+     * The default implementation will return {@code null}.
+     * @param connectorConfig the configuration that will be used for the connector.
+     * @return {@link ExactlyOnceSupport#SUPPORTED} if the connector can provide exactly-once support,
+     * and {@link ExactlyOnceSupport#UNSUPPORTED} if it cannot. If {@code null}, it is assumed that the
+     * connector cannot.
+     */
+    public ExactlyOnceSupport exactlyOnceSupport(Map<String, String> connectorConfig) {
+        return null;
+    }
+
+    /**
+     * Signals whether the connector can define its own transaction boundaries with the proposed
+     * configuration. Developers must override this method if they wish to add connector-defined
+     * transaction boundary support; if they do not, users will be unable to create instances of
+     * this connector that use connector-defined transaction boundaries. The default implementation
+     * will return {@code UNSUPPORTED}.

Review comment:
       LGTM. I added a small note on not returning `null` with the same language as in `SourceConnector::exactlyOnceSupport` but otherwise added this verbatim. LMKWYT

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java
##########
@@ -20,13 +20,45 @@
 import org.apache.kafka.clients.producer.RecordMetadata;
 
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * SourceTask is a Task that pulls records from another system for storage in Kafka.
  */
 public abstract class SourceTask implements Task {
 
+    /**
+     * <p>
+     * The configuration key that determines how source tasks will define transaction boundaries
+     * when exactly-once support is enabled.
+     * </p>
+     */
+    public static final String TRANSACTION_BOUNDARY_CONFIG = "transaction.boundary";
+
+    public enum TransactionBoundary {

Review comment:
       Just making sure: is there anything left to address for this comment?

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java
##########
@@ -20,13 +20,45 @@
 import org.apache.kafka.clients.producer.RecordMetadata;
 
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * SourceTask is a Task that pulls records from another system for storage in Kafka.
  */
 public abstract class SourceTask implements Task {
 
+    /**
+     * <p>
+     * The configuration key that determines how source tasks will define transaction boundaries
+     * when exactly-once support is enabled.
+     * </p>
+     */
+    public static final String TRANSACTION_BOUNDARY_CONFIG = "transaction.boundary";
+
+    public enum TransactionBoundary {
+        POLL,
+        INTERVAL,
+        CONNECTOR;
+
+        public static final TransactionBoundary DEFAULT = POLL;
+
+        public static List<String> options() {

Review comment:
       Neither of these patterns play nicely with the validators offered by `ConfigDef`, though. Both `ValidString` and `CaseInsensitiveValidString` require an array (or really, a varargs list) of strings in their constructors, which prevents us from using `Enum::values` directly to create one of these validators.
   
   I've removed `options()` (from both `TransactionBoundary` and `ExactlyOnceSupportLevel`) in favor of a reusable `Utils::enumOptions` method, which returns a `String[]` containing the names (retrieved via `toString`) of each value for an enum class. This prevents us from having to duplicate this logic and allows us to use it without adding new methods to public API.
   
   The actual implementations for both enums now follow the `ConnectorType` pattern, with an overridden `toString` method that returns the lowercase name of the enum and a static `fromProperty` method (which mirrors the `ConnectorType::fromValue` method) that parses a string case-insensitively into a value for the enum. I opted for a different method name for the latter since the values given to `fromProperty` should always be user-supplied values and the motivation for this separate method (instead of just invoking `Enum::valueOf` directly) is directly related to parsing user-supplied values.
   
   We could also add a general-purpose `EnumValidator` class to `ConfigDef` (for public use) or in a section of the Connect code base reserved for private API, but that would still leave us with the trouble of having to re-implement logic to follow the phrase "Permitted values are" in property docstrings, and it's probably best to hold off on implementing a public-facing validator like that until its exact behavior can be agreed on with a KIP.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceTaskContext.java
##########
@@ -38,4 +38,29 @@
      * Get the OffsetStorageReader for this SourceTask.
      */
     OffsetStorageReader offsetStorageReader();
+
+    /**
+     * Get a {@link TransactionContext} that can be used to define producer transaction boundaries
+     * when exactly-once support is enabled for the connector.
+     *
+     * <p>This method was added in Apache Kafka 3.0. Source tasks that use this method but want to
+     * maintain backward compatibility so they can also be deployed to older Connect runtimes
+     * should guard the call to this method with a try-catch block, since calling this method will result in a
+     * {@link NoSuchMethodException} or {@link NoClassDefFoundError} when the source connector is deployed to
+     * Connect runtimes older than Kafka 3.0. For example:
+     * <pre>
+     *     TransactionContext transactionContext;
+     *     try {
+     *         transactionContext = context.transactionContext();
+     *     } catch (NoSuchMethodError | NoClassDefFoundError e) {
+     *         transactionContext = null;
+     *     }
+     * </pre>
+     *
+     * @return the transaction context, or null if the connector was not configured to specify transaction boundaries
+     * @since 3.0

Review comment:
       I believe 3.2 is the correct version now, right? Will update to that.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java
##########
@@ -28,4 +30,39 @@
     protected SourceConnectorContext context() {
         return (SourceConnectorContext) context;
     }
+
+    /**
+     * Signals whether the connector supports exactly-once delivery guarantees with a proposed configuration.
+     * Developers can assume that worker-level exactly-once support is enabled when this method is invoked.
+     * For backwards compatibility, the default implementation will return {@code null}, but connector developers are

Review comment:
       Ack, done. (And added a few more paragraph breaks in other places too).

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/TransactionContext.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.source;
+
+/**
+ * Provided to source tasks to allow them to define their own producer transaction boundaries when
+ * exactly-once support is enabled.
+ */
+public interface TransactionContext {

Review comment:
       The behavior specified in the KIP is for `SourceTaskContext::transactionContext` to return "the transaction context, or null if the connector was not configured to specify transaction boundaries" (see the Javadocs for that method in the code snippets in [the relevant KIP section](https://cwiki.apache.org/confluence/display/KAFKA/KIP-618%3A+Exactly-Once+Support+for+Source+Connectors#KIP618:ExactlyOnceSupportforSourceConnectors-ConnectorAPIexpansions)).
   
   This is implemented in the [`ExactlyOnceWorkerSourceTask`](https://github.com/C0urante/kafka/blob/c06207d64a76286316b862e16143f909181501c3/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java#L113-L117) class and should prevent connectors from invoking `TransactionContext` methods and accumulating records when the user has not configured the connector to define its own transaction boundaries.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java
##########
@@ -192,6 +199,44 @@
     public static final String INTER_WORKER_VERIFICATION_ALGORITHMS_DOC = "A list of permitted algorithms for verifying internal requests";
     public static final List<String> INTER_WORKER_VERIFICATION_ALGORITHMS_DEFAULT = Collections.singletonList(INTER_WORKER_SIGNATURE_ALGORITHM_DEFAULT);
 
+    private enum ExactlyOnceSourceSupport {
+        DISABLED(false),
+        PREPARING(true),
+        ENABLED(true);
+
+        public final boolean usesTransactionalLeader;
+
+        ExactlyOnceSourceSupport(boolean usesTransactionalLeader) {
+            this.usesTransactionalLeader = usesTransactionalLeader;
+        }
+
+        public static List<String> options() {
+            return Stream.of(values()).map(ExactlyOnceSourceSupport::toString).collect(Collectors.toList());
+        }
+
+        public static ExactlyOnceSourceSupport fromProperty(String property) {
+            return ExactlyOnceSourceSupport.valueOf(property.toUpperCase(Locale.ROOT));
+        }
+
+        @Override
+        public String toString() {
+            return name().toLowerCase(Locale.ROOT);
+        }
+    }
+
+    public static final String EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG = "exactly.once.source.support";
+    public static final String EXACTLY_ONCE_SOURCE_SUPPORT_DOC = "Whether to enable exactly-once support for source connectors in the cluster "
+            + "by writing source records and their offsets in a Kafka transaction, and by proactively fencing out old task generations before bringing up new ones. "

Review comment:
       Ack, done.

##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedConfigTest.java
##########
@@ -294,4 +298,41 @@ public void shouldRemoveCompactionFromStatusTopicSettings() {
         assertEquals(expectedTopicSettings, actual);
         assertNotEquals(topicSettings, actual);
     }
+
+    @Test
+    public void shouldIdentifyNeedForTransactionalLeader() {
+        Map<String, String> workerProps = configs();
+
+        workerProps.put(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG, "disabled");
+        assertFalse(new DistributedConfig(workerProps).transactionalLeaderEnabled());
+
+        workerProps.put(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG, "preparing");
+        assertTrue(new DistributedConfig(workerProps).transactionalLeaderEnabled());
+
+        workerProps.put(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG, "enabled");
+        assertTrue(new DistributedConfig(workerProps).transactionalLeaderEnabled());
+    }
+
+    @Test
+    public void shouldConstructExpectedTransactionalId() {
+        Map<String, String> workerProps = configs();
+
+        workerProps.put(GROUP_ID_CONFIG, "why did i stay up all night writing unit tests");
+        assertEquals(
+                "connect-cluster-why did i stay up all night writing unit tests",
+                new DistributedConfig(workerProps).transactionalProducerId()
+        );
+
+        workerProps.put(GROUP_ID_CONFIG, "connect-cluster");
+        assertEquals(
+                "connect-cluster-connect-cluster",
+                new DistributedConfig(workerProps).transactionalProducerId()
+        );
+
+        workerProps.put(GROUP_ID_CONFIG, "\u2603");
+        assertEquals(
+                "connect-cluster-\u2603",
+                new DistributedConfig(workerProps).transactionalProducerId()
+        );
+    }

Review comment:
       Ack, done. The tests are located elsewhere since the logic is implemented in a different class, but there are new cases for this in the `KafkaConfigBackingStoreTest`, `KafkaOffsetBackingStoreTest`, and `WorkerTest` test suites.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java
##########
@@ -125,28 +130,28 @@ public KafkaBasedLog(String topic,
      * Create a new KafkaBasedLog object. This does not start reading the log and writing is not permitted until
      * {@link #start()} is invoked.
      *
-     * @param topic              the topic to treat as a log
-     * @param producerConfigs    configuration options to use when creating the internal producer. At a minimum this must
+     * @param topic                     the topic to treat as a log
+     * @param producerConfigs           configuration options to use when creating the internal producer. At a minimum this must
      *                           contain compatible serializer settings for the generic types used on this class. Some
      *                           setting, such as the number of acks, will be overridden to ensure correct behavior of this
      *                           class.
-     * @param consumerConfigs    configuration options to use when creating the internal consumer. At a minimum this must
+     * @param consumerConfigs           configuration options to use when creating the internal consumer. At a minimum this must
      *                           contain compatible serializer settings for the generic types used on this class. Some
      *                           setting, such as the auto offset reset policy, will be overridden to ensure correct
      *                           behavior of this class.
-     * @param topicAdminSupplier supplier function for an admin client, the lifecycle of which is expected to be controlled
+     * @param topicAdminSupplier        supplier function for an admin client, the lifecycle of which is expected to be controlled
      *                           by the calling component; may not be null
-     * @param consumedCallback   callback to invoke for each {@link ConsumerRecord} consumed when tailing the log
-     * @param time               Time interface
-     * @param initializer        the function that should be run when this log is {@link #start() started}; may be null
+     * @param consumedCallback          callback to invoke for each {@link ConsumerRecord} consumed when tailing the log
+     * @param time                      Time interface
+     * @param initializer               the function that should be run when this log is {@link #start() started}; may be null
      */
     public KafkaBasedLog(String topic,
-            Map<String, Object> producerConfigs,
-            Map<String, Object> consumerConfigs,
-            Supplier<TopicAdmin> topicAdminSupplier,
-            Callback<ConsumerRecord<K, V>> consumedCallback,
-            Time time,
-            java.util.function.Consumer<TopicAdmin> initializer) {
+                         Map<String, Object> producerConfigs,
+                         Map<String, Object> consumerConfigs,
+                         Supplier<TopicAdmin> topicAdminSupplier,
+                         Callback<ConsumerRecord<K, V>> consumedCallback,
+                         Time time,
+                         java.util.function.Consumer<TopicAdmin> initializer) {

Review comment:
       Apologies, will revert. Believe this was left in from an older approach that involved more invasive changes to this class.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java
##########
@@ -365,6 +413,10 @@ private void readToLogEnd() {
                 // This may happen with really old brokers that don't support the auto topic creation
                 // field in metadata requests
                 log.debug("Reading to end of log offsets with consumer since admin client is unsupported: {}", e.getMessage());
+                if (requireAdminForOffsets) {
+                    // Should be handled by the caller during log startup
+                    throw e;
+                }

Review comment:
       Ack on both points; done 👍 

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -264,12 +279,62 @@ public void startConnector(
 
                 log.info("Creating connector {} of type {}", connName, connClass);
                 final Connector connector = plugins.newConnector(connClass);
-                final ConnectorConfig connConfig = ConnectUtils.isSinkConnector(connector)
-                        ? new SinkConnectorConfig(plugins, connProps)
-                        : new SourceConnectorConfig(plugins, connProps, config.topicCreationEnable());
+                final ConnectorConfig connConfig;
+                final CloseableOffsetStorageReader offsetReader;
+                if (ConnectUtils.isSinkConnector(connector)) {
+                    connConfig = new SinkConnectorConfig(plugins, connProps);
+                    offsetReader = null;

Review comment:
       Ah, good catch! I think this is one more place where `Utils::closeQuietly` comes in handy since it does the null check for us, won't interrupt shutdown if an exception is thrown, and further standardizes the cleanup logic for the code base.
   
   Alternatively, we could retain the current behavior, but I don't see much use in constructing an offset reader for sink connectors that won't be used now and is unlikely to be used in the future.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConnectorOffsetBackingStore.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.storage;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.connect.runtime.WorkerConfig;
+import org.apache.kafka.connect.util.Callback;
+import org.apache.kafka.connect.util.LoggingContext;
+import org.apache.kafka.connect.util.TopicAdmin;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+public class ConnectorOffsetBackingStore implements OffsetBackingStore {

Review comment:
       Ack, done. Added docs to the (now-three) static factory methods and to each of the overridden `OffsetBackingStore` methods.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConnectorOffsetBackingStore.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.storage;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.connect.runtime.WorkerConfig;
+import org.apache.kafka.connect.util.Callback;
+import org.apache.kafka.connect.util.LoggingContext;
+import org.apache.kafka.connect.util.TopicAdmin;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+public class ConnectorOffsetBackingStore implements OffsetBackingStore {
+
+    private static final Logger log = LoggerFactory.getLogger(ConnectorOffsetBackingStore.class);
+
+    private final Time time;
+    private final Supplier<LoggingContext> loggingContext;
+    private final String primaryOffsetsTopic;
+    private final OffsetBackingStore workerStore;
+    private final Optional<OffsetBackingStore> connectorStore;
+    private final Optional<TopicAdmin> connectorStoreAdmin;
+
+    public static ConnectorOffsetBackingStore withConnectorOffsetStore(

Review comment:
       Ack, done.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectUtils.java
##########
@@ -72,6 +74,45 @@ static String lookupKafkaClusterId(Admin adminClient) {
         }
     }
 
+    /**
+     * Log a warning when the user attempts to override a property that cannot be overridden.
+     * @param props the configuration properties provided by the user
+     * @param key the name of the property to check on
+     * @param expectedValue the expected value for the property
+     * @param justification the reason the property cannot be overridden.
+     *                      Will follow the phrase "The value... for the... property will be ignored as it cannot be overridden ".
+     *                      For example, one might supply the message "in connectors with the DLQ feature enabled" for this parameter.
+     * @param caseSensitive whether the value should match case-insensitively
+     */
+    public static void warnOnOverriddenProperty(
+            Map<String, ?> props,
+            String key,
+            String expectedValue,
+            String justification,
+            boolean caseSensitive) {
+        overriddenPropertyWarning(props, key, expectedValue, justification, caseSensitive).ifPresent(log::warn);
+    }
+
+    // Visible for testing
+    static Optional<String> overriddenPropertyWarning(
+            Map<String, ?> props,
+            String key,
+            String expectedValue,
+            String justification,
+            boolean caseSensitive) {
+        Predicate<String> matchesExpectedValue = caseSensitive ? expectedValue::equals : expectedValue::equalsIgnoreCase;
+        String value = Optional.ofNullable(props.get(key)).map(Object::toString).orElse(null);
+        if (value != null && !matchesExpectedValue.test(value)) {
+            return Optional.of(String.format(
+                    "The value '%s' for the '%s' property will be ignored as it cannot be overridden%s. "
+                            + "The value '%s' will be used instead.",

Review comment:
       I suppose not! There are two classes that currently leverage this utility method: `Worker` and `DistributedConfig`.
   
   In the former, each invocation of `warnOnOverriddenProperty` is followed closely (if not immediately) by a line that overrides any user-supplied values that don't match the expected value.
   
   However, in `DistributedConfig`, warnings are emitted for values that are ignored and overridden when constructing Kafka clients for some but not all `KafkaBasedLog` instances: specifically, the ones used for the config and offset backing stores. It's possible that a user might specify `enable.idempotence=false` in their worker config with the intention of disabling idempotent writes for the producers used for the status backing store (since the default for this property was updated from `false` to `true` in [KIP-679](https://cwiki.apache.org/confluence/display/KAFKA/KIP-679%3A+Producer+will+enable+the+strongest+delivery+guarantee+by+default)). With the current behavior for this PR, that change would have an effect (and the values logged for the producer for the status backing store would confirm this), but the warning message would still be logged, which might lead to some confusion.
   
   At the bare minimum we can and should update this message to be more specific about where exactly these values will be ignored, but in addition to that, I've taken a stab at restructuring the reusable utility function to actually perform the override in addition to just warning about it, like you suggested in a comment below. This makes it harder to log warnings about overrides without actually ensuring that the overrides have taken place.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -264,12 +279,62 @@ public void startConnector(
 
                 log.info("Creating connector {} of type {}", connName, connClass);
                 final Connector connector = plugins.newConnector(connClass);
-                final ConnectorConfig connConfig = ConnectUtils.isSinkConnector(connector)
-                        ? new SinkConnectorConfig(plugins, connProps)
-                        : new SourceConnectorConfig(plugins, connProps, config.topicCreationEnable());
+                final ConnectorConfig connConfig;
+                final CloseableOffsetStorageReader offsetReader;
+                if (ConnectUtils.isSinkConnector(connector)) {
+                    connConfig = new SinkConnectorConfig(plugins, connProps);
+                    offsetReader = null;
+                } else {
+                    SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connProps, config.topicCreationEnable());
+                    connConfig = sourceConfig;
+
+                    String connectorOffsetsTopic = null;
+                    if (sourceConfig.offsetsTopic() != null) {
+                        connectorOffsetsTopic = sourceConfig.offsetsTopic();
+                    } else if (config.exactlyOnceSourceEnabled()) {
+                        connectorOffsetsTopic = config.offsetsTopic();
+                    }
+

Review comment:
       Ack, done.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -476,22 +541,95 @@ public boolean isRunning(String connName) {
     }
 
     /**
-     * Start a task managed by this worker.
+     * Start a sink task managed by this worker.
+     *
+     * @param id the task ID.
+     * @param configState the most recent {@link ClusterConfigState} known to the worker
+     * @param connProps the connector properties.
+     * @param taskProps the tasks properties.
+     * @param statusListener a listener for the runtime status transitions of the task.
+     * @param initialState the initial state of the connector.
+     * @return true if the task started successfully.
+     */
+    public boolean startSinkTask(
+            ConnectorTaskId id,
+            ClusterConfigState configState,
+            Map<String, String> connProps,
+            Map<String, String> taskProps,
+            TaskStatus.Listener statusListener,
+            TargetState initialState
+    ) {
+        return startTask(id, connProps, taskProps, statusListener,
+                new SinkTaskBuilder(id, configState, statusListener, initialState));
+    }
+
+    /**
+     * Start a source task managed by this worker.

Review comment:
       Ack, done.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -692,21 +823,22 @@ private WorkerTask buildWorkerTask(ClusterConfigState configState,
         ConnectUtils.addMetricsContextProperties(consumerProps, config, clusterId);
         // Connector-specified overrides
         Map<String, Object> consumerOverrides =
-            connectorClientConfigOverrides(id, connConfig, connectorClass, ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX,
+            connectorClientConfigOverrides(connName, connConfig, connectorClass, ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX,
                                            ConnectorType.SINK, ConnectorClientConfigRequest.ClientType.CONSUMER,
                                            connectorClientConfigOverridePolicy);
         consumerProps.putAll(consumerOverrides);
 
         return consumerProps;
     }
 
-    static Map<String, Object> adminConfigs(ConnectorTaskId id,
+    static Map<String, Object> adminConfigs(String connName,

Review comment:
       We have to be able to construct admin clients for `Connector` instances now, whereas before we only used them for `Task` instances.
   
   I do sympathize with the desire for task-specific tweaks, though. Right now the pattern is for the `adminConfigs` method to define a new parameter for anything that should be derived from the task ID/connector name and place the onus on the caller to derive that value (for example, this is done with the `defaultClientId` parameter). I think this works for now but one pattern we could consider is to add separate wrapper `adminConfigs` methods for clients constructed on behalf of connectors and on behalf of tasks, similar to the new wrapper methods I've added for (regular and exactly-once) offset consumers and exactly-once task producers.
   
   Thoughts?

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -980,6 +1119,329 @@ WorkerMetricsGroup workerMetricsGroup() {
         return workerMetricsGroup;
     }
 
+    abstract class TaskBuilder {
+
+        private final ConnectorTaskId id;
+        private final ClusterConfigState configState;
+        private final TaskStatus.Listener statusListener;
+        private final TargetState initialState;
+
+        private Task task;
+        private ConnectorConfig connectorConfig = null;
+        private Converter keyConverter = null;
+        private Converter valueConverter = null;
+        private HeaderConverter headerConverter = null;
+        private ClassLoader classLoader = null;
+
+        public TaskBuilder(ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState) {
+            this.id = id;
+            this.configState = configState;
+            this.statusListener = statusListener;
+            this.initialState = initialState;
+        }
+
+        public TaskBuilder withTask(Task task) {
+            this.task = task;
+            return this;
+        }
+
+        public TaskBuilder withConnectorConfig(ConnectorConfig connectorConfig) {
+            this.connectorConfig = connectorConfig;
+            return this;
+        }
+
+        public TaskBuilder withKeyConverter(Converter keyConverter) {
+            this.keyConverter = keyConverter;
+            return this;
+        }
+
+        public TaskBuilder withValueConverter(Converter valueConverter) {
+            this.valueConverter = valueConverter;
+            return this;
+        }
+
+        public TaskBuilder withHeaderConverter(HeaderConverter headerConverter) {
+            this.headerConverter = headerConverter;
+            return this;
+        }
+
+        public TaskBuilder withClassloader(ClassLoader classLoader) {
+            this.classLoader = classLoader;
+            return this;
+        }
+
+        public WorkerTask build() {
+            Objects.requireNonNull(task, "Task cannot be null");
+            Objects.requireNonNull(connectorConfig, "Connector config used by task cannot be null");
+            Objects.requireNonNull(keyConverter, "Key converter used by task cannot be null");
+            Objects.requireNonNull(valueConverter, "Value converter used by task cannot be null");
+            Objects.requireNonNull(headerConverter, "Header converter used by task cannot be null");
+            Objects.requireNonNull(classLoader, "Classloader used by task cannot be null");
+
+            ErrorHandlingMetrics errorHandlingMetrics = errorHandlingMetrics(id);
+            final Class<? extends Connector> connectorClass = plugins.connectorClass(
+                    connectorConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG));
+            RetryWithToleranceOperator retryWithToleranceOperator = new RetryWithToleranceOperator(connectorConfig.errorRetryTimeout(),
+                    connectorConfig.errorMaxDelayInMillis(), connectorConfig.errorToleranceType(), Time.SYSTEM);
+            retryWithToleranceOperator.metrics(errorHandlingMetrics);
+
+            return doBuild(task, id, configState, statusListener, initialState,
+                    connectorConfig, keyConverter, valueConverter, headerConverter, classLoader,
+                    errorHandlingMetrics, connectorClass, retryWithToleranceOperator);
+        }
+
+        abstract WorkerTask doBuild(Task task,
+                                    ConnectorTaskId id,
+                                    ClusterConfigState configState,
+                                    TaskStatus.Listener statusListener,
+                                    TargetState initialState,
+                                    ConnectorConfig connectorConfig,
+                                    Converter keyConverter,
+                                    Converter valueConverter,
+                                    HeaderConverter headerConverter,
+                                    ClassLoader classLoader,
+                                    ErrorHandlingMetrics errorHandlingMetrics,
+                                    Class<? extends Connector> connectorClass,
+                                    RetryWithToleranceOperator retryWithToleranceOperator);
+
+    }
+
+    class SinkTaskBuilder extends TaskBuilder {
+        public SinkTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            TransformationChain<SinkRecord> transformationChain = new TransformationChain<>(connectorConfig.<SinkRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+            SinkConnectorConfig sinkConfig = new SinkConnectorConfig(plugins, connectorConfig.originalsStrings());
+            retryWithToleranceOperator.reporters(sinkTaskReporters(id, sinkConfig, errorHandlingMetrics, connectorClass));
+            WorkerErrantRecordReporter workerErrantRecordReporter = createWorkerErrantRecordReporter(sinkConfig, retryWithToleranceOperator,
+                    keyConverter, valueConverter, headerConverter);
+
+            Map<String, Object> consumerProps = consumerConfigs(id.connector(),  "connector-consumer-" + id, config, connectorConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+            return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverter,
+                    valueConverter, headerConverter, transformationChain, consumer, classLoader, time,
+                    retryWithToleranceOperator, workerErrantRecordReporter, herder.statusBackingStore());
+        }
+    }
+
+    class SourceTaskBuilder extends TaskBuilder {
+        public SourceTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+            ConnectorOffsetBackingStore offsetStore = ConnectorOffsetBackingStore.withoutConnectorOffsetStore(
+                    () -> LoggingContext.forTask(id),
+                    globalOffsetBackingStore,
+                    config.offsetsTopic()
+            );
+            final TopicAdmin admin;
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (sourceConfig.offsetsTopic() != null || (config.topicCreationEnable() && sourceConfig.usesTopicCreation())) {

Review comment:
       I've extracted a lot of this logic into separate methods for testability and readability, but tried to stick with the general spirit of this comment while doing so and left a few comments and added some `final boolean` local variables in the newly-isolated methods.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -980,6 +1119,329 @@ WorkerMetricsGroup workerMetricsGroup() {
         return workerMetricsGroup;
     }
 
+    abstract class TaskBuilder {
+
+        private final ConnectorTaskId id;
+        private final ClusterConfigState configState;
+        private final TaskStatus.Listener statusListener;
+        private final TargetState initialState;
+
+        private Task task;
+        private ConnectorConfig connectorConfig = null;
+        private Converter keyConverter = null;
+        private Converter valueConverter = null;
+        private HeaderConverter headerConverter = null;
+        private ClassLoader classLoader = null;
+
+        public TaskBuilder(ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState) {
+            this.id = id;
+            this.configState = configState;
+            this.statusListener = statusListener;
+            this.initialState = initialState;
+        }
+
+        public TaskBuilder withTask(Task task) {
+            this.task = task;
+            return this;
+        }
+
+        public TaskBuilder withConnectorConfig(ConnectorConfig connectorConfig) {
+            this.connectorConfig = connectorConfig;
+            return this;
+        }
+
+        public TaskBuilder withKeyConverter(Converter keyConverter) {
+            this.keyConverter = keyConverter;
+            return this;
+        }
+
+        public TaskBuilder withValueConverter(Converter valueConverter) {
+            this.valueConverter = valueConverter;
+            return this;
+        }
+
+        public TaskBuilder withHeaderConverter(HeaderConverter headerConverter) {
+            this.headerConverter = headerConverter;
+            return this;
+        }
+
+        public TaskBuilder withClassloader(ClassLoader classLoader) {
+            this.classLoader = classLoader;
+            return this;
+        }
+
+        public WorkerTask build() {
+            Objects.requireNonNull(task, "Task cannot be null");
+            Objects.requireNonNull(connectorConfig, "Connector config used by task cannot be null");
+            Objects.requireNonNull(keyConverter, "Key converter used by task cannot be null");
+            Objects.requireNonNull(valueConverter, "Value converter used by task cannot be null");
+            Objects.requireNonNull(headerConverter, "Header converter used by task cannot be null");
+            Objects.requireNonNull(classLoader, "Classloader used by task cannot be null");
+
+            ErrorHandlingMetrics errorHandlingMetrics = errorHandlingMetrics(id);
+            final Class<? extends Connector> connectorClass = plugins.connectorClass(
+                    connectorConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG));
+            RetryWithToleranceOperator retryWithToleranceOperator = new RetryWithToleranceOperator(connectorConfig.errorRetryTimeout(),
+                    connectorConfig.errorMaxDelayInMillis(), connectorConfig.errorToleranceType(), Time.SYSTEM);
+            retryWithToleranceOperator.metrics(errorHandlingMetrics);
+
+            return doBuild(task, id, configState, statusListener, initialState,
+                    connectorConfig, keyConverter, valueConverter, headerConverter, classLoader,
+                    errorHandlingMetrics, connectorClass, retryWithToleranceOperator);
+        }
+
+        abstract WorkerTask doBuild(Task task,
+                                    ConnectorTaskId id,
+                                    ClusterConfigState configState,
+                                    TaskStatus.Listener statusListener,
+                                    TargetState initialState,
+                                    ConnectorConfig connectorConfig,
+                                    Converter keyConverter,
+                                    Converter valueConverter,
+                                    HeaderConverter headerConverter,
+                                    ClassLoader classLoader,
+                                    ErrorHandlingMetrics errorHandlingMetrics,
+                                    Class<? extends Connector> connectorClass,
+                                    RetryWithToleranceOperator retryWithToleranceOperator);
+
+    }
+
+    class SinkTaskBuilder extends TaskBuilder {
+        public SinkTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            TransformationChain<SinkRecord> transformationChain = new TransformationChain<>(connectorConfig.<SinkRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+            SinkConnectorConfig sinkConfig = new SinkConnectorConfig(plugins, connectorConfig.originalsStrings());
+            retryWithToleranceOperator.reporters(sinkTaskReporters(id, sinkConfig, errorHandlingMetrics, connectorClass));
+            WorkerErrantRecordReporter workerErrantRecordReporter = createWorkerErrantRecordReporter(sinkConfig, retryWithToleranceOperator,
+                    keyConverter, valueConverter, headerConverter);
+
+            Map<String, Object> consumerProps = consumerConfigs(id.connector(),  "connector-consumer-" + id, config, connectorConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+            return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverter,
+                    valueConverter, headerConverter, transformationChain, consumer, classLoader, time,
+                    retryWithToleranceOperator, workerErrantRecordReporter, herder.statusBackingStore());
+        }
+    }
+
+    class SourceTaskBuilder extends TaskBuilder {
+        public SourceTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+            ConnectorOffsetBackingStore offsetStore = ConnectorOffsetBackingStore.withoutConnectorOffsetStore(
+                    () -> LoggingContext.forTask(id),
+                    globalOffsetBackingStore,
+                    config.offsetsTopic()
+            );
+            final TopicAdmin admin;
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (sourceConfig.offsetsTopic() != null || (config.topicCreationEnable() && sourceConfig.usesTopicCreation())) {
+                Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,
+                        sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE);
+                Admin adminClient = Admin.create(adminOverrides);
+                admin = new TopicAdmin(adminOverrides.get(BOOTSTRAP_SERVERS_CONFIG), adminClient);
+
+                if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) {
+                    topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig);
+                }
+
+                if (sourceConfig.offsetsTopic() != null && config.connectorOffsetsTopicsPermitted()) {
+                    Map<String, Object> consumerProps = consumerConfigs(id.connector(), "connector-consumer-" + id, config, connectorConfig, connectorClass,
+                            connectorClientConfigOverridePolicy, kafkaClusterId);
+                    // Users can disable this if they want to; it won't affect delivery guarantees since the task isn't exactly-once anyways
+                    consumerProps.putIfAbsent(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString().toLowerCase(Locale.ROOT));
+                    KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+                    offsetStore = ConnectorOffsetBackingStore.withConnectorOffsetStore(
+                            () -> LoggingContext.forTask(id),
+                            globalOffsetBackingStore,
+                            sourceConfig.offsetsTopic(),
+                            producer,
+                            consumer,
+                            admin
+                    );
+                }
+            } else {
+                admin = null;
+            }
+            offsetStore.configure(config);
+
+            CloseableOffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+            OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+
+            // Note we pass the configState as it performs dynamic transformations under the covers
+            return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter, valueConverter,
+                    headerConverter, transformationChain, producer, admin, topicCreationGroups,
+                    offsetReader, offsetWriter, offsetStore, config, configState, metrics, classLoader, time,
+                    retryWithToleranceOperator, herder.statusBackingStore(), executor);
+        }
+    }
+
+    class ExactlyOnceSourceTaskBuilder extends TaskBuilder {
+        private final Runnable preProducerCheck;
+        private final Runnable postProducerCheck;
+
+        public ExactlyOnceSourceTaskBuilder(ConnectorTaskId id,
+                                            ClusterConfigState configState,
+                                            TaskStatus.Listener statusListener,
+                                            TargetState initialState,
+                                            Runnable preProducerCheck,
+                                            Runnable postProducerCheck) {
+            super(id, configState, statusListener, initialState);
+            this.preProducerCheck = preProducerCheck;
+            this.postProducerCheck = postProducerCheck;
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                                  ConnectorTaskId id,
+                                  ClusterConfigState configState,
+                                  TaskStatus.Listener statusListener,
+                                  TargetState initialState,
+                                  ConnectorConfig connectorConfig,
+                                  Converter keyConverter,
+                                  Converter valueConverter,
+                                  HeaderConverter headerConverter,
+                                  ClassLoader classLoader,
+                                  ErrorHandlingMetrics errorHandlingMetrics,
+                                  Class<? extends Connector> connectorClass,
+                                  RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,

Review comment:
       This section has been pretty heavily refactored; I hope the current state of things honors the intent of this comment but if not LMK.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -980,6 +1119,329 @@ WorkerMetricsGroup workerMetricsGroup() {
         return workerMetricsGroup;
     }
 
+    abstract class TaskBuilder {
+
+        private final ConnectorTaskId id;
+        private final ClusterConfigState configState;
+        private final TaskStatus.Listener statusListener;
+        private final TargetState initialState;
+
+        private Task task;
+        private ConnectorConfig connectorConfig = null;
+        private Converter keyConverter = null;
+        private Converter valueConverter = null;
+        private HeaderConverter headerConverter = null;
+        private ClassLoader classLoader = null;
+
+        public TaskBuilder(ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState) {
+            this.id = id;
+            this.configState = configState;
+            this.statusListener = statusListener;
+            this.initialState = initialState;
+        }
+
+        public TaskBuilder withTask(Task task) {
+            this.task = task;
+            return this;
+        }
+
+        public TaskBuilder withConnectorConfig(ConnectorConfig connectorConfig) {
+            this.connectorConfig = connectorConfig;
+            return this;
+        }
+
+        public TaskBuilder withKeyConverter(Converter keyConverter) {
+            this.keyConverter = keyConverter;
+            return this;
+        }
+
+        public TaskBuilder withValueConverter(Converter valueConverter) {
+            this.valueConverter = valueConverter;
+            return this;
+        }
+
+        public TaskBuilder withHeaderConverter(HeaderConverter headerConverter) {
+            this.headerConverter = headerConverter;
+            return this;
+        }
+
+        public TaskBuilder withClassloader(ClassLoader classLoader) {
+            this.classLoader = classLoader;
+            return this;
+        }
+
+        public WorkerTask build() {
+            Objects.requireNonNull(task, "Task cannot be null");
+            Objects.requireNonNull(connectorConfig, "Connector config used by task cannot be null");
+            Objects.requireNonNull(keyConverter, "Key converter used by task cannot be null");
+            Objects.requireNonNull(valueConverter, "Value converter used by task cannot be null");
+            Objects.requireNonNull(headerConverter, "Header converter used by task cannot be null");
+            Objects.requireNonNull(classLoader, "Classloader used by task cannot be null");
+
+            ErrorHandlingMetrics errorHandlingMetrics = errorHandlingMetrics(id);
+            final Class<? extends Connector> connectorClass = plugins.connectorClass(
+                    connectorConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG));
+            RetryWithToleranceOperator retryWithToleranceOperator = new RetryWithToleranceOperator(connectorConfig.errorRetryTimeout(),
+                    connectorConfig.errorMaxDelayInMillis(), connectorConfig.errorToleranceType(), Time.SYSTEM);
+            retryWithToleranceOperator.metrics(errorHandlingMetrics);
+
+            return doBuild(task, id, configState, statusListener, initialState,
+                    connectorConfig, keyConverter, valueConverter, headerConverter, classLoader,
+                    errorHandlingMetrics, connectorClass, retryWithToleranceOperator);
+        }
+
+        abstract WorkerTask doBuild(Task task,
+                                    ConnectorTaskId id,
+                                    ClusterConfigState configState,
+                                    TaskStatus.Listener statusListener,
+                                    TargetState initialState,
+                                    ConnectorConfig connectorConfig,
+                                    Converter keyConverter,
+                                    Converter valueConverter,
+                                    HeaderConverter headerConverter,
+                                    ClassLoader classLoader,
+                                    ErrorHandlingMetrics errorHandlingMetrics,
+                                    Class<? extends Connector> connectorClass,
+                                    RetryWithToleranceOperator retryWithToleranceOperator);
+
+    }
+
+    class SinkTaskBuilder extends TaskBuilder {
+        public SinkTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            TransformationChain<SinkRecord> transformationChain = new TransformationChain<>(connectorConfig.<SinkRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+            SinkConnectorConfig sinkConfig = new SinkConnectorConfig(plugins, connectorConfig.originalsStrings());
+            retryWithToleranceOperator.reporters(sinkTaskReporters(id, sinkConfig, errorHandlingMetrics, connectorClass));
+            WorkerErrantRecordReporter workerErrantRecordReporter = createWorkerErrantRecordReporter(sinkConfig, retryWithToleranceOperator,
+                    keyConverter, valueConverter, headerConverter);
+
+            Map<String, Object> consumerProps = consumerConfigs(id.connector(),  "connector-consumer-" + id, config, connectorConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+            return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverter,
+                    valueConverter, headerConverter, transformationChain, consumer, classLoader, time,
+                    retryWithToleranceOperator, workerErrantRecordReporter, herder.statusBackingStore());
+        }
+    }
+
+    class SourceTaskBuilder extends TaskBuilder {
+        public SourceTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+            ConnectorOffsetBackingStore offsetStore = ConnectorOffsetBackingStore.withoutConnectorOffsetStore(
+                    () -> LoggingContext.forTask(id),
+                    globalOffsetBackingStore,
+                    config.offsetsTopic()
+            );
+            final TopicAdmin admin;
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (sourceConfig.offsetsTopic() != null || (config.topicCreationEnable() && sourceConfig.usesTopicCreation())) {
+                Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,
+                        sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE);
+                Admin adminClient = Admin.create(adminOverrides);
+                admin = new TopicAdmin(adminOverrides.get(BOOTSTRAP_SERVERS_CONFIG), adminClient);
+
+                if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) {
+                    topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig);
+                }
+
+                if (sourceConfig.offsetsTopic() != null && config.connectorOffsetsTopicsPermitted()) {
+                    Map<String, Object> consumerProps = consumerConfigs(id.connector(), "connector-consumer-" + id, config, connectorConfig, connectorClass,
+                            connectorClientConfigOverridePolicy, kafkaClusterId);
+                    // Users can disable this if they want to; it won't affect delivery guarantees since the task isn't exactly-once anyways
+                    consumerProps.putIfAbsent(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString().toLowerCase(Locale.ROOT));
+                    KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+                    offsetStore = ConnectorOffsetBackingStore.withConnectorOffsetStore(
+                            () -> LoggingContext.forTask(id),
+                            globalOffsetBackingStore,
+                            sourceConfig.offsetsTopic(),
+                            producer,
+                            consumer,
+                            admin
+                    );
+                }
+            } else {
+                admin = null;
+            }
+            offsetStore.configure(config);
+
+            CloseableOffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+            OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+
+            // Note we pass the configState as it performs dynamic transformations under the covers
+            return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter, valueConverter,
+                    headerConverter, transformationChain, producer, admin, topicCreationGroups,
+                    offsetReader, offsetWriter, offsetStore, config, configState, metrics, classLoader, time,
+                    retryWithToleranceOperator, herder.statusBackingStore(), executor);
+        }
+    }
+
+    class ExactlyOnceSourceTaskBuilder extends TaskBuilder {
+        private final Runnable preProducerCheck;
+        private final Runnable postProducerCheck;
+
+        public ExactlyOnceSourceTaskBuilder(ConnectorTaskId id,
+                                            ClusterConfigState configState,
+                                            TaskStatus.Listener statusListener,
+                                            TargetState initialState,
+                                            Runnable preProducerCheck,
+                                            Runnable postProducerCheck) {
+            super(id, configState, statusListener, initialState);
+            this.preProducerCheck = preProducerCheck;
+            this.postProducerCheck = postProducerCheck;
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                                  ConnectorTaskId id,
+                                  ClusterConfigState configState,
+                                  TaskStatus.Listener statusListener,
+                                  TargetState initialState,
+                                  ConnectorConfig connectorConfig,
+                                  Converter keyConverter,
+                                  Converter valueConverter,
+                                  HeaderConverter headerConverter,
+                                  ClassLoader classLoader,
+                                  ErrorHandlingMetrics errorHandlingMetrics,
+                                  Class<? extends Connector> connectorClass,
+                                  RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,
+                    sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE);
+            Admin adminClient = Admin.create(adminOverrides);
+            TopicAdmin topicAdmin = new TopicAdmin(adminOverrides.get(BOOTSTRAP_SERVERS_CONFIG), adminClient);
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) {
+                topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig);
+            }
+
+            String transactionalId = transactionalId(id);
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            ConnectUtils.warnOnOverriddenProperty(
+                    producerProps, ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true",
+                    "for connectors when exactly-once source support is enabled",
+                    false
+            );
+            ConnectUtils.warnOnOverriddenProperty(
+                    producerProps, ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId,
+                    "for connectors when exactly-once source support is enabled",
+                    true
+            );
+            producerProps.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, true);
+            producerProps.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId);

Review comment:
       Yep, done.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -980,6 +1119,329 @@ WorkerMetricsGroup workerMetricsGroup() {
         return workerMetricsGroup;
     }
 
+    abstract class TaskBuilder {
+
+        private final ConnectorTaskId id;
+        private final ClusterConfigState configState;
+        private final TaskStatus.Listener statusListener;
+        private final TargetState initialState;
+
+        private Task task;
+        private ConnectorConfig connectorConfig = null;
+        private Converter keyConverter = null;
+        private Converter valueConverter = null;
+        private HeaderConverter headerConverter = null;
+        private ClassLoader classLoader = null;
+
+        public TaskBuilder(ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState) {
+            this.id = id;
+            this.configState = configState;
+            this.statusListener = statusListener;
+            this.initialState = initialState;
+        }
+
+        public TaskBuilder withTask(Task task) {
+            this.task = task;
+            return this;
+        }
+
+        public TaskBuilder withConnectorConfig(ConnectorConfig connectorConfig) {
+            this.connectorConfig = connectorConfig;
+            return this;
+        }
+
+        public TaskBuilder withKeyConverter(Converter keyConverter) {
+            this.keyConverter = keyConverter;
+            return this;
+        }
+
+        public TaskBuilder withValueConverter(Converter valueConverter) {
+            this.valueConverter = valueConverter;
+            return this;
+        }
+
+        public TaskBuilder withHeaderConverter(HeaderConverter headerConverter) {
+            this.headerConverter = headerConverter;
+            return this;
+        }
+
+        public TaskBuilder withClassloader(ClassLoader classLoader) {
+            this.classLoader = classLoader;
+            return this;
+        }
+
+        public WorkerTask build() {
+            Objects.requireNonNull(task, "Task cannot be null");
+            Objects.requireNonNull(connectorConfig, "Connector config used by task cannot be null");
+            Objects.requireNonNull(keyConverter, "Key converter used by task cannot be null");
+            Objects.requireNonNull(valueConverter, "Value converter used by task cannot be null");
+            Objects.requireNonNull(headerConverter, "Header converter used by task cannot be null");
+            Objects.requireNonNull(classLoader, "Classloader used by task cannot be null");
+
+            ErrorHandlingMetrics errorHandlingMetrics = errorHandlingMetrics(id);
+            final Class<? extends Connector> connectorClass = plugins.connectorClass(
+                    connectorConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG));
+            RetryWithToleranceOperator retryWithToleranceOperator = new RetryWithToleranceOperator(connectorConfig.errorRetryTimeout(),
+                    connectorConfig.errorMaxDelayInMillis(), connectorConfig.errorToleranceType(), Time.SYSTEM);
+            retryWithToleranceOperator.metrics(errorHandlingMetrics);
+
+            return doBuild(task, id, configState, statusListener, initialState,
+                    connectorConfig, keyConverter, valueConverter, headerConverter, classLoader,
+                    errorHandlingMetrics, connectorClass, retryWithToleranceOperator);
+        }
+
+        abstract WorkerTask doBuild(Task task,
+                                    ConnectorTaskId id,
+                                    ClusterConfigState configState,
+                                    TaskStatus.Listener statusListener,
+                                    TargetState initialState,
+                                    ConnectorConfig connectorConfig,
+                                    Converter keyConverter,
+                                    Converter valueConverter,
+                                    HeaderConverter headerConverter,
+                                    ClassLoader classLoader,
+                                    ErrorHandlingMetrics errorHandlingMetrics,
+                                    Class<? extends Connector> connectorClass,
+                                    RetryWithToleranceOperator retryWithToleranceOperator);
+
+    }
+
+    class SinkTaskBuilder extends TaskBuilder {
+        public SinkTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            TransformationChain<SinkRecord> transformationChain = new TransformationChain<>(connectorConfig.<SinkRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+            SinkConnectorConfig sinkConfig = new SinkConnectorConfig(plugins, connectorConfig.originalsStrings());
+            retryWithToleranceOperator.reporters(sinkTaskReporters(id, sinkConfig, errorHandlingMetrics, connectorClass));
+            WorkerErrantRecordReporter workerErrantRecordReporter = createWorkerErrantRecordReporter(sinkConfig, retryWithToleranceOperator,
+                    keyConverter, valueConverter, headerConverter);
+
+            Map<String, Object> consumerProps = consumerConfigs(id.connector(),  "connector-consumer-" + id, config, connectorConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+            return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverter,
+                    valueConverter, headerConverter, transformationChain, consumer, classLoader, time,
+                    retryWithToleranceOperator, workerErrantRecordReporter, herder.statusBackingStore());
+        }
+    }
+
+    class SourceTaskBuilder extends TaskBuilder {
+        public SourceTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+            ConnectorOffsetBackingStore offsetStore = ConnectorOffsetBackingStore.withoutConnectorOffsetStore(
+                    () -> LoggingContext.forTask(id),
+                    globalOffsetBackingStore,
+                    config.offsetsTopic()
+            );
+            final TopicAdmin admin;
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (sourceConfig.offsetsTopic() != null || (config.topicCreationEnable() && sourceConfig.usesTopicCreation())) {
+                Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,
+                        sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE);
+                Admin adminClient = Admin.create(adminOverrides);
+                admin = new TopicAdmin(adminOverrides.get(BOOTSTRAP_SERVERS_CONFIG), adminClient);
+
+                if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) {
+                    topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig);
+                }
+
+                if (sourceConfig.offsetsTopic() != null && config.connectorOffsetsTopicsPermitted()) {
+                    Map<String, Object> consumerProps = consumerConfigs(id.connector(), "connector-consumer-" + id, config, connectorConfig, connectorClass,
+                            connectorClientConfigOverridePolicy, kafkaClusterId);
+                    // Users can disable this if they want to; it won't affect delivery guarantees since the task isn't exactly-once anyways
+                    consumerProps.putIfAbsent(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString().toLowerCase(Locale.ROOT));
+                    KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+                    offsetStore = ConnectorOffsetBackingStore.withConnectorOffsetStore(
+                            () -> LoggingContext.forTask(id),
+                            globalOffsetBackingStore,
+                            sourceConfig.offsetsTopic(),
+                            producer,
+                            consumer,
+                            admin
+                    );
+                }
+            } else {
+                admin = null;
+            }
+            offsetStore.configure(config);
+
+            CloseableOffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+            OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+
+            // Note we pass the configState as it performs dynamic transformations under the covers
+            return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter, valueConverter,
+                    headerConverter, transformationChain, producer, admin, topicCreationGroups,
+                    offsetReader, offsetWriter, offsetStore, config, configState, metrics, classLoader, time,
+                    retryWithToleranceOperator, herder.statusBackingStore(), executor);
+        }
+    }
+
+    class ExactlyOnceSourceTaskBuilder extends TaskBuilder {
+        private final Runnable preProducerCheck;
+        private final Runnable postProducerCheck;
+
+        public ExactlyOnceSourceTaskBuilder(ConnectorTaskId id,
+                                            ClusterConfigState configState,
+                                            TaskStatus.Listener statusListener,
+                                            TargetState initialState,
+                                            Runnable preProducerCheck,
+                                            Runnable postProducerCheck) {
+            super(id, configState, statusListener, initialState);
+            this.preProducerCheck = preProducerCheck;
+            this.postProducerCheck = postProducerCheck;
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                                  ConnectorTaskId id,
+                                  ClusterConfigState configState,
+                                  TaskStatus.Listener statusListener,
+                                  TargetState initialState,
+                                  ConnectorConfig connectorConfig,
+                                  Converter keyConverter,
+                                  Converter valueConverter,
+                                  HeaderConverter headerConverter,
+                                  ClassLoader classLoader,
+                                  ErrorHandlingMetrics errorHandlingMetrics,
+                                  Class<? extends Connector> connectorClass,
+                                  RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,
+                    sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE);
+            Admin adminClient = Admin.create(adminOverrides);
+            TopicAdmin topicAdmin = new TopicAdmin(adminOverrides.get(BOOTSTRAP_SERVERS_CONFIG), adminClient);
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) {
+                topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig);
+            }
+
+            String transactionalId = transactionalId(id);
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            ConnectUtils.warnOnOverriddenProperty(
+                    producerProps, ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true",
+                    "for connectors when exactly-once source support is enabled",
+                    false
+            );
+            ConnectUtils.warnOnOverriddenProperty(
+                    producerProps, ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId,
+                    "for connectors when exactly-once source support is enabled",
+                    true
+            );
+            producerProps.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, true);
+            producerProps.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId);
+            KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+            Map<String, Object> consumerProps = consumerConfigs(id.connector(), "connector-consumer-" + id, config, connectorConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            ConnectUtils.warnOnOverriddenProperty(
+                    consumerProps, ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.name().toLowerCase(Locale.ROOT),
+                    "for connectors when exactly-once source support is enabled",
+                    false
+            );
+            consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString().toLowerCase(Locale.ROOT));
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+            String offsetsTopic = Optional.ofNullable(sourceConfig.offsetsTopic()).orElse(config.offsetsTopic());
+
+            ConnectorOffsetBackingStore offsetStore;
+            // No need to do secondary writes to the global offsets topic if we're certain that the task's local offset store
+            // is going to be targeting it anyway
+            // Note that this may lead to a false positive if the user provides an overridden bootstrap servers value for their
+            // producer that resolves to the same Kafka cluster; we might consider looking up the Kafka cluster ID in the future
+            // to prevent these false positives but at the moment this is probably adequate, especially since we probably don't
+            // want to put a ping to a remote Kafka cluster inside the herder's tick thread (which is where this logic takes place
+            // right now) in case that takes a while.

Review comment:
       Ack, done. This logic has been moved into an isolated method and I've moved the relevant portions of the comment with it.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java
##########
@@ -342,6 +343,10 @@ private void logPluginPathConfigProviderWarning(Map<String, String> rawOriginals
         }
     }
 
+    public String bootstrapServers() {
+        return String.join(",", getList(BOOTSTRAP_SERVERS_CONFIG));
+    }

Review comment:
       Sorry, I'm a little unclear on what the benefits of Javadocs here are. I've added tags for each of the relevant methods but would you mind elaborating? In IntelliJ, I can see where this method is invoked by cmd+clicking the `bootstrapServers()` declaration, and wherever it's invoked, I can go to the declaration by cmd+clicking the invocation.
   
   I'm aware that not everyone uses my IDE and certainly don't expect them to start doing so; just wondering if there's a different IDE out there with behavior that makes Javadocs like these significantly more powerful.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java
##########
@@ -74,15 +75,15 @@
     private volatile boolean cancelled; // indicates whether the Worker has cancelled the connector (e.g. because of slow shutdown)
 
     private State state;
-    private final OffsetStorageReader offsetStorageReader;
+    private final CloseableOffsetStorageReader offsetStorageReader;
 
     public WorkerConnector(String connName,
                            Connector connector,
                            ConnectorConfig connectorConfig,
                            CloseableConnectorContext ctx,
                            ConnectMetrics metrics,
                            ConnectorStatus.Listener statusListener,
-                           OffsetStorageReader offsetStorageReader,
+                           CloseableOffsetStorageReader offsetStorageReader,

Review comment:
       I opted for an alternative approach where the offset reader is allowed to be `null`, but I can also see the benefit of a conditional call to `Objects::requireNonNull`. I've added one in `WorkerConnector::initialize`; let me know what you think.




-- 
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 change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r785490692



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java
##########
@@ -192,6 +198,19 @@
     public static final String INTER_WORKER_VERIFICATION_ALGORITHMS_DOC = "A list of permitted algorithms for verifying internal requests";
     public static final List<String> INTER_WORKER_VERIFICATION_ALGORITHMS_DEFAULT = Collections.singletonList(INTER_WORKER_SIGNATURE_ALGORITHM_DEFAULT);
 
+    public static final String EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG = "exactly.once.source.support";
+    public static final String EXACTLY_ONCE_SOURCE_SUPPORT_DOC = "Whether to enable exactly-once support for source connectors in the cluster "
+            + "by writing source records and their offsets in a Kafka transaction, and by proactively fencing out old task generations before bringing up new ones. "
+            + "Note that this must be enabled on every worker in a cluster in order for exactly-once delivery to be guaranteed, "
+            + "and that some source connectors may still not be able to provide exactly-once delivery guarantees even with this support enabled. "
+            + "Permitted values are \"disabled\", \"preparing\", and \"enabled\". In order to safely enable exactly-once support for source connectors, "
+            + "all workers in the cluster must first be updated to use the \"preparing\" value for this property. "
+            + "Once this has been done, a second update of all of the workers in the cluster should be performed to change the value of this property to \"enabled\".";

Review comment:
       I've unresolved this as a general reminder to make sure to write the docs section for this feature before we release 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] C0urante commented on a change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r732232701



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java
##########
@@ -163,13 +181,24 @@ public synchronized boolean beginFlush() {
             }
 
             // And submit the data
-            log.debug("Submitting {} entries to backing store. The offsets are: {}", offsetsSerialized.size(), toFlush);
+            log.debug("Submitting {} entries to backing store. The offsets are: {}", offsetsSerialized.size(), flushed);
         }
 
-        return backingStore.set(offsetsSerialized, (error, result) -> {
-            boolean isCurrent = handleFinishWrite(flushId, error, result);
-            if (isCurrent && callback != null) {
-                callback.onCompletion(error, result);
+        return primaryBackingStore.set(offsetsSerialized, (primaryError, primaryResult) -> {
+            boolean isCurrent = handleFinishWrite(flushId, primaryError, primaryResult);
+            if (isCurrent) {
+                if (callback != null) {
+                    callback.onCompletion(primaryError, primaryResult);
+                }
+                if (secondaryBackingStore != null && primaryError == null) {
+                    secondaryBackingStore.set(offsetsSerialized, (secondaryError, secondaryResult) -> {
+                        if (secondaryError != null) {
+                            log.warn("Failed to write offsets ({}) to secondary backing store", flushed, secondaryError);
+                        } else {
+                            log.debug("Successfully flushed offsets ({}) to secondary backing store", flushed);

Review comment:
       Good catch; I believe there's also a case in the existing case where offset commit messages that are logged in a producer callback are also missing the MDC context. I've addressed both cases.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java
##########
@@ -20,13 +20,45 @@
 import org.apache.kafka.clients.producer.RecordMetadata;
 
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * SourceTask is a Task that pulls records from another system for storage in Kafka.
  */
 public abstract class SourceTask implements Task {
 
+    /**
+     * <p>

Review comment:
       Ack, removed.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java
##########
@@ -20,13 +20,45 @@
 import org.apache.kafka.clients.producer.RecordMetadata;
 
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * SourceTask is a Task that pulls records from another system for storage in Kafka.
  */
 public abstract class SourceTask implements Task {
 
+    /**
+     * <p>
+     * The configuration key that determines how source tasks will define transaction boundaries
+     * when exactly-once support is enabled.
+     * </p>

Review comment:
       Ack, removed.




-- 
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 change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r732231269



##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceTaskContext.java
##########
@@ -38,4 +38,30 @@
      * Get the OffsetStorageReader for this SourceTask.
      */
     OffsetStorageReader offsetStorageReader();
+
+    /**
+     * Get a {@link TransactionContext} that can be used to define producer transaction boundaries
+     * when exactly-once support is enabled for the connector.
+     *
+     * <p>This method was added in Apache Kafka 3.0. Source tasks that use this method but want to
+     * maintain backward compatibility so they can also be deployed to older Connect runtimes
+     * should guard the call to this method with a try-catch block, since calling this method will result in a
+     * {@link NoSuchMethodException} or {@link NoClassDefFoundError} when the source connector is deployed to
+     * Connect runtimes older than Kafka 3.0. For example:
+     * <pre>
+     *     TransactionContext transactionContext;
+     *     try {
+     *         transactionContext = context.transactionContext();
+     *     } catch (NoSuchMethodError | NoClassDefFoundError e) {
+     *         transactionContext = null;
+     *     }
+     * </pre>
+     *
+     * @return the transaction context, or null if the user does not want the connector to define
+     * its own transaction boundaries

Review comment:
       No objections to modifying Javadocs here instead of in a KIP (especially since you made that clear on the mailing list). It's easier to review these sorts of details in a PR IMO anyways.
   
   Can make the change to refer to the connector configuration. Agree that "user" is ambiguous.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java
##########
@@ -28,4 +30,31 @@
     protected SourceConnectorContext context() {
         return (SourceConnectorContext) context;
     }
+
+    /**
+     * Signals whether the connector supports exactly-once delivery guarantees with a proposed configuration.
+     * Developers can assume that worker-level exactly-once support is enabled when this method is invoked.
+     * The default implementation will return {@code null}.
+     * @param connectorConfig the configuration that will be used for the connector.
+     * @return {@link ExactlyOnceSupport#SUPPORTED} if the connector can provide exactly-once support,
+     * and {@link ExactlyOnceSupport#UNSUPPORTED} if it cannot. If {@code null}, it is assumed that the
+     * connector cannot.
+     */
+    public ExactlyOnceSupport exactlyOnceSupport(Map<String, String> connectorConfig) {
+        return null;
+    }
+
+    /**
+     * Signals whether the connector can define its own transaction boundaries with the proposed
+     * configuration. Developers must override this method if they wish to add connector-defined
+     * transaction boundary support; if they do not, users will be unable to create instances of
+     * this connector that use connector-defined transaction boundaries. The default implementation
+     * will return {@code UNSUPPORTED}.

Review comment:
       Can do. I think a reference to the existing `validate` method may help clarify things (especially since this method will be used in almost exactly the same way); LMK what you think.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java
##########
@@ -28,4 +30,31 @@
     protected SourceConnectorContext context() {
         return (SourceConnectorContext) context;
     }
+
+    /**
+     * Signals whether the connector supports exactly-once delivery guarantees with a proposed configuration.
+     * Developers can assume that worker-level exactly-once support is enabled when this method is invoked.
+     * The default implementation will return {@code null}.

Review comment:
       Ack, 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] C0urante commented on pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-1040883663


   I've completed a rebase onto the latest trunk, which brought in notable changes from https://github.com/apache/kafka/pull/11323, https://github.com/apache/kafka/pull/11382, https://github.com/apache/kafka/pull/11508 (thanks Mikael!), and https://github.com/apache/kafka/pull/11046.
   
   I'll start to break things up into smaller pieces tomorrow.


-- 
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 edited a comment on pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante edited a comment on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-1032801881


   Thanks Mickael, I can take a stab at that. I don't think I can get each PR to be buildable on its own without a lot of work but I can definitely isolate high-level sections into their own PRs, probably following the breakdown in the description for this one.
   
   I'm taking this week off but will hopefully be able to get a rebase done and split things up by Wednesday the 16th.


-- 
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] galeaspablo commented on pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
galeaspablo commented on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-1003836206


   Hey folks, I've kept an eye on this PR, its associated KIP, for a while. Is there anything I can do to help?
   
   Happy to take guidance to split the PR if that would make reviewing easier?
   
   P.S. I appreciate I haven't committed to this repository, but all my Kafka work was internal while I was at AWS. 


-- 
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] rhauch commented on pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
rhauch commented on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-962658200


   Thanks, @C0urante. I have been trying to get through this large PR, but that takes time and I often get sidetracked by other things, such as AK 3.1 release issues.
   
   Please keep this open. I do plan to complete my latest pass of this PR soon. Thanks for your continued patience.


-- 
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 change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r786341813



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java
##########
@@ -0,0 +1,643 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.connect.runtime;
+
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.metrics.stats.Avg;
+import org.apache.kafka.common.metrics.stats.CumulativeSum;
+import org.apache.kafka.common.metrics.stats.Max;
+import org.apache.kafka.common.metrics.stats.Rate;
+import org.apache.kafka.common.metrics.stats.Value;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.header.Headers;
+import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator;
+import org.apache.kafka.connect.runtime.errors.Stage;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.kafka.connect.source.SourceTaskContext;
+import org.apache.kafka.connect.storage.CloseableOffsetStorageReader;
+import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore;
+import org.apache.kafka.connect.storage.Converter;
+import org.apache.kafka.connect.storage.HeaderConverter;
+import org.apache.kafka.connect.storage.OffsetStorageWriter;
+import org.apache.kafka.connect.storage.StatusBackingStore;
+import org.apache.kafka.connect.util.ConnectUtils;
+import org.apache.kafka.connect.util.ConnectorTaskId;
+import org.apache.kafka.connect.util.TopicAdmin;
+import org.apache.kafka.connect.util.TopicCreation;
+import org.apache.kafka.connect.util.TopicCreationGroup;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_TRACKING_ENABLE_CONFIG;
+
+/**
+ * WorkerTask that contains shared logic for running source tasks with either standard or exactly-once delivery guarantees.
+ */
+public abstract class AbstractWorkerSourceTask extends WorkerTask {
+    private static final Logger log = LoggerFactory.getLogger(AbstractWorkerSourceTask.class);
+
+    private static final long SEND_FAILED_BACKOFF_MS = 100;
+
+    /**
+     * Hook to define custom startup behavior before the calling {@link SourceTask#initialize(SourceTaskContext)}
+     * and {@link SourceTask#start(Map)}.
+     */
+    protected abstract void prepareToInitializeTask();
+
+    /**
+     * Hook to define custom initialization behavior when preparing to begin the poll-convert-send loop for the first time,
+     * or when re-entering the loop after being paused.
+     */
+    protected abstract void prepareToEnterSendLoop();
+
+    /**
+     * Hook to define custom periodic checks for health, metrics, etc. Called whenever {@link SourceTask#poll()} is about to be invoked.
+     */
+    protected abstract void prepareToPollTask();
+
+    /**
+     * Invoked when a record provided by the task has been filtered out by a transform or the converter, will be
+     * discarded due to failures during transformation or conversion.
+     * @param record the pre-transform record that has been dropped; never null.
+     */
+    protected abstract void recordDropped(SourceRecord record);
+
+    /**
+     * Invoked when a record is about to be dispatched to the producer. May be invoked multiple times for the same
+     * record if retriable errors are encountered.
+     * @param sourceRecord the pre-transform {@link SourceRecord} provided by the source task; never null.
+     * @param producerRecord the {@link ProducerRecord} produced by transforming and converting the
+     * {@code sourceRecord}; never null;
+     * @param newRecord whether this is the first time the record is being dispatched to the producer.
+     */
+    protected abstract void prepareToSendRecord(SourceRecord sourceRecord, ProducerRecord<byte[], byte[]> producerRecord, boolean newRecord);
+
+    /**
+     * Invoked when a record has been transformed, converted, and dispatched to the producer successfully via
+     * {@link Producer#send}. Does not guarantee that the record has been sent to Kafka or ack'd by the required number
+     * of brokers, but does guarantee that it will never be re-processed.
+     * @param record the pre-transform {@link SourceRecord} that was successfully dispatched to the producer; never null.
+     */
+    protected abstract void recordDispatched(SourceRecord record);
+
+    /**
+     * Invoked when an entire batch of records returned from {@link SourceTask#poll} has been transformed, converted,
+     * and either discarded due to transform/conversion errors, filtered by a transform, or dispatched to the producer
+     * successfully via {@link Producer#send}. Does not guarantee that the records have been sent to Kafka or ack'd by the
+     * required number of brokers, but does guarantee that none of the records in the batch will ever be re-processed during
+     * the lifetime of this task. At most one record batch is polled from the task in between calls to this method.
+     */
+    protected abstract void batchDispatched();
+
+    /**
+     * Invoked when a record has been sent and ack'd by the Kafka cluster. Note that this method may be invoked
+     *  concurrently and should therefore be made thread-safe.
+     * @param sourceRecord  the pre-transform {@link SourceRecord} that was successfully sent to Kafka; never null.
+     * @param producerRecord the {@link ProducerRecord} produced by transforming and converting the
+     * {@code sourceRecord}; never null;
+     * @param recordMetadata the {@link RecordMetadata} for the corresponding producer record; never null.
+     */
+    protected abstract void recordSent(SourceRecord sourceRecord, ProducerRecord<byte[], byte[]> producerRecord, RecordMetadata recordMetadata);
+
+    /**
+     * Invoked when no more records will be polled from the task or dispatched to the producer. Should attempt to
+     * commit the offsets for any outstanding records when possible.
+     * @param failed whether the task is undergoing a healthy or an unhealthy shutdown
+     */
+    protected abstract void finalOffsetCommit(boolean failed);
+
+
+    protected final WorkerConfig workerConfig;
+    protected final WorkerSourceTaskContext sourceTaskContext;
+    protected final OffsetStorageWriter offsetWriter;
+    protected final Producer<byte[], byte[]> producer;
+
+    private final SourceTask task;
+    private final Converter keyConverter;
+    private final Converter valueConverter;
+    private final HeaderConverter headerConverter;
+    private final TransformationChain<SourceRecord> transformationChain;
+    private final TopicAdmin admin;
+    private final CloseableOffsetStorageReader offsetReader;
+    private final ConnectorOffsetBackingStore offsetStore;
+    private final AtomicReference<Exception> producerSendException;
+    private final SourceTaskMetricsGroup sourceTaskMetricsGroup;
+    private final CountDownLatch stopRequestedLatch;
+    private final boolean topicTrackingEnabled;
+    private final TopicCreation topicCreation;
+    private final Executor closeExecutor;
+
+    // Visible for testing
+    boolean lastSendFailed;
+    // Visible for testing
+    List<SourceRecord> toSend;
+    protected Map<String, String> taskConfig;
+    protected boolean started = false;
+
+    protected AbstractWorkerSourceTask(ConnectorTaskId id,
+                                       SourceTask task,
+                                       TaskStatus.Listener statusListener,
+                                       TargetState initialState,
+                                       Converter keyConverter,
+                                       Converter valueConverter,
+                                       HeaderConverter headerConverter,
+                                       TransformationChain<SourceRecord> transformationChain,
+                                       WorkerSourceTaskContext sourceTaskContext,
+                                       Producer<byte[], byte[]> producer,
+                                       TopicAdmin admin,
+                                       Map<String, TopicCreationGroup> topicGroups,
+                                       CloseableOffsetStorageReader offsetReader,
+                                       OffsetStorageWriter offsetWriter,
+                                       ConnectorOffsetBackingStore offsetStore,
+                                       WorkerConfig workerConfig,
+                                       ConnectMetrics connectMetrics,
+                                       ClassLoader loader,
+                                       Time time,
+                                       RetryWithToleranceOperator retryWithToleranceOperator,
+                                       StatusBackingStore statusBackingStore,
+                                       Executor closeExecutor) {
+
+        super(id, statusListener, initialState, loader, connectMetrics,
+                retryWithToleranceOperator, time, statusBackingStore);
+
+        this.workerConfig = workerConfig;
+        this.task = task;
+        this.keyConverter = keyConverter;
+        this.valueConverter = valueConverter;
+        this.headerConverter = headerConverter;
+        this.transformationChain = transformationChain;
+        this.producer = producer;
+        this.admin = admin;
+        this.offsetReader = offsetReader;
+        this.offsetWriter = offsetWriter;
+        this.offsetStore = offsetStore;
+        this.closeExecutor = closeExecutor;
+        this.sourceTaskContext = sourceTaskContext;
+
+        this.producerSendException = new AtomicReference<>();
+        this.stopRequestedLatch = new CountDownLatch(1);
+        this.sourceTaskMetricsGroup = new SourceTaskMetricsGroup(id, connectMetrics);
+        this.topicTrackingEnabled = workerConfig.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG);
+        this.topicCreation = TopicCreation.newTopicCreation(workerConfig, topicGroups);
+    }
+
+    @Override
+    public void initialize(TaskConfig taskConfig) {
+        try {
+            this.taskConfig = taskConfig.originalsStrings();
+        } catch (Throwable t) {
+            log.error("{} Task failed initialization and will not be started.", this, t);
+            onFailure(t);
+        }
+    }
+
+    @Override
+    protected void initializeAndStart() {
+        prepareToInitializeTask();
+        // If we try to start the task at all by invoking initialize, then count this as
+        // "started" and expect a subsequent call to the task's stop() method
+        // to properly clean up any resources allocated by its initialize() or
+        // start() methods. If the task throws an exception during stop(),
+        // the worst thing that happens is another exception gets logged for an already-
+        // failed task
+        started = true;
+        task.initialize(sourceTaskContext);
+        task.start(taskConfig);
+        log.info("{} Source task finished initialization and start", this);
+    }
+
+    @Override
+    public void cancel() {
+        super.cancel();
+        // Preemptively close the offset reader in case the task is blocked on an offset read.
+        offsetReader.close();
+        // We proactively close the producer here as the main work thread for the task may
+        // be blocked indefinitely in a call to Producer::send if automatic topic creation is
+        // not enabled on either the connector or the Kafka cluster. Closing the producer should
+        // unblock it in that case and allow shutdown to proceed normally.
+        // With a duration of 0, the producer's own shutdown logic should be fairly quick,
+        // but closing user-pluggable classes like interceptors may lag indefinitely. So, we
+        // call close on a separate thread in order to avoid blocking the herder's tick thread.
+        closeExecutor.execute(() -> closeProducer(Duration.ZERO));
+    }
+
+    @Override
+    public void stop() {
+        super.stop();
+        stopRequestedLatch.countDown();
+    }
+
+    @Override
+    public void removeMetrics() {
+        Utils.closeQuietly(sourceTaskMetricsGroup::close, "source task metrics tracker");
+        super.removeMetrics();
+    }
+
+    @Override
+    protected void close() {
+        if (started) {
+            Utils.closeQuietly(task::stop, "source task");
+        }
+
+        closeProducer(Duration.ofSeconds(30));
+
+        if (admin != null) {
+            Utils.closeQuietly(() -> admin.close(Duration.ofSeconds(30)), "source task admin");
+        }
+        Utils.closeQuietly(transformationChain, "transformation chain");
+        Utils.closeQuietly(retryWithToleranceOperator, "retry operator");
+        Utils.closeQuietly(offsetReader, "offset reader");
+        Utils.closeQuietly(offsetStore::stop, "offset backing store");
+    }
+
+    private void closeProducer(Duration duration) {
+        if (producer != null) {
+            Utils.closeQuietly(() -> producer.close(duration), "source task producer");
+        }
+    }
+
+    @Override
+    public void execute() {
+        try {
+            prepareToEnterSendLoop();
+            while (!isStopping()) {
+                if (shouldPause()) {
+                    onPause();
+                    if (awaitUnpause()) {
+                        onResume();
+                        prepareToEnterSendLoop();
+                    }
+                    continue;
+                }
+
+                if (toSend == null) {
+                    prepareToPollTask();
+
+                    log.trace("{} Nothing to send to Kafka. Polling source for additional records", this);
+                    long start = time.milliseconds();
+                    toSend = poll();
+                    if (toSend != null) {
+                        recordPollReturned(toSend.size(), time.milliseconds() - start);
+                    }
+                }
+                if (toSend == null)
+                    continue;
+                log.trace("{} About to send {} records to Kafka", this, toSend.size());
+                if (sendRecords()) {
+                    batchDispatched();
+                } else {
+                    stopRequestedLatch.await(SEND_FAILED_BACKOFF_MS, TimeUnit.MILLISECONDS);
+                }
+            }
+        } catch (InterruptedException e) {
+            // Ignore and allow to exit.
+        } catch (RuntimeException e) {
+            try {
+                finalOffsetCommit(true);
+            } catch (Exception offsetException) {
+                log.error("Failed to commit offsets for already-failing task", offsetException);
+            }
+            throw e;
+        }
+        finalOffsetCommit(false);
+    }
+
+    /**
+     * Try to send a batch of records. If a send fails and is retriable, this saves the remainder of the batch so it can
+     * be retried after backing off. If a send fails and is not retriable, this will throw a ConnectException.
+     * @return true if all messages were sent, false if some need to be retried
+     */
+    // Visible for testing
+    boolean sendRecords() {
+        int processed = 0;
+        recordBatch(toSend.size());
+        final SourceRecordWriteCounter counter =
+                toSend.size() > 0 ? new SourceRecordWriteCounter(toSend.size(), sourceTaskMetricsGroup) : null;
+        for (final SourceRecord preTransformRecord : toSend) {
+            retryWithToleranceOperator.sourceRecord(preTransformRecord);
+            final SourceRecord record = transformationChain.apply(preTransformRecord);
+            final ProducerRecord<byte[], byte[]> producerRecord = convertTransformedRecord(record);
+            if (producerRecord == null || retryWithToleranceOperator.failed()) {
+                counter.skipRecord();
+                recordDropped(preTransformRecord);
+                continue;
+            }
+
+            log.trace("{} Appending record to the topic {} with key {}, value {}", this, record.topic(), record.key(), record.value());
+            prepareToSendRecord(preTransformRecord, producerRecord, !lastSendFailed);
+            try {
+                final String topic = producerRecord.topic();
+                maybeCreateTopic(topic);
+                producer.send(
+                    producerRecord,
+                    (recordMetadata, e) -> {
+                        if (e != null) {
+                            log.debug("{} failed to send record to {}: ", AbstractWorkerSourceTask.this, topic, e);
+                            log.trace("{} Failed record: {}", AbstractWorkerSourceTask.this, preTransformRecord);
+                            if (isFatalProducerSendException(e)) {
+                                producerSendException.compareAndSet(null, e);
+                            }

Review comment:
       This will have to be tweaked slightly in order to invoke `SourceTask::commitRecord` and correctly track the number of in-flight records for metrics. Leaving as-is for now since this part of the PR will have to be updated anyways once rebased onto `trunk`.




-- 
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 change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r785355793



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
##########
@@ -16,95 +16,46 @@
  */
 package org.apache.kafka.connect.runtime;
 
-import org.apache.kafka.clients.admin.NewTopic;
-import org.apache.kafka.clients.admin.TopicDescription;
-import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.header.internals.RecordHeaders;
-import org.apache.kafka.common.metrics.Sensor;
-import org.apache.kafka.common.metrics.stats.Avg;
-import org.apache.kafka.common.metrics.stats.CumulativeSum;
-import org.apache.kafka.common.metrics.stats.Max;
-import org.apache.kafka.common.metrics.stats.Rate;
-import org.apache.kafka.common.metrics.stats.Value;
 import org.apache.kafka.common.utils.Time;
-import org.apache.kafka.common.utils.Utils;
-import org.apache.kafka.connect.errors.ConnectException;
-import org.apache.kafka.connect.errors.RetriableException;
-import org.apache.kafka.connect.header.Header;
-import org.apache.kafka.connect.header.Headers;
-import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup;
-import org.apache.kafka.connect.runtime.distributed.ClusterConfigState;
 import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator;
-import org.apache.kafka.connect.runtime.errors.Stage;
 import org.apache.kafka.connect.source.SourceRecord;
 import org.apache.kafka.connect.source.SourceTask;
 import org.apache.kafka.connect.storage.CloseableOffsetStorageReader;
+import org.apache.kafka.connect.storage.ClusterConfigState;
+import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore;
 import org.apache.kafka.connect.storage.Converter;
 import org.apache.kafka.connect.storage.HeaderConverter;
 import org.apache.kafka.connect.storage.OffsetStorageWriter;
 import org.apache.kafka.connect.storage.StatusBackingStore;
-import org.apache.kafka.connect.util.ConnectUtils;
 import org.apache.kafka.connect.util.ConnectorTaskId;
 import org.apache.kafka.connect.util.TopicAdmin;
-import org.apache.kafka.connect.util.TopicCreation;
 import org.apache.kafka.connect.util.TopicCreationGroup;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.time.Duration;
 import java.util.IdentityHashMap;
-import java.util.List;
 import java.util.Map;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executor;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_TRACKING_ENABLE_CONFIG;
 
 /**
  * WorkerTask that uses a SourceTask to ingest data into Kafka.
  */
-class WorkerSourceTask extends WorkerTask {
+class WorkerSourceTask extends AbstractWorkerSourceTask {

Review comment:
       May be best to hold off on this section for a bit; there's a fair amount of rebasing that needs to be done now that https://github.com/apache/kafka/pull/11323 has been merged.




-- 
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] mimaison commented on pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
mimaison commented on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-1029158871


   @C0urante Thanks for the PR! It's really massive! Can we try to cut it in smaller chunks? That would make it a lot easier to review and allow merging bit by bit.


-- 
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 closed pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante closed pull request #10907:
URL: https://github.com/apache/kafka/pull/10907


   


-- 
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 change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r654976449



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java
##########
@@ -337,15 +353,29 @@ private void readToLogEnd() {
                 } else {
                     log.trace("Behind end offset {} for {}; last-read offset is {}",
                             endOffset, topicPartition, lastConsumedOffset);
-                    poll(Integer.MAX_VALUE);
+                    if (topicContainsTransactions) {
+                        // The consumer won't return from its poll method if a transaction is aborted, even though
+                        // its position will advance. So, we poll for at most one second, then give ourselves another
+                        // chance to check whether we've reached the end of the topic.
+                        poll(1000);

Review comment:
       This causes a new consumer fetch request to be issued every second while there's an active read-to-end request for the log.
   
   If this is too expensive and ends up causing issues (either on the worker or on the broker), we might be able to retain the existing `poll(Integer.MAX_VALUE)` behavior by first producing a sentinel message to each topic partition we want to read to the end of. That way, even if a transaction that was initially blocking our read to the end of the partition is aborted, the call to `Consumer::poll` will return, since there will be a non-transactional message after it.
   
   Existing, older workers would be able to handle these sentinel messages gracefully (the only downside would be some `ERROR`-level log messages from the `KafkaConfigBackingStore` class upon encountering unrecognized in the config topic). However, we'd need to reason carefully about how this might work with multiple workers trying to read to the end of the same topic partitions at the same time.




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

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



[GitHub] [kafka] C0urante commented on pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-937394815


   @rhauch Is that second pass coming any time soon? It'd be nice if we could get this merged in time for the upcoming 3.1 release.
   
   I plan to address the existing comments next week.


-- 
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 change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r785355853



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java
##########
@@ -337,15 +353,29 @@ private void readToLogEnd() {
                 } else {
                     log.trace("Behind end offset {} for {}; last-read offset is {}",
                             endOffset, topicPartition, lastConsumedOffset);
-                    poll(Integer.MAX_VALUE);
+                    if (topicContainsTransactions) {
+                        // The consumer won't return from its poll method if a transaction is aborted, even though
+                        // its position will advance. So, we poll for at most one second, then give ourselves another
+                        // chance to check whether we've reached the end of the topic.
+                        poll(1000);

Review comment:
       Now that https://github.com/apache/kafka/pull/11046 has been merged and this PR has been rebased to include it, we can resolve this comment.




-- 
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 #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-1042573696


   @mimaison @rhauch I've broken things down into smaller PRs.
   
   These can all be reviewed immediately:
   - https://github.com/apache/kafka/pull/11777 (cc @hachikuji in case you want to take a look)
   - https://github.com/apache/kafka/pull/11772
   - https://github.com/apache/kafka/pull/11773
   - https://github.com/apache/kafka/pull/11774
   
   These have requirements on other PRs (either ones outlined above, or ones also in this list):
   - https://github.com/apache/kafka/pull/11775
   - https://github.com/apache/kafka/pull/11776
   - https://github.com/apache/kafka/pull/11778
   - https://github.com/apache/kafka/pull/11779
   - https://github.com/apache/kafka/pull/11780
   - https://github.com/apache/kafka/pull/11781
   
   And these should be reviewed last as they bring all of the above changes together and add integration and system tests for them, respectively:
   - https://github.com/apache/kafka/pull/11782
   - https://github.com/apache/kafka/pull/11783


-- 
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 change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r786341813



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java
##########
@@ -0,0 +1,643 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.connect.runtime;
+
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.metrics.stats.Avg;
+import org.apache.kafka.common.metrics.stats.CumulativeSum;
+import org.apache.kafka.common.metrics.stats.Max;
+import org.apache.kafka.common.metrics.stats.Rate;
+import org.apache.kafka.common.metrics.stats.Value;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.header.Headers;
+import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator;
+import org.apache.kafka.connect.runtime.errors.Stage;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.kafka.connect.source.SourceTaskContext;
+import org.apache.kafka.connect.storage.CloseableOffsetStorageReader;
+import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore;
+import org.apache.kafka.connect.storage.Converter;
+import org.apache.kafka.connect.storage.HeaderConverter;
+import org.apache.kafka.connect.storage.OffsetStorageWriter;
+import org.apache.kafka.connect.storage.StatusBackingStore;
+import org.apache.kafka.connect.util.ConnectUtils;
+import org.apache.kafka.connect.util.ConnectorTaskId;
+import org.apache.kafka.connect.util.TopicAdmin;
+import org.apache.kafka.connect.util.TopicCreation;
+import org.apache.kafka.connect.util.TopicCreationGroup;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_TRACKING_ENABLE_CONFIG;
+
+/**
+ * WorkerTask that contains shared logic for running source tasks with either standard or exactly-once delivery guarantees.
+ */
+public abstract class AbstractWorkerSourceTask extends WorkerTask {
+    private static final Logger log = LoggerFactory.getLogger(AbstractWorkerSourceTask.class);
+
+    private static final long SEND_FAILED_BACKOFF_MS = 100;
+
+    /**
+     * Hook to define custom startup behavior before the calling {@link SourceTask#initialize(SourceTaskContext)}
+     * and {@link SourceTask#start(Map)}.
+     */
+    protected abstract void prepareToInitializeTask();
+
+    /**
+     * Hook to define custom initialization behavior when preparing to begin the poll-convert-send loop for the first time,
+     * or when re-entering the loop after being paused.
+     */
+    protected abstract void prepareToEnterSendLoop();
+
+    /**
+     * Hook to define custom periodic checks for health, metrics, etc. Called whenever {@link SourceTask#poll()} is about to be invoked.
+     */
+    protected abstract void prepareToPollTask();
+
+    /**
+     * Invoked when a record provided by the task has been filtered out by a transform or the converter, will be
+     * discarded due to failures during transformation or conversion.
+     * @param record the pre-transform record that has been dropped; never null.
+     */
+    protected abstract void recordDropped(SourceRecord record);
+
+    /**
+     * Invoked when a record is about to be dispatched to the producer. May be invoked multiple times for the same
+     * record if retriable errors are encountered.
+     * @param sourceRecord the pre-transform {@link SourceRecord} provided by the source task; never null.
+     * @param producerRecord the {@link ProducerRecord} produced by transforming and converting the
+     * {@code sourceRecord}; never null;
+     * @param newRecord whether this is the first time the record is being dispatched to the producer.
+     */
+    protected abstract void prepareToSendRecord(SourceRecord sourceRecord, ProducerRecord<byte[], byte[]> producerRecord, boolean newRecord);
+
+    /**
+     * Invoked when a record has been transformed, converted, and dispatched to the producer successfully via
+     * {@link Producer#send}. Does not guarantee that the record has been sent to Kafka or ack'd by the required number
+     * of brokers, but does guarantee that it will never be re-processed.
+     * @param record the pre-transform {@link SourceRecord} that was successfully dispatched to the producer; never null.
+     */
+    protected abstract void recordDispatched(SourceRecord record);
+
+    /**
+     * Invoked when an entire batch of records returned from {@link SourceTask#poll} has been transformed, converted,
+     * and either discarded due to transform/conversion errors, filtered by a transform, or dispatched to the producer
+     * successfully via {@link Producer#send}. Does not guarantee that the records have been sent to Kafka or ack'd by the
+     * required number of brokers, but does guarantee that none of the records in the batch will ever be re-processed during
+     * the lifetime of this task. At most one record batch is polled from the task in between calls to this method.
+     */
+    protected abstract void batchDispatched();
+
+    /**
+     * Invoked when a record has been sent and ack'd by the Kafka cluster. Note that this method may be invoked
+     *  concurrently and should therefore be made thread-safe.
+     * @param sourceRecord  the pre-transform {@link SourceRecord} that was successfully sent to Kafka; never null.
+     * @param producerRecord the {@link ProducerRecord} produced by transforming and converting the
+     * {@code sourceRecord}; never null;
+     * @param recordMetadata the {@link RecordMetadata} for the corresponding producer record; never null.
+     */
+    protected abstract void recordSent(SourceRecord sourceRecord, ProducerRecord<byte[], byte[]> producerRecord, RecordMetadata recordMetadata);
+
+    /**
+     * Invoked when no more records will be polled from the task or dispatched to the producer. Should attempt to
+     * commit the offsets for any outstanding records when possible.
+     * @param failed whether the task is undergoing a healthy or an unhealthy shutdown
+     */
+    protected abstract void finalOffsetCommit(boolean failed);
+
+
+    protected final WorkerConfig workerConfig;
+    protected final WorkerSourceTaskContext sourceTaskContext;
+    protected final OffsetStorageWriter offsetWriter;
+    protected final Producer<byte[], byte[]> producer;
+
+    private final SourceTask task;
+    private final Converter keyConverter;
+    private final Converter valueConverter;
+    private final HeaderConverter headerConverter;
+    private final TransformationChain<SourceRecord> transformationChain;
+    private final TopicAdmin admin;
+    private final CloseableOffsetStorageReader offsetReader;
+    private final ConnectorOffsetBackingStore offsetStore;
+    private final AtomicReference<Exception> producerSendException;
+    private final SourceTaskMetricsGroup sourceTaskMetricsGroup;
+    private final CountDownLatch stopRequestedLatch;
+    private final boolean topicTrackingEnabled;
+    private final TopicCreation topicCreation;
+    private final Executor closeExecutor;
+
+    // Visible for testing
+    boolean lastSendFailed;
+    // Visible for testing
+    List<SourceRecord> toSend;
+    protected Map<String, String> taskConfig;
+    protected boolean started = false;
+
+    protected AbstractWorkerSourceTask(ConnectorTaskId id,
+                                       SourceTask task,
+                                       TaskStatus.Listener statusListener,
+                                       TargetState initialState,
+                                       Converter keyConverter,
+                                       Converter valueConverter,
+                                       HeaderConverter headerConverter,
+                                       TransformationChain<SourceRecord> transformationChain,
+                                       WorkerSourceTaskContext sourceTaskContext,
+                                       Producer<byte[], byte[]> producer,
+                                       TopicAdmin admin,
+                                       Map<String, TopicCreationGroup> topicGroups,
+                                       CloseableOffsetStorageReader offsetReader,
+                                       OffsetStorageWriter offsetWriter,
+                                       ConnectorOffsetBackingStore offsetStore,
+                                       WorkerConfig workerConfig,
+                                       ConnectMetrics connectMetrics,
+                                       ClassLoader loader,
+                                       Time time,
+                                       RetryWithToleranceOperator retryWithToleranceOperator,
+                                       StatusBackingStore statusBackingStore,
+                                       Executor closeExecutor) {
+
+        super(id, statusListener, initialState, loader, connectMetrics,
+                retryWithToleranceOperator, time, statusBackingStore);
+
+        this.workerConfig = workerConfig;
+        this.task = task;
+        this.keyConverter = keyConverter;
+        this.valueConverter = valueConverter;
+        this.headerConverter = headerConverter;
+        this.transformationChain = transformationChain;
+        this.producer = producer;
+        this.admin = admin;
+        this.offsetReader = offsetReader;
+        this.offsetWriter = offsetWriter;
+        this.offsetStore = offsetStore;
+        this.closeExecutor = closeExecutor;
+        this.sourceTaskContext = sourceTaskContext;
+
+        this.producerSendException = new AtomicReference<>();
+        this.stopRequestedLatch = new CountDownLatch(1);
+        this.sourceTaskMetricsGroup = new SourceTaskMetricsGroup(id, connectMetrics);
+        this.topicTrackingEnabled = workerConfig.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG);
+        this.topicCreation = TopicCreation.newTopicCreation(workerConfig, topicGroups);
+    }
+
+    @Override
+    public void initialize(TaskConfig taskConfig) {
+        try {
+            this.taskConfig = taskConfig.originalsStrings();
+        } catch (Throwable t) {
+            log.error("{} Task failed initialization and will not be started.", this, t);
+            onFailure(t);
+        }
+    }
+
+    @Override
+    protected void initializeAndStart() {
+        prepareToInitializeTask();
+        // If we try to start the task at all by invoking initialize, then count this as
+        // "started" and expect a subsequent call to the task's stop() method
+        // to properly clean up any resources allocated by its initialize() or
+        // start() methods. If the task throws an exception during stop(),
+        // the worst thing that happens is another exception gets logged for an already-
+        // failed task
+        started = true;
+        task.initialize(sourceTaskContext);
+        task.start(taskConfig);
+        log.info("{} Source task finished initialization and start", this);
+    }
+
+    @Override
+    public void cancel() {
+        super.cancel();
+        // Preemptively close the offset reader in case the task is blocked on an offset read.
+        offsetReader.close();
+        // We proactively close the producer here as the main work thread for the task may
+        // be blocked indefinitely in a call to Producer::send if automatic topic creation is
+        // not enabled on either the connector or the Kafka cluster. Closing the producer should
+        // unblock it in that case and allow shutdown to proceed normally.
+        // With a duration of 0, the producer's own shutdown logic should be fairly quick,
+        // but closing user-pluggable classes like interceptors may lag indefinitely. So, we
+        // call close on a separate thread in order to avoid blocking the herder's tick thread.
+        closeExecutor.execute(() -> closeProducer(Duration.ZERO));
+    }
+
+    @Override
+    public void stop() {
+        super.stop();
+        stopRequestedLatch.countDown();
+    }
+
+    @Override
+    public void removeMetrics() {
+        Utils.closeQuietly(sourceTaskMetricsGroup::close, "source task metrics tracker");
+        super.removeMetrics();
+    }
+
+    @Override
+    protected void close() {
+        if (started) {
+            Utils.closeQuietly(task::stop, "source task");
+        }
+
+        closeProducer(Duration.ofSeconds(30));
+
+        if (admin != null) {
+            Utils.closeQuietly(() -> admin.close(Duration.ofSeconds(30)), "source task admin");
+        }
+        Utils.closeQuietly(transformationChain, "transformation chain");
+        Utils.closeQuietly(retryWithToleranceOperator, "retry operator");
+        Utils.closeQuietly(offsetReader, "offset reader");
+        Utils.closeQuietly(offsetStore::stop, "offset backing store");
+    }
+
+    private void closeProducer(Duration duration) {
+        if (producer != null) {
+            Utils.closeQuietly(() -> producer.close(duration), "source task producer");
+        }
+    }
+
+    @Override
+    public void execute() {
+        try {
+            prepareToEnterSendLoop();
+            while (!isStopping()) {
+                if (shouldPause()) {
+                    onPause();
+                    if (awaitUnpause()) {
+                        onResume();
+                        prepareToEnterSendLoop();
+                    }
+                    continue;
+                }
+
+                if (toSend == null) {
+                    prepareToPollTask();
+
+                    log.trace("{} Nothing to send to Kafka. Polling source for additional records", this);
+                    long start = time.milliseconds();
+                    toSend = poll();
+                    if (toSend != null) {
+                        recordPollReturned(toSend.size(), time.milliseconds() - start);
+                    }
+                }
+                if (toSend == null)
+                    continue;
+                log.trace("{} About to send {} records to Kafka", this, toSend.size());
+                if (sendRecords()) {
+                    batchDispatched();
+                } else {
+                    stopRequestedLatch.await(SEND_FAILED_BACKOFF_MS, TimeUnit.MILLISECONDS);
+                }
+            }
+        } catch (InterruptedException e) {
+            // Ignore and allow to exit.
+        } catch (RuntimeException e) {
+            try {
+                finalOffsetCommit(true);
+            } catch (Exception offsetException) {
+                log.error("Failed to commit offsets for already-failing task", offsetException);
+            }
+            throw e;
+        }
+        finalOffsetCommit(false);
+    }
+
+    /**
+     * Try to send a batch of records. If a send fails and is retriable, this saves the remainder of the batch so it can
+     * be retried after backing off. If a send fails and is not retriable, this will throw a ConnectException.
+     * @return true if all messages were sent, false if some need to be retried
+     */
+    // Visible for testing
+    boolean sendRecords() {
+        int processed = 0;
+        recordBatch(toSend.size());
+        final SourceRecordWriteCounter counter =
+                toSend.size() > 0 ? new SourceRecordWriteCounter(toSend.size(), sourceTaskMetricsGroup) : null;
+        for (final SourceRecord preTransformRecord : toSend) {
+            retryWithToleranceOperator.sourceRecord(preTransformRecord);
+            final SourceRecord record = transformationChain.apply(preTransformRecord);
+            final ProducerRecord<byte[], byte[]> producerRecord = convertTransformedRecord(record);
+            if (producerRecord == null || retryWithToleranceOperator.failed()) {
+                counter.skipRecord();
+                recordDropped(preTransformRecord);
+                continue;
+            }
+
+            log.trace("{} Appending record to the topic {} with key {}, value {}", this, record.topic(), record.key(), record.value());
+            prepareToSendRecord(preTransformRecord, producerRecord, !lastSendFailed);
+            try {
+                final String topic = producerRecord.topic();
+                maybeCreateTopic(topic);
+                producer.send(
+                    producerRecord,
+                    (recordMetadata, e) -> {
+                        if (e != null) {
+                            log.debug("{} failed to send record to {}: ", AbstractWorkerSourceTask.this, topic, e);
+                            log.trace("{} Failed record: {}", AbstractWorkerSourceTask.this, preTransformRecord);
+                            if (isFatalProducerSendException(e)) {
+                                producerSendException.compareAndSet(null, e);
+                            }

Review comment:
       This will have to be tweaked slightly in order to invoke `SourceTask::commitRecord` and correctly track the number of in-flight records for metrics. Leaving as-is for now since this part of the PR will have to be updated once rebased onto `trunk`.




-- 
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 change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r786314621



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java
##########
@@ -0,0 +1,507 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.connect.runtime;
+
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.errors.InvalidProducerEpochException;
+import org.apache.kafka.common.errors.TransactionAbortedException;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.metrics.stats.Avg;
+import org.apache.kafka.common.metrics.stats.Max;
+import org.apache.kafka.common.metrics.stats.Min;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.kafka.connect.source.SourceTask.TransactionBoundary;
+import org.apache.kafka.connect.storage.CloseableOffsetStorageReader;
+import org.apache.kafka.connect.storage.ClusterConfigState;
+import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore;
+import org.apache.kafka.connect.storage.Converter;
+import org.apache.kafka.connect.storage.HeaderConverter;
+import org.apache.kafka.connect.storage.OffsetStorageWriter;
+import org.apache.kafka.connect.storage.StatusBackingStore;
+import org.apache.kafka.connect.util.ConnectorTaskId;
+import org.apache.kafka.connect.util.LoggingContext;
+import org.apache.kafka.connect.util.TopicAdmin;
+import org.apache.kafka.connect.util.TopicCreationGroup;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicReference;
+
+
+/**
+ * WorkerTask that uses a SourceTask to ingest data into Kafka, with support for exactly-once delivery guarantees.
+ */
+class ExactlyOnceWorkerSourceTask extends AbstractWorkerSourceTask {
+    private static final Logger log = LoggerFactory.getLogger(ExactlyOnceWorkerSourceTask.class);
+
+    private boolean transactionOpen;
+    private final LinkedHashMap<SourceRecord, RecordMetadata> commitableRecords;
+
+    private final TransactionManager transactionManager;
+    private final TransactionMetricsGroup transactionMetrics;
+
+    private final ConnectorOffsetBackingStore offsetBackingStore;
+    private final Runnable preProducerCheck;
+    private final Runnable postProducerCheck;
+
+    public ExactlyOnceWorkerSourceTask(ConnectorTaskId id,
+                                       SourceTask task,
+                                       TaskStatus.Listener statusListener,
+                                       TargetState initialState,
+                                       Converter keyConverter,
+                                       Converter valueConverter,
+                                       HeaderConverter headerConverter,
+                                       TransformationChain<SourceRecord> transformationChain,
+                                       Producer<byte[], byte[]> producer,
+                                       TopicAdmin admin,
+                                       Map<String, TopicCreationGroup> topicGroups,
+                                       CloseableOffsetStorageReader offsetReader,
+                                       OffsetStorageWriter offsetWriter,
+                                       ConnectorOffsetBackingStore offsetBackingStore,
+                                       WorkerConfig workerConfig,
+                                       ClusterConfigState configState,
+                                       ConnectMetrics connectMetrics,
+                                       ClassLoader loader,
+                                       Time time,
+                                       RetryWithToleranceOperator retryWithToleranceOperator,
+                                       StatusBackingStore statusBackingStore,
+                                       SourceConnectorConfig sourceConfig,
+                                       Executor closeExecutor,
+                                       Runnable preProducerCheck,
+                                       Runnable postProducerCheck) {
+        super(id, task, statusListener, initialState, keyConverter, valueConverter, headerConverter, transformationChain,
+                new WorkerSourceTaskContext(offsetReader, id, configState, buildTransactionContext(sourceConfig)),
+                producer, admin, topicGroups, offsetReader, offsetWriter, offsetBackingStore, workerConfig, connectMetrics,
+                loader, time, retryWithToleranceOperator, statusBackingStore, closeExecutor);
+
+        this.transactionOpen = false;
+        this.commitableRecords = new LinkedHashMap<>();
+        this.offsetBackingStore = offsetBackingStore;
+
+        this.preProducerCheck = preProducerCheck;
+        this.postProducerCheck = postProducerCheck;
+
+        this.transactionManager = buildTransactionManager(workerConfig, sourceConfig, sourceTaskContext.transactionContext());
+        this.transactionMetrics = new TransactionMetricsGroup(id, connectMetrics);
+    }
+
+    private static WorkerTransactionContext buildTransactionContext(SourceConnectorConfig sourceConfig) {
+        return TransactionBoundary.CONNECTOR.equals(sourceConfig.transactionBoundary())
+                ? new WorkerTransactionContext()
+                : null;
+    }
+
+    @Override
+    protected void prepareToInitializeTask() {
+        preProducerCheck.run();
+
+        // Try not to start up the offset store (which has its own producer and consumer) if we've already been shut down at this point
+        if (isStopping())
+            return;
+        offsetBackingStore.start();
+
+        // Try not to initialize the transactional producer (which may accidentally fence out other, later task generations) if we've already
+        // been shut down at this point
+        if (isStopping())
+            return;
+        producer.initTransactions();
+
+        postProducerCheck.run();
+    }
+
+    @Override
+    protected void prepareToEnterSendLoop() {
+        transactionManager.initialize();
+    }
+
+    @Override
+    protected void prepareToPollTask() {
+        // No-op
+    }
+
+    @Override
+    protected void batchDispatched() {
+        transactionManager.maybeCommitTransactionForBatch();
+    }
+
+    @Override
+    protected void recordDropped(SourceRecord record) {
+        synchronized (this) {
+            commitableRecords.put(record, null);
+        }
+        transactionManager.maybeCommitTransactionForRecord(record);
+    }
+
+    @Override
+    protected void prepareToSendRecord(SourceRecord sourceRecord, ProducerRecord<byte[], byte[]> producerRecord, boolean newRecord) {
+        if (offsetBackingStore.primaryOffsetsTopic().equals(producerRecord.topic())) {
+            // This is to prevent deadlock that occurs when:
+            //     1. A task provides a record whose topic is the task's offsets topic
+            //     2. That record is dispatched to the task's producer in a transaction that remains open
+            //        at least until the worker polls the task again
+            //     3. In the subsequent call to SourceTask::poll, the task requests offsets from the worker
+            //        (which requires a read to the end of the offsets topic, and will block until any open
+            //        transactions on the topic are either committed or aborted)
+            throw new ConnectException("Source tasks may not produce to their own offsets topics when exactly-once support is enabled");
+        }
+        maybeBeginTransaction();
+    }
+
+    @Override
+    protected void recordDispatched(SourceRecord record) {
+        // Offsets are converted & serialized in the OffsetWriter
+        // Important: we only save offsets for the record after it has been accepted by the producer; this way,
+        // we commit those offsets if and only if the record is sent successfully.
+        offsetWriter.offset(record.sourcePartition(), record.sourceOffset());
+        transactionMetrics.addRecord();
+        transactionManager.maybeCommitTransactionForRecord(record);
+    }
+
+    @Override
+    protected void recordSent(SourceRecord sourceRecord, ProducerRecord<byte[], byte[]> producerRecord, RecordMetadata recordMetadata) {
+        synchronized (this) {
+            commitableRecords.put(sourceRecord, recordMetadata);
+        }
+    }
+
+    @Override
+    protected void finalOffsetCommit(boolean failed) {
+        if (failed) {
+            log.debug("Skipping final offset commit as task has failed");
+            return;
+        }
+
+        // It should be safe to commit here even if we were in the middle of retrying on RetriableExceptions in the
+        // send loop since we only track source offsets for records that have been successfully dispatched to the
+        // producer.
+        // Any records that we were retrying on (and any records after them in the batch) won't be included in the
+        // transaction and their offsets won't be committed, but (unless the user has requested connector-defined
+        // transaction boundaries), it's better to commit some data than none.
+        transactionManager.maybeCommitFinalTransaction();
+    }
+
+    @Override
+    protected void onPause() {
+        super.onPause();
+        // Commit the transaction now so that we don't end up with a hanging transaction, or worse, get fenced out
+        // and fail the task once unpaused
+        transactionManager.maybeCommitFinalTransaction();
+    }
+
+    private void maybeBeginTransaction() {
+        if (!transactionOpen) {
+            producer.beginTransaction();
+            transactionOpen = true;
+        }
+    }
+
+    private void commitTransaction() {
+        log.debug("{} Committing offsets", this);
+
+        long started = time.milliseconds();
+
+        // Necessary as a workaround for https://issues.apache.org/jira/browse/KAFKA-9279,
+        // where some calls to Producer::send fail asynchronously but do not cause a subsequent call
+        // to Producer::commitTransaction to throw an exception
+        // We can remove this once a fix for KAFKA-9279 is merged
+        maybeThrowProducerSendException();

Review comment:
       I've filed https://github.com/apache/kafka/pull/11508 as a fix for [KAFKA-9279](https://issues.apache.org/jira/browse/KAFKA-9279). The workaround here isn't terribly awkward so it probably isn't necessary for us to block on that fix PR, but it'd be nice if someone could find the time to review 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] C0urante commented on pull request #10907: KAFKA-10000: Exactly-once support for source connectors

Posted by GitBox <gi...@apache.org>.
C0urante commented on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-865396213


   There was a small misunderstanding of the rebalance logic for distributed workers. It turns out that they already preemptively stop reconfigured tasks before (re)joining the group, so no additional logic is necessary to stop reconfigured source tasks before they are naturally fenced out by the leader. The KIP has been updated with this note and the changes in the PR related to it have been reverted.


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

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



[GitHub] [kafka] C0urante commented on a change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r732231502



##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java
##########
@@ -28,4 +30,31 @@
     protected SourceConnectorContext context() {
         return (SourceConnectorContext) context;
     }
+
+    /**
+     * Signals whether the connector supports exactly-once delivery guarantees with a proposed configuration.
+     * Developers can assume that worker-level exactly-once support is enabled when this method is invoked.
+     * The default implementation will return {@code null}.
+     * @param connectorConfig the configuration that will be used for the connector.
+     * @return {@link ExactlyOnceSupport#SUPPORTED} if the connector can provide exactly-once support,
+     * and {@link ExactlyOnceSupport#UNSUPPORTED} if it cannot. If {@code null}, it is assumed that the
+     * connector cannot.
+     */
+    public ExactlyOnceSupport exactlyOnceSupport(Map<String, String> connectorConfig) {
+        return null;

Review comment:
       > Should we just have another enum for UNKNOWN and make this more explicit than "null"?
   
   This was actually [suggested in the discussion thread](https://mail-archives.apache.org/mod_mbox/kafka-dev/202105.mbox/%3cCAMdOrUX4CvPsb+yjfTenHyRTtE=2aaw-_-_b2VbD+pvqzY7bxw@mail.gmail.com%3e):
   
   > what do you think about a new "exactlyOnce()" method to the SourceConnector class that can return a new ExactlyOnce enum with options of "SUPPORTED", "UNSUPPORTED", and "UNKNOWN", with a default implementation that returns "UNKNOWN"?
   
   And [decided against](https://mail-archives.apache.org/mod_mbox/kafka-dev/202105.mbox/%3cCADXUNmbsypos0LeJ8KXW9EAPCxC7wbtGXQDQHrpu6qRbJWiaYQ@mail.gmail.com%3e):
   
   > The problem with having an explicit UNKNOWN case is we really want connector developers to _not_ use it. That could mean it's deprecated from the start. Alternatively we could omit it from the enum and use null to mean unknown (we'd have to check for a null result anyway), with the contract for the method being that it should return non-null. Of course, this doesn't remove the ambiguous case, but avoids the need to eventually remove UNKNOWN in the future.
   
   (What I found especially convincing in the snippet above were the points that 1) we don't want people to return `UNKNOWN` from this method, and 2) no matter what, we're going to have to check for `null` anyways.)
   
   
   > Also, it seems like it would make sense to document that this method should be overridden by Connector developers, but has a default for backward compatibility.
   
   Ack, can do.
   
   > And it should state more clearly what should be returned for the various options.
   
   I've taken a shot at this, not sure how much clearer it can get but if you have thoughts let me know.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java
##########
@@ -20,13 +20,45 @@
 import org.apache.kafka.clients.producer.RecordMetadata;
 
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * SourceTask is a Task that pulls records from another system for storage in Kafka.
  */
 public abstract class SourceTask implements Task {
 
+    /**
+     * <p>
+     * The configuration key that determines how source tasks will define transaction boundaries
+     * when exactly-once support is enabled.
+     * </p>
+     */
+    public static final String TRANSACTION_BOUNDARY_CONFIG = "transaction.boundary";
+
+    public enum TransactionBoundary {

Review comment:
       Ack, done.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java
##########
@@ -20,13 +20,45 @@
 import org.apache.kafka.clients.producer.RecordMetadata;
 
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * SourceTask is a Task that pulls records from another system for storage in Kafka.
  */
 public abstract class SourceTask implements Task {
 
+    /**
+     * <p>
+     * The configuration key that determines how source tasks will define transaction boundaries
+     * when exactly-once support is enabled.
+     * </p>
+     */
+    public static final String TRANSACTION_BOUNDARY_CONFIG = "transaction.boundary";
+
+    public enum TransactionBoundary {
+        POLL,
+        INTERVAL,
+        CONNECTOR;
+
+        public static final TransactionBoundary DEFAULT = POLL;
+
+        public static List<String> options() {

Review comment:
       I wanted a convenient way to bring everything to lowercase, which is more standard for properties like this (see how [values for the consumer `isolation.level` property are rendered](https://github.com/apache/kafka/blob/da38a1df273ec9d3a077435b2a63d75053edd308/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java#L557), for example) and, IMO, more readable.
   
   We could remove this method and replace it with inline calls to `values()` followed by some streams magic to lowercase at the call site, but that seemed less clean than this approach.
   
   Alternatively, we could introduce a new utility method (such as `Utils::enumNames`) that does this for us in a centralized, reusable location and obviates the need for public-API methods that may cause headaches down the road.
   
   Thoughts?

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java
##########
@@ -192,6 +198,19 @@
     public static final String INTER_WORKER_VERIFICATION_ALGORITHMS_DOC = "A list of permitted algorithms for verifying internal requests";
     public static final List<String> INTER_WORKER_VERIFICATION_ALGORITHMS_DEFAULT = Collections.singletonList(INTER_WORKER_SIGNATURE_ALGORITHM_DEFAULT);
 
+    public static final String EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG = "exactly.once.source.support";
+    public static final String EXACTLY_ONCE_SOURCE_SUPPORT_DOC = "Whether to enable exactly-once support for source connectors in the cluster "
+            + "by writing source records and their offsets in a Kafka transaction, and by proactively fencing out old task generations before bringing up new ones. "
+            + "Note that this must be enabled on every worker in a cluster in order for exactly-once delivery to be guaranteed, "
+            + "and that some source connectors may still not be able to provide exactly-once delivery guarantees even with this support enabled. "
+            + "Permitted values are \"disabled\", \"preparing\", and \"enabled\". In order to safely enable exactly-once support for source connectors, "
+            + "all workers in the cluster must first be updated to use the \"preparing\" value for this property. "
+            + "Once this has been done, a second update of all of the workers in the cluster should be performed to change the value of this property to \"enabled\".";

Review comment:
       > It's clear to me that we should mention this, but it's not clear where we should do so. The user documentation generated from this doc string might be one spot that users will see routinely, so maybe it's a candidate. Another would be in the Kafka Connect docs about EOS for source connectors.
   
   I think both are acceptable. I'll add this to the docstring now and include it in the high-level docs when I write those as well.
   
   > BTW, can you add to this PR changes to the Kafka docs that describe this feature?
   
   Given how massive this PR is already, I'd like to do this in a follow-up, with the understanding that Kafka docs changes are a requisite for including this feature in a release.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java
##########
@@ -401,10 +429,46 @@ public Integer getRebalanceTimeout() {
         return getInt(DistributedConfig.REBALANCE_TIMEOUT_MS_CONFIG);
     }
 
+    @Override
+    public boolean exactlyOnceSourceEnabled() {
+        return EXACTLY_ONCE_SOURCE_SUPPORT_ENABLED.equalsIgnoreCase(
+                getString(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG)
+        );
+    }
+
+    public boolean transactionalLeaderEnabled() {
+        return Arrays.asList(EXACTLY_ONCE_SOURCE_SUPPORT_ENABLED, EXACTLY_ONCE_SOURCE_SUPPORT_PREPARING)
+                .contains(getString(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG).toLowerCase(Locale.ROOT));
+    }

Review comment:
       > Should we have an enum for the enabled, preparing and disabled literals, and should these make use of them?
   
   Sure, gave that a shot, LMK what you think.
   
   > Also, it might be useful to have JavaDoc on these methods, simply to help future developers understand the intent.
   
   Gave this a try too.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java
##########
@@ -156,6 +165,14 @@ public KafkaBasedLog(String topic,
         this.readLogEndOffsetCallbacks = new ArrayDeque<>();
         this.time = time;
         this.initializer = initializer != null ? initializer : admin -> { };
+        this.topicContainsTransactions = topicContainsTransactions;
+
+        // If the consumer is configured with isolation.level = read_committed, then its end offsets method cannot be relied on
+        // as it will not take records from currently-open transactions into account. We want to err on the side of caution in that
+        // case: when users request a read to the end of the log, we will read up to the point where the latest offsets visible to the
+        // consumer are at least as high as the (possibly-part-of-a-transaction) end offsets of the topic.
+        this.requireAdminForOffsets = IsolationLevel.READ_COMMITTED.name().toLowerCase(Locale.ROOT)
+                .equals(consumerConfigs.get(ConsumerConfig.ISOLATION_LEVEL_CONFIG));

Review comment:
       This was my first thought, but the `isolation.level` property is case-sensitive.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java
##########
@@ -118,35 +123,39 @@ public KafkaBasedLog(String topic,
                          Callback<ConsumerRecord<K, V>> consumedCallback,
                          Time time,
                          Runnable initializer) {
-        this(topic, producerConfigs, consumerConfigs, () -> null, consumedCallback, time, initializer != null ? admin -> initializer.run() : null);
+        this(topic, producerConfigs, consumerConfigs, () -> null, consumedCallback, time, initializer != null ? admin -> initializer.run() : null, false);
     }
 
     /**
      * Create a new KafkaBasedLog object. This does not start reading the log and writing is not permitted until
      * {@link #start()} is invoked.
      *
-     * @param topic              the topic to treat as a log
-     * @param producerConfigs    configuration options to use when creating the internal producer. At a minimum this must
+     * @param topic                     the topic to treat as a log
+     * @param producerConfigs           configuration options to use when creating the internal producer. At a minimum this must
      *                           contain compatible serializer settings for the generic types used on this class. Some
      *                           setting, such as the number of acks, will be overridden to ensure correct behavior of this
      *                           class.
-     * @param consumerConfigs    configuration options to use when creating the internal consumer. At a minimum this must
+     * @param consumerConfigs           configuration options to use when creating the internal consumer. At a minimum this must
      *                           contain compatible serializer settings for the generic types used on this class. Some
      *                           setting, such as the auto offset reset policy, will be overridden to ensure correct
      *                           behavior of this class.
-     * @param topicAdminSupplier supplier function for an admin client, the lifecycle of which is expected to be controlled
+     * @param topicAdminSupplier        supplier function for an admin client, the lifecycle of which is expected to be controlled
      *                           by the calling component; may not be null
-     * @param consumedCallback   callback to invoke for each {@link ConsumerRecord} consumed when tailing the log
-     * @param time               Time interface
-     * @param initializer        the function that should be run when this log is {@link #start() started}; may be null
+     * @param consumedCallback          callback to invoke for each {@link ConsumerRecord} consumed when tailing the log
+     * @param time                      Time interface
+     * @param initializer               the function that should be run when this log is {@link #start() started}; may be null
+     * @param topicContainsTransactions whether the topic being consumed contains (or is expected to contain) transactions;
+     *                                  if this is {@code false} and the topic does contain transactions, reads to the end of the log may block
+     *                                  indefinitely

Review comment:
       I'll rebase this PR on top of https://github.com/apache/kafka/pull/11046 and, after removing any changes to this class that become unnecessary as a result, will address this.
   
   I do believe a KIP is called for here though. As you've noted, this class is not public API and it's frustrating that development efforts against Connect are hampered by this unofficial and somewhat arbitrary restriction.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java
##########
@@ -210,26 +227,21 @@ public void stop() {
         synchronized (this) {
             stopRequested = true;
         }
-        consumer.wakeup();
-
-        try {
-            thread.join();
-        } catch (InterruptedException e) {
-            throw new ConnectException("Failed to stop KafkaBasedLog. Exiting without cleanly shutting " +
-                    "down it's producer and consumer.", e);
+        if (consumer != null) {
+            consumer.wakeup();
         }
 
-        try {
-            producer.close();
-        } catch (KafkaException e) {
-            log.error("Failed to stop KafkaBasedLog producer", e);
+        if (thread != null) {
+            try {
+                thread.join();
+            } catch (InterruptedException e) {
+                throw new ConnectException("Failed to stop KafkaBasedLog. Exiting without cleanly shutting " +
+                        "down it's producer and consumer.", e);
+            }
         }
 
-        try {
-            consumer.close();
-        } catch (KafkaException e) {
-            log.error("Failed to stop KafkaBasedLog consumer", e);
-        }
+        Utils.closeQuietly(producer, "KafkaBasedLog producer");
+        Utils.closeQuietly(consumer, "KafkaBasedLog consumer");

Review comment:
       Fewer lines, more standardized, and logging these at `ERROR` level is incorrect IMO.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectUtils.java
##########
@@ -72,6 +74,45 @@ static String lookupKafkaClusterId(Admin adminClient) {
         }
     }
 
+    /**
+     * Log a warning when the user attempts to override a property that cannot be overridden.
+     * @param props the configuration properties provided by the user
+     * @param key the name of the property to check on
+     * @param expectedValue the expected value for the property
+     * @param justification the reason the property cannot be overridden.
+     *                      Will follow the phrase "The value... for the... property will be ignored as it cannot be overridden ".
+     *                      For example, one might supply the message "in connectors with the DLQ feature enabled" for this parameter.
+     * @param caseSensitive whether the value should match case-insensitively
+     */
+    public static void warnOnOverriddenProperty(
+            Map<String, ?> props,
+            String key,
+            String expectedValue,
+            String justification,
+            boolean caseSensitive) {
+        overriddenPropertyWarning(props, key, expectedValue, justification, caseSensitive).ifPresent(log::warn);
+    }
+
+    // Visible for testing
+    static Optional<String> overriddenPropertyWarning(
+            Map<String, ?> props,
+            String key,
+            String expectedValue,
+            String justification,
+            boolean caseSensitive) {
+        Predicate<String> matchesExpectedValue = caseSensitive ? expectedValue::equals : expectedValue::equalsIgnoreCase;
+        String value = Optional.ofNullable(props.get(key)).map(Object::toString).orElse(null);
+        if (value != null && !matchesExpectedValue.test(value)) {
+            return Optional.of(String.format(
+                    "The value '%s' for the '%s' property will be ignored as it cannot be overridden %s. "
+                            + "The value '%s' will be used instead.",
+                    value, key, justification, expectedValue
+            ));
+        } else {
+            return Optional.empty();
+        }

Review comment:
       👍  &nbsp; SGTM

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java
##########
@@ -43,6 +44,13 @@
  * storage to achieve exactly once semantics).
  * </p>
  * <p>
+ * In order to support per-connector offsets topics but continue to back up progress to a
+ * cluster-global offsets topic, the writer accepts an optional <i>secondary backing store</i>.
+ * After successful flushes to the primary backing store, the writer will copy the flushed offsets
+ * over to the secondary backing store on a best-effort basis. Failures to write to the secondary
+ * store are logged but otherwise swallowed silently.
+ * </p>
+ * <p>

Review comment:
       This is a neat idea. It'd also simplify the offset read logic. I've taken a stab at this; LMK what you think.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java
##########
@@ -114,12 +124,18 @@ public synchronized boolean beginFlush() {
         if (data.isEmpty())
             return false;
 
-        assert !flushing();

Review comment:
       It's incorrect; this is a known issue where a race condition involving task failure during offset commit can cause this assertion to fail. The assertion itself has provided no value in catching, reproducing, or testing against this issue and at this point is more misleading than useful.




-- 
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 change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r732233337



##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/TransactionContext.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.source;
+
+/**
+ * Provided to source tasks to allow them to define their own producer transaction boundaries when
+ * exactly-once support is enabled.
+ */
+public interface TransactionContext {
+
+    /**
+     * Request a transaction commit after the next batch of records from {@link SourceTask#poll()}
+     * is processed.
+     */
+    void commitTransaction();
+
+    /**
+     * Request a transaction commit after a source record is processed. The source record will be the
+     * last record in the committed transaction.
+     * @param record the record to commit the transaction after.

Review comment:
       Ack, added "may not be null".

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/TransactionContext.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.source;
+
+/**
+ * Provided to source tasks to allow them to define their own producer transaction boundaries when
+ * exactly-once support is enabled.
+ */
+public interface TransactionContext {
+
+    /**
+     * Request a transaction commit after the next batch of records from {@link SourceTask#poll()}
+     * is processed.
+     */
+    void commitTransaction();
+
+    /**
+     * Request a transaction commit after a source record is processed. The source record will be the
+     * last record in the committed transaction.
+     * @param record the record to commit the transaction after.
+     */
+    void commitTransaction(SourceRecord record);
+
+    /**
+     * Requests a transaction abort the next batch of records from {@link SourceTask#poll()}. All of
+     * the records in that transaction will be discarded and will not appear in a committed transaction.
+     * However, offsets for that transaction will still be committed. If the data should be reprocessed,
+     * the task should not invoke this method and should instead throw an exception.
+     */
+    void abortTransaction();
+
+    /**
+     * Requests a transaction abort after a source record is processed. The source record will be the
+     * last record in the aborted transaction. All of the records in that transaction will be discarded
+     * and will not appear in a committed transaction. However, offsets for that transaction will still
+     * be committed. If the data should be reprocessed, the task should not invoke this method and
+     * should instead throw an exception.
+     * @param record the record to abort the transaction after.

Review comment:
       Ack, added "may not be null".




-- 
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] rhauch commented on a change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
rhauch commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r779947401



##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceTaskContext.java
##########
@@ -38,4 +38,29 @@
      * Get the OffsetStorageReader for this SourceTask.
      */
     OffsetStorageReader offsetStorageReader();
+
+    /**
+     * Get a {@link TransactionContext} that can be used to define producer transaction boundaries
+     * when exactly-once support is enabled for the connector.
+     *
+     * <p>This method was added in Apache Kafka 3.0. Source tasks that use this method but want to
+     * maintain backward compatibility so they can also be deployed to older Connect runtimes
+     * should guard the call to this method with a try-catch block, since calling this method will result in a
+     * {@link NoSuchMethodException} or {@link NoClassDefFoundError} when the source connector is deployed to
+     * Connect runtimes older than Kafka 3.0. For example:
+     * <pre>
+     *     TransactionContext transactionContext;
+     *     try {
+     *         transactionContext = context.transactionContext();
+     *     } catch (NoSuchMethodError | NoClassDefFoundError e) {
+     *         transactionContext = null;
+     *     }
+     * </pre>
+     *
+     * @return the transaction context, or null if the connector was not configured to specify transaction boundaries
+     * @since 3.0

Review comment:
       Please change all of these `@since 3.0` to `@since 3.1`, plus any other `Kafka 3.0` references in JavaDoc.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java
##########
@@ -125,28 +130,28 @@ public KafkaBasedLog(String topic,
      * Create a new KafkaBasedLog object. This does not start reading the log and writing is not permitted until
      * {@link #start()} is invoked.
      *
-     * @param topic              the topic to treat as a log
-     * @param producerConfigs    configuration options to use when creating the internal producer. At a minimum this must
+     * @param topic                     the topic to treat as a log
+     * @param producerConfigs           configuration options to use when creating the internal producer. At a minimum this must
      *                           contain compatible serializer settings for the generic types used on this class. Some
      *                           setting, such as the number of acks, will be overridden to ensure correct behavior of this
      *                           class.
-     * @param consumerConfigs    configuration options to use when creating the internal consumer. At a minimum this must
+     * @param consumerConfigs           configuration options to use when creating the internal consumer. At a minimum this must
      *                           contain compatible serializer settings for the generic types used on this class. Some
      *                           setting, such as the auto offset reset policy, will be overridden to ensure correct
      *                           behavior of this class.
-     * @param topicAdminSupplier supplier function for an admin client, the lifecycle of which is expected to be controlled
+     * @param topicAdminSupplier        supplier function for an admin client, the lifecycle of which is expected to be controlled
      *                           by the calling component; may not be null
-     * @param consumedCallback   callback to invoke for each {@link ConsumerRecord} consumed when tailing the log
-     * @param time               Time interface
-     * @param initializer        the function that should be run when this log is {@link #start() started}; may be null
+     * @param consumedCallback          callback to invoke for each {@link ConsumerRecord} consumed when tailing the log
+     * @param time                      Time interface
+     * @param initializer               the function that should be run when this log is {@link #start() started}; may be null
      */
     public KafkaBasedLog(String topic,
-            Map<String, Object> producerConfigs,
-            Map<String, Object> consumerConfigs,
-            Supplier<TopicAdmin> topicAdminSupplier,
-            Callback<ConsumerRecord<K, V>> consumedCallback,
-            Time time,
-            java.util.function.Consumer<TopicAdmin> initializer) {
+                         Map<String, Object> producerConfigs,
+                         Map<String, Object> consumerConfigs,
+                         Supplier<TopicAdmin> topicAdminSupplier,
+                         Callback<ConsumerRecord<K, V>> consumedCallback,
+                         Time time,
+                         java.util.function.Consumer<TopicAdmin> initializer) {

Review comment:
       Why make these unnecessary changes?

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java
##########
@@ -74,15 +75,15 @@
     private volatile boolean cancelled; // indicates whether the Worker has cancelled the connector (e.g. because of slow shutdown)
 
     private State state;
-    private final OffsetStorageReader offsetStorageReader;
+    private final CloseableOffsetStorageReader offsetStorageReader;
 
     public WorkerConnector(String connName,
                            Connector connector,
                            ConnectorConfig connectorConfig,
                            CloseableConnectorContext ctx,
                            ConnectMetrics metrics,
                            ConnectorStatus.Listener statusListener,
-                           OffsetStorageReader offsetStorageReader,
+                           CloseableOffsetStorageReader offsetStorageReader,

Review comment:
       As mentioned above, IIUC this PR will sometimes pass a null `offsetStorageReader` (IIRC for sink connectors), but this class currently expects that to be null. Might be worth adding a `Objects.requireNonNull(...)` call here to help catch that situation.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java
##########
@@ -28,4 +30,39 @@
     protected SourceConnectorContext context() {
         return (SourceConnectorContext) context;
     }
+
+    /**
+     * Signals whether the connector supports exactly-once delivery guarantees with a proposed configuration.
+     * Developers can assume that worker-level exactly-once support is enabled when this method is invoked.
+     * For backwards compatibility, the default implementation will return {@code null}, but connector developers are

Review comment:
       Nit: using a new paragraph makes this stand out more.
   ```suggestion
        * <p>For backwards compatibility, the default implementation will return {@code null}, but connector developers are
   ```

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/TransactionContext.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.source;
+
+/**
+ * Provided to source tasks to allow them to define their own producer transaction boundaries when
+ * exactly-once support is enabled.
+ */
+public interface TransactionContext {

Review comment:
       IIUC, the `WorkerTransactionContext` is the only implementation of this. That means that if a connector is configured with `transaction.boundary=poll` or `transaction.boundary=interval`, a poorly-implemented connector could still call these methods and they'd unnecessarily accumulate records.
   
   WDYT about in such cases the `SourceTaskContext#transactionContext()` method returning a no-op implementation of this interface, so no harm is done if a connector implementation still calls these methods when `transaction.boundary` is _not_ set to `connector`?
   
   Maybe we could consider a warning log message if these methods are called by a connector inappropriately. But we have to be careful. While such log messages might be useful for the **developer** of a connector plugin, I would argue that _prolific_ warnings are actually harmful for a **user** trying to _use_ a connector plugin they didn't develop with a connector configuration that includes `transaction.boundary=poll` or `transaction.boundary=interval`. So maybe it's worthwhile for the "no-op" implementation to only log each warning once per method per instance.

##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedConfigTest.java
##########
@@ -294,4 +298,41 @@ public void shouldRemoveCompactionFromStatusTopicSettings() {
         assertEquals(expectedTopicSettings, actual);
         assertNotEquals(topicSettings, actual);
     }
+
+    @Test
+    public void shouldIdentifyNeedForTransactionalLeader() {
+        Map<String, String> workerProps = configs();
+
+        workerProps.put(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG, "disabled");
+        assertFalse(new DistributedConfig(workerProps).transactionalLeaderEnabled());
+
+        workerProps.put(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG, "preparing");
+        assertTrue(new DistributedConfig(workerProps).transactionalLeaderEnabled());
+
+        workerProps.put(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG, "enabled");
+        assertTrue(new DistributedConfig(workerProps).transactionalLeaderEnabled());
+    }
+
+    @Test
+    public void shouldConstructExpectedTransactionalId() {
+        Map<String, String> workerProps = configs();
+
+        workerProps.put(GROUP_ID_CONFIG, "why did i stay up all night writing unit tests");
+        assertEquals(
+                "connect-cluster-why did i stay up all night writing unit tests",
+                new DistributedConfig(workerProps).transactionalProducerId()
+        );
+
+        workerProps.put(GROUP_ID_CONFIG, "connect-cluster");
+        assertEquals(
+                "connect-cluster-connect-cluster",
+                new DistributedConfig(workerProps).transactionalProducerId()
+        );
+
+        workerProps.put(GROUP_ID_CONFIG, "\u2603");
+        assertEquals(
+                "connect-cluster-\u2603",
+                new DistributedConfig(workerProps).transactionalProducerId()
+        );
+    }

Review comment:
       Let's add some negative tests that verify that invalid values for the new worker configuration properties are properly handled/identified by the validators.
   
   We should also add positive and negative tests for the new connector-level config properties in  `SourceConnectorConfigTest`

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java
##########
@@ -365,6 +413,10 @@ private void readToLogEnd() {
                 // This may happen with really old brokers that don't support the auto topic creation
                 // field in metadata requests
                 log.debug("Reading to end of log offsets with consumer since admin client is unsupported: {}", e.getMessage());
+                if (requireAdminForOffsets) {
+                    // Should be handled by the caller during log startup
+                    throw e;
+                }

Review comment:
       It's true that the `DistributedHerder.run()` is ultimately catching and handling this exception. But I feel like many users might not understand the significance of such an error nor how to correct their configuration. Rather than just re-throw that exception, we should probably wrap that exception with one that has a more instructive message, such as something like:
   > Enabling exactly once for source connectors requires a Kafka broker version that allows admin clients to read consumer offsets. Disable the worker's exactly once support for source connectors, or use a newer Kafka broker version.
   
   Plus, should this if block be before the `log.debug(...)` on the previous line? Seems like that log message might just confuse the situation since the worker will not read "to the end of log offsets with consumer".

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -264,12 +279,62 @@ public void startConnector(
 
                 log.info("Creating connector {} of type {}", connName, connClass);
                 final Connector connector = plugins.newConnector(connClass);
-                final ConnectorConfig connConfig = ConnectUtils.isSinkConnector(connector)
-                        ? new SinkConnectorConfig(plugins, connProps)
-                        : new SourceConnectorConfig(plugins, connProps, config.topicCreationEnable());
+                final ConnectorConfig connConfig;
+                final CloseableOffsetStorageReader offsetReader;
+                if (ConnectUtils.isSinkConnector(connector)) {
+                    connConfig = new SinkConnectorConfig(plugins, connProps);
+                    offsetReader = null;

Review comment:
       IIUC, this changes the behavior of the `WorkerConnector` created below. Prior to this PR, the `WorkerConnector` was always created with the `Worker.offsetBackingStore`, even for sink connectors. 
   
   However, with this PR, the `WorkerConnector` will be instantiated with a null `offsetReader` parameter, which will cause a NPE in `WorkerConnector#doShutdown()` and `WorkerConnector#cancel()`  since `WorkerConnector` does not check for a null parameter there.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConnectorOffsetBackingStore.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.storage;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.connect.runtime.WorkerConfig;
+import org.apache.kafka.connect.util.Callback;
+import org.apache.kafka.connect.util.LoggingContext;
+import org.apache.kafka.connect.util.TopicAdmin;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+public class ConnectorOffsetBackingStore implements OffsetBackingStore {

Review comment:
       Please add JavaDoc that explains the purpose of this class, and in particular, what the two modes are that correspond to the two static factory methods.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConnectorOffsetBackingStore.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.storage;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.connect.runtime.WorkerConfig;
+import org.apache.kafka.connect.util.Callback;
+import org.apache.kafka.connect.util.LoggingContext;
+import org.apache.kafka.connect.util.TopicAdmin;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+public class ConnectorOffsetBackingStore implements OffsetBackingStore {
+
+    private static final Logger log = LoggerFactory.getLogger(ConnectorOffsetBackingStore.class);
+
+    private final Time time;
+    private final Supplier<LoggingContext> loggingContext;
+    private final String primaryOffsetsTopic;
+    private final OffsetBackingStore workerStore;
+    private final Optional<OffsetBackingStore> connectorStore;
+    private final Optional<TopicAdmin> connectorStoreAdmin;
+
+    public static ConnectorOffsetBackingStore withConnectorOffsetStore(

Review comment:
       Nit: move these two static factory methods above the non-static member variables, so all static and non-static members are together.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java
##########
@@ -192,6 +199,44 @@
     public static final String INTER_WORKER_VERIFICATION_ALGORITHMS_DOC = "A list of permitted algorithms for verifying internal requests";
     public static final List<String> INTER_WORKER_VERIFICATION_ALGORITHMS_DEFAULT = Collections.singletonList(INTER_WORKER_SIGNATURE_ALGORITHM_DEFAULT);
 
+    private enum ExactlyOnceSourceSupport {
+        DISABLED(false),
+        PREPARING(true),
+        ENABLED(true);
+
+        public final boolean usesTransactionalLeader;
+
+        ExactlyOnceSourceSupport(boolean usesTransactionalLeader) {
+            this.usesTransactionalLeader = usesTransactionalLeader;
+        }
+
+        public static List<String> options() {
+            return Stream.of(values()).map(ExactlyOnceSourceSupport::toString).collect(Collectors.toList());
+        }
+
+        public static ExactlyOnceSourceSupport fromProperty(String property) {
+            return ExactlyOnceSourceSupport.valueOf(property.toUpperCase(Locale.ROOT));
+        }
+
+        @Override
+        public String toString() {
+            return name().toLowerCase(Locale.ROOT);
+        }
+    }
+
+    public static final String EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG = "exactly.once.source.support";
+    public static final String EXACTLY_ONCE_SOURCE_SUPPORT_DOC = "Whether to enable exactly-once support for source connectors in the cluster "
+            + "by writing source records and their offsets in a Kafka transaction, and by proactively fencing out old task generations before bringing up new ones. "

Review comment:
       Nit, to improve readability and precision, especially around how many Kafka transactions would be used:
   > Whether to enable exactly-once support for source connectors in the cluster by using transactions to write source records and their source offsets, and by proactively fencing out old task generations before bringing up new ones.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectUtils.java
##########
@@ -72,6 +74,45 @@ static String lookupKafkaClusterId(Admin adminClient) {
         }
     }
 
+    /**
+     * Log a warning when the user attempts to override a property that cannot be overridden.
+     * @param props the configuration properties provided by the user
+     * @param key the name of the property to check on
+     * @param expectedValue the expected value for the property
+     * @param justification the reason the property cannot be overridden.
+     *                      Will follow the phrase "The value... for the... property will be ignored as it cannot be overridden ".
+     *                      For example, one might supply the message "in connectors with the DLQ feature enabled" for this parameter.
+     * @param caseSensitive whether the value should match case-insensitively
+     */
+    public static void warnOnOverriddenProperty(
+            Map<String, ?> props,
+            String key,
+            String expectedValue,
+            String justification,
+            boolean caseSensitive) {
+        overriddenPropertyWarning(props, key, expectedValue, justification, caseSensitive).ifPresent(log::warn);
+    }
+
+    // Visible for testing
+    static Optional<String> overriddenPropertyWarning(
+            Map<String, ?> props,
+            String key,
+            String expectedValue,
+            String justification,
+            boolean caseSensitive) {
+        Predicate<String> matchesExpectedValue = caseSensitive ? expectedValue::equals : expectedValue::equalsIgnoreCase;
+        String value = Optional.ofNullable(props.get(key)).map(Object::toString).orElse(null);
+        if (value != null && !matchesExpectedValue.test(value)) {
+            return Optional.of(String.format(
+                    "The value '%s' for the '%s' property will be ignored as it cannot be overridden%s. "
+                            + "The value '%s' will be used instead.",

Review comment:
       Is this really true that all of these incorrectly-overridden property values are ignored?

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -264,12 +279,62 @@ public void startConnector(
 
                 log.info("Creating connector {} of type {}", connName, connClass);
                 final Connector connector = plugins.newConnector(connClass);
-                final ConnectorConfig connConfig = ConnectUtils.isSinkConnector(connector)
-                        ? new SinkConnectorConfig(plugins, connProps)
-                        : new SourceConnectorConfig(plugins, connProps, config.topicCreationEnable());
+                final ConnectorConfig connConfig;
+                final CloseableOffsetStorageReader offsetReader;
+                if (ConnectUtils.isSinkConnector(connector)) {
+                    connConfig = new SinkConnectorConfig(plugins, connProps);
+                    offsetReader = null;
+                } else {
+                    SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connProps, config.topicCreationEnable());
+                    connConfig = sourceConfig;
+
+                    String connectorOffsetsTopic = null;
+                    if (sourceConfig.offsetsTopic() != null) {
+                        connectorOffsetsTopic = sourceConfig.offsetsTopic();
+                    } else if (config.exactlyOnceSourceEnabled()) {
+                        connectorOffsetsTopic = config.offsetsTopic();
+                    }
+

Review comment:
       Suggestion:
   ```suggestion
   
                       // Set up the offset backing store for this connector instance
   ```

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -678,10 +810,9 @@ private WorkerTask buildWorkerTask(ClusterConfigState configState,
         // and through to the task
         Map<String, Object> consumerProps = new HashMap<>();
 
-        consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, SinkUtils.consumerGroupId(id.connector()));
-        consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, "connector-consumer-" + id);
-        consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG,
-                  Utils.join(config.getList(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG), ","));
+        consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, SinkUtils.consumerGroupId(connName));
+        consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, defaultClientId);
+        consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, config.bootstrapServers());

Review comment:
       Thanks! This should have been done quite some time ago.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -476,22 +541,95 @@ public boolean isRunning(String connName) {
     }
 
     /**
-     * Start a task managed by this worker.
+     * Start a sink task managed by this worker.
+     *
+     * @param id the task ID.
+     * @param configState the most recent {@link ClusterConfigState} known to the worker
+     * @param connProps the connector properties.
+     * @param taskProps the tasks properties.
+     * @param statusListener a listener for the runtime status transitions of the task.
+     * @param initialState the initial state of the connector.
+     * @return true if the task started successfully.
+     */
+    public boolean startSinkTask(
+            ConnectorTaskId id,
+            ClusterConfigState configState,
+            Map<String, String> connProps,
+            Map<String, String> taskProps,
+            TaskStatus.Listener statusListener,
+            TargetState initialState
+    ) {
+        return startTask(id, connProps, taskProps, statusListener,
+                new SinkTaskBuilder(id, configState, statusListener, initialState));
+    }
+
+    /**
+     * Start a source task managed by this worker.

Review comment:
       It's probably worth while to mention that this method starts the task for a source connector with older behavior (without exactly once support).

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -264,12 +279,62 @@ public void startConnector(
 
                 log.info("Creating connector {} of type {}", connName, connClass);
                 final Connector connector = plugins.newConnector(connClass);
-                final ConnectorConfig connConfig = ConnectUtils.isSinkConnector(connector)
-                        ? new SinkConnectorConfig(plugins, connProps)
-                        : new SourceConnectorConfig(plugins, connProps, config.topicCreationEnable());
+                final ConnectorConfig connConfig;
+                final CloseableOffsetStorageReader offsetReader;
+                if (ConnectUtils.isSinkConnector(connector)) {
+                    connConfig = new SinkConnectorConfig(plugins, connProps);
+                    offsetReader = null;
+                } else {
+                    SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connProps, config.topicCreationEnable());
+                    connConfig = sourceConfig;
+

Review comment:
       Suggestion:
   ```suggestion
   
                       // Use the desired topic for offsets
   ```

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -980,6 +1119,329 @@ WorkerMetricsGroup workerMetricsGroup() {
         return workerMetricsGroup;
     }
 
+    abstract class TaskBuilder {
+
+        private final ConnectorTaskId id;
+        private final ClusterConfigState configState;
+        private final TaskStatus.Listener statusListener;
+        private final TargetState initialState;
+
+        private Task task;
+        private ConnectorConfig connectorConfig = null;
+        private Converter keyConverter = null;
+        private Converter valueConverter = null;
+        private HeaderConverter headerConverter = null;
+        private ClassLoader classLoader = null;
+
+        public TaskBuilder(ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState) {
+            this.id = id;
+            this.configState = configState;
+            this.statusListener = statusListener;
+            this.initialState = initialState;
+        }
+
+        public TaskBuilder withTask(Task task) {
+            this.task = task;
+            return this;
+        }
+
+        public TaskBuilder withConnectorConfig(ConnectorConfig connectorConfig) {
+            this.connectorConfig = connectorConfig;
+            return this;
+        }
+
+        public TaskBuilder withKeyConverter(Converter keyConverter) {
+            this.keyConverter = keyConverter;
+            return this;
+        }
+
+        public TaskBuilder withValueConverter(Converter valueConverter) {
+            this.valueConverter = valueConverter;
+            return this;
+        }
+
+        public TaskBuilder withHeaderConverter(HeaderConverter headerConverter) {
+            this.headerConverter = headerConverter;
+            return this;
+        }
+
+        public TaskBuilder withClassloader(ClassLoader classLoader) {
+            this.classLoader = classLoader;
+            return this;
+        }
+
+        public WorkerTask build() {
+            Objects.requireNonNull(task, "Task cannot be null");
+            Objects.requireNonNull(connectorConfig, "Connector config used by task cannot be null");
+            Objects.requireNonNull(keyConverter, "Key converter used by task cannot be null");
+            Objects.requireNonNull(valueConverter, "Value converter used by task cannot be null");
+            Objects.requireNonNull(headerConverter, "Header converter used by task cannot be null");
+            Objects.requireNonNull(classLoader, "Classloader used by task cannot be null");
+
+            ErrorHandlingMetrics errorHandlingMetrics = errorHandlingMetrics(id);
+            final Class<? extends Connector> connectorClass = plugins.connectorClass(
+                    connectorConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG));
+            RetryWithToleranceOperator retryWithToleranceOperator = new RetryWithToleranceOperator(connectorConfig.errorRetryTimeout(),
+                    connectorConfig.errorMaxDelayInMillis(), connectorConfig.errorToleranceType(), Time.SYSTEM);
+            retryWithToleranceOperator.metrics(errorHandlingMetrics);
+
+            return doBuild(task, id, configState, statusListener, initialState,
+                    connectorConfig, keyConverter, valueConverter, headerConverter, classLoader,
+                    errorHandlingMetrics, connectorClass, retryWithToleranceOperator);
+        }
+
+        abstract WorkerTask doBuild(Task task,
+                                    ConnectorTaskId id,
+                                    ClusterConfigState configState,
+                                    TaskStatus.Listener statusListener,
+                                    TargetState initialState,
+                                    ConnectorConfig connectorConfig,
+                                    Converter keyConverter,
+                                    Converter valueConverter,
+                                    HeaderConverter headerConverter,
+                                    ClassLoader classLoader,
+                                    ErrorHandlingMetrics errorHandlingMetrics,
+                                    Class<? extends Connector> connectorClass,
+                                    RetryWithToleranceOperator retryWithToleranceOperator);
+
+    }
+
+    class SinkTaskBuilder extends TaskBuilder {
+        public SinkTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            TransformationChain<SinkRecord> transformationChain = new TransformationChain<>(connectorConfig.<SinkRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+            SinkConnectorConfig sinkConfig = new SinkConnectorConfig(plugins, connectorConfig.originalsStrings());
+            retryWithToleranceOperator.reporters(sinkTaskReporters(id, sinkConfig, errorHandlingMetrics, connectorClass));
+            WorkerErrantRecordReporter workerErrantRecordReporter = createWorkerErrantRecordReporter(sinkConfig, retryWithToleranceOperator,
+                    keyConverter, valueConverter, headerConverter);
+
+            Map<String, Object> consumerProps = consumerConfigs(id.connector(),  "connector-consumer-" + id, config, connectorConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+            return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverter,
+                    valueConverter, headerConverter, transformationChain, consumer, classLoader, time,
+                    retryWithToleranceOperator, workerErrantRecordReporter, herder.statusBackingStore());
+        }
+    }
+
+    class SourceTaskBuilder extends TaskBuilder {
+        public SourceTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+            ConnectorOffsetBackingStore offsetStore = ConnectorOffsetBackingStore.withoutConnectorOffsetStore(
+                    () -> LoggingContext.forTask(id),
+                    globalOffsetBackingStore,
+                    config.offsetsTopic()
+            );
+            final TopicAdmin admin;
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (sourceConfig.offsetsTopic() != null || (config.topicCreationEnable() && sourceConfig.usesTopicCreation())) {

Review comment:
       I think adding some comments here will help future developers. There are enough subtleties here (e.g., the `||` in this line) that makes this more challenging than it should be to understand the behavior. 
   
   Or, it might be worth adding some final booleans here to make these conditions a bit more clear and readable, especially since this method is a series of relatively independent parts/sections. For example:
   ```
   final boolean customOffsetTopic = sourceConfig.offsetTopic();
   final boolean createTopicsEnabled = config.topicCreationEnable() && sourceConfig.usesTopicCreation();
   if (customOffsetTopic || createTopicsEnabled) {
     // Create an admin client
     ...
     if (createTopicsEnabled) {
       topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig);
     }
     if (customOffsetTopic) {
       // Build custom offset store
       ...
     }
   } else {
     // No need for admin
     admin = null;
   }
   ```

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -692,21 +823,22 @@ private WorkerTask buildWorkerTask(ClusterConfigState configState,
         ConnectUtils.addMetricsContextProperties(consumerProps, config, clusterId);
         // Connector-specified overrides
         Map<String, Object> consumerOverrides =
-            connectorClientConfigOverrides(id, connConfig, connectorClass, ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX,
+            connectorClientConfigOverrides(connName, connConfig, connectorClass, ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX,
                                            ConnectorType.SINK, ConnectorClientConfigRequest.ClientType.CONSUMER,
                                            connectorClientConfigOverridePolicy);
         consumerProps.putAll(consumerOverrides);
 
         return consumerProps;
     }
 
-    static Map<String, Object> adminConfigs(ConnectorTaskId id,
+    static Map<String, Object> adminConfigs(String connName,

Review comment:
       Do we need to change these signatures from `ConnectorTaskId` to `String`? The `ConnectorTaskId` gives us the ability to define tasks-specific client configuration properties if necessary/desired. I'm afraid that switching to `String` will make it harder and more invasive to add that back in. Plus, if there's not a good reason to remove these, let's leave that for smaller PRs.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java
##########
@@ -342,6 +343,10 @@ private void logPluginPathConfigProviderWarning(Map<String, String> rawOriginals
         }
     }
 
+    public String bootstrapServers() {
+        return String.join(",", getList(BOOTSTRAP_SERVERS_CONFIG));
+    }

Review comment:
       Very nice. For this and the other new getter methods, can you add some JavaDoc just so that it's easier to follow in an IDE where these methods are _used_? I wanted that several times as I was reviewing this code.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
##########
@@ -16,95 +16,46 @@
  */
 package org.apache.kafka.connect.runtime;
 
-import org.apache.kafka.clients.admin.NewTopic;
-import org.apache.kafka.clients.admin.TopicDescription;
-import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.header.internals.RecordHeaders;
-import org.apache.kafka.common.metrics.Sensor;
-import org.apache.kafka.common.metrics.stats.Avg;
-import org.apache.kafka.common.metrics.stats.CumulativeSum;
-import org.apache.kafka.common.metrics.stats.Max;
-import org.apache.kafka.common.metrics.stats.Rate;
-import org.apache.kafka.common.metrics.stats.Value;
 import org.apache.kafka.common.utils.Time;
-import org.apache.kafka.common.utils.Utils;
-import org.apache.kafka.connect.errors.ConnectException;
-import org.apache.kafka.connect.errors.RetriableException;
-import org.apache.kafka.connect.header.Header;
-import org.apache.kafka.connect.header.Headers;
-import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup;
-import org.apache.kafka.connect.runtime.distributed.ClusterConfigState;
 import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator;
-import org.apache.kafka.connect.runtime.errors.Stage;
 import org.apache.kafka.connect.source.SourceRecord;
 import org.apache.kafka.connect.source.SourceTask;
 import org.apache.kafka.connect.storage.CloseableOffsetStorageReader;
+import org.apache.kafka.connect.storage.ClusterConfigState;
+import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore;
 import org.apache.kafka.connect.storage.Converter;
 import org.apache.kafka.connect.storage.HeaderConverter;
 import org.apache.kafka.connect.storage.OffsetStorageWriter;
 import org.apache.kafka.connect.storage.StatusBackingStore;
-import org.apache.kafka.connect.util.ConnectUtils;
 import org.apache.kafka.connect.util.ConnectorTaskId;
 import org.apache.kafka.connect.util.TopicAdmin;
-import org.apache.kafka.connect.util.TopicCreation;
 import org.apache.kafka.connect.util.TopicCreationGroup;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.time.Duration;
 import java.util.IdentityHashMap;
-import java.util.List;
 import java.util.Map;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executor;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_TRACKING_ENABLE_CONFIG;
 
 /**
  * WorkerTask that uses a SourceTask to ingest data into Kafka.
  */
-class WorkerSourceTask extends WorkerTask {
+class WorkerSourceTask extends AbstractWorkerSourceTask {

Review comment:
       Note to future me: I didn't get this far in the PR.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -980,6 +1119,329 @@ WorkerMetricsGroup workerMetricsGroup() {
         return workerMetricsGroup;
     }
 
+    abstract class TaskBuilder {
+
+        private final ConnectorTaskId id;
+        private final ClusterConfigState configState;
+        private final TaskStatus.Listener statusListener;
+        private final TargetState initialState;
+
+        private Task task;
+        private ConnectorConfig connectorConfig = null;
+        private Converter keyConverter = null;
+        private Converter valueConverter = null;
+        private HeaderConverter headerConverter = null;
+        private ClassLoader classLoader = null;
+
+        public TaskBuilder(ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState) {
+            this.id = id;
+            this.configState = configState;
+            this.statusListener = statusListener;
+            this.initialState = initialState;
+        }
+
+        public TaskBuilder withTask(Task task) {
+            this.task = task;
+            return this;
+        }
+
+        public TaskBuilder withConnectorConfig(ConnectorConfig connectorConfig) {
+            this.connectorConfig = connectorConfig;
+            return this;
+        }
+
+        public TaskBuilder withKeyConverter(Converter keyConverter) {
+            this.keyConverter = keyConverter;
+            return this;
+        }
+
+        public TaskBuilder withValueConverter(Converter valueConverter) {
+            this.valueConverter = valueConverter;
+            return this;
+        }
+
+        public TaskBuilder withHeaderConverter(HeaderConverter headerConverter) {
+            this.headerConverter = headerConverter;
+            return this;
+        }
+
+        public TaskBuilder withClassloader(ClassLoader classLoader) {
+            this.classLoader = classLoader;
+            return this;
+        }
+
+        public WorkerTask build() {
+            Objects.requireNonNull(task, "Task cannot be null");
+            Objects.requireNonNull(connectorConfig, "Connector config used by task cannot be null");
+            Objects.requireNonNull(keyConverter, "Key converter used by task cannot be null");
+            Objects.requireNonNull(valueConverter, "Value converter used by task cannot be null");
+            Objects.requireNonNull(headerConverter, "Header converter used by task cannot be null");
+            Objects.requireNonNull(classLoader, "Classloader used by task cannot be null");
+
+            ErrorHandlingMetrics errorHandlingMetrics = errorHandlingMetrics(id);
+            final Class<? extends Connector> connectorClass = plugins.connectorClass(
+                    connectorConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG));
+            RetryWithToleranceOperator retryWithToleranceOperator = new RetryWithToleranceOperator(connectorConfig.errorRetryTimeout(),
+                    connectorConfig.errorMaxDelayInMillis(), connectorConfig.errorToleranceType(), Time.SYSTEM);
+            retryWithToleranceOperator.metrics(errorHandlingMetrics);
+
+            return doBuild(task, id, configState, statusListener, initialState,
+                    connectorConfig, keyConverter, valueConverter, headerConverter, classLoader,
+                    errorHandlingMetrics, connectorClass, retryWithToleranceOperator);
+        }
+
+        abstract WorkerTask doBuild(Task task,
+                                    ConnectorTaskId id,
+                                    ClusterConfigState configState,
+                                    TaskStatus.Listener statusListener,
+                                    TargetState initialState,
+                                    ConnectorConfig connectorConfig,
+                                    Converter keyConverter,
+                                    Converter valueConverter,
+                                    HeaderConverter headerConverter,
+                                    ClassLoader classLoader,
+                                    ErrorHandlingMetrics errorHandlingMetrics,
+                                    Class<? extends Connector> connectorClass,
+                                    RetryWithToleranceOperator retryWithToleranceOperator);
+
+    }
+
+    class SinkTaskBuilder extends TaskBuilder {
+        public SinkTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            TransformationChain<SinkRecord> transformationChain = new TransformationChain<>(connectorConfig.<SinkRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+            SinkConnectorConfig sinkConfig = new SinkConnectorConfig(plugins, connectorConfig.originalsStrings());
+            retryWithToleranceOperator.reporters(sinkTaskReporters(id, sinkConfig, errorHandlingMetrics, connectorClass));
+            WorkerErrantRecordReporter workerErrantRecordReporter = createWorkerErrantRecordReporter(sinkConfig, retryWithToleranceOperator,
+                    keyConverter, valueConverter, headerConverter);
+
+            Map<String, Object> consumerProps = consumerConfigs(id.connector(),  "connector-consumer-" + id, config, connectorConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+            return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverter,
+                    valueConverter, headerConverter, transformationChain, consumer, classLoader, time,
+                    retryWithToleranceOperator, workerErrantRecordReporter, herder.statusBackingStore());
+        }
+    }
+
+    class SourceTaskBuilder extends TaskBuilder {
+        public SourceTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+            ConnectorOffsetBackingStore offsetStore = ConnectorOffsetBackingStore.withoutConnectorOffsetStore(
+                    () -> LoggingContext.forTask(id),
+                    globalOffsetBackingStore,
+                    config.offsetsTopic()
+            );
+            final TopicAdmin admin;
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (sourceConfig.offsetsTopic() != null || (config.topicCreationEnable() && sourceConfig.usesTopicCreation())) {
+                Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,
+                        sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE);
+                Admin adminClient = Admin.create(adminOverrides);
+                admin = new TopicAdmin(adminOverrides.get(BOOTSTRAP_SERVERS_CONFIG), adminClient);
+
+                if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) {
+                    topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig);
+                }
+
+                if (sourceConfig.offsetsTopic() != null && config.connectorOffsetsTopicsPermitted()) {
+                    Map<String, Object> consumerProps = consumerConfigs(id.connector(), "connector-consumer-" + id, config, connectorConfig, connectorClass,
+                            connectorClientConfigOverridePolicy, kafkaClusterId);
+                    // Users can disable this if they want to; it won't affect delivery guarantees since the task isn't exactly-once anyways
+                    consumerProps.putIfAbsent(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString().toLowerCase(Locale.ROOT));
+                    KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+                    offsetStore = ConnectorOffsetBackingStore.withConnectorOffsetStore(
+                            () -> LoggingContext.forTask(id),
+                            globalOffsetBackingStore,
+                            sourceConfig.offsetsTopic(),
+                            producer,
+                            consumer,
+                            admin
+                    );
+                }
+            } else {
+                admin = null;
+            }
+            offsetStore.configure(config);
+
+            CloseableOffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+            OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+
+            // Note we pass the configState as it performs dynamic transformations under the covers
+            return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter, valueConverter,
+                    headerConverter, transformationChain, producer, admin, topicCreationGroups,
+                    offsetReader, offsetWriter, offsetStore, config, configState, metrics, classLoader, time,
+                    retryWithToleranceOperator, herder.statusBackingStore(), executor);
+        }
+    }
+
+    class ExactlyOnceSourceTaskBuilder extends TaskBuilder {
+        private final Runnable preProducerCheck;
+        private final Runnable postProducerCheck;
+
+        public ExactlyOnceSourceTaskBuilder(ConnectorTaskId id,
+                                            ClusterConfigState configState,
+                                            TaskStatus.Listener statusListener,
+                                            TargetState initialState,
+                                            Runnable preProducerCheck,
+                                            Runnable postProducerCheck) {
+            super(id, configState, statusListener, initialState);
+            this.preProducerCheck = preProducerCheck;
+            this.postProducerCheck = postProducerCheck;
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                                  ConnectorTaskId id,
+                                  ClusterConfigState configState,
+                                  TaskStatus.Listener statusListener,
+                                  TargetState initialState,
+                                  ConnectorConfig connectorConfig,
+                                  Converter keyConverter,
+                                  Converter valueConverter,
+                                  HeaderConverter headerConverter,
+                                  ClassLoader classLoader,
+                                  ErrorHandlingMetrics errorHandlingMetrics,
+                                  Class<? extends Connector> connectorClass,
+                                  RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,

Review comment:
       In my comment about `SourceTaskBuilder#doBuild`, I suggested adding a few comments to help the different parts/sections of this method stand out more clearly. The same would be true here, such as:
   ```suggestion
               // Create an admin client
               Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,
   ```

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -980,6 +1119,329 @@ WorkerMetricsGroup workerMetricsGroup() {
         return workerMetricsGroup;
     }
 
+    abstract class TaskBuilder {
+
+        private final ConnectorTaskId id;
+        private final ClusterConfigState configState;
+        private final TaskStatus.Listener statusListener;
+        private final TargetState initialState;
+
+        private Task task;
+        private ConnectorConfig connectorConfig = null;
+        private Converter keyConverter = null;
+        private Converter valueConverter = null;
+        private HeaderConverter headerConverter = null;
+        private ClassLoader classLoader = null;
+
+        public TaskBuilder(ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState) {
+            this.id = id;
+            this.configState = configState;
+            this.statusListener = statusListener;
+            this.initialState = initialState;
+        }
+
+        public TaskBuilder withTask(Task task) {
+            this.task = task;
+            return this;
+        }
+
+        public TaskBuilder withConnectorConfig(ConnectorConfig connectorConfig) {
+            this.connectorConfig = connectorConfig;
+            return this;
+        }
+
+        public TaskBuilder withKeyConverter(Converter keyConverter) {
+            this.keyConverter = keyConverter;
+            return this;
+        }
+
+        public TaskBuilder withValueConverter(Converter valueConverter) {
+            this.valueConverter = valueConverter;
+            return this;
+        }
+
+        public TaskBuilder withHeaderConverter(HeaderConverter headerConverter) {
+            this.headerConverter = headerConverter;
+            return this;
+        }
+
+        public TaskBuilder withClassloader(ClassLoader classLoader) {
+            this.classLoader = classLoader;
+            return this;
+        }
+
+        public WorkerTask build() {
+            Objects.requireNonNull(task, "Task cannot be null");
+            Objects.requireNonNull(connectorConfig, "Connector config used by task cannot be null");
+            Objects.requireNonNull(keyConverter, "Key converter used by task cannot be null");
+            Objects.requireNonNull(valueConverter, "Value converter used by task cannot be null");
+            Objects.requireNonNull(headerConverter, "Header converter used by task cannot be null");
+            Objects.requireNonNull(classLoader, "Classloader used by task cannot be null");
+
+            ErrorHandlingMetrics errorHandlingMetrics = errorHandlingMetrics(id);
+            final Class<? extends Connector> connectorClass = plugins.connectorClass(
+                    connectorConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG));
+            RetryWithToleranceOperator retryWithToleranceOperator = new RetryWithToleranceOperator(connectorConfig.errorRetryTimeout(),
+                    connectorConfig.errorMaxDelayInMillis(), connectorConfig.errorToleranceType(), Time.SYSTEM);
+            retryWithToleranceOperator.metrics(errorHandlingMetrics);
+
+            return doBuild(task, id, configState, statusListener, initialState,
+                    connectorConfig, keyConverter, valueConverter, headerConverter, classLoader,
+                    errorHandlingMetrics, connectorClass, retryWithToleranceOperator);
+        }
+
+        abstract WorkerTask doBuild(Task task,
+                                    ConnectorTaskId id,
+                                    ClusterConfigState configState,
+                                    TaskStatus.Listener statusListener,
+                                    TargetState initialState,
+                                    ConnectorConfig connectorConfig,
+                                    Converter keyConverter,
+                                    Converter valueConverter,
+                                    HeaderConverter headerConverter,
+                                    ClassLoader classLoader,
+                                    ErrorHandlingMetrics errorHandlingMetrics,
+                                    Class<? extends Connector> connectorClass,
+                                    RetryWithToleranceOperator retryWithToleranceOperator);
+
+    }
+
+    class SinkTaskBuilder extends TaskBuilder {
+        public SinkTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            TransformationChain<SinkRecord> transformationChain = new TransformationChain<>(connectorConfig.<SinkRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+            SinkConnectorConfig sinkConfig = new SinkConnectorConfig(plugins, connectorConfig.originalsStrings());
+            retryWithToleranceOperator.reporters(sinkTaskReporters(id, sinkConfig, errorHandlingMetrics, connectorClass));
+            WorkerErrantRecordReporter workerErrantRecordReporter = createWorkerErrantRecordReporter(sinkConfig, retryWithToleranceOperator,
+                    keyConverter, valueConverter, headerConverter);
+
+            Map<String, Object> consumerProps = consumerConfigs(id.connector(),  "connector-consumer-" + id, config, connectorConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+            return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverter,
+                    valueConverter, headerConverter, transformationChain, consumer, classLoader, time,
+                    retryWithToleranceOperator, workerErrantRecordReporter, herder.statusBackingStore());
+        }
+    }
+
+    class SourceTaskBuilder extends TaskBuilder {
+        public SourceTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+            ConnectorOffsetBackingStore offsetStore = ConnectorOffsetBackingStore.withoutConnectorOffsetStore(
+                    () -> LoggingContext.forTask(id),
+                    globalOffsetBackingStore,
+                    config.offsetsTopic()
+            );
+            final TopicAdmin admin;
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (sourceConfig.offsetsTopic() != null || (config.topicCreationEnable() && sourceConfig.usesTopicCreation())) {
+                Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,
+                        sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE);
+                Admin adminClient = Admin.create(adminOverrides);
+                admin = new TopicAdmin(adminOverrides.get(BOOTSTRAP_SERVERS_CONFIG), adminClient);
+
+                if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) {
+                    topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig);
+                }
+
+                if (sourceConfig.offsetsTopic() != null && config.connectorOffsetsTopicsPermitted()) {
+                    Map<String, Object> consumerProps = consumerConfigs(id.connector(), "connector-consumer-" + id, config, connectorConfig, connectorClass,
+                            connectorClientConfigOverridePolicy, kafkaClusterId);
+                    // Users can disable this if they want to; it won't affect delivery guarantees since the task isn't exactly-once anyways
+                    consumerProps.putIfAbsent(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString().toLowerCase(Locale.ROOT));
+                    KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+                    offsetStore = ConnectorOffsetBackingStore.withConnectorOffsetStore(
+                            () -> LoggingContext.forTask(id),
+                            globalOffsetBackingStore,
+                            sourceConfig.offsetsTopic(),
+                            producer,
+                            consumer,
+                            admin
+                    );
+                }
+            } else {
+                admin = null;
+            }
+            offsetStore.configure(config);
+
+            CloseableOffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+            OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+
+            // Note we pass the configState as it performs dynamic transformations under the covers
+            return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter, valueConverter,
+                    headerConverter, transformationChain, producer, admin, topicCreationGroups,
+                    offsetReader, offsetWriter, offsetStore, config, configState, metrics, classLoader, time,
+                    retryWithToleranceOperator, herder.statusBackingStore(), executor);
+        }
+    }
+
+    class ExactlyOnceSourceTaskBuilder extends TaskBuilder {
+        private final Runnable preProducerCheck;
+        private final Runnable postProducerCheck;
+
+        public ExactlyOnceSourceTaskBuilder(ConnectorTaskId id,
+                                            ClusterConfigState configState,
+                                            TaskStatus.Listener statusListener,
+                                            TargetState initialState,
+                                            Runnable preProducerCheck,
+                                            Runnable postProducerCheck) {
+            super(id, configState, statusListener, initialState);
+            this.preProducerCheck = preProducerCheck;
+            this.postProducerCheck = postProducerCheck;
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                                  ConnectorTaskId id,
+                                  ClusterConfigState configState,
+                                  TaskStatus.Listener statusListener,
+                                  TargetState initialState,
+                                  ConnectorConfig connectorConfig,
+                                  Converter keyConverter,
+                                  Converter valueConverter,
+                                  HeaderConverter headerConverter,
+                                  ClassLoader classLoader,
+                                  ErrorHandlingMetrics errorHandlingMetrics,
+                                  Class<? extends Connector> connectorClass,
+                                  RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,
+                    sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE);
+            Admin adminClient = Admin.create(adminOverrides);
+            TopicAdmin topicAdmin = new TopicAdmin(adminOverrides.get(BOOTSTRAP_SERVERS_CONFIG), adminClient);
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) {
+                topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig);
+            }
+
+            String transactionalId = transactionalId(id);
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            ConnectUtils.warnOnOverriddenProperty(
+                    producerProps, ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true",
+                    "for connectors when exactly-once source support is enabled",
+                    false
+            );
+            ConnectUtils.warnOnOverriddenProperty(
+                    producerProps, ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId,
+                    "for connectors when exactly-once source support is enabled",
+                    true
+            );
+            producerProps.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, true);
+            producerProps.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId);
+            KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+            Map<String, Object> consumerProps = consumerConfigs(id.connector(), "connector-consumer-" + id, config, connectorConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            ConnectUtils.warnOnOverriddenProperty(
+                    consumerProps, ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.name().toLowerCase(Locale.ROOT),
+                    "for connectors when exactly-once source support is enabled",
+                    false
+            );
+            consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString().toLowerCase(Locale.ROOT));
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+            String offsetsTopic = Optional.ofNullable(sourceConfig.offsetsTopic()).orElse(config.offsetsTopic());
+
+            ConnectorOffsetBackingStore offsetStore;
+            // No need to do secondary writes to the global offsets topic if we're certain that the task's local offset store
+            // is going to be targeting it anyway
+            // Note that this may lead to a false positive if the user provides an overridden bootstrap servers value for their
+            // producer that resolves to the same Kafka cluster; we might consider looking up the Kafka cluster ID in the future
+            // to prevent these false positives but at the moment this is probably adequate, especially since we probably don't
+            // want to put a ping to a remote Kafka cluster inside the herder's tick thread (which is where this logic takes place
+            // right now) in case that takes a while.

Review comment:
       This is more of a nit-type comment.
   
   It seems like this comment tries to describe why we don't need to create a connector-specific offset store **AND** then talk about the check might not always be accurate. But I found the comment a bit hard to follow.
   
   Also, the `ConnectorOffsetBackingStore offsetStore;` line should move under the multi-line comment.
   
   In such cases, it might be worth changing the code to help make things more clear. For example:
   ```
   // We can simply reuse the worker's offset store when the connector-specific offset topic
   // is the same as the worker's. We can check the offset topic name and the Kafka cluster's
   // bootstrap servers, although this isn't exact and can lead to some false positives if the user
   // provides an overridden bootstrap servers value for their producer that is different than
   // the worker's but still resolves to the same Kafka cluster used by the worker.
   // At the moment this is probably adequate, especially since we probably don't want to put
   // a network ping to a remote Kafka cluster inside the herder's tick thread (which is where this
   // logic takes place right now) in case that takes a while.
   ConnectorOffsetBackingStore offsetStore;
   final boolean sameOffsetTopicAsWorker = offsetsTopic.equals(config.offsetsTopic())
                       && producerProps.get(BOOTSTRAP_SERVERS_CONFIG).equals(config.bootstrapServers();
   if (sameOffsetTopicAsWorker) {
       offsetStore = ConnectorOffsetBackingStore.withoutConnectorOffsetStore(
       ...
   } else {
       offsetStore = ConnectorOffsetBackingStore.withConnectorOffsetStore(
       ...
   }
   ```

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java
##########
@@ -20,13 +20,45 @@
 import org.apache.kafka.clients.producer.RecordMetadata;
 
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * SourceTask is a Task that pulls records from another system for storage in Kafka.
  */
 public abstract class SourceTask implements Task {
 
+    /**
+     * <p>
+     * The configuration key that determines how source tasks will define transaction boundaries
+     * when exactly-once support is enabled.
+     * </p>
+     */
+    public static final String TRANSACTION_BOUNDARY_CONFIG = "transaction.boundary";
+
+    public enum TransactionBoundary {
+        POLL,
+        INTERVAL,
+        CONNECTOR;
+
+        public static final TransactionBoundary DEFAULT = POLL;
+
+        public static List<String> options() {

Review comment:
       I think the risk of introducing `options()` is that some developers might accidentally use `values()`. 
   
   The pattern used in `ConnectorType` is far better, as it overrides the `toString()` method. That doesn't handle the case-independence for parsing, though `ConverterType` is a better pattern to follow if that's required.
   
   Let's be consistent with the new enums, and have each follow one of those two patterns depending upon whether parsing case-independently is required.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java
##########
@@ -28,4 +30,31 @@
     protected SourceConnectorContext context() {
         return (SourceConnectorContext) context;
     }
+
+    /**
+     * Signals whether the connector supports exactly-once delivery guarantees with a proposed configuration.
+     * Developers can assume that worker-level exactly-once support is enabled when this method is invoked.
+     * The default implementation will return {@code null}.
+     * @param connectorConfig the configuration that will be used for the connector.
+     * @return {@link ExactlyOnceSupport#SUPPORTED} if the connector can provide exactly-once support,
+     * and {@link ExactlyOnceSupport#UNSUPPORTED} if it cannot. If {@code null}, it is assumed that the
+     * connector cannot.
+     */
+    public ExactlyOnceSupport exactlyOnceSupport(Map<String, String> connectorConfig) {
+        return null;
+    }
+
+    /**
+     * Signals whether the connector can define its own transaction boundaries with the proposed
+     * configuration. Developers must override this method if they wish to add connector-defined
+     * transaction boundary support; if they do not, users will be unable to create instances of
+     * this connector that use connector-defined transaction boundaries. The default implementation
+     * will return {@code UNSUPPORTED}.

Review comment:
       WDYT about something like this:
   ```
   /**
    * Signals whether the connector implementation is capable of defining the transaction boundaries for a
    * connector with the given configuration. This method is called before {@link #start(Map)}, only when the
    * runtime supports exactly-once and the connector configuration includes {@code transaction.boundary=connector}.
    *
    * <p>This method need not be implemented if the connector implementation does not support definiting
    * transaction boundaries.
    *
    * @param connectorConfig the configuration that will be used for the connector
    * @return {@link ConnectorTransactionBoundaries.SUPPORTED} if the connector will define its own transaction boundaries, 
    *         or {@link ConnectorTransactionBoundaries.UNSUPPORTED} otherwise.
    * @see TransactionContext
    */
   ```
   

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -980,6 +1119,329 @@ WorkerMetricsGroup workerMetricsGroup() {
         return workerMetricsGroup;
     }
 
+    abstract class TaskBuilder {
+
+        private final ConnectorTaskId id;
+        private final ClusterConfigState configState;
+        private final TaskStatus.Listener statusListener;
+        private final TargetState initialState;
+
+        private Task task;
+        private ConnectorConfig connectorConfig = null;
+        private Converter keyConverter = null;
+        private Converter valueConverter = null;
+        private HeaderConverter headerConverter = null;
+        private ClassLoader classLoader = null;
+
+        public TaskBuilder(ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState) {
+            this.id = id;
+            this.configState = configState;
+            this.statusListener = statusListener;
+            this.initialState = initialState;
+        }
+
+        public TaskBuilder withTask(Task task) {
+            this.task = task;
+            return this;
+        }
+
+        public TaskBuilder withConnectorConfig(ConnectorConfig connectorConfig) {
+            this.connectorConfig = connectorConfig;
+            return this;
+        }
+
+        public TaskBuilder withKeyConverter(Converter keyConverter) {
+            this.keyConverter = keyConverter;
+            return this;
+        }
+
+        public TaskBuilder withValueConverter(Converter valueConverter) {
+            this.valueConverter = valueConverter;
+            return this;
+        }
+
+        public TaskBuilder withHeaderConverter(HeaderConverter headerConverter) {
+            this.headerConverter = headerConverter;
+            return this;
+        }
+
+        public TaskBuilder withClassloader(ClassLoader classLoader) {
+            this.classLoader = classLoader;
+            return this;
+        }
+
+        public WorkerTask build() {
+            Objects.requireNonNull(task, "Task cannot be null");
+            Objects.requireNonNull(connectorConfig, "Connector config used by task cannot be null");
+            Objects.requireNonNull(keyConverter, "Key converter used by task cannot be null");
+            Objects.requireNonNull(valueConverter, "Value converter used by task cannot be null");
+            Objects.requireNonNull(headerConverter, "Header converter used by task cannot be null");
+            Objects.requireNonNull(classLoader, "Classloader used by task cannot be null");
+
+            ErrorHandlingMetrics errorHandlingMetrics = errorHandlingMetrics(id);
+            final Class<? extends Connector> connectorClass = plugins.connectorClass(
+                    connectorConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG));
+            RetryWithToleranceOperator retryWithToleranceOperator = new RetryWithToleranceOperator(connectorConfig.errorRetryTimeout(),
+                    connectorConfig.errorMaxDelayInMillis(), connectorConfig.errorToleranceType(), Time.SYSTEM);
+            retryWithToleranceOperator.metrics(errorHandlingMetrics);
+
+            return doBuild(task, id, configState, statusListener, initialState,
+                    connectorConfig, keyConverter, valueConverter, headerConverter, classLoader,
+                    errorHandlingMetrics, connectorClass, retryWithToleranceOperator);
+        }
+
+        abstract WorkerTask doBuild(Task task,
+                                    ConnectorTaskId id,
+                                    ClusterConfigState configState,
+                                    TaskStatus.Listener statusListener,
+                                    TargetState initialState,
+                                    ConnectorConfig connectorConfig,
+                                    Converter keyConverter,
+                                    Converter valueConverter,
+                                    HeaderConverter headerConverter,
+                                    ClassLoader classLoader,
+                                    ErrorHandlingMetrics errorHandlingMetrics,
+                                    Class<? extends Connector> connectorClass,
+                                    RetryWithToleranceOperator retryWithToleranceOperator);
+
+    }
+
+    class SinkTaskBuilder extends TaskBuilder {
+        public SinkTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            TransformationChain<SinkRecord> transformationChain = new TransformationChain<>(connectorConfig.<SinkRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+            SinkConnectorConfig sinkConfig = new SinkConnectorConfig(plugins, connectorConfig.originalsStrings());
+            retryWithToleranceOperator.reporters(sinkTaskReporters(id, sinkConfig, errorHandlingMetrics, connectorClass));
+            WorkerErrantRecordReporter workerErrantRecordReporter = createWorkerErrantRecordReporter(sinkConfig, retryWithToleranceOperator,
+                    keyConverter, valueConverter, headerConverter);
+
+            Map<String, Object> consumerProps = consumerConfigs(id.connector(),  "connector-consumer-" + id, config, connectorConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+            return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverter,
+                    valueConverter, headerConverter, transformationChain, consumer, classLoader, time,
+                    retryWithToleranceOperator, workerErrantRecordReporter, herder.statusBackingStore());
+        }
+    }
+
+    class SourceTaskBuilder extends TaskBuilder {
+        public SourceTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+            ConnectorOffsetBackingStore offsetStore = ConnectorOffsetBackingStore.withoutConnectorOffsetStore(
+                    () -> LoggingContext.forTask(id),
+                    globalOffsetBackingStore,
+                    config.offsetsTopic()
+            );
+            final TopicAdmin admin;
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (sourceConfig.offsetsTopic() != null || (config.topicCreationEnable() && sourceConfig.usesTopicCreation())) {
+                Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,
+                        sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE);
+                Admin adminClient = Admin.create(adminOverrides);
+                admin = new TopicAdmin(adminOverrides.get(BOOTSTRAP_SERVERS_CONFIG), adminClient);
+
+                if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) {
+                    topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig);
+                }
+
+                if (sourceConfig.offsetsTopic() != null && config.connectorOffsetsTopicsPermitted()) {
+                    Map<String, Object> consumerProps = consumerConfigs(id.connector(), "connector-consumer-" + id, config, connectorConfig, connectorClass,
+                            connectorClientConfigOverridePolicy, kafkaClusterId);
+                    // Users can disable this if they want to; it won't affect delivery guarantees since the task isn't exactly-once anyways
+                    consumerProps.putIfAbsent(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString().toLowerCase(Locale.ROOT));
+                    KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+                    offsetStore = ConnectorOffsetBackingStore.withConnectorOffsetStore(
+                            () -> LoggingContext.forTask(id),
+                            globalOffsetBackingStore,
+                            sourceConfig.offsetsTopic(),
+                            producer,
+                            consumer,
+                            admin
+                    );
+                }
+            } else {
+                admin = null;
+            }
+            offsetStore.configure(config);
+
+            CloseableOffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+            OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+
+            // Note we pass the configState as it performs dynamic transformations under the covers
+            return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter, valueConverter,
+                    headerConverter, transformationChain, producer, admin, topicCreationGroups,
+                    offsetReader, offsetWriter, offsetStore, config, configState, metrics, classLoader, time,
+                    retryWithToleranceOperator, herder.statusBackingStore(), executor);
+        }
+    }
+
+    class ExactlyOnceSourceTaskBuilder extends TaskBuilder {
+        private final Runnable preProducerCheck;
+        private final Runnable postProducerCheck;
+
+        public ExactlyOnceSourceTaskBuilder(ConnectorTaskId id,
+                                            ClusterConfigState configState,
+                                            TaskStatus.Listener statusListener,
+                                            TargetState initialState,
+                                            Runnable preProducerCheck,
+                                            Runnable postProducerCheck) {
+            super(id, configState, statusListener, initialState);
+            this.preProducerCheck = preProducerCheck;
+            this.postProducerCheck = postProducerCheck;
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                                  ConnectorTaskId id,
+                                  ClusterConfigState configState,
+                                  TaskStatus.Listener statusListener,
+                                  TargetState initialState,
+                                  ConnectorConfig connectorConfig,
+                                  Converter keyConverter,
+                                  Converter valueConverter,
+                                  HeaderConverter headerConverter,
+                                  ClassLoader classLoader,
+                                  ErrorHandlingMetrics errorHandlingMetrics,
+                                  Class<? extends Connector> connectorClass,
+                                  RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,
+                    sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE);
+            Admin adminClient = Admin.create(adminOverrides);
+            TopicAdmin topicAdmin = new TopicAdmin(adminOverrides.get(BOOTSTRAP_SERVERS_CONFIG), adminClient);
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) {
+                topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig);
+            }
+
+            String transactionalId = transactionalId(id);
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            ConnectUtils.warnOnOverriddenProperty(
+                    producerProps, ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true",
+                    "for connectors when exactly-once source support is enabled",
+                    false
+            );
+            ConnectUtils.warnOnOverriddenProperty(
+                    producerProps, ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId,
+                    "for connectors when exactly-once source support is enabled",
+                    true
+            );
+            producerProps.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, true);
+            producerProps.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId);

Review comment:
       Ah, so here's one example of how these properties cannot be overridden by the connector config. But that's not quite so obvious for some other calls to `ConnectUtils.warnOnOverriddenProperty`.
   
   Is it not feasible to reset the expected value on the props within the `ConnectUtils#warnOnOverriddenProperty` method?




-- 
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] rhauch commented on a change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
rhauch commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r779947401



##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceTaskContext.java
##########
@@ -38,4 +38,29 @@
      * Get the OffsetStorageReader for this SourceTask.
      */
     OffsetStorageReader offsetStorageReader();
+
+    /**
+     * Get a {@link TransactionContext} that can be used to define producer transaction boundaries
+     * when exactly-once support is enabled for the connector.
+     *
+     * <p>This method was added in Apache Kafka 3.0. Source tasks that use this method but want to
+     * maintain backward compatibility so they can also be deployed to older Connect runtimes
+     * should guard the call to this method with a try-catch block, since calling this method will result in a
+     * {@link NoSuchMethodException} or {@link NoClassDefFoundError} when the source connector is deployed to
+     * Connect runtimes older than Kafka 3.0. For example:
+     * <pre>
+     *     TransactionContext transactionContext;
+     *     try {
+     *         transactionContext = context.transactionContext();
+     *     } catch (NoSuchMethodError | NoClassDefFoundError e) {
+     *         transactionContext = null;
+     *     }
+     * </pre>
+     *
+     * @return the transaction context, or null if the connector was not configured to specify transaction boundaries
+     * @since 3.0

Review comment:
       Please change all of these `@since 3.0` to `@since 3.1`, plus any other `Kafka 3.0` references in JavaDoc.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java
##########
@@ -125,28 +130,28 @@ public KafkaBasedLog(String topic,
      * Create a new KafkaBasedLog object. This does not start reading the log and writing is not permitted until
      * {@link #start()} is invoked.
      *
-     * @param topic              the topic to treat as a log
-     * @param producerConfigs    configuration options to use when creating the internal producer. At a minimum this must
+     * @param topic                     the topic to treat as a log
+     * @param producerConfigs           configuration options to use when creating the internal producer. At a minimum this must
      *                           contain compatible serializer settings for the generic types used on this class. Some
      *                           setting, such as the number of acks, will be overridden to ensure correct behavior of this
      *                           class.
-     * @param consumerConfigs    configuration options to use when creating the internal consumer. At a minimum this must
+     * @param consumerConfigs           configuration options to use when creating the internal consumer. At a minimum this must
      *                           contain compatible serializer settings for the generic types used on this class. Some
      *                           setting, such as the auto offset reset policy, will be overridden to ensure correct
      *                           behavior of this class.
-     * @param topicAdminSupplier supplier function for an admin client, the lifecycle of which is expected to be controlled
+     * @param topicAdminSupplier        supplier function for an admin client, the lifecycle of which is expected to be controlled
      *                           by the calling component; may not be null
-     * @param consumedCallback   callback to invoke for each {@link ConsumerRecord} consumed when tailing the log
-     * @param time               Time interface
-     * @param initializer        the function that should be run when this log is {@link #start() started}; may be null
+     * @param consumedCallback          callback to invoke for each {@link ConsumerRecord} consumed when tailing the log
+     * @param time                      Time interface
+     * @param initializer               the function that should be run when this log is {@link #start() started}; may be null
      */
     public KafkaBasedLog(String topic,
-            Map<String, Object> producerConfigs,
-            Map<String, Object> consumerConfigs,
-            Supplier<TopicAdmin> topicAdminSupplier,
-            Callback<ConsumerRecord<K, V>> consumedCallback,
-            Time time,
-            java.util.function.Consumer<TopicAdmin> initializer) {
+                         Map<String, Object> producerConfigs,
+                         Map<String, Object> consumerConfigs,
+                         Supplier<TopicAdmin> topicAdminSupplier,
+                         Callback<ConsumerRecord<K, V>> consumedCallback,
+                         Time time,
+                         java.util.function.Consumer<TopicAdmin> initializer) {

Review comment:
       Why make these unnecessary changes?

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java
##########
@@ -74,15 +75,15 @@
     private volatile boolean cancelled; // indicates whether the Worker has cancelled the connector (e.g. because of slow shutdown)
 
     private State state;
-    private final OffsetStorageReader offsetStorageReader;
+    private final CloseableOffsetStorageReader offsetStorageReader;
 
     public WorkerConnector(String connName,
                            Connector connector,
                            ConnectorConfig connectorConfig,
                            CloseableConnectorContext ctx,
                            ConnectMetrics metrics,
                            ConnectorStatus.Listener statusListener,
-                           OffsetStorageReader offsetStorageReader,
+                           CloseableOffsetStorageReader offsetStorageReader,

Review comment:
       As mentioned above, IIUC this PR will sometimes pass a null `offsetStorageReader` (IIRC for sink connectors), but this class currently expects that to be null. Might be worth adding a `Objects.requireNonNull(...)` call here to help catch that situation.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java
##########
@@ -28,4 +30,39 @@
     protected SourceConnectorContext context() {
         return (SourceConnectorContext) context;
     }
+
+    /**
+     * Signals whether the connector supports exactly-once delivery guarantees with a proposed configuration.
+     * Developers can assume that worker-level exactly-once support is enabled when this method is invoked.
+     * For backwards compatibility, the default implementation will return {@code null}, but connector developers are

Review comment:
       Nit: using a new paragraph makes this stand out more.
   ```suggestion
        * <p>For backwards compatibility, the default implementation will return {@code null}, but connector developers are
   ```

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/TransactionContext.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.source;
+
+/**
+ * Provided to source tasks to allow them to define their own producer transaction boundaries when
+ * exactly-once support is enabled.
+ */
+public interface TransactionContext {

Review comment:
       IIUC, the `WorkerTransactionContext` is the only implementation of this. That means that if a connector is configured with `transaction.boundary=poll` or `transaction.boundary=interval`, a poorly-implemented connector could still call these methods and they'd unnecessarily accumulate records.
   
   WDYT about in such cases the `SourceTaskContext#transactionContext()` method returning a no-op implementation of this interface, so no harm is done if a connector implementation still calls these methods when `transaction.boundary` is _not_ set to `connector`?
   
   Maybe we could consider a warning log message if these methods are called by a connector inappropriately. But we have to be careful. While such log messages might be useful for the **developer** of a connector plugin, I would argue that _prolific_ warnings are actually harmful for a **user** trying to _use_ a connector plugin they didn't develop with a connector configuration that includes `transaction.boundary=poll` or `transaction.boundary=interval`. So maybe it's worthwhile for the "no-op" implementation to only log each warning once per method per instance.

##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedConfigTest.java
##########
@@ -294,4 +298,41 @@ public void shouldRemoveCompactionFromStatusTopicSettings() {
         assertEquals(expectedTopicSettings, actual);
         assertNotEquals(topicSettings, actual);
     }
+
+    @Test
+    public void shouldIdentifyNeedForTransactionalLeader() {
+        Map<String, String> workerProps = configs();
+
+        workerProps.put(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG, "disabled");
+        assertFalse(new DistributedConfig(workerProps).transactionalLeaderEnabled());
+
+        workerProps.put(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG, "preparing");
+        assertTrue(new DistributedConfig(workerProps).transactionalLeaderEnabled());
+
+        workerProps.put(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG, "enabled");
+        assertTrue(new DistributedConfig(workerProps).transactionalLeaderEnabled());
+    }
+
+    @Test
+    public void shouldConstructExpectedTransactionalId() {
+        Map<String, String> workerProps = configs();
+
+        workerProps.put(GROUP_ID_CONFIG, "why did i stay up all night writing unit tests");
+        assertEquals(
+                "connect-cluster-why did i stay up all night writing unit tests",
+                new DistributedConfig(workerProps).transactionalProducerId()
+        );
+
+        workerProps.put(GROUP_ID_CONFIG, "connect-cluster");
+        assertEquals(
+                "connect-cluster-connect-cluster",
+                new DistributedConfig(workerProps).transactionalProducerId()
+        );
+
+        workerProps.put(GROUP_ID_CONFIG, "\u2603");
+        assertEquals(
+                "connect-cluster-\u2603",
+                new DistributedConfig(workerProps).transactionalProducerId()
+        );
+    }

Review comment:
       Let's add some negative tests that verify that invalid values for the new worker configuration properties are properly handled/identified by the validators.
   
   We should also add positive and negative tests for the new connector-level config properties in  `SourceConnectorConfigTest`

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java
##########
@@ -365,6 +413,10 @@ private void readToLogEnd() {
                 // This may happen with really old brokers that don't support the auto topic creation
                 // field in metadata requests
                 log.debug("Reading to end of log offsets with consumer since admin client is unsupported: {}", e.getMessage());
+                if (requireAdminForOffsets) {
+                    // Should be handled by the caller during log startup
+                    throw e;
+                }

Review comment:
       It's true that the `DistributedHerder.run()` is ultimately catching and handling this exception. But I feel like many users might not understand the significance of such an error nor how to correct their configuration. Rather than just re-throw that exception, we should probably wrap that exception with one that has a more instructive message, such as something like:
   > Enabling exactly once for source connectors requires a Kafka broker version that allows admin clients to read consumer offsets. Disable the worker's exactly once support for source connectors, or use a newer Kafka broker version.
   
   Plus, should this if block be before the `log.debug(...)` on the previous line? Seems like that log message might just confuse the situation since the worker will not read "to the end of log offsets with consumer".

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -264,12 +279,62 @@ public void startConnector(
 
                 log.info("Creating connector {} of type {}", connName, connClass);
                 final Connector connector = plugins.newConnector(connClass);
-                final ConnectorConfig connConfig = ConnectUtils.isSinkConnector(connector)
-                        ? new SinkConnectorConfig(plugins, connProps)
-                        : new SourceConnectorConfig(plugins, connProps, config.topicCreationEnable());
+                final ConnectorConfig connConfig;
+                final CloseableOffsetStorageReader offsetReader;
+                if (ConnectUtils.isSinkConnector(connector)) {
+                    connConfig = new SinkConnectorConfig(plugins, connProps);
+                    offsetReader = null;

Review comment:
       IIUC, this changes the behavior of the `WorkerConnector` created below. Prior to this PR, the `WorkerConnector` was always created with the `Worker.offsetBackingStore`, even for sink connectors. 
   
   However, with this PR, the `WorkerConnector` will be instantiated with a null `offsetReader` parameter, which will cause a NPE in `WorkerConnector#doShutdown()` and `WorkerConnector#cancel()`  since `WorkerConnector` does not check for a null parameter there.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConnectorOffsetBackingStore.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.storage;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.connect.runtime.WorkerConfig;
+import org.apache.kafka.connect.util.Callback;
+import org.apache.kafka.connect.util.LoggingContext;
+import org.apache.kafka.connect.util.TopicAdmin;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+public class ConnectorOffsetBackingStore implements OffsetBackingStore {

Review comment:
       Please add JavaDoc that explains the purpose of this class, and in particular, what the two modes are that correspond to the two static factory methods.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConnectorOffsetBackingStore.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.storage;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.connect.runtime.WorkerConfig;
+import org.apache.kafka.connect.util.Callback;
+import org.apache.kafka.connect.util.LoggingContext;
+import org.apache.kafka.connect.util.TopicAdmin;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+public class ConnectorOffsetBackingStore implements OffsetBackingStore {
+
+    private static final Logger log = LoggerFactory.getLogger(ConnectorOffsetBackingStore.class);
+
+    private final Time time;
+    private final Supplier<LoggingContext> loggingContext;
+    private final String primaryOffsetsTopic;
+    private final OffsetBackingStore workerStore;
+    private final Optional<OffsetBackingStore> connectorStore;
+    private final Optional<TopicAdmin> connectorStoreAdmin;
+
+    public static ConnectorOffsetBackingStore withConnectorOffsetStore(

Review comment:
       Nit: move these two static factory methods above the non-static member variables, so all static and non-static members are together.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java
##########
@@ -192,6 +199,44 @@
     public static final String INTER_WORKER_VERIFICATION_ALGORITHMS_DOC = "A list of permitted algorithms for verifying internal requests";
     public static final List<String> INTER_WORKER_VERIFICATION_ALGORITHMS_DEFAULT = Collections.singletonList(INTER_WORKER_SIGNATURE_ALGORITHM_DEFAULT);
 
+    private enum ExactlyOnceSourceSupport {
+        DISABLED(false),
+        PREPARING(true),
+        ENABLED(true);
+
+        public final boolean usesTransactionalLeader;
+
+        ExactlyOnceSourceSupport(boolean usesTransactionalLeader) {
+            this.usesTransactionalLeader = usesTransactionalLeader;
+        }
+
+        public static List<String> options() {
+            return Stream.of(values()).map(ExactlyOnceSourceSupport::toString).collect(Collectors.toList());
+        }
+
+        public static ExactlyOnceSourceSupport fromProperty(String property) {
+            return ExactlyOnceSourceSupport.valueOf(property.toUpperCase(Locale.ROOT));
+        }
+
+        @Override
+        public String toString() {
+            return name().toLowerCase(Locale.ROOT);
+        }
+    }
+
+    public static final String EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG = "exactly.once.source.support";
+    public static final String EXACTLY_ONCE_SOURCE_SUPPORT_DOC = "Whether to enable exactly-once support for source connectors in the cluster "
+            + "by writing source records and their offsets in a Kafka transaction, and by proactively fencing out old task generations before bringing up new ones. "

Review comment:
       Nit, to improve readability and precision, especially around how many Kafka transactions would be used:
   > Whether to enable exactly-once support for source connectors in the cluster by using transactions to write source records and their source offsets, and by proactively fencing out old task generations before bringing up new ones.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectUtils.java
##########
@@ -72,6 +74,45 @@ static String lookupKafkaClusterId(Admin adminClient) {
         }
     }
 
+    /**
+     * Log a warning when the user attempts to override a property that cannot be overridden.
+     * @param props the configuration properties provided by the user
+     * @param key the name of the property to check on
+     * @param expectedValue the expected value for the property
+     * @param justification the reason the property cannot be overridden.
+     *                      Will follow the phrase "The value... for the... property will be ignored as it cannot be overridden ".
+     *                      For example, one might supply the message "in connectors with the DLQ feature enabled" for this parameter.
+     * @param caseSensitive whether the value should match case-insensitively
+     */
+    public static void warnOnOverriddenProperty(
+            Map<String, ?> props,
+            String key,
+            String expectedValue,
+            String justification,
+            boolean caseSensitive) {
+        overriddenPropertyWarning(props, key, expectedValue, justification, caseSensitive).ifPresent(log::warn);
+    }
+
+    // Visible for testing
+    static Optional<String> overriddenPropertyWarning(
+            Map<String, ?> props,
+            String key,
+            String expectedValue,
+            String justification,
+            boolean caseSensitive) {
+        Predicate<String> matchesExpectedValue = caseSensitive ? expectedValue::equals : expectedValue::equalsIgnoreCase;
+        String value = Optional.ofNullable(props.get(key)).map(Object::toString).orElse(null);
+        if (value != null && !matchesExpectedValue.test(value)) {
+            return Optional.of(String.format(
+                    "The value '%s' for the '%s' property will be ignored as it cannot be overridden%s. "
+                            + "The value '%s' will be used instead.",

Review comment:
       Is this really true that all of these incorrectly-overridden property values are ignored?

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -264,12 +279,62 @@ public void startConnector(
 
                 log.info("Creating connector {} of type {}", connName, connClass);
                 final Connector connector = plugins.newConnector(connClass);
-                final ConnectorConfig connConfig = ConnectUtils.isSinkConnector(connector)
-                        ? new SinkConnectorConfig(plugins, connProps)
-                        : new SourceConnectorConfig(plugins, connProps, config.topicCreationEnable());
+                final ConnectorConfig connConfig;
+                final CloseableOffsetStorageReader offsetReader;
+                if (ConnectUtils.isSinkConnector(connector)) {
+                    connConfig = new SinkConnectorConfig(plugins, connProps);
+                    offsetReader = null;
+                } else {
+                    SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connProps, config.topicCreationEnable());
+                    connConfig = sourceConfig;
+
+                    String connectorOffsetsTopic = null;
+                    if (sourceConfig.offsetsTopic() != null) {
+                        connectorOffsetsTopic = sourceConfig.offsetsTopic();
+                    } else if (config.exactlyOnceSourceEnabled()) {
+                        connectorOffsetsTopic = config.offsetsTopic();
+                    }
+

Review comment:
       Suggestion:
   ```suggestion
   
                       // Set up the offset backing store for this connector instance
   ```

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -678,10 +810,9 @@ private WorkerTask buildWorkerTask(ClusterConfigState configState,
         // and through to the task
         Map<String, Object> consumerProps = new HashMap<>();
 
-        consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, SinkUtils.consumerGroupId(id.connector()));
-        consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, "connector-consumer-" + id);
-        consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG,
-                  Utils.join(config.getList(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG), ","));
+        consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, SinkUtils.consumerGroupId(connName));
+        consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, defaultClientId);
+        consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, config.bootstrapServers());

Review comment:
       Thanks! This should have been done quite some time ago.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -476,22 +541,95 @@ public boolean isRunning(String connName) {
     }
 
     /**
-     * Start a task managed by this worker.
+     * Start a sink task managed by this worker.
+     *
+     * @param id the task ID.
+     * @param configState the most recent {@link ClusterConfigState} known to the worker
+     * @param connProps the connector properties.
+     * @param taskProps the tasks properties.
+     * @param statusListener a listener for the runtime status transitions of the task.
+     * @param initialState the initial state of the connector.
+     * @return true if the task started successfully.
+     */
+    public boolean startSinkTask(
+            ConnectorTaskId id,
+            ClusterConfigState configState,
+            Map<String, String> connProps,
+            Map<String, String> taskProps,
+            TaskStatus.Listener statusListener,
+            TargetState initialState
+    ) {
+        return startTask(id, connProps, taskProps, statusListener,
+                new SinkTaskBuilder(id, configState, statusListener, initialState));
+    }
+
+    /**
+     * Start a source task managed by this worker.

Review comment:
       It's probably worth while to mention that this method starts the task for a source connector with older behavior (without exactly once support).

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -264,12 +279,62 @@ public void startConnector(
 
                 log.info("Creating connector {} of type {}", connName, connClass);
                 final Connector connector = plugins.newConnector(connClass);
-                final ConnectorConfig connConfig = ConnectUtils.isSinkConnector(connector)
-                        ? new SinkConnectorConfig(plugins, connProps)
-                        : new SourceConnectorConfig(plugins, connProps, config.topicCreationEnable());
+                final ConnectorConfig connConfig;
+                final CloseableOffsetStorageReader offsetReader;
+                if (ConnectUtils.isSinkConnector(connector)) {
+                    connConfig = new SinkConnectorConfig(plugins, connProps);
+                    offsetReader = null;
+                } else {
+                    SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connProps, config.topicCreationEnable());
+                    connConfig = sourceConfig;
+

Review comment:
       Suggestion:
   ```suggestion
   
                       // Use the desired topic for offsets
   ```

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -980,6 +1119,329 @@ WorkerMetricsGroup workerMetricsGroup() {
         return workerMetricsGroup;
     }
 
+    abstract class TaskBuilder {
+
+        private final ConnectorTaskId id;
+        private final ClusterConfigState configState;
+        private final TaskStatus.Listener statusListener;
+        private final TargetState initialState;
+
+        private Task task;
+        private ConnectorConfig connectorConfig = null;
+        private Converter keyConverter = null;
+        private Converter valueConverter = null;
+        private HeaderConverter headerConverter = null;
+        private ClassLoader classLoader = null;
+
+        public TaskBuilder(ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState) {
+            this.id = id;
+            this.configState = configState;
+            this.statusListener = statusListener;
+            this.initialState = initialState;
+        }
+
+        public TaskBuilder withTask(Task task) {
+            this.task = task;
+            return this;
+        }
+
+        public TaskBuilder withConnectorConfig(ConnectorConfig connectorConfig) {
+            this.connectorConfig = connectorConfig;
+            return this;
+        }
+
+        public TaskBuilder withKeyConverter(Converter keyConverter) {
+            this.keyConverter = keyConverter;
+            return this;
+        }
+
+        public TaskBuilder withValueConverter(Converter valueConverter) {
+            this.valueConverter = valueConverter;
+            return this;
+        }
+
+        public TaskBuilder withHeaderConverter(HeaderConverter headerConverter) {
+            this.headerConverter = headerConverter;
+            return this;
+        }
+
+        public TaskBuilder withClassloader(ClassLoader classLoader) {
+            this.classLoader = classLoader;
+            return this;
+        }
+
+        public WorkerTask build() {
+            Objects.requireNonNull(task, "Task cannot be null");
+            Objects.requireNonNull(connectorConfig, "Connector config used by task cannot be null");
+            Objects.requireNonNull(keyConverter, "Key converter used by task cannot be null");
+            Objects.requireNonNull(valueConverter, "Value converter used by task cannot be null");
+            Objects.requireNonNull(headerConverter, "Header converter used by task cannot be null");
+            Objects.requireNonNull(classLoader, "Classloader used by task cannot be null");
+
+            ErrorHandlingMetrics errorHandlingMetrics = errorHandlingMetrics(id);
+            final Class<? extends Connector> connectorClass = plugins.connectorClass(
+                    connectorConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG));
+            RetryWithToleranceOperator retryWithToleranceOperator = new RetryWithToleranceOperator(connectorConfig.errorRetryTimeout(),
+                    connectorConfig.errorMaxDelayInMillis(), connectorConfig.errorToleranceType(), Time.SYSTEM);
+            retryWithToleranceOperator.metrics(errorHandlingMetrics);
+
+            return doBuild(task, id, configState, statusListener, initialState,
+                    connectorConfig, keyConverter, valueConverter, headerConverter, classLoader,
+                    errorHandlingMetrics, connectorClass, retryWithToleranceOperator);
+        }
+
+        abstract WorkerTask doBuild(Task task,
+                                    ConnectorTaskId id,
+                                    ClusterConfigState configState,
+                                    TaskStatus.Listener statusListener,
+                                    TargetState initialState,
+                                    ConnectorConfig connectorConfig,
+                                    Converter keyConverter,
+                                    Converter valueConverter,
+                                    HeaderConverter headerConverter,
+                                    ClassLoader classLoader,
+                                    ErrorHandlingMetrics errorHandlingMetrics,
+                                    Class<? extends Connector> connectorClass,
+                                    RetryWithToleranceOperator retryWithToleranceOperator);
+
+    }
+
+    class SinkTaskBuilder extends TaskBuilder {
+        public SinkTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            TransformationChain<SinkRecord> transformationChain = new TransformationChain<>(connectorConfig.<SinkRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+            SinkConnectorConfig sinkConfig = new SinkConnectorConfig(plugins, connectorConfig.originalsStrings());
+            retryWithToleranceOperator.reporters(sinkTaskReporters(id, sinkConfig, errorHandlingMetrics, connectorClass));
+            WorkerErrantRecordReporter workerErrantRecordReporter = createWorkerErrantRecordReporter(sinkConfig, retryWithToleranceOperator,
+                    keyConverter, valueConverter, headerConverter);
+
+            Map<String, Object> consumerProps = consumerConfigs(id.connector(),  "connector-consumer-" + id, config, connectorConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+            return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverter,
+                    valueConverter, headerConverter, transformationChain, consumer, classLoader, time,
+                    retryWithToleranceOperator, workerErrantRecordReporter, herder.statusBackingStore());
+        }
+    }
+
+    class SourceTaskBuilder extends TaskBuilder {
+        public SourceTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+            ConnectorOffsetBackingStore offsetStore = ConnectorOffsetBackingStore.withoutConnectorOffsetStore(
+                    () -> LoggingContext.forTask(id),
+                    globalOffsetBackingStore,
+                    config.offsetsTopic()
+            );
+            final TopicAdmin admin;
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (sourceConfig.offsetsTopic() != null || (config.topicCreationEnable() && sourceConfig.usesTopicCreation())) {

Review comment:
       I think adding some comments here will help future developers. There are enough subtleties here (e.g., the `||` in this line) that makes this more challenging than it should be to understand the behavior. 
   
   Or, it might be worth adding some final booleans here to make these conditions a bit more clear and readable, especially since this method is a series of relatively independent parts/sections. For example:
   ```
   final boolean customOffsetTopic = sourceConfig.offsetTopic();
   final boolean createTopicsEnabled = config.topicCreationEnable() && sourceConfig.usesTopicCreation();
   if (customOffsetTopic || createTopicsEnabled) {
     // Create an admin client
     ...
     if (createTopicsEnabled) {
       topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig);
     }
     if (customOffsetTopic) {
       // Build custom offset store
       ...
     }
   } else {
     // No need for admin
     admin = null;
   }
   ```

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -692,21 +823,22 @@ private WorkerTask buildWorkerTask(ClusterConfigState configState,
         ConnectUtils.addMetricsContextProperties(consumerProps, config, clusterId);
         // Connector-specified overrides
         Map<String, Object> consumerOverrides =
-            connectorClientConfigOverrides(id, connConfig, connectorClass, ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX,
+            connectorClientConfigOverrides(connName, connConfig, connectorClass, ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX,
                                            ConnectorType.SINK, ConnectorClientConfigRequest.ClientType.CONSUMER,
                                            connectorClientConfigOverridePolicy);
         consumerProps.putAll(consumerOverrides);
 
         return consumerProps;
     }
 
-    static Map<String, Object> adminConfigs(ConnectorTaskId id,
+    static Map<String, Object> adminConfigs(String connName,

Review comment:
       Do we need to change these signatures from `ConnectorTaskId` to `String`? The `ConnectorTaskId` gives us the ability to define tasks-specific client configuration properties if necessary/desired. I'm afraid that switching to `String` will make it harder and more invasive to add that back in. Plus, if there's not a good reason to remove these, let's leave that for smaller PRs.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java
##########
@@ -342,6 +343,10 @@ private void logPluginPathConfigProviderWarning(Map<String, String> rawOriginals
         }
     }
 
+    public String bootstrapServers() {
+        return String.join(",", getList(BOOTSTRAP_SERVERS_CONFIG));
+    }

Review comment:
       Very nice. For this and the other new getter methods, can you add some JavaDoc just so that it's easier to follow in an IDE where these methods are _used_? I wanted that several times as I was reviewing this code.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
##########
@@ -16,95 +16,46 @@
  */
 package org.apache.kafka.connect.runtime;
 
-import org.apache.kafka.clients.admin.NewTopic;
-import org.apache.kafka.clients.admin.TopicDescription;
-import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.header.internals.RecordHeaders;
-import org.apache.kafka.common.metrics.Sensor;
-import org.apache.kafka.common.metrics.stats.Avg;
-import org.apache.kafka.common.metrics.stats.CumulativeSum;
-import org.apache.kafka.common.metrics.stats.Max;
-import org.apache.kafka.common.metrics.stats.Rate;
-import org.apache.kafka.common.metrics.stats.Value;
 import org.apache.kafka.common.utils.Time;
-import org.apache.kafka.common.utils.Utils;
-import org.apache.kafka.connect.errors.ConnectException;
-import org.apache.kafka.connect.errors.RetriableException;
-import org.apache.kafka.connect.header.Header;
-import org.apache.kafka.connect.header.Headers;
-import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup;
-import org.apache.kafka.connect.runtime.distributed.ClusterConfigState;
 import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator;
-import org.apache.kafka.connect.runtime.errors.Stage;
 import org.apache.kafka.connect.source.SourceRecord;
 import org.apache.kafka.connect.source.SourceTask;
 import org.apache.kafka.connect.storage.CloseableOffsetStorageReader;
+import org.apache.kafka.connect.storage.ClusterConfigState;
+import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore;
 import org.apache.kafka.connect.storage.Converter;
 import org.apache.kafka.connect.storage.HeaderConverter;
 import org.apache.kafka.connect.storage.OffsetStorageWriter;
 import org.apache.kafka.connect.storage.StatusBackingStore;
-import org.apache.kafka.connect.util.ConnectUtils;
 import org.apache.kafka.connect.util.ConnectorTaskId;
 import org.apache.kafka.connect.util.TopicAdmin;
-import org.apache.kafka.connect.util.TopicCreation;
 import org.apache.kafka.connect.util.TopicCreationGroup;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.time.Duration;
 import java.util.IdentityHashMap;
-import java.util.List;
 import java.util.Map;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executor;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_TRACKING_ENABLE_CONFIG;
 
 /**
  * WorkerTask that uses a SourceTask to ingest data into Kafka.
  */
-class WorkerSourceTask extends WorkerTask {
+class WorkerSourceTask extends AbstractWorkerSourceTask {

Review comment:
       Note to future me: I didn't get this far in the PR.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -980,6 +1119,329 @@ WorkerMetricsGroup workerMetricsGroup() {
         return workerMetricsGroup;
     }
 
+    abstract class TaskBuilder {
+
+        private final ConnectorTaskId id;
+        private final ClusterConfigState configState;
+        private final TaskStatus.Listener statusListener;
+        private final TargetState initialState;
+
+        private Task task;
+        private ConnectorConfig connectorConfig = null;
+        private Converter keyConverter = null;
+        private Converter valueConverter = null;
+        private HeaderConverter headerConverter = null;
+        private ClassLoader classLoader = null;
+
+        public TaskBuilder(ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState) {
+            this.id = id;
+            this.configState = configState;
+            this.statusListener = statusListener;
+            this.initialState = initialState;
+        }
+
+        public TaskBuilder withTask(Task task) {
+            this.task = task;
+            return this;
+        }
+
+        public TaskBuilder withConnectorConfig(ConnectorConfig connectorConfig) {
+            this.connectorConfig = connectorConfig;
+            return this;
+        }
+
+        public TaskBuilder withKeyConverter(Converter keyConverter) {
+            this.keyConverter = keyConverter;
+            return this;
+        }
+
+        public TaskBuilder withValueConverter(Converter valueConverter) {
+            this.valueConverter = valueConverter;
+            return this;
+        }
+
+        public TaskBuilder withHeaderConverter(HeaderConverter headerConverter) {
+            this.headerConverter = headerConverter;
+            return this;
+        }
+
+        public TaskBuilder withClassloader(ClassLoader classLoader) {
+            this.classLoader = classLoader;
+            return this;
+        }
+
+        public WorkerTask build() {
+            Objects.requireNonNull(task, "Task cannot be null");
+            Objects.requireNonNull(connectorConfig, "Connector config used by task cannot be null");
+            Objects.requireNonNull(keyConverter, "Key converter used by task cannot be null");
+            Objects.requireNonNull(valueConverter, "Value converter used by task cannot be null");
+            Objects.requireNonNull(headerConverter, "Header converter used by task cannot be null");
+            Objects.requireNonNull(classLoader, "Classloader used by task cannot be null");
+
+            ErrorHandlingMetrics errorHandlingMetrics = errorHandlingMetrics(id);
+            final Class<? extends Connector> connectorClass = plugins.connectorClass(
+                    connectorConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG));
+            RetryWithToleranceOperator retryWithToleranceOperator = new RetryWithToleranceOperator(connectorConfig.errorRetryTimeout(),
+                    connectorConfig.errorMaxDelayInMillis(), connectorConfig.errorToleranceType(), Time.SYSTEM);
+            retryWithToleranceOperator.metrics(errorHandlingMetrics);
+
+            return doBuild(task, id, configState, statusListener, initialState,
+                    connectorConfig, keyConverter, valueConverter, headerConverter, classLoader,
+                    errorHandlingMetrics, connectorClass, retryWithToleranceOperator);
+        }
+
+        abstract WorkerTask doBuild(Task task,
+                                    ConnectorTaskId id,
+                                    ClusterConfigState configState,
+                                    TaskStatus.Listener statusListener,
+                                    TargetState initialState,
+                                    ConnectorConfig connectorConfig,
+                                    Converter keyConverter,
+                                    Converter valueConverter,
+                                    HeaderConverter headerConverter,
+                                    ClassLoader classLoader,
+                                    ErrorHandlingMetrics errorHandlingMetrics,
+                                    Class<? extends Connector> connectorClass,
+                                    RetryWithToleranceOperator retryWithToleranceOperator);
+
+    }
+
+    class SinkTaskBuilder extends TaskBuilder {
+        public SinkTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            TransformationChain<SinkRecord> transformationChain = new TransformationChain<>(connectorConfig.<SinkRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+            SinkConnectorConfig sinkConfig = new SinkConnectorConfig(plugins, connectorConfig.originalsStrings());
+            retryWithToleranceOperator.reporters(sinkTaskReporters(id, sinkConfig, errorHandlingMetrics, connectorClass));
+            WorkerErrantRecordReporter workerErrantRecordReporter = createWorkerErrantRecordReporter(sinkConfig, retryWithToleranceOperator,
+                    keyConverter, valueConverter, headerConverter);
+
+            Map<String, Object> consumerProps = consumerConfigs(id.connector(),  "connector-consumer-" + id, config, connectorConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+            return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverter,
+                    valueConverter, headerConverter, transformationChain, consumer, classLoader, time,
+                    retryWithToleranceOperator, workerErrantRecordReporter, herder.statusBackingStore());
+        }
+    }
+
+    class SourceTaskBuilder extends TaskBuilder {
+        public SourceTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+            ConnectorOffsetBackingStore offsetStore = ConnectorOffsetBackingStore.withoutConnectorOffsetStore(
+                    () -> LoggingContext.forTask(id),
+                    globalOffsetBackingStore,
+                    config.offsetsTopic()
+            );
+            final TopicAdmin admin;
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (sourceConfig.offsetsTopic() != null || (config.topicCreationEnable() && sourceConfig.usesTopicCreation())) {
+                Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,
+                        sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE);
+                Admin adminClient = Admin.create(adminOverrides);
+                admin = new TopicAdmin(adminOverrides.get(BOOTSTRAP_SERVERS_CONFIG), adminClient);
+
+                if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) {
+                    topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig);
+                }
+
+                if (sourceConfig.offsetsTopic() != null && config.connectorOffsetsTopicsPermitted()) {
+                    Map<String, Object> consumerProps = consumerConfigs(id.connector(), "connector-consumer-" + id, config, connectorConfig, connectorClass,
+                            connectorClientConfigOverridePolicy, kafkaClusterId);
+                    // Users can disable this if they want to; it won't affect delivery guarantees since the task isn't exactly-once anyways
+                    consumerProps.putIfAbsent(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString().toLowerCase(Locale.ROOT));
+                    KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+                    offsetStore = ConnectorOffsetBackingStore.withConnectorOffsetStore(
+                            () -> LoggingContext.forTask(id),
+                            globalOffsetBackingStore,
+                            sourceConfig.offsetsTopic(),
+                            producer,
+                            consumer,
+                            admin
+                    );
+                }
+            } else {
+                admin = null;
+            }
+            offsetStore.configure(config);
+
+            CloseableOffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+            OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+
+            // Note we pass the configState as it performs dynamic transformations under the covers
+            return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter, valueConverter,
+                    headerConverter, transformationChain, producer, admin, topicCreationGroups,
+                    offsetReader, offsetWriter, offsetStore, config, configState, metrics, classLoader, time,
+                    retryWithToleranceOperator, herder.statusBackingStore(), executor);
+        }
+    }
+
+    class ExactlyOnceSourceTaskBuilder extends TaskBuilder {
+        private final Runnable preProducerCheck;
+        private final Runnable postProducerCheck;
+
+        public ExactlyOnceSourceTaskBuilder(ConnectorTaskId id,
+                                            ClusterConfigState configState,
+                                            TaskStatus.Listener statusListener,
+                                            TargetState initialState,
+                                            Runnable preProducerCheck,
+                                            Runnable postProducerCheck) {
+            super(id, configState, statusListener, initialState);
+            this.preProducerCheck = preProducerCheck;
+            this.postProducerCheck = postProducerCheck;
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                                  ConnectorTaskId id,
+                                  ClusterConfigState configState,
+                                  TaskStatus.Listener statusListener,
+                                  TargetState initialState,
+                                  ConnectorConfig connectorConfig,
+                                  Converter keyConverter,
+                                  Converter valueConverter,
+                                  HeaderConverter headerConverter,
+                                  ClassLoader classLoader,
+                                  ErrorHandlingMetrics errorHandlingMetrics,
+                                  Class<? extends Connector> connectorClass,
+                                  RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,

Review comment:
       In my comment about `SourceTaskBuilder#doBuild`, I suggested adding a few comments to help the different parts/sections of this method stand out more clearly. The same would be true here, such as:
   ```suggestion
               // Create an admin client
               Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,
   ```

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -980,6 +1119,329 @@ WorkerMetricsGroup workerMetricsGroup() {
         return workerMetricsGroup;
     }
 
+    abstract class TaskBuilder {
+
+        private final ConnectorTaskId id;
+        private final ClusterConfigState configState;
+        private final TaskStatus.Listener statusListener;
+        private final TargetState initialState;
+
+        private Task task;
+        private ConnectorConfig connectorConfig = null;
+        private Converter keyConverter = null;
+        private Converter valueConverter = null;
+        private HeaderConverter headerConverter = null;
+        private ClassLoader classLoader = null;
+
+        public TaskBuilder(ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState) {
+            this.id = id;
+            this.configState = configState;
+            this.statusListener = statusListener;
+            this.initialState = initialState;
+        }
+
+        public TaskBuilder withTask(Task task) {
+            this.task = task;
+            return this;
+        }
+
+        public TaskBuilder withConnectorConfig(ConnectorConfig connectorConfig) {
+            this.connectorConfig = connectorConfig;
+            return this;
+        }
+
+        public TaskBuilder withKeyConverter(Converter keyConverter) {
+            this.keyConverter = keyConverter;
+            return this;
+        }
+
+        public TaskBuilder withValueConverter(Converter valueConverter) {
+            this.valueConverter = valueConverter;
+            return this;
+        }
+
+        public TaskBuilder withHeaderConverter(HeaderConverter headerConverter) {
+            this.headerConverter = headerConverter;
+            return this;
+        }
+
+        public TaskBuilder withClassloader(ClassLoader classLoader) {
+            this.classLoader = classLoader;
+            return this;
+        }
+
+        public WorkerTask build() {
+            Objects.requireNonNull(task, "Task cannot be null");
+            Objects.requireNonNull(connectorConfig, "Connector config used by task cannot be null");
+            Objects.requireNonNull(keyConverter, "Key converter used by task cannot be null");
+            Objects.requireNonNull(valueConverter, "Value converter used by task cannot be null");
+            Objects.requireNonNull(headerConverter, "Header converter used by task cannot be null");
+            Objects.requireNonNull(classLoader, "Classloader used by task cannot be null");
+
+            ErrorHandlingMetrics errorHandlingMetrics = errorHandlingMetrics(id);
+            final Class<? extends Connector> connectorClass = plugins.connectorClass(
+                    connectorConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG));
+            RetryWithToleranceOperator retryWithToleranceOperator = new RetryWithToleranceOperator(connectorConfig.errorRetryTimeout(),
+                    connectorConfig.errorMaxDelayInMillis(), connectorConfig.errorToleranceType(), Time.SYSTEM);
+            retryWithToleranceOperator.metrics(errorHandlingMetrics);
+
+            return doBuild(task, id, configState, statusListener, initialState,
+                    connectorConfig, keyConverter, valueConverter, headerConverter, classLoader,
+                    errorHandlingMetrics, connectorClass, retryWithToleranceOperator);
+        }
+
+        abstract WorkerTask doBuild(Task task,
+                                    ConnectorTaskId id,
+                                    ClusterConfigState configState,
+                                    TaskStatus.Listener statusListener,
+                                    TargetState initialState,
+                                    ConnectorConfig connectorConfig,
+                                    Converter keyConverter,
+                                    Converter valueConverter,
+                                    HeaderConverter headerConverter,
+                                    ClassLoader classLoader,
+                                    ErrorHandlingMetrics errorHandlingMetrics,
+                                    Class<? extends Connector> connectorClass,
+                                    RetryWithToleranceOperator retryWithToleranceOperator);
+
+    }
+
+    class SinkTaskBuilder extends TaskBuilder {
+        public SinkTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            TransformationChain<SinkRecord> transformationChain = new TransformationChain<>(connectorConfig.<SinkRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+            SinkConnectorConfig sinkConfig = new SinkConnectorConfig(plugins, connectorConfig.originalsStrings());
+            retryWithToleranceOperator.reporters(sinkTaskReporters(id, sinkConfig, errorHandlingMetrics, connectorClass));
+            WorkerErrantRecordReporter workerErrantRecordReporter = createWorkerErrantRecordReporter(sinkConfig, retryWithToleranceOperator,
+                    keyConverter, valueConverter, headerConverter);
+
+            Map<String, Object> consumerProps = consumerConfigs(id.connector(),  "connector-consumer-" + id, config, connectorConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+            return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverter,
+                    valueConverter, headerConverter, transformationChain, consumer, classLoader, time,
+                    retryWithToleranceOperator, workerErrantRecordReporter, herder.statusBackingStore());
+        }
+    }
+
+    class SourceTaskBuilder extends TaskBuilder {
+        public SourceTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+            ConnectorOffsetBackingStore offsetStore = ConnectorOffsetBackingStore.withoutConnectorOffsetStore(
+                    () -> LoggingContext.forTask(id),
+                    globalOffsetBackingStore,
+                    config.offsetsTopic()
+            );
+            final TopicAdmin admin;
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (sourceConfig.offsetsTopic() != null || (config.topicCreationEnable() && sourceConfig.usesTopicCreation())) {
+                Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,
+                        sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE);
+                Admin adminClient = Admin.create(adminOverrides);
+                admin = new TopicAdmin(adminOverrides.get(BOOTSTRAP_SERVERS_CONFIG), adminClient);
+
+                if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) {
+                    topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig);
+                }
+
+                if (sourceConfig.offsetsTopic() != null && config.connectorOffsetsTopicsPermitted()) {
+                    Map<String, Object> consumerProps = consumerConfigs(id.connector(), "connector-consumer-" + id, config, connectorConfig, connectorClass,
+                            connectorClientConfigOverridePolicy, kafkaClusterId);
+                    // Users can disable this if they want to; it won't affect delivery guarantees since the task isn't exactly-once anyways
+                    consumerProps.putIfAbsent(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString().toLowerCase(Locale.ROOT));
+                    KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+                    offsetStore = ConnectorOffsetBackingStore.withConnectorOffsetStore(
+                            () -> LoggingContext.forTask(id),
+                            globalOffsetBackingStore,
+                            sourceConfig.offsetsTopic(),
+                            producer,
+                            consumer,
+                            admin
+                    );
+                }
+            } else {
+                admin = null;
+            }
+            offsetStore.configure(config);
+
+            CloseableOffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+            OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+
+            // Note we pass the configState as it performs dynamic transformations under the covers
+            return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter, valueConverter,
+                    headerConverter, transformationChain, producer, admin, topicCreationGroups,
+                    offsetReader, offsetWriter, offsetStore, config, configState, metrics, classLoader, time,
+                    retryWithToleranceOperator, herder.statusBackingStore(), executor);
+        }
+    }
+
+    class ExactlyOnceSourceTaskBuilder extends TaskBuilder {
+        private final Runnable preProducerCheck;
+        private final Runnable postProducerCheck;
+
+        public ExactlyOnceSourceTaskBuilder(ConnectorTaskId id,
+                                            ClusterConfigState configState,
+                                            TaskStatus.Listener statusListener,
+                                            TargetState initialState,
+                                            Runnable preProducerCheck,
+                                            Runnable postProducerCheck) {
+            super(id, configState, statusListener, initialState);
+            this.preProducerCheck = preProducerCheck;
+            this.postProducerCheck = postProducerCheck;
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                                  ConnectorTaskId id,
+                                  ClusterConfigState configState,
+                                  TaskStatus.Listener statusListener,
+                                  TargetState initialState,
+                                  ConnectorConfig connectorConfig,
+                                  Converter keyConverter,
+                                  Converter valueConverter,
+                                  HeaderConverter headerConverter,
+                                  ClassLoader classLoader,
+                                  ErrorHandlingMetrics errorHandlingMetrics,
+                                  Class<? extends Connector> connectorClass,
+                                  RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,
+                    sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE);
+            Admin adminClient = Admin.create(adminOverrides);
+            TopicAdmin topicAdmin = new TopicAdmin(adminOverrides.get(BOOTSTRAP_SERVERS_CONFIG), adminClient);
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) {
+                topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig);
+            }
+
+            String transactionalId = transactionalId(id);
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            ConnectUtils.warnOnOverriddenProperty(
+                    producerProps, ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true",
+                    "for connectors when exactly-once source support is enabled",
+                    false
+            );
+            ConnectUtils.warnOnOverriddenProperty(
+                    producerProps, ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId,
+                    "for connectors when exactly-once source support is enabled",
+                    true
+            );
+            producerProps.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, true);
+            producerProps.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId);
+            KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+            Map<String, Object> consumerProps = consumerConfigs(id.connector(), "connector-consumer-" + id, config, connectorConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            ConnectUtils.warnOnOverriddenProperty(
+                    consumerProps, ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.name().toLowerCase(Locale.ROOT),
+                    "for connectors when exactly-once source support is enabled",
+                    false
+            );
+            consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString().toLowerCase(Locale.ROOT));
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+            String offsetsTopic = Optional.ofNullable(sourceConfig.offsetsTopic()).orElse(config.offsetsTopic());
+
+            ConnectorOffsetBackingStore offsetStore;
+            // No need to do secondary writes to the global offsets topic if we're certain that the task's local offset store
+            // is going to be targeting it anyway
+            // Note that this may lead to a false positive if the user provides an overridden bootstrap servers value for their
+            // producer that resolves to the same Kafka cluster; we might consider looking up the Kafka cluster ID in the future
+            // to prevent these false positives but at the moment this is probably adequate, especially since we probably don't
+            // want to put a ping to a remote Kafka cluster inside the herder's tick thread (which is where this logic takes place
+            // right now) in case that takes a while.

Review comment:
       This is more of a nit-type comment.
   
   It seems like this comment tries to describe why we don't need to create a connector-specific offset store **AND** then talk about the check might not always be accurate. But I found the comment a bit hard to follow.
   
   Also, the `ConnectorOffsetBackingStore offsetStore;` line should move under the multi-line comment.
   
   In such cases, it might be worth changing the code to help make things more clear. For example:
   ```
   // We can simply reuse the worker's offset store when the connector-specific offset topic
   // is the same as the worker's. We can check the offset topic name and the Kafka cluster's
   // bootstrap servers, although this isn't exact and can lead to some false positives if the user
   // provides an overridden bootstrap servers value for their producer that is different than
   // the worker's but still resolves to the same Kafka cluster used by the worker.
   // At the moment this is probably adequate, especially since we probably don't want to put
   // a network ping to a remote Kafka cluster inside the herder's tick thread (which is where this
   // logic takes place right now) in case that takes a while.
   ConnectorOffsetBackingStore offsetStore;
   final boolean sameOffsetTopicAsWorker = offsetsTopic.equals(config.offsetsTopic())
                       && producerProps.get(BOOTSTRAP_SERVERS_CONFIG).equals(config.bootstrapServers();
   if (sameOffsetTopicAsWorker) {
       offsetStore = ConnectorOffsetBackingStore.withoutConnectorOffsetStore(
       ...
   } else {
       offsetStore = ConnectorOffsetBackingStore.withConnectorOffsetStore(
       ...
   }
   ```

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java
##########
@@ -20,13 +20,45 @@
 import org.apache.kafka.clients.producer.RecordMetadata;
 
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * SourceTask is a Task that pulls records from another system for storage in Kafka.
  */
 public abstract class SourceTask implements Task {
 
+    /**
+     * <p>
+     * The configuration key that determines how source tasks will define transaction boundaries
+     * when exactly-once support is enabled.
+     * </p>
+     */
+    public static final String TRANSACTION_BOUNDARY_CONFIG = "transaction.boundary";
+
+    public enum TransactionBoundary {
+        POLL,
+        INTERVAL,
+        CONNECTOR;
+
+        public static final TransactionBoundary DEFAULT = POLL;
+
+        public static List<String> options() {

Review comment:
       I think the risk of introducing `options()` is that some developers might accidentally use `values()`. 
   
   The pattern used in `ConnectorType` is far better, as it overrides the `toString()` method. That doesn't handle the case-independence for parsing, though `ConverterType` is a better pattern to follow if that's required.
   
   Let's be consistent with the new enums, and have each follow one of those two patterns depending upon whether parsing case-independently is required.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java
##########
@@ -28,4 +30,31 @@
     protected SourceConnectorContext context() {
         return (SourceConnectorContext) context;
     }
+
+    /**
+     * Signals whether the connector supports exactly-once delivery guarantees with a proposed configuration.
+     * Developers can assume that worker-level exactly-once support is enabled when this method is invoked.
+     * The default implementation will return {@code null}.
+     * @param connectorConfig the configuration that will be used for the connector.
+     * @return {@link ExactlyOnceSupport#SUPPORTED} if the connector can provide exactly-once support,
+     * and {@link ExactlyOnceSupport#UNSUPPORTED} if it cannot. If {@code null}, it is assumed that the
+     * connector cannot.
+     */
+    public ExactlyOnceSupport exactlyOnceSupport(Map<String, String> connectorConfig) {
+        return null;
+    }
+
+    /**
+     * Signals whether the connector can define its own transaction boundaries with the proposed
+     * configuration. Developers must override this method if they wish to add connector-defined
+     * transaction boundary support; if they do not, users will be unable to create instances of
+     * this connector that use connector-defined transaction boundaries. The default implementation
+     * will return {@code UNSUPPORTED}.

Review comment:
       WDYT about something like this:
   ```
   /**
    * Signals whether the connector implementation is capable of defining the transaction boundaries for a
    * connector with the given configuration. This method is called before {@link #start(Map)}, only when the
    * runtime supports exactly-once and the connector configuration includes {@code transaction.boundary=connector}.
    *
    * <p>This method need not be implemented if the connector implementation does not support definiting
    * transaction boundaries.
    *
    * @param connectorConfig the configuration that will be used for the connector
    * @return {@link ConnectorTransactionBoundaries.SUPPORTED} if the connector will define its own transaction boundaries, 
    *         or {@link ConnectorTransactionBoundaries.UNSUPPORTED} otherwise.
    * @see TransactionContext
    */
   ```
   

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -980,6 +1119,329 @@ WorkerMetricsGroup workerMetricsGroup() {
         return workerMetricsGroup;
     }
 
+    abstract class TaskBuilder {
+
+        private final ConnectorTaskId id;
+        private final ClusterConfigState configState;
+        private final TaskStatus.Listener statusListener;
+        private final TargetState initialState;
+
+        private Task task;
+        private ConnectorConfig connectorConfig = null;
+        private Converter keyConverter = null;
+        private Converter valueConverter = null;
+        private HeaderConverter headerConverter = null;
+        private ClassLoader classLoader = null;
+
+        public TaskBuilder(ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState) {
+            this.id = id;
+            this.configState = configState;
+            this.statusListener = statusListener;
+            this.initialState = initialState;
+        }
+
+        public TaskBuilder withTask(Task task) {
+            this.task = task;
+            return this;
+        }
+
+        public TaskBuilder withConnectorConfig(ConnectorConfig connectorConfig) {
+            this.connectorConfig = connectorConfig;
+            return this;
+        }
+
+        public TaskBuilder withKeyConverter(Converter keyConverter) {
+            this.keyConverter = keyConverter;
+            return this;
+        }
+
+        public TaskBuilder withValueConverter(Converter valueConverter) {
+            this.valueConverter = valueConverter;
+            return this;
+        }
+
+        public TaskBuilder withHeaderConverter(HeaderConverter headerConverter) {
+            this.headerConverter = headerConverter;
+            return this;
+        }
+
+        public TaskBuilder withClassloader(ClassLoader classLoader) {
+            this.classLoader = classLoader;
+            return this;
+        }
+
+        public WorkerTask build() {
+            Objects.requireNonNull(task, "Task cannot be null");
+            Objects.requireNonNull(connectorConfig, "Connector config used by task cannot be null");
+            Objects.requireNonNull(keyConverter, "Key converter used by task cannot be null");
+            Objects.requireNonNull(valueConverter, "Value converter used by task cannot be null");
+            Objects.requireNonNull(headerConverter, "Header converter used by task cannot be null");
+            Objects.requireNonNull(classLoader, "Classloader used by task cannot be null");
+
+            ErrorHandlingMetrics errorHandlingMetrics = errorHandlingMetrics(id);
+            final Class<? extends Connector> connectorClass = plugins.connectorClass(
+                    connectorConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG));
+            RetryWithToleranceOperator retryWithToleranceOperator = new RetryWithToleranceOperator(connectorConfig.errorRetryTimeout(),
+                    connectorConfig.errorMaxDelayInMillis(), connectorConfig.errorToleranceType(), Time.SYSTEM);
+            retryWithToleranceOperator.metrics(errorHandlingMetrics);
+
+            return doBuild(task, id, configState, statusListener, initialState,
+                    connectorConfig, keyConverter, valueConverter, headerConverter, classLoader,
+                    errorHandlingMetrics, connectorClass, retryWithToleranceOperator);
+        }
+
+        abstract WorkerTask doBuild(Task task,
+                                    ConnectorTaskId id,
+                                    ClusterConfigState configState,
+                                    TaskStatus.Listener statusListener,
+                                    TargetState initialState,
+                                    ConnectorConfig connectorConfig,
+                                    Converter keyConverter,
+                                    Converter valueConverter,
+                                    HeaderConverter headerConverter,
+                                    ClassLoader classLoader,
+                                    ErrorHandlingMetrics errorHandlingMetrics,
+                                    Class<? extends Connector> connectorClass,
+                                    RetryWithToleranceOperator retryWithToleranceOperator);
+
+    }
+
+    class SinkTaskBuilder extends TaskBuilder {
+        public SinkTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            TransformationChain<SinkRecord> transformationChain = new TransformationChain<>(connectorConfig.<SinkRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+            SinkConnectorConfig sinkConfig = new SinkConnectorConfig(plugins, connectorConfig.originalsStrings());
+            retryWithToleranceOperator.reporters(sinkTaskReporters(id, sinkConfig, errorHandlingMetrics, connectorClass));
+            WorkerErrantRecordReporter workerErrantRecordReporter = createWorkerErrantRecordReporter(sinkConfig, retryWithToleranceOperator,
+                    keyConverter, valueConverter, headerConverter);
+
+            Map<String, Object> consumerProps = consumerConfigs(id.connector(),  "connector-consumer-" + id, config, connectorConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+            return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverter,
+                    valueConverter, headerConverter, transformationChain, consumer, classLoader, time,
+                    retryWithToleranceOperator, workerErrantRecordReporter, herder.statusBackingStore());
+        }
+    }
+
+    class SourceTaskBuilder extends TaskBuilder {
+        public SourceTaskBuilder(ConnectorTaskId id,
+                               ClusterConfigState configState,
+                               TaskStatus.Listener statusListener,
+                               TargetState initialState) {
+            super(id, configState, statusListener, initialState);
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                           ConnectorTaskId id,
+                           ClusterConfigState configState,
+                           TaskStatus.Listener statusListener,
+                           TargetState initialState,
+                           ConnectorConfig connectorConfig,
+                           Converter keyConverter,
+                           Converter valueConverter,
+                           HeaderConverter headerConverter,
+                           ClassLoader classLoader,
+                           ErrorHandlingMetrics errorHandlingMetrics,
+                           Class<? extends Connector> connectorClass,
+                           RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(producerProps);
+
+            ConnectorOffsetBackingStore offsetStore = ConnectorOffsetBackingStore.withoutConnectorOffsetStore(
+                    () -> LoggingContext.forTask(id),
+                    globalOffsetBackingStore,
+                    config.offsetsTopic()
+            );
+            final TopicAdmin admin;
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (sourceConfig.offsetsTopic() != null || (config.topicCreationEnable() && sourceConfig.usesTopicCreation())) {
+                Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,
+                        sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE);
+                Admin adminClient = Admin.create(adminOverrides);
+                admin = new TopicAdmin(adminOverrides.get(BOOTSTRAP_SERVERS_CONFIG), adminClient);
+
+                if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) {
+                    topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig);
+                }
+
+                if (sourceConfig.offsetsTopic() != null && config.connectorOffsetsTopicsPermitted()) {
+                    Map<String, Object> consumerProps = consumerConfigs(id.connector(), "connector-consumer-" + id, config, connectorConfig, connectorClass,
+                            connectorClientConfigOverridePolicy, kafkaClusterId);
+                    // Users can disable this if they want to; it won't affect delivery guarantees since the task isn't exactly-once anyways
+                    consumerProps.putIfAbsent(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString().toLowerCase(Locale.ROOT));
+                    KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
+
+                    offsetStore = ConnectorOffsetBackingStore.withConnectorOffsetStore(
+                            () -> LoggingContext.forTask(id),
+                            globalOffsetBackingStore,
+                            sourceConfig.offsetsTopic(),
+                            producer,
+                            consumer,
+                            admin
+                    );
+                }
+            } else {
+                admin = null;
+            }
+            offsetStore.configure(config);
+
+            CloseableOffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+            OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
+
+            // Note we pass the configState as it performs dynamic transformations under the covers
+            return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter, valueConverter,
+                    headerConverter, transformationChain, producer, admin, topicCreationGroups,
+                    offsetReader, offsetWriter, offsetStore, config, configState, metrics, classLoader, time,
+                    retryWithToleranceOperator, herder.statusBackingStore(), executor);
+        }
+    }
+
+    class ExactlyOnceSourceTaskBuilder extends TaskBuilder {
+        private final Runnable preProducerCheck;
+        private final Runnable postProducerCheck;
+
+        public ExactlyOnceSourceTaskBuilder(ConnectorTaskId id,
+                                            ClusterConfigState configState,
+                                            TaskStatus.Listener statusListener,
+                                            TargetState initialState,
+                                            Runnable preProducerCheck,
+                                            Runnable postProducerCheck) {
+            super(id, configState, statusListener, initialState);
+            this.preProducerCheck = preProducerCheck;
+            this.postProducerCheck = postProducerCheck;
+        }
+
+        @Override
+        public WorkerTask doBuild(Task task,
+                                  ConnectorTaskId id,
+                                  ClusterConfigState configState,
+                                  TaskStatus.Listener statusListener,
+                                  TargetState initialState,
+                                  ConnectorConfig connectorConfig,
+                                  Converter keyConverter,
+                                  Converter valueConverter,
+                                  HeaderConverter headerConverter,
+                                  ClassLoader classLoader,
+                                  ErrorHandlingMetrics errorHandlingMetrics,
+                                  Class<? extends Connector> connectorClass,
+                                  RetryWithToleranceOperator retryWithToleranceOperator) {
+
+            SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins,
+                    connectorConfig.originalsStrings(), config.topicCreationEnable());
+            retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
+            TransformationChain<SourceRecord> transformationChain = new TransformationChain<>(sourceConfig.<SourceRecord>transformations(), retryWithToleranceOperator);
+            log.info("Initializing: {}", transformationChain);
+
+            Map<String, Object> adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config,
+                    sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE);
+            Admin adminClient = Admin.create(adminOverrides);
+            TopicAdmin topicAdmin = new TopicAdmin(adminOverrides.get(BOOTSTRAP_SERVERS_CONFIG), adminClient);
+            Map<String, TopicCreationGroup> topicCreationGroups = null;
+            if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) {
+                topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig);
+            }
+
+            String transactionalId = transactionalId(id);
+            Map<String, Object> producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass,
+                    connectorClientConfigOverridePolicy, kafkaClusterId);
+            ConnectUtils.warnOnOverriddenProperty(
+                    producerProps, ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true",
+                    "for connectors when exactly-once source support is enabled",
+                    false
+            );
+            ConnectUtils.warnOnOverriddenProperty(
+                    producerProps, ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId,
+                    "for connectors when exactly-once source support is enabled",
+                    true
+            );
+            producerProps.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, true);
+            producerProps.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId);

Review comment:
       Ah, so here's one example of how these properties cannot be overridden by the connector config. But that's not quite so obvious for some other calls to `ConnectUtils.warnOnOverriddenProperty`.
   
   Is it not feasible to reset the expected value on the props within the `ConnectUtils#warnOnOverriddenProperty` method?




-- 
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 #10907: KAFKA-10000: Exactly-once support for source connectors

Posted by GitBox <gi...@apache.org>.
C0urante commented on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-865396213


   There was a small misunderstanding of the rebalance logic for distributed workers. It turns out that they already preemptively stop reconfigured tasks before (re)joining the group, so no additional logic is necessary to stop reconfigured source tasks before they are naturally fenced out by the leader. The KIP has been updated with this note and the changes in the PR related to it have been reverted.


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

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



[GitHub] [kafka] C0urante commented on pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-1032801881


   Thanks Mikael, I can take a stab at that. I don't think I can get each PR to be buildable on its own without a lot of work but I can definitely isolate high-level sections into their own PRs, probably following the breakdown in the description for this one.
   
   I'm taking this week off but will hopefully be able to get a rebase done and split things up by Wednesday the 16th.


-- 
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] galeaspablo removed a comment on pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
galeaspablo removed a comment on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-1003836206


   Hey folks, I've kept an eye on this PR, its associated KIP, for a while. Is there anything I can do to help?
   
   Happy to take guidance to split the PR if that would make reviewing easier?
   
   P.S. I appreciate I haven't committed to this repository, but all my Kafka work was internal while I was at AWS. 


-- 
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 #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-956207072


   Removed from the upcoming 3.1 release plan due to lack of review.
   
   @rhauch do you think we can get this merged in time for the 3.2/4.0 release? I'd rather not have to continue cleaning up merge conflicts for this PR and the KIP was accepted several months ago with only one round of review since then. If there just isn't time to look at this please let me know and I can close the 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 pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-1014923523


   Thanks Randall, I've addressed all of your comments either with code changes or an inline response in the appropriate thread.
   
   I've noted this elsewhere but it bears repeating: the `WorkerSourceTask` and `AbstractWorkerSourceTask` classes may be best to save for later given that there are some conflicts that will need to be resolved between this PR and the current `trunk` branch.
   
   The `ExactlyOnceWorkerSourceTask` class should be fine for review, as well as the zombie fencing logic added to the `DistributedHerder`, `KafkaConfigBackingStore` and `Worker` classes.
   
   The new admin client API for fencing producer transactions should also be safe to review (although there are some conflicts in the `KafkaAdminClientTest` that need to be resolved).
   
   > (Also, I responded on a few threads from my previous review. I resolved most of that review's threads where I was happy with your fixes. Hopefully that makes it easier to see the outstanding threads.)
   
   This made it very easy to follow along, thanks 👍 


-- 
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 #10907: KAFKA-10000: Exactly-once support for source connectors

Posted by GitBox <gi...@apache.org>.
C0urante commented on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-864380630


   @gharris1727 you've been tremendously helpful so far in the design process; if you have time would you like to take a look at the implementation 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.

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



[GitHub] [kafka] rhauch commented on a change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
rhauch commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r688675632



##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceTaskContext.java
##########
@@ -38,4 +38,30 @@
      * Get the OffsetStorageReader for this SourceTask.
      */
     OffsetStorageReader offsetStorageReader();
+
+    /**
+     * Get a {@link TransactionContext} that can be used to define producer transaction boundaries
+     * when exactly-once support is enabled for the connector.
+     *
+     * <p>This method was added in Apache Kafka 3.0. Source tasks that use this method but want to
+     * maintain backward compatibility so they can also be deployed to older Connect runtimes
+     * should guard the call to this method with a try-catch block, since calling this method will result in a
+     * {@link NoSuchMethodException} or {@link NoClassDefFoundError} when the source connector is deployed to
+     * Connect runtimes older than Kafka 3.0. For example:
+     * <pre>
+     *     TransactionContext transactionContext;
+     *     try {
+     *         transactionContext = context.transactionContext();
+     *     } catch (NoSuchMethodError | NoClassDefFoundError e) {
+     *         transactionContext = null;
+     *     }
+     * </pre>
+     *
+     * @return the transaction context, or null if the user does not want the connector to define
+     * its own transaction boundaries

Review comment:
       Nit: what does "user" mean here? Should we instead refer to the connector configuration, perhaps something like: "or null if the connector was configured to not specify transaction boundaries"?
   
   (I know this is taken directly from the KIP, but I think we should improve this JavaDoc during the implementation.)

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java
##########
@@ -28,4 +30,31 @@
     protected SourceConnectorContext context() {
         return (SourceConnectorContext) context;
     }
+
+    /**
+     * Signals whether the connector supports exactly-once delivery guarantees with a proposed configuration.
+     * Developers can assume that worker-level exactly-once support is enabled when this method is invoked.
+     * The default implementation will return {@code null}.
+     * @param connectorConfig the configuration that will be used for the connector.
+     * @return {@link ExactlyOnceSupport#SUPPORTED} if the connector can provide exactly-once support,
+     * and {@link ExactlyOnceSupport#UNSUPPORTED} if it cannot. If {@code null}, it is assumed that the
+     * connector cannot.
+     */
+    public ExactlyOnceSupport exactlyOnceSupport(Map<String, String> connectorConfig) {
+        return null;
+    }
+
+    /**
+     * Signals whether the connector can define its own transaction boundaries with the proposed
+     * configuration. Developers must override this method if they wish to add connector-defined
+     * transaction boundary support; if they do not, users will be unable to create instances of
+     * this connector that use connector-defined transaction boundaries. The default implementation
+     * will return {@code UNSUPPORTED}.

Review comment:
       I think this JavaDoc should more clearly specify when an implementation may expect this method is called relative to other methods. We don't want to box ourselves in with respect to the implementation, but we also need a clear contract that Connector developers can rely upon.
   
   For example, maybe add something like: 
   > This method may be called by the runtime before the {@link #start} method when the connector is being run with exactly-once support and when the connector is to determine the transactions boundaries.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java
##########
@@ -28,4 +30,31 @@
     protected SourceConnectorContext context() {
         return (SourceConnectorContext) context;
     }
+
+    /**
+     * Signals whether the connector supports exactly-once delivery guarantees with a proposed configuration.
+     * Developers can assume that worker-level exactly-once support is enabled when this method is invoked.
+     * The default implementation will return {@code null}.
+     * @param connectorConfig the configuration that will be used for the connector.
+     * @return {@link ExactlyOnceSupport#SUPPORTED} if the connector can provide exactly-once support,
+     * and {@link ExactlyOnceSupport#UNSUPPORTED} if it cannot. If {@code null}, it is assumed that the
+     * connector cannot.
+     */
+    public ExactlyOnceSupport exactlyOnceSupport(Map<String, String> connectorConfig) {
+        return null;

Review comment:
       It's difficult to discern in the KIP, but IIUC there is a difference between this method returning `null` versus `ExactlyOnceSupport#UNSUPPORTED`:
   * Returning `null` really seems to equate to `UNKNOWN` and a Connect user is still allowed sort of force the use of EOS by setting the `exactly.once.support=requested` in the connector configuration.
   * Returning `UNSUPPORTED` means that a Connect user is not allowed to force the use of EOS.
   
   Should we just have another enum for `UNKNOWN` and make this more explicit than "null"?
   
   Also, it seems like it would make sense to document that this method should be overridden by Connector developers, but has a default for backward compatibility. And it should state more clearly what should be returned for the various options.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java
##########
@@ -28,4 +30,31 @@
     protected SourceConnectorContext context() {
         return (SourceConnectorContext) context;
     }
+
+    /**
+     * Signals whether the connector supports exactly-once delivery guarantees with a proposed configuration.
+     * Developers can assume that worker-level exactly-once support is enabled when this method is invoked.
+     * The default implementation will return {@code null}.

Review comment:
       I think this JavaDoc should more clearly specify when an implementation may expect this method is called relative to other methods. We don't want to box ourselves in with respect to the implementation, but we also need a clear contract that Connector developers can rely upon.
   
   For example, maybe add something like: 
   > This method may be called by the runtime before the {@link #start} method when the connector is being run with exactly-once support.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java
##########
@@ -20,13 +20,45 @@
 import org.apache.kafka.clients.producer.RecordMetadata;
 
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * SourceTask is a Task that pulls records from another system for storage in Kafka.
  */
 public abstract class SourceTask implements Task {
 
+    /**
+     * <p>
+     * The configuration key that determines how source tasks will define transaction boundaries
+     * when exactly-once support is enabled.
+     * </p>
+     */
+    public static final String TRANSACTION_BOUNDARY_CONFIG = "transaction.boundary";
+
+    public enum TransactionBoundary {
+        POLL,
+        INTERVAL,
+        CONNECTOR;
+
+        public static final TransactionBoundary DEFAULT = POLL;
+
+        public static List<String> options() {

Review comment:
       Do we really need this method when `values()` is already available and more standard? It looks like this is used in only two places: one expects an Array (where `values()` would work better), and the other just uses it to build a documentation (where `Arrays.toString(...)` might work just as well).

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java
##########
@@ -401,10 +429,46 @@ public Integer getRebalanceTimeout() {
         return getInt(DistributedConfig.REBALANCE_TIMEOUT_MS_CONFIG);
     }
 
+    @Override
+    public boolean exactlyOnceSourceEnabled() {
+        return EXACTLY_ONCE_SOURCE_SUPPORT_ENABLED.equalsIgnoreCase(
+                getString(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG)
+        );
+    }
+
+    public boolean transactionalLeaderEnabled() {
+        return Arrays.asList(EXACTLY_ONCE_SOURCE_SUPPORT_ENABLED, EXACTLY_ONCE_SOURCE_SUPPORT_PREPARING)
+                .contains(getString(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG).toLowerCase(Locale.ROOT));
+    }

Review comment:
       Should we have an enum for the `enabled`, `preparing` and `disabled` literals, and should these make use of them? Also, it might be useful to have JavaDoc on these methods, simply to help future developers understand the intent.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/TransactionContext.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.source;
+
+/**
+ * Provided to source tasks to allow them to define their own producer transaction boundaries when
+ * exactly-once support is enabled.
+ */
+public interface TransactionContext {
+
+    /**
+     * Request a transaction commit after the next batch of records from {@link SourceTask#poll()}
+     * is processed.
+     */
+    void commitTransaction();
+
+    /**
+     * Request a transaction commit after a source record is processed. The source record will be the
+     * last record in the committed transaction.
+     * @param record the record to commit the transaction after.

Review comment:
       Is the SourceTask implementation calling this method allowed to pass a null value here? If so, what happens? If not, please add that to the JavaDoc.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java
##########
@@ -20,13 +20,45 @@
 import org.apache.kafka.clients.producer.RecordMetadata;
 
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * SourceTask is a Task that pulls records from another system for storage in Kafka.
  */
 public abstract class SourceTask implements Task {
 
+    /**
+     * <p>

Review comment:
       We don't need this `<p>` tag.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java
##########
@@ -163,13 +181,24 @@ public synchronized boolean beginFlush() {
             }
 
             // And submit the data
-            log.debug("Submitting {} entries to backing store. The offsets are: {}", offsetsSerialized.size(), toFlush);
+            log.debug("Submitting {} entries to backing store. The offsets are: {}", offsetsSerialized.size(), flushed);
         }
 
-        return backingStore.set(offsetsSerialized, (error, result) -> {
-            boolean isCurrent = handleFinishWrite(flushId, error, result);
-            if (isCurrent && callback != null) {
-                callback.onCompletion(error, result);
+        return primaryBackingStore.set(offsetsSerialized, (primaryError, primaryResult) -> {
+            boolean isCurrent = handleFinishWrite(flushId, primaryError, primaryResult);
+            if (isCurrent) {
+                if (callback != null) {
+                    callback.onCompletion(primaryError, primaryResult);
+                }
+                if (secondaryBackingStore != null && primaryError == null) {
+                    secondaryBackingStore.set(offsetsSerialized, (secondaryError, secondaryResult) -> {
+                        if (secondaryError != null) {
+                            log.warn("Failed to write offsets ({}) to secondary backing store", flushed, secondaryError);
+                        } else {
+                            log.debug("Successfully flushed offsets ({}) to secondary backing store", flushed);

Review comment:
       Do these log messages include the Connector context for EOS-enabled source connectors?

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java
##########
@@ -43,6 +44,13 @@
  * storage to achieve exactly once semantics).
  * </p>
  * <p>
+ * In order to support per-connector offsets topics but continue to back up progress to a
+ * cluster-global offsets topic, the writer accepts an optional <i>secondary backing store</i>.
+ * After successful flushes to the primary backing store, the writer will copy the flushed offsets
+ * over to the secondary backing store on a best-effort basis. Failures to write to the secondary
+ * store are logged but otherwise swallowed silently.
+ * </p>
+ * <p>

Review comment:
       Did you consider introducing a new `OffsetBackingStore` implementation that writes to two other `OffsetBackingStore` implementations? That might simplify the logic in this class and better encapsulate the double write behavior.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java
##########
@@ -337,15 +353,29 @@ private void readToLogEnd() {
                 } else {
                     log.trace("Behind end offset {} for {}; last-read offset is {}",
                             endOffset, topicPartition, lastConsumedOffset);
-                    poll(Integer.MAX_VALUE);
+                    if (topicContainsTransactions) {
+                        // The consumer won't return from its poll method if a transaction is aborted, even though
+                        // its position will advance. So, we poll for at most one second, then give ourselves another
+                        // chance to check whether we've reached the end of the topic.
+                        poll(1000);

Review comment:
       And for reference, here is @C0urante's PR for that fix: #11046
   
   I agree, it would be good to avoid this hack if possible.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/TransactionContext.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.source;
+
+/**
+ * Provided to source tasks to allow them to define their own producer transaction boundaries when
+ * exactly-once support is enabled.
+ */
+public interface TransactionContext {
+
+    /**
+     * Request a transaction commit after the next batch of records from {@link SourceTask#poll()}
+     * is processed.
+     */
+    void commitTransaction();
+
+    /**
+     * Request a transaction commit after a source record is processed. The source record will be the
+     * last record in the committed transaction.
+     * @param record the record to commit the transaction after.
+     */
+    void commitTransaction(SourceRecord record);
+
+    /**
+     * Requests a transaction abort the next batch of records from {@link SourceTask#poll()}. All of
+     * the records in that transaction will be discarded and will not appear in a committed transaction.
+     * However, offsets for that transaction will still be committed. If the data should be reprocessed,
+     * the task should not invoke this method and should instead throw an exception.
+     */
+    void abortTransaction();
+
+    /**
+     * Requests a transaction abort after a source record is processed. The source record will be the
+     * last record in the aborted transaction. All of the records in that transaction will be discarded
+     * and will not appear in a committed transaction. However, offsets for that transaction will still
+     * be committed. If the data should be reprocessed, the task should not invoke this method and
+     * should instead throw an exception.
+     * @param record the record to abort the transaction after.

Review comment:
       Is the SourceTask implementation calling this method allowed to pass a null value here? If so, what happens? If not, please add that to the JavaDoc.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java
##########
@@ -20,13 +20,45 @@
 import org.apache.kafka.clients.producer.RecordMetadata;
 
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * SourceTask is a Task that pulls records from another system for storage in Kafka.
  */
 public abstract class SourceTask implements Task {
 
+    /**
+     * <p>
+     * The configuration key that determines how source tasks will define transaction boundaries
+     * when exactly-once support is enabled.
+     * </p>
+     */
+    public static final String TRANSACTION_BOUNDARY_CONFIG = "transaction.boundary";
+
+    public enum TransactionBoundary {

Review comment:
       The `SourceTask` class is part of the public API for Connect, and so we should have JavaDoc on this enum and its literals and methods.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectUtils.java
##########
@@ -72,6 +74,45 @@ static String lookupKafkaClusterId(Admin adminClient) {
         }
     }
 
+    /**
+     * Log a warning when the user attempts to override a property that cannot be overridden.
+     * @param props the configuration properties provided by the user
+     * @param key the name of the property to check on
+     * @param expectedValue the expected value for the property
+     * @param justification the reason the property cannot be overridden.
+     *                      Will follow the phrase "The value... for the... property will be ignored as it cannot be overridden ".
+     *                      For example, one might supply the message "in connectors with the DLQ feature enabled" for this parameter.
+     * @param caseSensitive whether the value should match case-insensitively
+     */
+    public static void warnOnOverriddenProperty(
+            Map<String, ?> props,
+            String key,
+            String expectedValue,
+            String justification,
+            boolean caseSensitive) {
+        overriddenPropertyWarning(props, key, expectedValue, justification, caseSensitive).ifPresent(log::warn);
+    }
+
+    // Visible for testing
+    static Optional<String> overriddenPropertyWarning(
+            Map<String, ?> props,
+            String key,
+            String expectedValue,
+            String justification,
+            boolean caseSensitive) {
+        Predicate<String> matchesExpectedValue = caseSensitive ? expectedValue::equals : expectedValue::equalsIgnoreCase;
+        String value = Optional.ofNullable(props.get(key)).map(Object::toString).orElse(null);
+        if (value != null && !matchesExpectedValue.test(value)) {
+            return Optional.of(String.format(
+                    "The value '%s' for the '%s' property will be ignored as it cannot be overridden %s. "
+                            + "The value '%s' will be used instead.",
+                    value, key, justification, expectedValue
+            ));
+        } else {
+            return Optional.empty();
+        }

Review comment:
       Nit: what do you think about tolerating an empty or null `justification` string, since this methods does not ensure that one is provided?

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java
##########
@@ -192,6 +198,19 @@
     public static final String INTER_WORKER_VERIFICATION_ALGORITHMS_DOC = "A list of permitted algorithms for verifying internal requests";
     public static final List<String> INTER_WORKER_VERIFICATION_ALGORITHMS_DEFAULT = Collections.singletonList(INTER_WORKER_SIGNATURE_ALGORITHM_DEFAULT);
 
+    public static final String EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG = "exactly.once.source.support";
+    public static final String EXACTLY_ONCE_SOURCE_SUPPORT_DOC = "Whether to enable exactly-once support for source connectors in the cluster "
+            + "by writing source records and their offsets in a Kafka transaction, and by proactively fencing out old task generations before bringing up new ones. "
+            + "Note that this must be enabled on every worker in a cluster in order for exactly-once delivery to be guaranteed, "
+            + "and that some source connectors may still not be able to provide exactly-once delivery guarantees even with this support enabled. "
+            + "Permitted values are \"disabled\", \"preparing\", and \"enabled\". In order to safely enable exactly-once support for source connectors, "
+            + "all workers in the cluster must first be updated to use the \"preparing\" value for this property. "
+            + "Once this has been done, a second update of all of the workers in the cluster should be performed to change the value of this property to \"enabled\".";

Review comment:
       We should mention in the public docs that when enabling exactly once support for source connectors, consumers of the topics to which the EOS source connectors write should be configured with `isolation.level=read_committed`? After all, the default for consumers is `isolation.level=read_uncommitted` (see [ConsumerConfig.java](https://github.com/apache/kafka/blob/db1f581da7f3440cfd5be93800b4a9a2d7327a35/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java#L302)).
   
   It's clear to me that we should mention this, but it's not clear where we should do so. The user documentation generated from this doc string might be one spot that users will see routinely, so maybe it's a candidate. Another would be in the Kafka Connect docs about EOS for source connectors.
   
   BTW, can you add to this PR changes to the Kafka docs that describe this feature?
   

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java
##########
@@ -118,35 +123,39 @@ public KafkaBasedLog(String topic,
                          Callback<ConsumerRecord<K, V>> consumedCallback,
                          Time time,
                          Runnable initializer) {
-        this(topic, producerConfigs, consumerConfigs, () -> null, consumedCallback, time, initializer != null ? admin -> initializer.run() : null);
+        this(topic, producerConfigs, consumerConfigs, () -> null, consumedCallback, time, initializer != null ? admin -> initializer.run() : null, false);
     }
 
     /**
      * Create a new KafkaBasedLog object. This does not start reading the log and writing is not permitted until
      * {@link #start()} is invoked.
      *
-     * @param topic              the topic to treat as a log
-     * @param producerConfigs    configuration options to use when creating the internal producer. At a minimum this must
+     * @param topic                     the topic to treat as a log
+     * @param producerConfigs           configuration options to use when creating the internal producer. At a minimum this must
      *                           contain compatible serializer settings for the generic types used on this class. Some
      *                           setting, such as the number of acks, will be overridden to ensure correct behavior of this
      *                           class.
-     * @param consumerConfigs    configuration options to use when creating the internal consumer. At a minimum this must
+     * @param consumerConfigs           configuration options to use when creating the internal consumer. At a minimum this must
      *                           contain compatible serializer settings for the generic types used on this class. Some
      *                           setting, such as the auto offset reset policy, will be overridden to ensure correct
      *                           behavior of this class.
-     * @param topicAdminSupplier supplier function for an admin client, the lifecycle of which is expected to be controlled
+     * @param topicAdminSupplier        supplier function for an admin client, the lifecycle of which is expected to be controlled
      *                           by the calling component; may not be null
-     * @param consumedCallback   callback to invoke for each {@link ConsumerRecord} consumed when tailing the log
-     * @param time               Time interface
-     * @param initializer        the function that should be run when this log is {@link #start() started}; may be null
+     * @param consumedCallback          callback to invoke for each {@link ConsumerRecord} consumed when tailing the log
+     * @param time                      Time interface
+     * @param initializer               the function that should be run when this log is {@link #start() started}; may be null
+     * @param topicContainsTransactions whether the topic being consumed contains (or is expected to contain) transactions;
+     *                                  if this is {@code false} and the topic does contain transactions, reads to the end of the log may block
+     *                                  indefinitely

Review comment:
       There are projects outside of Apache Kafka that do use this class. While this class is not a public API and we technically don't have to avoid breaking compatibility, it's fairly straightforward to maintain API compatibility and so we should do that.

##########
File path: connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java
##########
@@ -20,13 +20,45 @@
 import org.apache.kafka.clients.producer.RecordMetadata;
 
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * SourceTask is a Task that pulls records from another system for storage in Kafka.
  */
 public abstract class SourceTask implements Task {
 
+    /**
+     * <p>
+     * The configuration key that determines how source tasks will define transaction boundaries
+     * when exactly-once support is enabled.
+     * </p>

Review comment:
       The `</p>` has no meaning in JavaDoc, so we should remove these.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java
##########
@@ -210,26 +227,21 @@ public void stop() {
         synchronized (this) {
             stopRequested = true;
         }
-        consumer.wakeup();
-
-        try {
-            thread.join();
-        } catch (InterruptedException e) {
-            throw new ConnectException("Failed to stop KafkaBasedLog. Exiting without cleanly shutting " +
-                    "down it's producer and consumer.", e);
+        if (consumer != null) {
+            consumer.wakeup();
         }
 
-        try {
-            producer.close();
-        } catch (KafkaException e) {
-            log.error("Failed to stop KafkaBasedLog producer", e);
+        if (thread != null) {
+            try {
+                thread.join();
+            } catch (InterruptedException e) {
+                throw new ConnectException("Failed to stop KafkaBasedLog. Exiting without cleanly shutting " +
+                        "down it's producer and consumer.", e);
+            }
         }
 
-        try {
-            consumer.close();
-        } catch (KafkaException e) {
-            log.error("Failed to stop KafkaBasedLog consumer", e);
-        }
+        Utils.closeQuietly(producer, "KafkaBasedLog producer");
+        Utils.closeQuietly(consumer, "KafkaBasedLog consumer");

Review comment:
       This change does alter the level at which these problems are logged. Is that intentional, and if so why?

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java
##########
@@ -156,6 +165,14 @@ public KafkaBasedLog(String topic,
         this.readLogEndOffsetCallbacks = new ArrayDeque<>();
         this.time = time;
         this.initializer = initializer != null ? initializer : admin -> { };
+        this.topicContainsTransactions = topicContainsTransactions;
+
+        // If the consumer is configured with isolation.level = read_committed, then its end offsets method cannot be relied on
+        // as it will not take records from currently-open transactions into account. We want to err on the side of caution in that
+        // case: when users request a read to the end of the log, we will read up to the point where the latest offsets visible to the
+        // consumer are at least as high as the (possibly-part-of-a-transaction) end offsets of the topic.
+        this.requireAdminForOffsets = IsolationLevel.READ_COMMITTED.name().toLowerCase(Locale.ROOT)
+                .equals(consumerConfigs.get(ConsumerConfig.ISOLATION_LEVEL_CONFIG));

Review comment:
       Why not `equalsIgnoreCase(...)` here instead of lowercasing and then calling `equals(...)`?

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java
##########
@@ -114,12 +124,18 @@ public synchronized boolean beginFlush() {
         if (data.isEmpty())
             return false;
 
-        assert !flushing();

Review comment:
       Why remove this assertion?




-- 
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 #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-931698206


   Resolved merge conflicts. Will try to address review comments soon.


-- 
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 change in pull request #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#discussion_r730964164



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java
##########
@@ -192,6 +198,19 @@
     public static final String INTER_WORKER_VERIFICATION_ALGORITHMS_DOC = "A list of permitted algorithms for verifying internal requests";
     public static final List<String> INTER_WORKER_VERIFICATION_ALGORITHMS_DEFAULT = Collections.singletonList(INTER_WORKER_SIGNATURE_ALGORITHM_DEFAULT);
 
+    public static final String EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG = "exactly.once.source.support";
+    public static final String EXACTLY_ONCE_SOURCE_SUPPORT_DOC = "Whether to enable exactly-once support for source connectors in the cluster "
+            + "by writing source records and their offsets in a Kafka transaction, and by proactively fencing out old task generations before bringing up new ones. "
+            + "Note that this must be enabled on every worker in a cluster in order for exactly-once delivery to be guaranteed, "
+            + "and that some source connectors may still not be able to provide exactly-once delivery guarantees even with this support enabled. "
+            + "Permitted values are \"disabled\", \"preparing\", and \"enabled\". In order to safely enable exactly-once support for source connectors, "
+            + "all workers in the cluster must first be updated to use the \"preparing\" value for this property. "
+            + "Once this has been done, a second update of all of the workers in the cluster should be performed to change the value of this property to \"enabled\".";

Review comment:
       > It's clear to me that we should mention this, but it's not clear where we should do so. The user documentation generated from this doc string might be one spot that users will see routinely, so maybe it's a candidate. Another would be in the Kafka Connect docs about EOS for source connectors.
   
   I think both are acceptable. I'll add this to the docstring now and include it in the high-level docs when I write those as well.
   
   > BTW, can you add to this PR changes to the Kafka docs that describe this feature?
   
   Given how massive this PR is already, I'd like to do this in a follow-up, with the understanding that Kafka docs changes are a requisite for including this feature in a release.




-- 
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 #10907: KAFKA-10000: Exactly-once support for source connectors (KIP-618)

Posted by GitBox <gi...@apache.org>.
C0urante commented on pull request #10907:
URL: https://github.com/apache/kafka/pull/10907#issuecomment-947780934


   Rebased onto https://github.com/apache/kafka/pull/11046 and cleaned up the `KafkaBasedLog` changes to remove the ugly hack for handling aborted transactions.


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