ableegoldman commented on a change in pull request #9848: URL: https://github.com/apache/kafka/pull/9848#discussion_r556058982
########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/RepartitionTopics.java ########## @@ -0,0 +1,211 @@ +/* + * 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.processor.internals; + +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.streams.errors.MissingSourceTopicException; +import org.apache.kafka.streams.errors.TaskAssignmentException; +import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo; +import org.apache.kafka.streams.processor.internals.assignment.CopartitionedTopicsEnforcer; +import org.slf4j.Logger; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +public class RepartitionTopics { + + private final InternalTopicManager internalTopicManager; + private final InternalTopologyBuilder internalTopologyBuilder; + private final Cluster clusterMetadata; + private final CopartitionedTopicsEnforcer copartitionedTopicsEnforcer; + private final Logger log; + private final Map<TopicPartition, PartitionInfo> topicPartitionInfos = new HashMap<>(); + + public RepartitionTopics(final InternalTopologyBuilder internalTopologyBuilder, + final InternalTopicManager internalTopicManager, + final Cluster clusterMetadata, + final String logPrefix) { + this.internalTopologyBuilder = internalTopologyBuilder; + this.internalTopicManager = internalTopicManager; + this.clusterMetadata = clusterMetadata; + this.copartitionedTopicsEnforcer = new CopartitionedTopicsEnforcer(logPrefix); + final LogContext logContext = new LogContext(logPrefix); + log = logContext.logger(getClass()); + } + + public void setup() { + final Map<Integer, TopicsInfo> topicGroups = internalTopologyBuilder.topicGroups(); + final Map<String, InternalTopicConfig> repartitionTopicMetadata = computeRepartitionTopicConfig(topicGroups, clusterMetadata); + + // ensure the co-partitioning topics within the group have the same number of partitions, + // and enforce the number of partitions for those repartition topics to be the same if they + // are co-partitioned as well. + ensureCopartitioning(internalTopologyBuilder.copartitionGroups(), repartitionTopicMetadata, clusterMetadata); + + // make sure the repartition source topics exist with the right number of partitions, + // create these topics if necessary + internalTopicManager.makeReady(repartitionTopicMetadata); + + // augment the metadata with the newly computed number of partitions for all the + // repartition source topics + for (final Map.Entry<String, InternalTopicConfig> entry : repartitionTopicMetadata.entrySet()) { + final String topic = entry.getKey(); + final int numPartitions = entry.getValue().numberOfPartitions().orElse(-1); + + for (int partition = 0; partition < numPartitions; partition++) { + topicPartitionInfos.put( + new TopicPartition(topic, partition), + new PartitionInfo(topic, partition, null, new Node[0], new Node[0]) + ); + } + } + } + + public Map<TopicPartition, PartitionInfo> topicPartitionsInfo() { + return Collections.unmodifiableMap(topicPartitionInfos); + } + + private Map<String, InternalTopicConfig> computeRepartitionTopicConfig(final Map<Integer, TopicsInfo> topicGroups, + final Cluster clusterMetadata) { + + final Map<String, InternalTopicConfig> repartitionTopicsMetadata = new HashMap<>(); + for (final TopicsInfo topicsInfo : topicGroups.values()) { + checkIfExternalSourceTopicsExist(topicsInfo, clusterMetadata); + repartitionTopicsMetadata.putAll(topicsInfo.repartitionSourceTopics.values().stream() + .collect(Collectors.toMap(InternalTopicConfig::name, topicConfig -> topicConfig))); + } + setRepartitionSourceTopicPartitionCount(repartitionTopicsMetadata, topicGroups, clusterMetadata); + + return repartitionTopicsMetadata; + } + + private void ensureCopartitioning(final Collection<Set<String>> copartitionGroups, + final Map<String, InternalTopicConfig> repartitionTopicMetadata, + final Cluster clusterMetadata) { + for (final Set<String> copartitionGroup : copartitionGroups) { + copartitionedTopicsEnforcer.enforce(copartitionGroup, repartitionTopicMetadata, clusterMetadata); + } + } + + private void checkIfExternalSourceTopicsExist(final TopicsInfo topicsInfo, + final Cluster clusterMetadata) { + final Set<String> externalSourceTopics = new HashSet<>(topicsInfo.sourceTopics); Review comment: prop: rename to `missingExternalSourceTopics` or similar ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/RepartitionTopics.java ########## @@ -0,0 +1,212 @@ +/* + * 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.processor.internals; + +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.streams.errors.MissingSourceTopicException; +import org.apache.kafka.streams.errors.TaskAssignmentException; +import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo; +import org.apache.kafka.streams.processor.internals.assignment.CopartitionedTopicsEnforcer; +import org.slf4j.Logger; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +public class RepartitionTopics { + + private final InternalTopicManager internalTopicManager; + private final InternalTopologyBuilder internalTopologyBuilder; + private final Cluster clusterMetadata; + private final CopartitionedTopicsEnforcer copartitionedTopicsEnforcer; + private final Logger log; + private final Map<TopicPartition, PartitionInfo> topicPartitionInfos = new HashMap<>(); + + public RepartitionTopics(final InternalTopologyBuilder internalTopologyBuilder, + final InternalTopicManager internalTopicManager, + final CopartitionedTopicsEnforcer copartitionedTopicsEnforcer, + final Cluster clusterMetadata, + final String logPrefix) { + this.internalTopologyBuilder = internalTopologyBuilder; + this.internalTopicManager = internalTopicManager; + this.clusterMetadata = clusterMetadata; + this.copartitionedTopicsEnforcer = copartitionedTopicsEnforcer; + final LogContext logContext = new LogContext(logPrefix); + log = logContext.logger(getClass()); + } + + public void setup() { + final Map<Integer, TopicsInfo> topicGroups = internalTopologyBuilder.topicGroups(); + final Map<String, InternalTopicConfig> repartitionTopicMetadata = computeRepartitionTopicConfig(topicGroups, clusterMetadata); + + // ensure the co-partitioning topics within the group have the same number of partitions, + // and enforce the number of partitions for those repartition topics to be the same if they + // are co-partitioned as well. + ensureCopartitioning(internalTopologyBuilder.copartitionGroups(), repartitionTopicMetadata, clusterMetadata); + + // make sure the repartition source topics exist with the right number of partitions, + // create these topics if necessary + internalTopicManager.makeReady(repartitionTopicMetadata); + + // augment the metadata with the newly computed number of partitions for all the + // repartition source topics + for (final Map.Entry<String, InternalTopicConfig> entry : repartitionTopicMetadata.entrySet()) { + final String topic = entry.getKey(); + final int numPartitions = entry.getValue().numberOfPartitions().orElse(-1); + + for (int partition = 0; partition < numPartitions; partition++) { + topicPartitionInfos.put( + new TopicPartition(topic, partition), + new PartitionInfo(topic, partition, null, new Node[0], new Node[0]) + ); + } + } + } + + public Map<TopicPartition, PartitionInfo> topicPartitionsInfo() { + return Collections.unmodifiableMap(topicPartitionInfos); + } + + private Map<String, InternalTopicConfig> computeRepartitionTopicConfig(final Map<Integer, TopicsInfo> topicGroups, + final Cluster clusterMetadata) { + + final Map<String, InternalTopicConfig> repartitionTopicConfigs = new HashMap<>(); + for (final TopicsInfo topicsInfo : topicGroups.values()) { + checkIfExternalSourceTopicsExist(topicsInfo, clusterMetadata); + repartitionTopicConfigs.putAll(topicsInfo.repartitionSourceTopics.values().stream() + .collect(Collectors.toMap(InternalTopicConfig::name, topicConfig -> topicConfig))); + } + setRepartitionSourceTopicPartitionCount(repartitionTopicConfigs, topicGroups, clusterMetadata); + + return repartitionTopicConfigs; + } + + private void ensureCopartitioning(final Collection<Set<String>> copartitionGroups, + final Map<String, InternalTopicConfig> repartitionTopicMetadata, + final Cluster clusterMetadata) { + for (final Set<String> copartitionGroup : copartitionGroups) { + copartitionedTopicsEnforcer.enforce(copartitionGroup, repartitionTopicMetadata, clusterMetadata); + } + } + + private void checkIfExternalSourceTopicsExist(final TopicsInfo topicsInfo, + final Cluster clusterMetadata) { + final Set<String> externalSourceTopics = new HashSet<>(topicsInfo.sourceTopics); + externalSourceTopics.removeAll(topicsInfo.repartitionSourceTopics.keySet()); + externalSourceTopics.removeAll(clusterMetadata.topics()); + if (!externalSourceTopics.isEmpty()) { + log.error("The following source topics are missing/unknown: {}. Please make sure all source topics " + + "have been pre-created before starting the Streams application. ", + externalSourceTopics); + throw new MissingSourceTopicException("Missing source topics."); + } + } + + /** + * Computes the number of partitions and sets it for each repartition topic in repartitionTopicMetadata + */ + private void setRepartitionSourceTopicPartitionCount(final Map<String, InternalTopicConfig> repartitionTopicMetadata, + final Map<Integer, TopicsInfo> topicGroups, + final Cluster clusterMetadata) { + boolean partitionCountNeeded; + do { + partitionCountNeeded = false; + boolean progressMadeThisIteration = false; // avoid infinitely looping without making any progress on unknown repartitions + + for (final TopicsInfo topicsInfo : topicGroups.values()) { + for (final String repartitionSourceTopic : topicsInfo.repartitionSourceTopics.keySet()) { + final Optional<Integer> repartitionSourceTopicPartitionCount = + repartitionTopicMetadata.get(repartitionSourceTopic).numberOfPartitions(); + + if (!repartitionSourceTopicPartitionCount.isPresent()) { + final Integer numPartitions = computePartitionCount( + repartitionTopicMetadata, + topicGroups, + clusterMetadata, + repartitionSourceTopic + ); + + if (numPartitions == null) { + partitionCountNeeded = true; + log.trace("Unable to determine number of partitions for {}, another iteration is needed", + repartitionSourceTopic); + } else { + repartitionTopicMetadata.get(repartitionSourceTopic).setNumberOfPartitions(numPartitions); + progressMadeThisIteration = true; + } + } + } + } + if (!progressMadeThisIteration && partitionCountNeeded) { + throw new TaskAssignmentException("Failed to compute number of partitions for all repartition topics"); + } + } while (partitionCountNeeded); + } + + private Integer computePartitionCount(final Map<String, InternalTopicConfig> repartitionTopicMetadata, + final Map<Integer, TopicsInfo> topicGroups, + final Cluster clusterMetadata, + final String repartitionSourceTopic) { + Integer partitionCount = null; + // try set the number of partitions for this repartition topic if it is not set yet + for (final TopicsInfo upstreamTopicsInfo : topicGroups.values()) { Review comment: It's a bit confusing to call this `upstreamTopicsInfo` since that will only be true of one (or some) of these topic groups. Most of them are not upstream of the current ########## File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionTopicsTest.java ########## @@ -0,0 +1,434 @@ +/* + * 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.processor.internals; + +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.streams.errors.MissingSourceTopicException; +import org.apache.kafka.streams.errors.TaskAssignmentException; +import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo; +import org.apache.kafka.streams.processor.internals.assignment.CopartitionedTopicsEnforcer; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.common.utils.Utils.mkSet; +import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.eq; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.mock; +import static org.easymock.EasyMock.niceMock; +import static org.easymock.EasyMock.replay; +import static org.easymock.EasyMock.verify; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertThrows; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({Cluster.class}) +public class RepartitionTopicsTest { + + private static final String SOURCE_TOPIC_NAME1 = "source1"; + private static final String SOURCE_TOPIC_NAME2 = "source2"; + private static final String SOURCE_TOPIC_NAME3 = "source3"; + private static final String SINK_TOPIC_NAME1 = "sink1"; + private static final String SINK_TOPIC_NAME2 = "sink2"; + private static final String REPARTITION_TOPIC_NAME1 = "repartition1"; + private static final String REPARTITION_TOPIC_NAME2 = "repartition2"; + private static final String REPARTITION_TOPIC_NAME3 = "repartition3"; + private static final String REPARTITION_TOPIC_NAME4 = "repartition4"; + private static final String REPARTITION_WITHOUT_PARTITION_COUNT = "repartitionWithoutPartitionCount"; + private static final String SOME_OTHER_TOPIC = "someOtherTopic"; + private static final Map<String, String> TOPIC_CONFIG1 = Collections.singletonMap("config1", "val1"); + private static final Map<String, String> TOPIC_CONFIG2 = Collections.singletonMap("config2", "val2"); + private static final Map<String, String> TOPIC_CONFIG5 = Collections.singletonMap("config5", "val5"); + private static final RepartitionTopicConfig REPARTITION_TOPIC_CONFIG1 = + new RepartitionTopicConfig(REPARTITION_TOPIC_NAME1, TOPIC_CONFIG1, 4, true); + private static final RepartitionTopicConfig REPARTITION_TOPIC_CONFIG2 = + new RepartitionTopicConfig(REPARTITION_TOPIC_NAME2, TOPIC_CONFIG2, 2, true); + private static final TopicsInfo TOPICS_INFO1 = new TopicsInfo( + mkSet(REPARTITION_TOPIC_NAME1), + mkSet(SOURCE_TOPIC_NAME1, SOURCE_TOPIC_NAME2), + mkMap( + mkEntry(REPARTITION_TOPIC_NAME1, REPARTITION_TOPIC_CONFIG1), + mkEntry(REPARTITION_TOPIC_NAME2, REPARTITION_TOPIC_CONFIG2) + ), + Collections.emptyMap() + ); + private static final TopicsInfo TOPICS_INFO2 = new TopicsInfo( + mkSet(SINK_TOPIC_NAME1), + mkSet(REPARTITION_TOPIC_NAME1), + mkMap(mkEntry(REPARTITION_TOPIC_NAME1, REPARTITION_TOPIC_CONFIG1)), + Collections.emptyMap() + ); + + final InternalTopologyBuilder internalTopologyBuilder = mock(InternalTopologyBuilder.class); + final InternalTopicManager internalTopicManager = mock(InternalTopicManager.class); + final CopartitionedTopicsEnforcer copartitionedTopicsEnforcer = mock(CopartitionedTopicsEnforcer.class); + final Cluster clusterMetadata = niceMock(Cluster.class); + + @Before + public void setUp() { Review comment: Did you mean to leave in an empty method? ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/RepartitionTopics.java ########## @@ -0,0 +1,211 @@ +/* + * 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.processor.internals; + +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.streams.errors.MissingSourceTopicException; +import org.apache.kafka.streams.errors.TaskAssignmentException; +import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo; +import org.apache.kafka.streams.processor.internals.assignment.CopartitionedTopicsEnforcer; +import org.slf4j.Logger; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +public class RepartitionTopics { + + private final InternalTopicManager internalTopicManager; + private final InternalTopologyBuilder internalTopologyBuilder; + private final Cluster clusterMetadata; + private final CopartitionedTopicsEnforcer copartitionedTopicsEnforcer; + private final Logger log; + private final Map<TopicPartition, PartitionInfo> topicPartitionInfos = new HashMap<>(); + + public RepartitionTopics(final InternalTopologyBuilder internalTopologyBuilder, + final InternalTopicManager internalTopicManager, + final Cluster clusterMetadata, + final String logPrefix) { + this.internalTopologyBuilder = internalTopologyBuilder; + this.internalTopicManager = internalTopicManager; + this.clusterMetadata = clusterMetadata; + this.copartitionedTopicsEnforcer = new CopartitionedTopicsEnforcer(logPrefix); + final LogContext logContext = new LogContext(logPrefix); + log = logContext.logger(getClass()); + } + + public void setup() { + final Map<Integer, TopicsInfo> topicGroups = internalTopologyBuilder.topicGroups(); + final Map<String, InternalTopicConfig> repartitionTopicMetadata = computeRepartitionTopicConfig(topicGroups, clusterMetadata); + + // ensure the co-partitioning topics within the group have the same number of partitions, + // and enforce the number of partitions for those repartition topics to be the same if they + // are co-partitioned as well. + ensureCopartitioning(internalTopologyBuilder.copartitionGroups(), repartitionTopicMetadata, clusterMetadata); + + // make sure the repartition source topics exist with the right number of partitions, + // create these topics if necessary + internalTopicManager.makeReady(repartitionTopicMetadata); + + // augment the metadata with the newly computed number of partitions for all the + // repartition source topics + for (final Map.Entry<String, InternalTopicConfig> entry : repartitionTopicMetadata.entrySet()) { + final String topic = entry.getKey(); + final int numPartitions = entry.getValue().numberOfPartitions().orElse(-1); + + for (int partition = 0; partition < numPartitions; partition++) { + topicPartitionInfos.put( + new TopicPartition(topic, partition), + new PartitionInfo(topic, partition, null, new Node[0], new Node[0]) + ); + } + } + } + + public Map<TopicPartition, PartitionInfo> topicPartitionsInfo() { + return Collections.unmodifiableMap(topicPartitionInfos); + } + + private Map<String, InternalTopicConfig> computeRepartitionTopicConfig(final Map<Integer, TopicsInfo> topicGroups, + final Cluster clusterMetadata) { + + final Map<String, InternalTopicConfig> repartitionTopicsMetadata = new HashMap<>(); + for (final TopicsInfo topicsInfo : topicGroups.values()) { + checkIfExternalSourceTopicsExist(topicsInfo, clusterMetadata); + repartitionTopicsMetadata.putAll(topicsInfo.repartitionSourceTopics.values().stream() + .collect(Collectors.toMap(InternalTopicConfig::name, topicConfig -> topicConfig))); + } + setRepartitionSourceTopicPartitionCount(repartitionTopicsMetadata, topicGroups, clusterMetadata); + + return repartitionTopicsMetadata; + } + + private void ensureCopartitioning(final Collection<Set<String>> copartitionGroups, + final Map<String, InternalTopicConfig> repartitionTopicMetadata, + final Cluster clusterMetadata) { + for (final Set<String> copartitionGroup : copartitionGroups) { + copartitionedTopicsEnforcer.enforce(copartitionGroup, repartitionTopicMetadata, clusterMetadata); + } + } + + private void checkIfExternalSourceTopicsExist(final TopicsInfo topicsInfo, + final Cluster clusterMetadata) { + final Set<String> externalSourceTopics = new HashSet<>(topicsInfo.sourceTopics); + externalSourceTopics.removeAll(topicsInfo.repartitionSourceTopics.keySet()); + externalSourceTopics.removeAll(clusterMetadata.topics()); + if (!externalSourceTopics.isEmpty()) { + log.error("The following source topics are missing/unknown: {}. Please make sure all source topics " + + "have been pre-created before starting the Streams application. ", + externalSourceTopics); + throw new MissingSourceTopicException("Missing source topics."); + } + } + + /** + * Computes the number of partitions and sets it for each repartition topic in repartitionTopicMetadata + */ + private void setRepartitionSourceTopicPartitionCount(final Map<String, InternalTopicConfig> repartitionTopicMetadata, + final Map<Integer, TopicsInfo> topicGroups, + final Cluster clusterMetadata) { + boolean partitionCountNeeded; + do { + partitionCountNeeded = false; + boolean progressMadeThisIteration = false; // avoid infinitely looping without making any progress on unknown repartitions + + for (final TopicsInfo topicsInfo : topicGroups.values()) { + for (final String repartitionSourceTopic : topicsInfo.repartitionSourceTopics.keySet()) { + final Optional<Integer> repartitionSourceTopicPartitionCount = + repartitionTopicMetadata.get(repartitionSourceTopic).numberOfPartitions(); + + if (!repartitionSourceTopicPartitionCount.isPresent()) { + final Integer numPartitions = computePartitionCount( + repartitionTopicMetadata, + topicGroups, + clusterMetadata, + repartitionSourceTopic + ); + + if (numPartitions == null) { + partitionCountNeeded = true; + log.trace("Unable to determine number of partitions for {}, another iteration is needed", Review comment: @lct45 sort of; the point of `progressMadeThisIteration` is to break out of the loop entirely if we detect that we're infinitely looping, as was the case in a recent-ish bug. We want to continue looping only as long as we're actually gaining new information/filling in the partition count of some topic group (subtopology). So yes, if `progressMadeThisIteration = false` after the full loop, but we still don't have all the partition counts (`partitionCountNeeded = true`), then oh well. Something went wrong 🙂 We should throw ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org