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


##########
connect/runtime/src/test/java/org/apache/kafka/connect/integration/OffsetsApiIntegrationTest.java:
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.integration;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.connect.runtime.ConnectorConfig;
+import org.apache.kafka.connect.runtime.SourceConnectorConfig;
+import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffset;
+import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets;
+import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException;
+import org.apache.kafka.connect.storage.StringConverter;
+import org.apache.kafka.connect.util.SinkUtils;
+import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster;
+import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG;
+import static 
org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG;
+import static 
org.apache.kafka.connect.runtime.SinkConnectorConfig.TOPICS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX;
+import static 
org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG;
+import static 
org.apache.kafka.connect.runtime.WorkerConfig.KEY_CONVERTER_CLASS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Integration tests for Kafka Connect's connector offset management REST APIs
+ */
+@Category(IntegrationTest.class)
+public class OffsetsApiIntegrationTest {
+
+    private static final String CONNECTOR_NAME = "test-connector";
+    private static final String TOPIC = "test-topic";
+    private static final Integer NUM_TASKS = 2;
+    private static final long OFFSET_COMMIT_INTERVAL_MS = 
TimeUnit.SECONDS.toMillis(1);
+    private static final int NUM_WORKERS = 3;
+    private EmbeddedConnectCluster connect;
+
+    @Before
+    public void setup() {
+        // setup Connect worker properties
+        Map<String, String> workerProps = new HashMap<>();
+        workerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, 
String.valueOf(OFFSET_COMMIT_INTERVAL_MS));
+
+        // build a Connect cluster backed by Kafka and Zk
+        connect = new EmbeddedConnectCluster.Builder()
+                .name("connect-cluster")
+                .numWorkers(NUM_WORKERS)
+                .workerProps(workerProps)
+                .build();
+        connect.start();
+    }
+
+    @After
+    public void tearDown() {
+        connect.stop();
+    }
+
+    @Test
+    public void testGetNonExistentConnectorOffsets() {
+        ConnectRestException e = assertThrows(ConnectRestException.class,
+                () -> connect.connectorOffsets("non-existent-connector"));
+        assertEquals(404, e.errorCode());
+    }
+
+    @Test
+    public void testGetSinkConnectorOffsets() throws Exception {
+        getAndVerifySinkConnectorOffsets(baseSinkConnectorConfigs(), 
connect.kafka());
+    }
+
+    @Test
+    public void testGetSinkConnectorOffsetsOverriddenConsumerGroupId() throws 
Exception {
+        Map<String, String> connectorConfigs = baseSinkConnectorConfigs();
+        
connectorConfigs.put(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX 
+ CommonClientConfigs.GROUP_ID_CONFIG,
+                "overridden-group-id");
+        getAndVerifySinkConnectorOffsets(connectorConfigs, connect.kafka());
+    }
+
+    @Test
+    public void testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted() 
throws Exception {
+        EmbeddedKafkaCluster kafkaCluster = new EmbeddedKafkaCluster(1, new 
Properties());
+        kafkaCluster.start();
+
+        Map<String, String> connectorConfigs = baseSinkConnectorConfigs();
+        
connectorConfigs.put(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX 
+ CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG,
+                kafkaCluster.bootstrapServers());
+        
connectorConfigs.put(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX + 
CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG,
+                kafkaCluster.bootstrapServers());
+
+        getAndVerifySinkConnectorOffsets(connectorConfigs, kafkaCluster);
+
+        kafkaCluster.stop();
+    }
+
+    private void getAndVerifySinkConnectorOffsets(Map<String, String> 
connectorConfigs, EmbeddedKafkaCluster kafkaCluster) throws Exception {
+        kafkaCluster.createTopic(TOPIC, 5);
+
+        // Produce 10 messages to each partition
+        for (int partition = 0; partition < 5; partition++) {
+            for (int message = 0; message < 10; message++) {
+                kafkaCluster.produce(TOPIC, partition, "key", "value");
+            }
+        }
+
+        // Create sink connector
+        connect.configureConnector(CONNECTOR_NAME, connectorConfigs);
+        
connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(CONNECTOR_NAME,
 NUM_TASKS,
+                "Connector tasks did not start in time.");
+
+        TestUtils.waitForCondition(() -> {
+            ConnectorOffsets offsets = 
connect.connectorOffsets(CONNECTOR_NAME);
+            // There should be 5 topic partitions
+            if (offsets.offsets().size() != 5) {
+                return false;
+            }
+            for (ConnectorOffset offset: offsets.offsets()) {
+                assertEquals("test-topic", 
offset.partition().get(SinkUtils.KAFKA_TOPIC_KEY));
+                if ((Integer) offset.offset().get(SinkUtils.KAFKA_OFFSET_KEY) 
!= 10) {
+                    return false;
+                }
+            }
+            return true;
+        }, "Sink connector consumer group offsets should catch up to the topic 
end offsets");
+
+        // Produce 10 more messages to each partition
+        for (int partition = 0; partition < 5; partition++) {
+            for (int message = 0; message < 10; message++) {
+                kafkaCluster.produce(TOPIC, partition, "key", "value");
+            }
+        }
+
+        TestUtils.waitForCondition(() -> {
+            ConnectorOffsets offsets = 
connect.connectorOffsets(CONNECTOR_NAME);
+            // There should be 5 topic partitions
+            if (offsets.offsets().size() != 5) {
+                return false;
+            }
+            for (ConnectorOffset offset: offsets.offsets()) {
+                assertEquals("test-topic", 
offset.partition().get(SinkUtils.KAFKA_TOPIC_KEY));
+                if ((Integer) offset.offset().get(SinkUtils.KAFKA_OFFSET_KEY) 
!= 20) {
+                    return false;
+                }
+            }
+            return true;
+        }, "Sink connector consumer group offsets should catch up to the topic 
end offsets");
+    }
+
+    @Test
+    public void testGetSourceConnectorOffsets() throws Exception {
+        getAndVerifySourceConnectorOffsets(baseSourceConnectorConfigs());
+    }
+
+    @Test
+    public void testGetSourceConnectorOffsetsCustomOffsetsTopic() throws 
Exception {
+        Map<String, String> connectorConfigs = baseSourceConnectorConfigs();
+        connectorConfigs.put(SourceConnectorConfig.OFFSETS_TOPIC_CONFIG, 
"custom-offsets-topic");
+        getAndVerifySourceConnectorOffsets(connectorConfigs);
+    }
+
+    @Test
+    public void testGetSourceConnectorOffsetsDifferentKafkaClusterTargeted() 
throws Exception {
+        EmbeddedKafkaCluster kafkaCluster = new EmbeddedKafkaCluster(1, new 
Properties());
+        kafkaCluster.start();

Review Comment:
   (Same thought RE try-with-resources for the embedded Kafka cluster)



##########
connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java:
##########
@@ -109,6 +121,69 @@ public void testThreadName() {
                 
.newThread(EMPTY_RUNNABLE).getName().startsWith(FileOffsetBackingStore.class.getSimpleName()));
     }
 
+    @Test
+    public void testConnectorPartitions() throws Exception {
+        @SuppressWarnings("unchecked")
+        Callback<Void> setCallback = mock(Callback.class);
+
+        JsonConverter jsonConverter = new JsonConverter();
+        
jsonConverter.configure(Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG,
 "false"), true);
+
+        Map<ByteBuffer, ByteBuffer> serializedPartitionOffsets = new 
HashMap<>();
+        serializedPartitionOffsets.put(
+                ByteBuffer.wrap(jsonConverter.fromConnectData("", null,
+                        Arrays.asList("connector1", 
Collections.singletonMap("partitionKey", "partitionValue1")))),
+                ByteBuffer.wrap(jsonConverter.fromConnectData("", null,
+                        Collections.singletonMap("offsetKey", "offsetValue")))
+        );
+        serializedPartitionOffsets.put(

Review Comment:
   Doesn't this overwrite the mapping we just added at line 133?



##########
connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java:
##########
@@ -109,6 +121,69 @@ public void testThreadName() {
                 
.newThread(EMPTY_RUNNABLE).getName().startsWith(FileOffsetBackingStore.class.getSimpleName()));
     }
 
+    @Test
+    public void testConnectorPartitions() throws Exception {
+        @SuppressWarnings("unchecked")
+        Callback<Void> setCallback = mock(Callback.class);
+
+        JsonConverter jsonConverter = new JsonConverter();
+        
jsonConverter.configure(Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG,
 "false"), true);
+
+        Map<ByteBuffer, ByteBuffer> serializedPartitionOffsets = new 
HashMap<>();
+        serializedPartitionOffsets.put(
+                ByteBuffer.wrap(jsonConverter.fromConnectData("", null,
+                        Arrays.asList("connector1", 
Collections.singletonMap("partitionKey", "partitionValue1")))),
+                ByteBuffer.wrap(jsonConverter.fromConnectData("", null,
+                        Collections.singletonMap("offsetKey", "offsetValue")))
+        );
+        serializedPartitionOffsets.put(
+                ByteBuffer.wrap(jsonConverter.fromConnectData("", null,
+                        Arrays.asList("connector1", 
Collections.singletonMap("partitionKey", "partitionValue1")))),
+                ByteBuffer.wrap(jsonConverter.fromConnectData("", null,
+                        Collections.singletonMap("offsetKey", "offsetValue2")))
+        );
+        serializedPartitionOffsets.put(
+                ByteBuffer.wrap(jsonConverter.fromConnectData("", null,
+                        Arrays.asList("connector1", 
Collections.singletonMap("partitionKey", "partitionValue2")))),
+                ByteBuffer.wrap(jsonConverter.fromConnectData("", null,
+                        Collections.singletonMap("offsetKey", "offsetValue")))
+        );
+        serializedPartitionOffsets.put(
+                ByteBuffer.wrap(jsonConverter.fromConnectData("", null,
+                        Arrays.asList("connector2", 
Collections.singletonMap("partitionKey", "partitionValue")))),
+                ByteBuffer.wrap(jsonConverter.fromConnectData("", null,
+                        Collections.singletonMap("offsetKey", "offsetValue")))
+        );
+
+        store.set(serializedPartitionOffsets, setCallback).get();
+        store.stop();
+
+        // Restore into a new store to ensure correct reload from scratch
+        FileOffsetBackingStore restore = new 
FileOffsetBackingStore(jsonConverter);
+        restore.configure(config);
+        restore.start();
+
+        Set<Map<String, Object>> connectorPartitions1 = 
restore.connectorPartitions("connector1");
+        assertEquals(2, connectorPartitions1.size());
+
+        Set<Map<String, Object>> expectedConnectorPartition1 = new HashSet<>();
+        
expectedConnectorPartition1.add(Collections.singletonMap("partitionKey", 
"partitionValue1"));
+        
expectedConnectorPartition1.add(Collections.singletonMap("partitionKey", 
"partitionValue2"));
+
+        
assertTrue(connectorPartitions1.containsAll(expectedConnectorPartition1));
+        
assertTrue(expectedConnectorPartition1.containsAll(connectorPartitions1));
+
+        Set<Map<String, Object>> connectorPartitions2 = 
restore.connectorPartitions("connector2");
+        assertEquals(1, connectorPartitions2.size());
+
+        Set<Map<String, Object>> expectedConnectorPartition2 = 
Collections.singleton(Collections.singletonMap("partitionKey", 
"partitionValue"));
+
+        
assertTrue(connectorPartitions2.containsAll(expectedConnectorPartition2));
+        
assertTrue(expectedConnectorPartition2.containsAll(connectorPartitions2));

Review Comment:
   Worth adding a case after here where we emit a null value for one of the 
source partitions already present in the store and verify that it gets wiped?



##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java:
##########
@@ -1722,6 +1727,128 @@ public void testZombieFencing() {
         verifyGenericIsolation();
     }
 
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testGetSinkConnectorOffsets() throws Exception {
+        mockKafkaClusterId();
+
+        String connectorClass = SampleSinkConnector.class.getName();
+        connectorProps.put(CONNECTOR_CLASS_CONFIG, connectorClass);
+
+        worker = new Worker(WORKER_ID, new MockTime(), plugins, config, 
offsetBackingStore, executorService,
+                allConnectorClientConfigOverridePolicy);
+        worker.start();
+
+        Map<TopicPartition, OffsetAndMetadata> consumerGroupOffsets =
+                Collections.singletonMap(new TopicPartition("test-topic", 0), 
new OffsetAndMetadata(10));
+        Map<String, Map<TopicPartition, OffsetAndMetadata>> 
consumerGroupToOffsetsMap =
+                
Collections.singletonMap(SinkUtils.consumerGroupId(CONNECTOR_ID), 
consumerGroupOffsets);
+
+        Admin admin = mock(Admin.class);
+        ListConsumerGroupOffsetsResult result = 
mock(ListConsumerGroupOffsetsResult.class);
+        when(admin.listConsumerGroupOffsets(anyString(), 
any(ListConsumerGroupOffsetsOptions.class))).thenReturn(result);
+        KafkaFuture<Map<String, Map<TopicPartition, OffsetAndMetadata>>> 
adminFuture = mock(KafkaFuture.class);
+        when(result.all()).thenReturn(adminFuture);
+        when(adminFuture.whenComplete(any())).thenAnswer(invocation -> {
+            ((KafkaFuture.BiConsumer<Map<String, Map<TopicPartition, 
OffsetAndMetadata>>, Throwable>) invocation.getArgument(0))
+                    .accept(consumerGroupToOffsetsMap, null);
+            return null;
+        });
+
+        FutureCallback<ConnectorOffsets> cb = new FutureCallback<>();
+        worker.sinkConnectorOffsets(CONNECTOR_ID, sinkConnector, 
connectorProps, cb, config -> admin);
+        ConnectorOffsets offsets = cb.get(1000, TimeUnit.MILLISECONDS);
+
+        assertEquals(1, offsets.offsets().size());
+        assertEquals(10L, 
offsets.offsets().get(0).offset().get(SinkUtils.KAFKA_OFFSET_KEY));
+        assertEquals(0, 
offsets.offsets().get(0).partition().get(SinkUtils.KAFKA_PARTITION_KEY));
+        assertEquals("test-topic", 
offsets.offsets().get(0).partition().get(SinkUtils.KAFKA_TOPIC_KEY));
+
+        
verify(admin).listConsumerGroupOffsets(eq(SinkUtils.consumerGroupId(CONNECTOR_ID)),
 any(ListConsumerGroupOffsetsOptions.class));
+        verify(admin).close();
+        verifyKafkaClusterId();
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testGetSinkConnectorOffsetsOverriddenConsumerGroupId() throws 
Exception {

Review Comment:
   Can we also add test cases for synchronous and asynchronous failures? Both 
can probably take place with the `Admin::listConsumerGroupOffsets` invocation 
(one when it's called, and one in the future that it returns).



##########
connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSourceConnector.java:
##########
@@ -191,12 +194,16 @@ public void start(Map<String, String> props) {
         @Override
         public List<SourceRecord> poll() {
             if (!stopped) {
+                // Don't return any more records since we've already produced 
the configured maximum number.
+                if (seqno >= maxMessages) {
+                    return null;
+                }
                 if (throttler.shouldThrottle(seqno - startingSeqno, 
System.currentTimeMillis())) {
                     throttler.throttle();
                 }
                 taskHandle.record(batchSize);

Review Comment:
   Doesn't this line need to be updated since we may no longer be returning 
`batchSize` records from `poll`?
   
   It might help to pull `Math.min(maxMessages - seqno, batchSize)` out into a 
local variable so that we don't accidentally use `batchSize` in this method 
where we shouldn't.



##########
connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java:
##########
@@ -109,6 +121,69 @@ public void testThreadName() {
                 
.newThread(EMPTY_RUNNABLE).getName().startsWith(FileOffsetBackingStore.class.getSimpleName()));
     }
 
+    @Test
+    public void testConnectorPartitions() throws Exception {
+        @SuppressWarnings("unchecked")
+        Callback<Void> setCallback = mock(Callback.class);
+
+        JsonConverter jsonConverter = new JsonConverter();
+        
jsonConverter.configure(Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG,
 "false"), true);
+
+        Map<ByteBuffer, ByteBuffer> serializedPartitionOffsets = new 
HashMap<>();
+        serializedPartitionOffsets.put(
+                ByteBuffer.wrap(jsonConverter.fromConnectData("", null,
+                        Arrays.asList("connector1", 
Collections.singletonMap("partitionKey", "partitionValue1")))),
+                ByteBuffer.wrap(jsonConverter.fromConnectData("", null,
+                        Collections.singletonMap("offsetKey", "offsetValue")))

Review Comment:
   This would be a bit easier to read with a couple small utility methods:
   
   ```java
   private static ByteBuffer serializeKey(Converter converter, String 
connectorName, Map<String, Object> sourcePartition) {
       List<Object> nameAndPartition = Arrays.asList(connectorName, 
sourcePartition);
       return serialize(converter, nameAndPartition);
   }
   
   private static ByteBuffer serialize(Converter converter, Object value) {
       byte[] serialized = converter.fromConnectData("", null, value);
       return ByteBuffer.wrap(serialized);
   }
   ```
   
   We could then turn lines like this into:
   ```suggestion
                   serializeKey(jsonConverter, "connector1", 
Collections.singletonMap("partitionKey", "partitionValue1")),
                   serialize(jsonConverter, 
Collections.singletonMap("offsetKey", "offsetValue"))
   ```



##########
connect/runtime/src/test/java/org/apache/kafka/connect/integration/OffsetsApiIntegrationTest.java:
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.integration;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.connect.runtime.ConnectorConfig;
+import org.apache.kafka.connect.runtime.SourceConnectorConfig;
+import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffset;
+import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets;
+import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException;
+import org.apache.kafka.connect.storage.StringConverter;
+import org.apache.kafka.connect.util.SinkUtils;
+import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster;
+import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG;
+import static 
org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG;
+import static 
org.apache.kafka.connect.runtime.SinkConnectorConfig.TOPICS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX;
+import static 
org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG;
+import static 
org.apache.kafka.connect.runtime.WorkerConfig.KEY_CONVERTER_CLASS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Integration tests for Kafka Connect's connector offset management REST APIs
+ */
+@Category(IntegrationTest.class)
+public class OffsetsApiIntegrationTest {
+
+    private static final String CONNECTOR_NAME = "test-connector";
+    private static final String TOPIC = "test-topic";
+    private static final Integer NUM_TASKS = 2;
+    private static final long OFFSET_COMMIT_INTERVAL_MS = 
TimeUnit.SECONDS.toMillis(1);
+    private static final int NUM_WORKERS = 3;
+    private EmbeddedConnectCluster connect;
+
+    @Before
+    public void setup() {
+        // setup Connect worker properties
+        Map<String, String> workerProps = new HashMap<>();
+        workerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, 
String.valueOf(OFFSET_COMMIT_INTERVAL_MS));
+
+        // build a Connect cluster backed by Kafka and Zk
+        connect = new EmbeddedConnectCluster.Builder()
+                .name("connect-cluster")
+                .numWorkers(NUM_WORKERS)
+                .workerProps(workerProps)
+                .build();
+        connect.start();
+    }
+
+    @After
+    public void tearDown() {
+        connect.stop();
+    }
+
+    @Test
+    public void testGetNonExistentConnectorOffsets() {
+        ConnectRestException e = assertThrows(ConnectRestException.class,
+                () -> connect.connectorOffsets("non-existent-connector"));
+        assertEquals(404, e.errorCode());
+    }
+
+    @Test
+    public void testGetSinkConnectorOffsets() throws Exception {
+        getAndVerifySinkConnectorOffsets(baseSinkConnectorConfigs(), 
connect.kafka());
+    }
+
+    @Test
+    public void testGetSinkConnectorOffsetsOverriddenConsumerGroupId() throws 
Exception {
+        Map<String, String> connectorConfigs = baseSinkConnectorConfigs();
+        
connectorConfigs.put(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX 
+ CommonClientConfigs.GROUP_ID_CONFIG,

Review Comment:
   This feels a little brittle... we don't really confirm in this test that the 
connector is using the group ID we give it here, we just configure it in a way 
that we believe will do that and then run the same test that we do for when the 
group ID is not overridden.
   
   Do you think it'd be worth it to add a small sanity check on the targeted 
Kafka cluster to ensure that the consumer group exists on that cluster, and 
possibly that the would-be default doesn't?



##########
connect/runtime/src/test/java/org/apache/kafka/connect/integration/OffsetsApiIntegrationTest.java:
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.integration;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.connect.runtime.ConnectorConfig;
+import org.apache.kafka.connect.runtime.SourceConnectorConfig;
+import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffset;
+import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets;
+import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException;
+import org.apache.kafka.connect.storage.StringConverter;
+import org.apache.kafka.connect.util.SinkUtils;
+import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster;
+import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG;
+import static 
org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG;
+import static 
org.apache.kafka.connect.runtime.SinkConnectorConfig.TOPICS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX;
+import static 
org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG;
+import static 
org.apache.kafka.connect.runtime.WorkerConfig.KEY_CONVERTER_CLASS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Integration tests for Kafka Connect's connector offset management REST APIs
+ */
+@Category(IntegrationTest.class)
+public class OffsetsApiIntegrationTest {
+
+    private static final String CONNECTOR_NAME = "test-connector";
+    private static final String TOPIC = "test-topic";
+    private static final Integer NUM_TASKS = 2;
+    private static final long OFFSET_COMMIT_INTERVAL_MS = 
TimeUnit.SECONDS.toMillis(1);
+    private static final int NUM_WORKERS = 3;
+    private EmbeddedConnectCluster connect;
+
+    @Before
+    public void setup() {
+        // setup Connect worker properties
+        Map<String, String> workerProps = new HashMap<>();
+        workerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, 
String.valueOf(OFFSET_COMMIT_INTERVAL_MS));
+
+        // build a Connect cluster backed by Kafka and Zk
+        connect = new EmbeddedConnectCluster.Builder()
+                .name("connect-cluster")
+                .numWorkers(NUM_WORKERS)
+                .workerProps(workerProps)
+                .build();
+        connect.start();
+    }
+
+    @After
+    public void tearDown() {
+        connect.stop();
+    }
+
+    @Test
+    public void testGetNonExistentConnectorOffsets() {
+        ConnectRestException e = assertThrows(ConnectRestException.class,
+                () -> connect.connectorOffsets("non-existent-connector"));
+        assertEquals(404, e.errorCode());
+    }
+
+    @Test
+    public void testGetSinkConnectorOffsets() throws Exception {
+        getAndVerifySinkConnectorOffsets(baseSinkConnectorConfigs(), 
connect.kafka());
+    }
+
+    @Test
+    public void testGetSinkConnectorOffsetsOverriddenConsumerGroupId() throws 
Exception {
+        Map<String, String> connectorConfigs = baseSinkConnectorConfigs();
+        
connectorConfigs.put(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX 
+ CommonClientConfigs.GROUP_ID_CONFIG,
+                "overridden-group-id");
+        getAndVerifySinkConnectorOffsets(connectorConfigs, connect.kafka());
+    }
+
+    @Test
+    public void testGetSinkConnectorOffsetsDifferentKafkaClusterTargeted() 
throws Exception {
+        EmbeddedKafkaCluster kafkaCluster = new EmbeddedKafkaCluster(1, new 
Properties());
+        kafkaCluster.start();
+
+        Map<String, String> connectorConfigs = baseSinkConnectorConfigs();
+        
connectorConfigs.put(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX 
+ CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG,
+                kafkaCluster.bootstrapServers());
+        
connectorConfigs.put(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX + 
CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG,
+                kafkaCluster.bootstrapServers());
+
+        getAndVerifySinkConnectorOffsets(connectorConfigs, kafkaCluster);
+
+        kafkaCluster.stop();

Review Comment:
   Nit: we can use try-with-resources to prevent failing tests from leaking 
embedded Kafka clusters:
   ```suggestion
           try (AutoCloseable cleanUpCluster = kafkaCluster::stop;) {
               kafkaCluster.start();
   
               Map<String, String> connectorConfigs = 
baseSinkConnectorConfigs();
               
connectorConfigs.put(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX 
+ CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG,
                       kafkaCluster.bootstrapServers());
               
connectorConfigs.put(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX + 
CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG,
                       kafkaCluster.bootstrapServers());
   
               getAndVerifySinkConnectorOffsets(connectorConfigs, kafkaCluster);
           }
   ```



##########
connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java:
##########
@@ -57,9 +66,12 @@ public class FileOffsetBackingStoreTest {
 
     @Before
     public void setup() throws IOException {
-        store = new FileOffsetBackingStore();
+        converter = mock(Converter.class);
+        when(converter.toConnectData(anyString(), 
any(byte[].class))).thenReturn(new SchemaAndValue(null,

Review Comment:
   A comment here could be helpful:
   ```suggestion
           // This is only needed for storing deserialized connector 
partitions, which we don't test in most of the cases here
           when(converter.toConnectData(anyString(), 
any(byte[].class))).thenReturn(new SchemaAndValue(null,
   ```



##########
connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java:
##########
@@ -109,6 +121,69 @@ public void testThreadName() {
                 
.newThread(EMPTY_RUNNABLE).getName().startsWith(FileOffsetBackingStore.class.getSimpleName()));
     }
 
+    @Test
+    public void testConnectorPartitions() throws Exception {
+        @SuppressWarnings("unchecked")
+        Callback<Void> setCallback = mock(Callback.class);
+
+        JsonConverter jsonConverter = new JsonConverter();

Review Comment:
   We can add an accompanying comment here:
   
   ```suggestion
           // This test actually requires the offset store to track 
deserialized source partitions, so we can't use the mock converter we created 
in setup()
           JsonConverter jsonConverter = new JsonConverter();
   ```



##########
connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java:
##########
@@ -460,6 +471,74 @@ public void testClientIds() {
         assertEquals(expectedClientId, 
capturedConsumerProps.getValue().get(CLIENT_ID_CONFIG));
     }
 
+    @Test
+    public void testConnectorPartitions() throws Exception {
+        JsonConverter jsonConverter = new JsonConverter();
+        
jsonConverter.configure(Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG,
 "false"), true);
+        store = spy(new KafkaOffsetBackingStore(() -> {
+            fail("Should not attempt to instantiate admin in these tests");
+            return null;
+        }, () -> CLIENT_ID_BASE, jsonConverter));
+
+        
doReturn(storeLog).when(store).createKafkaBasedLog(capturedTopic.capture(), 
capturedProducerProps.capture(),
+                capturedConsumerProps.capture(), 
capturedConsumedCallback.capture(),
+                capturedNewTopic.capture(), capturedAdminSupplier.capture());
+
+        store.configure(mockConfig(props));
+        store.start();
+
+        verify(storeLog).start();
+
+        doAnswer(invocation -> {
+            capturedConsumedCallback.getValue().onCompletion(null,
+                    new ConsumerRecord<>(TOPIC, 0, 0, 0L, 
TimestampType.CREATE_TIME, 0, 0,
+                            jsonConverter.fromConnectData("", null, 
Arrays.asList("connector1",
+                                    Collections.singletonMap("partitionKey", 
"partitionValue1"))), TP0_VALUE.array(),
+                            new RecordHeaders(), Optional.empty()));
+            capturedConsumedCallback.getValue().onCompletion(null,
+                    new ConsumerRecord<>(TOPIC, 1, 0, 0L, 
TimestampType.CREATE_TIME, 0, 0,
+                            jsonConverter.fromConnectData("", null, 
Arrays.asList("connector1",
+                                    Collections.singletonMap("partitionKey", 
"partitionValue1"))), TP1_VALUE.array(),
+                            new RecordHeaders(), Optional.empty()));
+            capturedConsumedCallback.getValue().onCompletion(null,
+                    new ConsumerRecord<>(TOPIC, 2, 0, 0L, 
TimestampType.CREATE_TIME, 0, 0,
+                            jsonConverter.fromConnectData("", null, 
Arrays.asList("connector1",
+                                    Collections.singletonMap("partitionKey", 
"partitionValue2"))), TP2_VALUE.array(),
+                            new RecordHeaders(), Optional.empty()));
+            capturedConsumedCallback.getValue().onCompletion(null,
+                    new ConsumerRecord<>(TOPIC, 3, 0, 0L, 
TimestampType.CREATE_TIME, 0, 0,
+                            jsonConverter.fromConnectData("", null, 
Arrays.asList("connector2",
+                                    Collections.singletonMap("partitionKey", 
"partitionValue"))), TP1_VALUE.array(),
+                            new RecordHeaders(), Optional.empty()));
+            storeLogCallbackArgumentCaptor.getValue().onCompletion(null, null);
+            return null;
+        }).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture());
+
+        // Trigger a read to the end of the log
+        store.get(Collections.emptyList()).get(10000, TimeUnit.MILLISECONDS);
+
+        Set<Map<String, Object>> connectorPartitions1 = 
store.connectorPartitions("connector1");
+        assertEquals(2, connectorPartitions1.size());
+
+        Set<Map<String, Object>> expectedConnectorPartition1 = new HashSet<>();
+        
expectedConnectorPartition1.add(Collections.singletonMap("partitionKey", 
"partitionValue1"));
+        
expectedConnectorPartition1.add(Collections.singletonMap("partitionKey", 
"partitionValue2"));
+
+        
assertTrue(connectorPartitions1.containsAll(expectedConnectorPartition1));
+        
assertTrue(expectedConnectorPartition1.containsAll(connectorPartitions1));

Review Comment:
   Any reason to do this instead of `assertEquals(expectedConnectorPartition1, 
connectorPartitions1);`?



##########
connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java:
##########
@@ -460,6 +471,74 @@ public void testClientIds() {
         assertEquals(expectedClientId, 
capturedConsumerProps.getValue().get(CLIENT_ID_CONFIG));
     }
 
+    @Test
+    public void testConnectorPartitions() throws Exception {
+        JsonConverter jsonConverter = new JsonConverter();
+        
jsonConverter.configure(Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG,
 "false"), true);
+        store = spy(new KafkaOffsetBackingStore(() -> {
+            fail("Should not attempt to instantiate admin in these tests");
+            return null;
+        }, () -> CLIENT_ID_BASE, jsonConverter));
+
+        
doReturn(storeLog).when(store).createKafkaBasedLog(capturedTopic.capture(), 
capturedProducerProps.capture(),
+                capturedConsumerProps.capture(), 
capturedConsumedCallback.capture(),
+                capturedNewTopic.capture(), capturedAdminSupplier.capture());
+
+        store.configure(mockConfig(props));
+        store.start();
+
+        verify(storeLog).start();
+
+        doAnswer(invocation -> {
+            capturedConsumedCallback.getValue().onCompletion(null,
+                    new ConsumerRecord<>(TOPIC, 0, 0, 0L, 
TimestampType.CREATE_TIME, 0, 0,
+                            jsonConverter.fromConnectData("", null, 
Arrays.asList("connector1",
+                                    Collections.singletonMap("partitionKey", 
"partitionValue1"))), TP0_VALUE.array(),
+                            new RecordHeaders(), Optional.empty()));
+            capturedConsumedCallback.getValue().onCompletion(null,
+                    new ConsumerRecord<>(TOPIC, 1, 0, 0L, 
TimestampType.CREATE_TIME, 0, 0,

Review Comment:
   Shouldn't we be incrementing the offset of the consumer record instead of 
the partition in this line and the few that follow it?



##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java:
##########
@@ -96,13 +104,10 @@
 import java.util.function.Function;
 
 import static 
org.apache.kafka.clients.admin.AdminClientConfig.RETRY_BACKOFF_MS_CONFIG;
-import static 
org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX;

Review Comment:
   I prefer the static import style (we don't need longer lines when using 
these already-verbose constant names).



##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java:
##########
@@ -96,13 +104,10 @@
 import java.util.function.Function;
 
 import static 
org.apache.kafka.clients.admin.AdminClientConfig.RETRY_BACKOFF_MS_CONFIG;
-import static 
org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX;
-import static 
org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
-
 import static 
org.apache.kafka.clients.consumer.ConsumerConfig.ISOLATION_LEVEL_CONFIG;
 import static 
org.apache.kafka.clients.producer.ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG;
 import static 
org.apache.kafka.clients.producer.ProducerConfig.TRANSACTIONAL_ID_CONFIG;
-import static 
org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX;

Review Comment:
   (Same thought RE static imports)



##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java:
##########
@@ -1722,6 +1727,128 @@ public void testZombieFencing() {
         verifyGenericIsolation();
     }
 
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testGetSinkConnectorOffsets() throws Exception {
+        mockKafkaClusterId();
+
+        String connectorClass = SampleSinkConnector.class.getName();
+        connectorProps.put(CONNECTOR_CLASS_CONFIG, connectorClass);
+
+        worker = new Worker(WORKER_ID, new MockTime(), plugins, config, 
offsetBackingStore, executorService,
+                allConnectorClientConfigOverridePolicy);
+        worker.start();
+
+        Map<TopicPartition, OffsetAndMetadata> consumerGroupOffsets =
+                Collections.singletonMap(new TopicPartition("test-topic", 0), 
new OffsetAndMetadata(10));
+        Map<String, Map<TopicPartition, OffsetAndMetadata>> 
consumerGroupToOffsetsMap =
+                
Collections.singletonMap(SinkUtils.consumerGroupId(CONNECTOR_ID), 
consumerGroupOffsets);
+
+        Admin admin = mock(Admin.class);
+        ListConsumerGroupOffsetsResult result = 
mock(ListConsumerGroupOffsetsResult.class);
+        when(admin.listConsumerGroupOffsets(anyString(), 
any(ListConsumerGroupOffsetsOptions.class))).thenReturn(result);
+        KafkaFuture<Map<String, Map<TopicPartition, OffsetAndMetadata>>> 
adminFuture = mock(KafkaFuture.class);
+        when(result.all()).thenReturn(adminFuture);
+        when(adminFuture.whenComplete(any())).thenAnswer(invocation -> {
+            ((KafkaFuture.BiConsumer<Map<String, Map<TopicPartition, 
OffsetAndMetadata>>, Throwable>) invocation.getArgument(0))
+                    .accept(consumerGroupToOffsetsMap, null);
+            return null;
+        });
+
+        FutureCallback<ConnectorOffsets> cb = new FutureCallback<>();
+        worker.sinkConnectorOffsets(CONNECTOR_ID, sinkConnector, 
connectorProps, cb, config -> admin);
+        ConnectorOffsets offsets = cb.get(1000, TimeUnit.MILLISECONDS);
+
+        assertEquals(1, offsets.offsets().size());
+        assertEquals(10L, 
offsets.offsets().get(0).offset().get(SinkUtils.KAFKA_OFFSET_KEY));
+        assertEquals(0, 
offsets.offsets().get(0).partition().get(SinkUtils.KAFKA_PARTITION_KEY));
+        assertEquals("test-topic", 
offsets.offsets().get(0).partition().get(SinkUtils.KAFKA_TOPIC_KEY));
+
+        
verify(admin).listConsumerGroupOffsets(eq(SinkUtils.consumerGroupId(CONNECTOR_ID)),
 any(ListConsumerGroupOffsetsOptions.class));
+        verify(admin).close();
+        verifyKafkaClusterId();
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testGetSinkConnectorOffsetsOverriddenConsumerGroupId() throws 
Exception {
+        mockKafkaClusterId();
+
+        String connectorClass = SampleSinkConnector.class.getName();
+        connectorProps.put(CONNECTOR_CLASS_CONFIG, connectorClass);
+        
connectorProps.put(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX + 
CommonClientConfigs.GROUP_ID_CONFIG, "overridden-group-id");
+
+        worker = new Worker(WORKER_ID, new MockTime(), plugins, config, 
offsetBackingStore, executorService,
+                allConnectorClientConfigOverridePolicy);
+        worker.start();
+
+        Map<String, Map<TopicPartition, OffsetAndMetadata>> 
consumerGroupToOffsetsMap = new HashMap<>();
+        consumerGroupToOffsetsMap.put("overridden-group-id", 
Collections.singletonMap(new TopicPartition("test-topic", 0), new 
OffsetAndMetadata(10)));
+        consumerGroupToOffsetsMap.put(SinkUtils.consumerGroupId(CONNECTOR_ID),
+                Collections.singletonMap(new TopicPartition("test-topic-2", 
1), new OffsetAndMetadata(0)));
+
+        Admin admin = mock(Admin.class);
+        ListConsumerGroupOffsetsResult result = 
mock(ListConsumerGroupOffsetsResult.class);
+        when(admin.listConsumerGroupOffsets(anyString(), 
any(ListConsumerGroupOffsetsOptions.class))).thenReturn(result);
+        KafkaFuture<Map<String, Map<TopicPartition, OffsetAndMetadata>>> 
adminFuture = mock(KafkaFuture.class);
+        when(result.all()).thenReturn(adminFuture);
+        when(adminFuture.whenComplete(any())).thenAnswer(invocation -> {
+            ((KafkaFuture.BiConsumer<Map<String, Map<TopicPartition, 
OffsetAndMetadata>>, Throwable>) invocation.getArgument(0))
+                    .accept(consumerGroupToOffsetsMap, null);
+            return null;
+        });
+
+        FutureCallback<ConnectorOffsets> cb = new FutureCallback<>();
+        worker.sinkConnectorOffsets(CONNECTOR_ID, sinkConnector, 
connectorProps, cb, config -> admin);
+        ConnectorOffsets offsets = cb.get(1000, TimeUnit.MILLISECONDS);
+
+        assertEquals(1, offsets.offsets().size());
+        assertEquals(10L, 
offsets.offsets().get(0).offset().get(SinkUtils.KAFKA_OFFSET_KEY));
+        assertEquals(0, 
offsets.offsets().get(0).partition().get(SinkUtils.KAFKA_PARTITION_KEY));
+        assertEquals("test-topic", 
offsets.offsets().get(0).partition().get(SinkUtils.KAFKA_TOPIC_KEY));
+
+        verify(admin).listConsumerGroupOffsets(eq("overridden-group-id"), 
any(ListConsumerGroupOffsetsOptions.class));
+        verify(admin).close();
+        verifyKafkaClusterId();
+    }
+
+    @Test
+    public void testGetSourceConnectorOffsets() throws Exception {

Review Comment:
   Can we also add a test case where we fail to read source connector offsets 
(can probably happen during a call to `offsetStore::connectorPartitions`) and 
verify that the store was stopped?
   
   It'd also be nice to verify that the callback gets invoked, but it looks 
like that'd be tricky without reshuffling the main (non-testing) logic, so feel 
free to leave that out if it's easier.



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