lucasbru commented on code in PR #18856: URL: https://github.com/apache/kafka/pull/18856#discussion_r1952314367
########## streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/IQv2EndpointToPartitionsIntegrationTest.java: ########## @@ -0,0 +1,229 @@ +/* + * 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.streams.integration; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.StreamsMetadata; +import org.apache.kafka.streams.ThreadMetadata; +import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.test.TestUtils; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.TestInfo; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Locale; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Stream; + +import static org.apache.kafka.streams.utils.TestUtils.safeUniqueTestName; +import static org.apache.kafka.test.TestUtils.waitForCondition; +import static org.junit.jupiter.api.Assertions.assertEquals; + +@Timeout(600) +@Tag("integration") +public class IQv2EndpointToPartitionsIntegrationTest { + private String appId; + private String inputTopicTwoPartitions; + private String outputTopicTwoPartitions; + private Properties streamsApplicationProperties = new Properties(); + private Properties streamsSecondApplicationProperties = new Properties(); + + private static EmbeddedKafkaCluster cluster; + private static final int NUM_BROKERS = 3; + private static final Logger LOG = LoggerFactory.getLogger(IQv2EndpointToPartitionsIntegrationTest.class); + + public void startCluster(final int standbyConfig) throws IOException { + final Properties properties = new Properties(); + properties.put(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,consumer,streams"); + properties.put(GroupCoordinatorConfig.STREAMS_GROUP_NUM_STANDBY_REPLICAS_CONFIG, standbyConfig); + cluster = new EmbeddedKafkaCluster(NUM_BROKERS, properties); + cluster.start(); + } + + public void setUp(final TestInfo testInfo) throws InterruptedException { + appId = safeUniqueTestName("endpointIntegrationTest"); + inputTopicTwoPartitions = appId + "-input-two"; + outputTopicTwoPartitions = appId + "-output-two"; + cluster.createTopic(inputTopicTwoPartitions, 2, 1); + cluster.createTopic(outputTopicTwoPartitions, 2, 1); + } + + public void closeCluster() { + cluster.stop(); + } + + @AfterEach + public void tearDown() throws Exception { + IntegrationTestUtils.purgeLocalStreamsState(streamsApplicationProperties); + if (!streamsSecondApplicationProperties.isEmpty()) { + IntegrationTestUtils.purgeLocalStreamsState(streamsSecondApplicationProperties); + } + } + + @ParameterizedTest(name = "{3}") + @MethodSource("groupProtocolParameters") + public void shouldGetCorrectHostPartitionInformation(final String groupProtocolConfig, + final boolean usingStandbyReplicas, + final int numStandbyReplicas, + final String testName) throws Exception { + try { + startCluster(usingStandbyReplicas ? numStandbyReplicas : 0); + setUp(null); + + final Properties streamOneProperties = new Properties(); + streamOneProperties.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath() + "-ks1"); + streamOneProperties.put(StreamsConfig.CLIENT_ID_CONFIG, appId + "-ks1"); + streamOneProperties.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "localhost:2020"); + streamOneProperties.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocolConfig); + if (usingStandbyReplicas) { + streamOneProperties.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, numStandbyReplicas); + } + streamsApplicationProperties = props(streamOneProperties); + + final Properties streamTwoProperties = new Properties(); + streamTwoProperties.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath() + "-ks2"); + streamTwoProperties.put(StreamsConfig.CLIENT_ID_CONFIG, appId + "-ks2"); + streamTwoProperties.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "localhost:3030"); + streamTwoProperties.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocolConfig); + if (usingStandbyReplicas) { + streamTwoProperties.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, numStandbyReplicas); + } + streamsSecondApplicationProperties = props(streamTwoProperties); + + final Topology topology = complexTopology(); + try (final KafkaStreams streamsOne = new KafkaStreams(topology, streamsApplicationProperties)) { + IntegrationTestUtils.startApplicationAndWaitUntilRunning(streamsOne); + final List<StreamsMetadata> streamsMetadataAllClients = new ArrayList<>(streamsOne.metadataForAllStreamsClients()); + assertEquals(1, streamsMetadataAllClients.size()); + final StreamsMetadata streamsOneMetadataOne = streamsMetadataAllClients.get(0); + final Set<TopicPartition> topicPartitions = streamsOneMetadataOne.topicPartitions(); + assertEquals(2020, streamsOneMetadataOne.hostInfo().port()); + assertEquals(4, topicPartitions.size()); + assertEquals(0, streamsOneMetadataOne.standbyTopicPartitions().size()); + + final long repartitionTopicTaskCount = topicPartitions.stream().filter(tp -> tp.topic().contains("-repartition")).count(); + final long sourceTopicTaskCount = topicPartitions.stream().filter(tp -> tp.topic().contains("-input-two")).count(); + assertEquals(2, repartitionTopicTaskCount); + assertEquals(2, sourceTopicTaskCount); + final int expectedStandbyTopicPartitionCount = usingStandbyReplicas ? 1 : 0; + + try (final KafkaStreams streamsTwo = new KafkaStreams(topology, streamsSecondApplicationProperties)) { + streamsTwo.start(); + waitForCondition(() -> KafkaStreams.State.RUNNING == streamsTwo.state() && KafkaStreams.State.RUNNING == streamsOne.state(), + IntegrationTestUtils.DEFAULT_TIMEOUT, + () -> "Kafka Streams one or two never transitioned to a RUNNING state."); + + waitForCondition(() -> { + final ThreadMetadata threadMetadata = streamsOne.metadataForLocalThreads().iterator().next(); + return threadMetadata.activeTasks().size() == 2 && threadMetadata.standbyTasks().size() == expectedStandbyTopicPartitionCount; + }, TestUtils.DEFAULT_MAX_WAIT_MS, + "KafkaStreams one never released active tasks and received standby task"); + + waitForCondition(() -> { + final ThreadMetadata threadMetadata = streamsTwo.metadataForLocalThreads().iterator().next(); + return threadMetadata.activeTasks().size() == 2 && threadMetadata.standbyTasks().size() == expectedStandbyTopicPartitionCount; + }, TestUtils.DEFAULT_MAX_WAIT_MS, + "KafkaStreams two never received active tasks and standby"); + + waitForCondition(() -> { + final List<StreamsMetadata> metadata = new ArrayList<>(streamsTwo.metadataForAllStreamsClients()); + return metadata.size() == 2 && + metadata.get(0).standbyTopicPartitions().size() == expectedStandbyTopicPartitionCount && + metadata.get(1).standbyTopicPartitions().size() == expectedStandbyTopicPartitionCount; + }, TestUtils.DEFAULT_MAX_WAIT_MS, + "Kafka Streams clients 1 and 2 never got metadata about standby tasks"); + + final List<StreamsMetadata> allClientMetadataUpdated = new ArrayList<>(streamsTwo.metadataForAllStreamsClients()); + final StreamsMetadata streamsOneMetadataOneUpdated = allClientMetadataUpdated.get(0); + final Set<TopicPartition> streamsOneTopicPartitions = streamsOneMetadataOneUpdated.topicPartitions(); + assertEquals(2020, streamsOneMetadataOneUpdated.hostInfo().port()); + assertEquals(2, streamsOneTopicPartitions.size()); + final long streamsOneRepartitionTopicTaskCount = streamsOneTopicPartitions.stream().filter(tp -> tp.topic().contains("-repartition")).count(); + final long streamsOneSourceTopicTaskCount = streamsOneTopicPartitions.stream().filter(tp -> tp.topic().contains("-input-two")).count(); + assertEquals(1, streamsOneRepartitionTopicTaskCount); + assertEquals(1, streamsOneSourceTopicTaskCount); + + final StreamsMetadata streamsOneMetadataTwo = allClientMetadataUpdated.get(1); + final Set<TopicPartition> streamsTwoTopicPartitions = streamsOneMetadataTwo.topicPartitions(); + assertEquals(3030, streamsOneMetadataTwo.hostInfo().port()); + assertEquals(2, streamsTwoTopicPartitions.size()); + assertEquals(expectedStandbyTopicPartitionCount, streamsOneMetadataTwo.standbyTopicPartitions().size()); Review Comment: Should we check the contents of the standbyTopicPartitions? The javadoc states that they are changelog partitions, but it looks like we are not using changelog topics (up in the implementation). I am utterly confused now which topic partitions are included in which set now. Can we also check the result of `standbyStateStoreNames`? It seems it is derived from `standbyTopicPartitions`, it should run the same logic in both protocols. -- 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