aliehsaeedii commented on code in PR #20347:
URL: https://github.com/apache/kafka/pull/20347#discussion_r2272990965


##########
streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java:
##########
@@ -815,28 +828,28 @@ private void createStateForRestoration(final String 
changelogTopic, final int st
     }
 
     private void setCommittedOffset(final String topic, final int limitDelta) {
-        final Properties consumerConfig = new Properties();
-        consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers());
-        consumerConfig.put(ConsumerConfig.GROUP_ID_CONFIG, appId);
-        consumerConfig.put(ConsumerConfig.CLIENT_ID_CONFIG, "commit-consumer");
-        consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, 
IntegerDeserializer.class);
-        consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, 
IntegerDeserializer.class);
-
-        final Consumer<Integer, Integer> consumer = new 
KafkaConsumer<>(consumerConfig);
-        final List<TopicPartition> partitions = asList(
-                new TopicPartition(topic, 0),
-                new TopicPartition(topic, 1));
-
-        consumer.assign(partitions);
-        consumer.seekToEnd(partitions);
-
-        for (final TopicPartition partition : partitions) {
-            final long position = consumer.position(partition);
-            consumer.seek(partition, position - limitDelta);
-        }
+        try {
+            final List<TopicPartition> partitions = asList(
+                    new TopicPartition(topic, 0),
+                    new TopicPartition(topic, 1));
+
+            final Map<TopicPartition, OffsetSpec> offsetSpecs = 
partitions.stream()
+                    .collect(Collectors.toMap(tp -> tp, tp -> 
OffsetSpec.latest()));
+            
+            final Map<TopicPartition, ListOffsetsResult.ListOffsetsResultInfo> 
endOffsets = 
+                    admin.listOffsets(offsetSpecs).all().get();
+
+            final Map<TopicPartition, OffsetAndMetadata> offsetsToCommit = new 
HashMap<>();
+            for (final TopicPartition partition : partitions) {
+                final long endOffset = endOffsets.get(partition).offset();
+                final long targetOffset = Math.max(0, endOffset - limitDelta);
+                offsetsToCommit.put(partition, new 
OffsetAndMetadata(targetOffset));
+            }
 
-        consumer.commitSync();
-        consumer.close();
+            admin.alterConsumerGroupOffsets(appId, 
offsetsToCommit).all().get();

Review Comment:
   Please call `admin.alterStreamsGroupOffsets`



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to