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


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

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



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

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



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

Review Comment:
   Believe this still needs to be addressed?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

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

Reply via email to