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


##########
streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java:
##########
@@ -535,18 +660,20 @@ public void 
shouldRecycleStateFromStandbyTaskPromotedToActiveTaskAndNotRestore(f
 
             assertThat(restoreListener.totalNumRestored(), 
CoreMatchers.equalTo(initialNunRestoredCount));
 
-            // After stopping instance 2 and letting instance 1 take over its 
tasks, we should have closed just two stores
-            // total: the active and standby tasks on instance 2
-            assertThat(CloseCountingInMemoryStore.numStoresClosed(), 
equalTo(initialStoreCloseCount + 2));
+            // After stopping instance 2 and letting instance 1 take over its 
tasks, we should have closed the stores on instance 2.
+            // Under the new group protocol, an extra store close can occur 
during rebalance; account for that here.
+            final int expectedAfterStreams2Close = initialStoreCloseCount + 
(useNewProtocol ? 3 : 2);
+            assertThat(CloseCountingInMemoryStore.numStoresClosed(), 
equalTo(expectedAfterStreams2Close));
         } finally {
             streams1.close();
         }

Review Comment:
   I know this was pre-existing but I have the same comment about closing with 
the overload that accepts a timeout.



##########
streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java:
##########
@@ -344,15 +404,42 @@ public void 
shouldRestoreStateFromSourceTopicForGlobalTable(final boolean stateU
         kafkaStreams.start();
 
         assertTrue(startupLatch.await(30, TimeUnit.SECONDS));
+
+        if (useNewProtocol) {
+            // For new protocol, we need to stop the streams instance before 
altering offsets
+            kafkaStreams.close();
+            setCommittedOffset(inputStream, offsetLimitDelta, useNewProtocol);
+
+            // Restart the streams instance with a new startup latch
+            final CountDownLatch restartLatch = new CountDownLatch(1);
+            kafkaStreams = new KafkaStreams(builder.build(props), props);

Review Comment:
   Same here



##########
streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java:
##########
@@ -293,26 +322,57 @@ public void 
shouldRestoreStateFromSourceTopicForReadOnlyStore(final boolean stat
         kafkaStreams.start();
 
         assertTrue(startupLatch.await(30, TimeUnit.SECONDS));
+
+        if (useNewProtocol) {
+            // For new protocol, we need to stop the streams instance before 
altering offsets
+            kafkaStreams.close();
+            setCommittedOffset(inputStream, offsetLimitDelta, useNewProtocol);
+            
+            // Restart the streams instance with a new startup latch
+            final CountDownLatch restartLatch = new CountDownLatch(1);
+            kafkaStreams = new KafkaStreams(topology, props);
+            kafkaStreams.setStateListener((newState, oldState) -> {
+                if (newState == KafkaStreams.State.RUNNING && oldState == 
KafkaStreams.State.REBALANCING) {
+                    restartLatch.countDown();
+                }
+            });
+            kafkaStreams.setGlobalStateRestoreListener(new 
TrackingStateRestoreListener(restored));
+            kafkaStreams.start();

Review Comment:
   There's a ` 
IntegrationTestUtils.startApplicationAndWaitUntilRunning(KafkaStreams 
instance)` method you can use that returns your KS instance once it's in a 
running state which will simplify your logic here - no need for `CountDownLatch`



##########
streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java:
##########
@@ -814,29 +946,54 @@ 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);
+    private void setCommittedOffset(final String topic, final int limitDelta, 
final boolean useNewProtocol) {
+        if (!useNewProtocol) {
+            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);
+
+            try (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);
+                }
+
+                consumer.commitSync();
+            }
+        } else {
+            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));
+                }
+
+                admin.alterStreamsGroupOffsets(appId, 
offsetsToCommit).all().get();
+            } catch (final Exception e) {
+                throw new RuntimeException("Failed to set committed offsets", 
e);

Review Comment:
   maybe use `fail(...._`



##########
streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java:
##########
@@ -293,26 +322,57 @@ public void 
shouldRestoreStateFromSourceTopicForReadOnlyStore(final boolean stat
         kafkaStreams.start();
 
         assertTrue(startupLatch.await(30, TimeUnit.SECONDS));
+
+        if (useNewProtocol) {
+            // For new protocol, we need to stop the streams instance before 
altering offsets
+            kafkaStreams.close();

Review Comment:
   Probably best to use `close(Duration timeout)` overload, otherwise the test 
can get stuck



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