cadonna commented on code in PR #18268: URL: https://github.com/apache/kafka/pull/18268#discussion_r1893795554
########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredInternalTopic.java: ########## @@ -0,0 +1,156 @@ +/* + * 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.coordinator.group.streams.topics; + +import org.apache.kafka.common.internals.Topic; +import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; + +import java.util.Collections; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * ConfiguredInternalTopic captures the properties required for configuring the internal topics we create for change-logs and repartitioning + * etc. + * <p> + * It is derived from the topology sent by the client, and the current state of the topics inside the broker. If the topics on the broker + * changes, the internal topic may need to be reconfigured. + */ +public class ConfiguredInternalTopic { + + private final String name; + private final Map<String, String> topicConfigs; + private final Optional<Short> replicationFactor; + private final boolean enforceNumberOfPartitions; + private Optional<Integer> numberOfPartitions; + + public ConfiguredInternalTopic(final String name) { + this(name, Collections.emptyMap(), Optional.empty(), Optional.empty()); + } + + public ConfiguredInternalTopic(final String name, + final Map<String, String> topicConfigs) { + this(name, topicConfigs, Optional.empty(), Optional.empty()); + } + + public ConfiguredInternalTopic(final String name, + final Map<String, String> topicConfigs, + final Optional<Integer> numberOfPartitions, + final Optional<Short> replicationFactor) { + this.name = Objects.requireNonNull(name, "name can't be null"); + Topic.validate(name); + numberOfPartitions.ifPresent(ConfiguredInternalTopic::validateNumberOfPartitions); + this.topicConfigs = Objects.requireNonNull(topicConfigs, "topicConfigs can't be null"); + this.numberOfPartitions = numberOfPartitions; + this.replicationFactor = replicationFactor; + this.enforceNumberOfPartitions = numberOfPartitions.isPresent(); + } + + private static void validateNumberOfPartitions(final int numberOfPartitions) { + if (numberOfPartitions < 1) { + throw new IllegalArgumentException("Number of partitions must be at least 1."); + } + } + + public Map<String, String> topicConfigs() { + return topicConfigs; + } + + public boolean hasEnforcedNumberOfPartitions() { + return enforceNumberOfPartitions; + } + + public String name() { + return name; + } + + public Optional<Integer> numberOfPartitions() { + return numberOfPartitions; + } + + public Optional<Short> replicationFactor() { + return replicationFactor; + } + + public ConfiguredInternalTopic setNumberOfPartitions(final int numberOfPartitions) { Review Comment: As discussed offline, could you please change the visibility to package-private? ########## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredInternalTopicTest.java: ########## @@ -0,0 +1,125 @@ +/* + * 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.coordinator.group.streams.topics; + +import org.apache.kafka.common.errors.InvalidTopicException; +import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; + +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class ConfiguredInternalTopicTest { + + @Test + public void testConstructorAndGetters() { + Map<String, String> topicConfigs = new HashMap<>(); + topicConfigs.put("retention.ms", "1000"); + + ConfiguredInternalTopic config = new ConfiguredInternalTopic("test-topic", topicConfigs, Optional.of(3), Optional.of((short) 2)); + + assertEquals("test-topic", config.name()); + assertEquals(topicConfigs, config.topicConfigs()); + assertEquals(Optional.of(3), config.numberOfPartitions()); + assertEquals(Optional.of((short) 2), config.replicationFactor()); + } + + @Test + public void testConstructorWithNullName() { + assertThrows(NullPointerException.class, () -> new ConfiguredInternalTopic(null, Collections.emptyMap())); + } + + @Test + public void testConstructorWithInvalidName() { + assertThrows(InvalidTopicException.class, () -> new ConfiguredInternalTopic("invalid topic name", Collections.emptyMap())); + } + + @Test + public void testConstructorWithNullTopicConfigs() { + assertThrows(NullPointerException.class, () -> new ConfiguredInternalTopic("test-topic", null)); + } + + @Test + public void testSetNumberOfPartitions() { + ConfiguredInternalTopic config = new ConfiguredInternalTopic("test-topic", Collections.emptyMap()); + config.setNumberOfPartitions(3); + assertEquals(Optional.of(3), config.numberOfPartitions()); + } + + @Test + public void testSetNumberOfPartitionsInvalid() { + ConfiguredInternalTopic config = new ConfiguredInternalTopic("test-topic", Collections.emptyMap()); + assertThrows(IllegalArgumentException.class, () -> config.setNumberOfPartitions(0)); + } + + @Test + public void testSetNumberOfPartitionsUnsupportedOperation() { + ConfiguredInternalTopic config = new ConfiguredInternalTopic("test-topic", Collections.emptyMap(), Optional.of(3), + Optional.empty()); + assertThrows(UnsupportedOperationException.class, () -> config.setNumberOfPartitions(4)); + } + + @Test + public void testEqualsAndHashCode() { + Map<String, String> topicConfigs = new HashMap<>(); + topicConfigs.put("retention.ms", "1000"); + + ConfiguredInternalTopic config1 = new ConfiguredInternalTopic("test-topic", topicConfigs, Optional.of(3), Optional.of((short) 2)); + ConfiguredInternalTopic config2 = new ConfiguredInternalTopic("test-topic", topicConfigs, Optional.of(3), Optional.of((short) 2)); + + assertEquals(config1, config2); + assertEquals(config1.hashCode(), config2.hashCode()); + } + + @Test + public void testToString() { Review Comment: nit: Is this needed? The `toString()` is mainly for debug purposes, right? If you want to leave it, that is fine with me. Just asking. ########## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredInternalTopicTest.java: ########## @@ -0,0 +1,125 @@ +/* + * 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.coordinator.group.streams.topics; + +import org.apache.kafka.common.errors.InvalidTopicException; +import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; + +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class ConfiguredInternalTopicTest { + + @Test + public void testConstructorAndGetters() { + Map<String, String> topicConfigs = new HashMap<>(); + topicConfigs.put("retention.ms", "1000"); + + ConfiguredInternalTopic config = new ConfiguredInternalTopic("test-topic", topicConfigs, Optional.of(3), Optional.of((short) 2)); + + assertEquals("test-topic", config.name()); + assertEquals(topicConfigs, config.topicConfigs()); + assertEquals(Optional.of(3), config.numberOfPartitions()); + assertEquals(Optional.of((short) 2), config.replicationFactor()); + } + + @Test + public void testConstructorWithNullName() { + assertThrows(NullPointerException.class, () -> new ConfiguredInternalTopic(null, Collections.emptyMap())); + } + + @Test + public void testConstructorWithInvalidName() { + assertThrows(InvalidTopicException.class, () -> new ConfiguredInternalTopic("invalid topic name", Collections.emptyMap())); + } + + @Test + public void testConstructorWithNullTopicConfigs() { + assertThrows(NullPointerException.class, () -> new ConfiguredInternalTopic("test-topic", null)); + } + + @Test + public void testSetNumberOfPartitions() { + ConfiguredInternalTopic config = new ConfiguredInternalTopic("test-topic", Collections.emptyMap()); + config.setNumberOfPartitions(3); + assertEquals(Optional.of(3), config.numberOfPartitions()); + } + + @Test + public void testSetNumberOfPartitionsInvalid() { + ConfiguredInternalTopic config = new ConfiguredInternalTopic("test-topic", Collections.emptyMap()); + assertThrows(IllegalArgumentException.class, () -> config.setNumberOfPartitions(0)); + } + + @Test + public void testSetNumberOfPartitionsUnsupportedOperation() { + ConfiguredInternalTopic config = new ConfiguredInternalTopic("test-topic", Collections.emptyMap(), Optional.of(3), + Optional.empty()); + assertThrows(UnsupportedOperationException.class, () -> config.setNumberOfPartitions(4)); + } + + @Test + public void testEqualsAndHashCode() { + Map<String, String> topicConfigs = new HashMap<>(); + topicConfigs.put("retention.ms", "1000"); + + ConfiguredInternalTopic config1 = new ConfiguredInternalTopic("test-topic", topicConfigs, Optional.of(3), Optional.of((short) 2)); + ConfiguredInternalTopic config2 = new ConfiguredInternalTopic("test-topic", topicConfigs, Optional.of(3), Optional.of((short) 2)); + + assertEquals(config1, config2); + assertEquals(config1.hashCode(), config2.hashCode()); Review Comment: Could you please also add a negative test for each field that is triggers an inequality? ########## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredSubtopologyTest.java: ########## @@ -0,0 +1,172 @@ +/* + * 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.coordinator.group.streams.topics; + +import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class ConfiguredSubtopologyTest { + + @Test + public void testConstructorAndGetters() { + Set<String> repartitionSinkTopics = Set.of("repartitionSinkTopic1", "repartitionSinkTopic2"); + Set<String> sourceTopics = Set.of("sourceTopic1", "sourceTopic2"); + Map<String, ConfiguredInternalTopic> repartitionSourceTopics = new HashMap<>(); + Map<String, ConfiguredInternalTopic> stateChangelogTopics = new HashMap<>(); + + ConfiguredSubtopology configuredSubtopology = new ConfiguredSubtopology(repartitionSinkTopics, sourceTopics, + repartitionSourceTopics, stateChangelogTopics); + + assertEquals(repartitionSinkTopics, configuredSubtopology.repartitionSinkTopics()); + assertEquals(sourceTopics, configuredSubtopology.sourceTopics()); + assertEquals(repartitionSourceTopics, configuredSubtopology.repartitionSourceTopics()); + assertEquals(stateChangelogTopics, configuredSubtopology.stateChangelogTopics()); + } + + @Test + public void testSetters() { + ConfiguredSubtopology configuredSubtopology = new ConfiguredSubtopology(); + + Set<String> repartitionSinkTopics = Set.of("repartitionSinkTopic1", "repartitionSinkTopic2"); + ConfiguredSubtopology ret1 = configuredSubtopology.setRepartitionSinkTopics(repartitionSinkTopics); + assertEquals(repartitionSinkTopics, configuredSubtopology.repartitionSinkTopics()); + assertEquals(ret1, configuredSubtopology); + + Set<String> sourceTopics = Set.of("sourceTopic1", "sourceTopic2"); + ConfiguredSubtopology ret2 = configuredSubtopology.setSourceTopics(sourceTopics); + assertEquals(sourceTopics, configuredSubtopology.sourceTopics()); + assertEquals(ret2, configuredSubtopology); + + Map<String, ConfiguredInternalTopic> repartitionSourceTopics = new HashMap<>(); + ConfiguredSubtopology ret3 = configuredSubtopology.setRepartitionSourceTopics(repartitionSourceTopics); + assertEquals(repartitionSourceTopics, configuredSubtopology.repartitionSourceTopics()); + assertEquals(ret3, configuredSubtopology); + + Map<String, ConfiguredInternalTopic> stateChangelogTopics = new HashMap<>(); + ConfiguredSubtopology ret4 = configuredSubtopology.setStateChangelogTopics(stateChangelogTopics); + assertEquals(stateChangelogTopics, configuredSubtopology.stateChangelogTopics()); + assertEquals(ret4, configuredSubtopology); + } + + @Test + public void testNonSourceChangelogTopics() { + Map<String, ConfiguredInternalTopic> stateChangelogTopics = new HashMap<>(); + stateChangelogTopics.put("changelogTopic1", new ConfiguredInternalTopic("changelogTopic1")); + stateChangelogTopics.put("sourceTopic1", new ConfiguredInternalTopic("sourceTopic1")); + + ConfiguredSubtopology configuredSubtopology = new ConfiguredSubtopology( + Collections.emptySet(), + Collections.singleton("sourceTopic1"), + Collections.emptyMap(), + stateChangelogTopics + ); + + Set<ConfiguredInternalTopic> nonSourceChangelogTopics = configuredSubtopology.nonSourceChangelogTopics(); + assertEquals(1, nonSourceChangelogTopics.size()); + assertTrue(nonSourceChangelogTopics.contains(new ConfiguredInternalTopic("changelogTopic1"))); Review Comment: Could you please add a negative example? That is, a source changelog topic that is not returned by `nonSourceChangelogTopics()`. ########## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredSubtopologyTest.java: ########## @@ -0,0 +1,172 @@ +/* + * 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.coordinator.group.streams.topics; + +import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class ConfiguredSubtopologyTest { + + @Test + public void testConstructorAndGetters() { + Set<String> repartitionSinkTopics = Set.of("repartitionSinkTopic1", "repartitionSinkTopic2"); + Set<String> sourceTopics = Set.of("sourceTopic1", "sourceTopic2"); + Map<String, ConfiguredInternalTopic> repartitionSourceTopics = new HashMap<>(); + Map<String, ConfiguredInternalTopic> stateChangelogTopics = new HashMap<>(); + + ConfiguredSubtopology configuredSubtopology = new ConfiguredSubtopology(repartitionSinkTopics, sourceTopics, + repartitionSourceTopics, stateChangelogTopics); + + assertEquals(repartitionSinkTopics, configuredSubtopology.repartitionSinkTopics()); + assertEquals(sourceTopics, configuredSubtopology.sourceTopics()); + assertEquals(repartitionSourceTopics, configuredSubtopology.repartitionSourceTopics()); + assertEquals(stateChangelogTopics, configuredSubtopology.stateChangelogTopics()); + } + + @Test + public void testSetters() { + ConfiguredSubtopology configuredSubtopology = new ConfiguredSubtopology(); + + Set<String> repartitionSinkTopics = Set.of("repartitionSinkTopic1", "repartitionSinkTopic2"); + ConfiguredSubtopology ret1 = configuredSubtopology.setRepartitionSinkTopics(repartitionSinkTopics); + assertEquals(repartitionSinkTopics, configuredSubtopology.repartitionSinkTopics()); + assertEquals(ret1, configuredSubtopology); + + Set<String> sourceTopics = Set.of("sourceTopic1", "sourceTopic2"); + ConfiguredSubtopology ret2 = configuredSubtopology.setSourceTopics(sourceTopics); + assertEquals(sourceTopics, configuredSubtopology.sourceTopics()); + assertEquals(ret2, configuredSubtopology); + + Map<String, ConfiguredInternalTopic> repartitionSourceTopics = new HashMap<>(); + ConfiguredSubtopology ret3 = configuredSubtopology.setRepartitionSourceTopics(repartitionSourceTopics); + assertEquals(repartitionSourceTopics, configuredSubtopology.repartitionSourceTopics()); + assertEquals(ret3, configuredSubtopology); + + Map<String, ConfiguredInternalTopic> stateChangelogTopics = new HashMap<>(); + ConfiguredSubtopology ret4 = configuredSubtopology.setStateChangelogTopics(stateChangelogTopics); + assertEquals(stateChangelogTopics, configuredSubtopology.stateChangelogTopics()); + assertEquals(ret4, configuredSubtopology); + } + + @Test + public void testNonSourceChangelogTopics() { + Map<String, ConfiguredInternalTopic> stateChangelogTopics = new HashMap<>(); + stateChangelogTopics.put("changelogTopic1", new ConfiguredInternalTopic("changelogTopic1")); + stateChangelogTopics.put("sourceTopic1", new ConfiguredInternalTopic("sourceTopic1")); + + ConfiguredSubtopology configuredSubtopology = new ConfiguredSubtopology( + Collections.emptySet(), + Collections.singleton("sourceTopic1"), + Collections.emptyMap(), + stateChangelogTopics + ); + + Set<ConfiguredInternalTopic> nonSourceChangelogTopics = configuredSubtopology.nonSourceChangelogTopics(); + assertEquals(1, nonSourceChangelogTopics.size()); + assertTrue(nonSourceChangelogTopics.contains(new ConfiguredInternalTopic("changelogTopic1"))); + } + + @Test + public void testEquals() { + Set<String> repartitionSinkTopics = new HashSet<>(Arrays.asList("repartitionSinkTopic1", "repartitionSinkTopic2")); + Set<String> sourceTopics = new HashSet<>(Arrays.asList("sourceTopic1", "sourceTopic2")); + Map<String, ConfiguredInternalTopic> repartitionSourceTopics = new HashMap<>(); + Map<String, ConfiguredInternalTopic> stateChangelogTopics = new HashMap<>(); + + ConfiguredSubtopology configuredSubtopology1 = new ConfiguredSubtopology(repartitionSinkTopics, sourceTopics, + repartitionSourceTopics, stateChangelogTopics); + ConfiguredSubtopology configuredSubtopology2 = new ConfiguredSubtopology(repartitionSinkTopics, sourceTopics, + repartitionSourceTopics, stateChangelogTopics); + + assertEquals(configuredSubtopology1, configuredSubtopology2); + } + + @Test + public void testHashCode() { Review Comment: Consider merging this test with `testEquals()`. Might save some code line. Do not forget the neg. examples. ########## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredInternalTopicTest.java: ########## @@ -0,0 +1,125 @@ +/* + * 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.coordinator.group.streams.topics; + +import org.apache.kafka.common.errors.InvalidTopicException; +import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; + +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class ConfiguredInternalTopicTest { + + @Test + public void testConstructorAndGetters() { + Map<String, String> topicConfigs = new HashMap<>(); + topicConfigs.put("retention.ms", "1000"); + + ConfiguredInternalTopic config = new ConfiguredInternalTopic("test-topic", topicConfigs, Optional.of(3), Optional.of((short) 2)); + + assertEquals("test-topic", config.name()); + assertEquals(topicConfigs, config.topicConfigs()); + assertEquals(Optional.of(3), config.numberOfPartitions()); + assertEquals(Optional.of((short) 2), config.replicationFactor()); + } Review Comment: nit: Do you think this is needed? I am fine if you want to keep it. I usually do not test this rather simple constructor and getters. This also applies to the other test classes. ########## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/TopicConfigurationException.java: ########## @@ -0,0 +1,73 @@ +/* + * 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.coordinator.group.streams.topics; + +import org.apache.kafka.common.requests.StreamsGroupHeartbeatResponse; +import org.apache.kafka.common.requests.StreamsGroupHeartbeatResponse.Status; + +import java.util.Objects; + +public class TopicConfigurationException extends RuntimeException { + + private final Status status; + + public TopicConfigurationException(StreamsGroupHeartbeatResponse.Status status, String message) { + super(message); + this.status = status; + } + + public Status status() { + return status; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final TopicConfigurationException that = (TopicConfigurationException) o; + return status == that.status && Objects.equals(getMessage(), that.getMessage()); + } + + @Override + public int hashCode() { + return Objects.hash(status, getMessage()); + } + + @Override + public String toString() { + return "TopicConfigurationException{" + + "status=" + status + + ", getMessage='" + getMessage() + '\'' + + '}'; + } Review Comment: Do we need those overrides? ########## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredSubtopologyTest.java: ########## @@ -0,0 +1,172 @@ +/* + * 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.coordinator.group.streams.topics; + +import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class ConfiguredSubtopologyTest { + + @Test + public void testConstructorAndGetters() { + Set<String> repartitionSinkTopics = Set.of("repartitionSinkTopic1", "repartitionSinkTopic2"); + Set<String> sourceTopics = Set.of("sourceTopic1", "sourceTopic2"); + Map<String, ConfiguredInternalTopic> repartitionSourceTopics = new HashMap<>(); + Map<String, ConfiguredInternalTopic> stateChangelogTopics = new HashMap<>(); + + ConfiguredSubtopology configuredSubtopology = new ConfiguredSubtopology(repartitionSinkTopics, sourceTopics, + repartitionSourceTopics, stateChangelogTopics); + + assertEquals(repartitionSinkTopics, configuredSubtopology.repartitionSinkTopics()); + assertEquals(sourceTopics, configuredSubtopology.sourceTopics()); + assertEquals(repartitionSourceTopics, configuredSubtopology.repartitionSourceTopics()); + assertEquals(stateChangelogTopics, configuredSubtopology.stateChangelogTopics()); + } + + @Test + public void testSetters() { + ConfiguredSubtopology configuredSubtopology = new ConfiguredSubtopology(); + + Set<String> repartitionSinkTopics = Set.of("repartitionSinkTopic1", "repartitionSinkTopic2"); + ConfiguredSubtopology ret1 = configuredSubtopology.setRepartitionSinkTopics(repartitionSinkTopics); + assertEquals(repartitionSinkTopics, configuredSubtopology.repartitionSinkTopics()); + assertEquals(ret1, configuredSubtopology); + + Set<String> sourceTopics = Set.of("sourceTopic1", "sourceTopic2"); + ConfiguredSubtopology ret2 = configuredSubtopology.setSourceTopics(sourceTopics); + assertEquals(sourceTopics, configuredSubtopology.sourceTopics()); + assertEquals(ret2, configuredSubtopology); + + Map<String, ConfiguredInternalTopic> repartitionSourceTopics = new HashMap<>(); + ConfiguredSubtopology ret3 = configuredSubtopology.setRepartitionSourceTopics(repartitionSourceTopics); + assertEquals(repartitionSourceTopics, configuredSubtopology.repartitionSourceTopics()); + assertEquals(ret3, configuredSubtopology); + + Map<String, ConfiguredInternalTopic> stateChangelogTopics = new HashMap<>(); + ConfiguredSubtopology ret4 = configuredSubtopology.setStateChangelogTopics(stateChangelogTopics); + assertEquals(stateChangelogTopics, configuredSubtopology.stateChangelogTopics()); + assertEquals(ret4, configuredSubtopology); + } + + @Test + public void testNonSourceChangelogTopics() { + Map<String, ConfiguredInternalTopic> stateChangelogTopics = new HashMap<>(); + stateChangelogTopics.put("changelogTopic1", new ConfiguredInternalTopic("changelogTopic1")); + stateChangelogTopics.put("sourceTopic1", new ConfiguredInternalTopic("sourceTopic1")); + + ConfiguredSubtopology configuredSubtopology = new ConfiguredSubtopology( + Collections.emptySet(), + Collections.singleton("sourceTopic1"), + Collections.emptyMap(), + stateChangelogTopics + ); + + Set<ConfiguredInternalTopic> nonSourceChangelogTopics = configuredSubtopology.nonSourceChangelogTopics(); + assertEquals(1, nonSourceChangelogTopics.size()); + assertTrue(nonSourceChangelogTopics.contains(new ConfiguredInternalTopic("changelogTopic1"))); + } + + @Test + public void testEquals() { + Set<String> repartitionSinkTopics = new HashSet<>(Arrays.asList("repartitionSinkTopic1", "repartitionSinkTopic2")); + Set<String> sourceTopics = new HashSet<>(Arrays.asList("sourceTopic1", "sourceTopic2")); + Map<String, ConfiguredInternalTopic> repartitionSourceTopics = new HashMap<>(); + Map<String, ConfiguredInternalTopic> stateChangelogTopics = new HashMap<>(); + + ConfiguredSubtopology configuredSubtopology1 = new ConfiguredSubtopology(repartitionSinkTopics, sourceTopics, + repartitionSourceTopics, stateChangelogTopics); + ConfiguredSubtopology configuredSubtopology2 = new ConfiguredSubtopology(repartitionSinkTopics, sourceTopics, + repartitionSourceTopics, stateChangelogTopics); + + assertEquals(configuredSubtopology1, configuredSubtopology2); Review Comment: Could you please add neg. examples? ########## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredTopologyTest.java: ########## @@ -0,0 +1,123 @@ +/* + * 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.coordinator.group.streams.topics; + +import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic; +import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; + +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class ConfiguredTopologyTest { + + @Test + public void testConstructorAndGetters() { + int topologyEpoch = 1; + Map<String, ConfiguredSubtopology> subtopologies = new HashMap<>(); + Map<String, CreatableTopic> internalTopicsToBeCreated = new HashMap<>(); + Optional<TopicConfigurationException> topicConfigurationException = Optional.empty(); + + ConfiguredTopology configuredTopology = new ConfiguredTopology( + topologyEpoch, subtopologies, internalTopicsToBeCreated, topicConfigurationException); + + assertEquals(topologyEpoch, configuredTopology.topologyEpoch()); + assertEquals(subtopologies, configuredTopology.subtopologies()); + assertEquals(internalTopicsToBeCreated, configuredTopology.internalTopicsToBeCreated()); + assertEquals(topicConfigurationException, configuredTopology.topicConfigurationException()); + } + + @Test + public void testIsReady() { + ConfiguredTopology readyTopology = new ConfiguredTopology( + 1, new HashMap<>(), new HashMap<>(), Optional.empty()); + assertTrue(readyTopology.isReady()); + + ConfiguredTopology notReadyTopology = new ConfiguredTopology( + 1, new HashMap<>(), new HashMap<>(), Optional.of(TopicConfigurationException.missingSourceTopics("missing"))); + assertFalse(notReadyTopology.isReady()); + } + + @Test + public void testEqualsAndHashCode() { + int topologyEpoch = 1; + Map<String, ConfiguredSubtopology> subtopologies = new HashMap<>(); + Map<String, CreatableTopic> internalTopicsToBeCreated = new HashMap<>(); + Optional<TopicConfigurationException> topicConfigurationException = Optional.empty(); + + ConfiguredTopology configuredTopology1 = new ConfiguredTopology( + topologyEpoch, subtopologies, internalTopicsToBeCreated, topicConfigurationException); + ConfiguredTopology configuredTopology2 = new ConfiguredTopology( + topologyEpoch, subtopologies, internalTopicsToBeCreated, topicConfigurationException); + + assertEquals(configuredTopology1, configuredTopology2); + assertEquals(configuredTopology1.hashCode(), configuredTopology2.hashCode()); Review Comment: Could you please add negative examples? -- 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]
