mirror of https://github.com/apache/kafka.git
KAFKA-19044: Handle tasks that are not present in the current topology (#19722)
A heartbeat might be sent to the group coordinator, claiming to own tasks that we do not know about. We need some logic to handle those requests. In KIP-1071, we propose to return `INVALID_REQUEST` error whenever this happens, effectively letting the clients crash. This behavior will, however, make topology updates impossible. Bruno Cadonna proposed to only check that owned tasks match our set of expected tasks if the topology epochs between the group and the client match. The aim of this change is to implement a check and a behavior for the first version of the protocol, which is to always return `INVALID_REQUEST` if an unknown task is sent to the group coordinator. We can relax this constraint once we allow topology updating with topology epochs. To efficiently check this whenever we receive a heartbeat containing tasks, we precompute the number of tasks for each subtopology. This also benefits the performance of the assignor. Reviewers: Bill Bejeck <bbejeck@apache.org>
This commit is contained in:
parent
949617b0b2
commit
678d456ad7
|
|
@ -156,6 +156,7 @@ import org.apache.kafka.coordinator.group.streams.TasksTuple;
|
|||
import org.apache.kafka.coordinator.group.streams.assignor.StickyTaskAssignor;
|
||||
import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignor;
|
||||
import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignorException;
|
||||
import org.apache.kafka.coordinator.group.streams.topics.ConfiguredSubtopology;
|
||||
import org.apache.kafka.coordinator.group.streams.topics.ConfiguredTopology;
|
||||
import org.apache.kafka.coordinator.group.streams.topics.EndpointToPartitionsManager;
|
||||
import org.apache.kafka.coordinator.group.streams.topics.InternalTopicManager;
|
||||
|
|
@ -195,6 +196,7 @@ import java.util.Objects;
|
|||
import java.util.Optional;
|
||||
import java.util.OptionalInt;
|
||||
import java.util.Set;
|
||||
import java.util.SortedMap;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.BiFunction;
|
||||
|
|
@ -1665,6 +1667,34 @@ public class GroupMetadataManager {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Validates that the requested tasks exist in the configured topology and partitions are valid.
|
||||
* If tasks is null, does nothing. If an invalid task is found, throws InvalidRequestException.
|
||||
*
|
||||
* @param subtopologySortedMap The configured topology.
|
||||
* @param tasks The list of requested tasks.
|
||||
*/
|
||||
private static void throwIfRequestContainsInvalidTasks(
|
||||
SortedMap<String, ConfiguredSubtopology> subtopologySortedMap,
|
||||
List<StreamsGroupHeartbeatRequestData.TaskIds> tasks
|
||||
) {
|
||||
if (tasks == null || tasks.isEmpty()) return;
|
||||
for (StreamsGroupHeartbeatRequestData.TaskIds task : tasks) {
|
||||
String subtopologyId = task.subtopologyId();
|
||||
ConfiguredSubtopology subtopology = subtopologySortedMap.get(subtopologyId);
|
||||
if (subtopology == null) {
|
||||
throw new InvalidRequestException("Subtopology " + subtopologyId + " does not exist in the topology.");
|
||||
}
|
||||
int numTasks = subtopology.numberOfTasks();
|
||||
for (Integer partition : task.partitions()) {
|
||||
if (partition < 0 || partition >= numTasks) {
|
||||
throw new InvalidRequestException("Task " + partition + " for subtopology " + subtopologyId +
|
||||
" is invalid. Number of tasks for this subtopology: " + numTasks);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Validates if the received classic member protocols are supported by the group.
|
||||
*
|
||||
|
|
@ -1917,6 +1947,13 @@ public class GroupMetadataManager {
|
|||
updatedConfiguredTopology = group.configuredTopology().get();
|
||||
}
|
||||
|
||||
if (updatedConfiguredTopology.isReady()) {
|
||||
SortedMap<String, ConfiguredSubtopology> subtopologySortedMap = updatedConfiguredTopology.subtopologies().get();
|
||||
throwIfRequestContainsInvalidTasks(subtopologySortedMap, ownedActiveTasks);
|
||||
throwIfRequestContainsInvalidTasks(subtopologySortedMap, ownedStandbyTasks);
|
||||
throwIfRequestContainsInvalidTasks(subtopologySortedMap, ownedWarmupTasks);
|
||||
}
|
||||
|
||||
// Actually bump the group epoch
|
||||
int groupEpoch = group.groupEpoch();
|
||||
if (bumpGroupEpoch) {
|
||||
|
|
|
|||
|
|
@ -19,14 +19,12 @@ package org.apache.kafka.coordinator.group.streams;
|
|||
import org.apache.kafka.coordinator.group.streams.assignor.TopologyDescriber;
|
||||
import org.apache.kafka.coordinator.group.streams.topics.ConfiguredSubtopology;
|
||||
import org.apache.kafka.image.MetadataImage;
|
||||
import org.apache.kafka.image.TopicImage;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.Objects;
|
||||
import java.util.SortedMap;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* The topology metadata class is used by the {@link org.apache.kafka.coordinator.group.streams.assignor.TaskAssignor} to get topic and
|
||||
|
|
@ -42,14 +40,6 @@ public record TopologyMetadata(MetadataImage metadataImage, SortedMap<String, Co
|
|||
subtopologyMap = Objects.requireNonNull(Collections.unmodifiableSortedMap(subtopologyMap));
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The metadata image in topology metadata.
|
||||
*/
|
||||
@Override
|
||||
public MetadataImage metadataImage() {
|
||||
return this.metadataImage;
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks whether the given subtopology is associated with a changelog topic.
|
||||
*
|
||||
|
|
@ -85,18 +75,7 @@ public record TopologyMetadata(MetadataImage metadataImage, SortedMap<String, Co
|
|||
@Override
|
||||
public int maxNumInputPartitions(String subtopologyId) {
|
||||
final ConfiguredSubtopology subtopology = getSubtopologyOrFail(subtopologyId);
|
||||
return Stream.concat(
|
||||
subtopology.sourceTopics().stream(),
|
||||
subtopology.repartitionSourceTopics().keySet().stream()
|
||||
).map(topic -> {
|
||||
TopicImage topicImage = metadataImage.topics().getTopic(topic);
|
||||
if (topicImage == null) {
|
||||
throw new IllegalStateException("Topic " + topic + " not found in metadata image");
|
||||
}
|
||||
return topicImage.partitions().size();
|
||||
}).max(Integer::compareTo).orElseThrow(
|
||||
() -> new IllegalStateException("Subtopology does not contain any source topics")
|
||||
);
|
||||
return subtopology.numberOfTasks();
|
||||
}
|
||||
|
||||
private ConfiguredSubtopology getSubtopologyOrFail(String subtopologyId) {
|
||||
|
|
|
|||
|
|
@ -85,8 +85,10 @@ public class ChangelogTopics {
|
|||
}
|
||||
}
|
||||
|
||||
log.debug("Expecting state changelog topic partitions {} for the requested topology.",
|
||||
changelogTopicPartitions.entrySet().stream().map(e -> e.getKey() + ":" + e.getValue()).collect(Collectors.joining(", ")));
|
||||
if (!changelogTopicPartitions.isEmpty()) {
|
||||
log.debug("Expecting state changelog topic partitions {} for the requested topology.",
|
||||
changelogTopicPartitions.entrySet().stream().map(e -> e.getKey() + ":" + e.getValue()).collect(Collectors.joining(", ")));
|
||||
}
|
||||
|
||||
return changelogTopicPartitions;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -31,17 +31,25 @@ import java.util.Set;
|
|||
* <p>
|
||||
* Configured subtopologies may be recreated every time the input topics used by the subtopology are modified.
|
||||
*
|
||||
* @param numberOfTasks Precomputed number of tasks. Not that not every source topic may have a partition for
|
||||
* every task, in cases where there are multiple source topics with an unequal number of
|
||||
* partitions (e.g., one topic has 3 partitions and another has 5 and both are used in a
|
||||
* merge).
|
||||
* @param sourceTopics The source topics of the subtopology.
|
||||
* @param repartitionSourceTopics The repartition source topics of the subtopology.
|
||||
* @param repartitionSinkTopics The repartition sink topics of the subtopology.
|
||||
* @param stateChangelogTopics The state changelog topics of the subtopology.
|
||||
*/
|
||||
public record ConfiguredSubtopology(Set<String> sourceTopics,
|
||||
public record ConfiguredSubtopology(int numberOfTasks,
|
||||
Set<String> sourceTopics,
|
||||
Map<String, ConfiguredInternalTopic> repartitionSourceTopics,
|
||||
Set<String> repartitionSinkTopics,
|
||||
Map<String, ConfiguredInternalTopic> stateChangelogTopics) {
|
||||
|
||||
public ConfiguredSubtopology {
|
||||
if (numberOfTasks <= 0) {
|
||||
throw new IllegalArgumentException("Number of tasks must be positive");
|
||||
}
|
||||
Objects.requireNonNull(sourceTopics, "sourceTopics can't be null");
|
||||
Objects.requireNonNull(repartitionSourceTopics, "repartitionSourceTopics can't be null");
|
||||
Objects.requireNonNull(repartitionSinkTopics, "repartitionSinkTopics can't be null");
|
||||
|
|
@ -61,4 +69,6 @@ public record ConfiguredSubtopology(Set<String> sourceTopics,
|
|||
.sorted(Comparator.comparing(StreamsGroupDescribeResponseData.TopicInfo::name)).toList());
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
|
@ -82,7 +82,7 @@ public class InternalTopicManager {
|
|||
subtopologies.stream()
|
||||
.collect(Collectors.toMap(
|
||||
StreamsGroupTopologyValue.Subtopology::subtopologyId,
|
||||
x -> fromPersistedSubtopology(x, decidedPartitionCountsForInternalTopics),
|
||||
x -> fromPersistedSubtopology(x, topicsImage, decidedPartitionCountsForInternalTopics),
|
||||
(v1, v2) -> {
|
||||
throw new RuntimeException(String.format("Duplicate key for values %s and %s", v1, v2));
|
||||
},
|
||||
|
|
@ -264,9 +264,11 @@ public class InternalTopicManager {
|
|||
}
|
||||
|
||||
private static ConfiguredSubtopology fromPersistedSubtopology(final StreamsGroupTopologyValue.Subtopology subtopology,
|
||||
final TopicsImage topicsImage,
|
||||
final Map<String, Integer> decidedPartitionCountsForInternalTopics
|
||||
) {
|
||||
return new ConfiguredSubtopology(
|
||||
computeNumberOfTasks(subtopology, topicsImage, decidedPartitionCountsForInternalTopics),
|
||||
new HashSet<>(subtopology.sourceTopics()),
|
||||
subtopology.repartitionSourceTopics().stream()
|
||||
.map(x -> fromPersistedTopicInfo(x, decidedPartitionCountsForInternalTopics))
|
||||
|
|
@ -278,6 +280,21 @@ public class InternalTopicManager {
|
|||
);
|
||||
}
|
||||
|
||||
private static int computeNumberOfTasks(final StreamsGroupTopologyValue.Subtopology subtopology,
|
||||
final TopicsImage topicsImage,
|
||||
final Map<String, Integer> decidedPartitionCountsForInternalTopics) {
|
||||
return Stream.concat(
|
||||
subtopology.sourceTopics().stream(),
|
||||
subtopology.repartitionSourceTopics().stream().map(StreamsGroupTopologyValue.TopicInfo::name)
|
||||
).map(
|
||||
topic -> getPartitionCount(topicsImage, topic, decidedPartitionCountsForInternalTopics).orElseThrow(
|
||||
() -> new IllegalStateException("Number of partitions must be set for topic " + topic)
|
||||
)
|
||||
).max(Integer::compareTo).orElseThrow(
|
||||
() -> new IllegalStateException("Subtopology does not contain any source topics")
|
||||
);
|
||||
}
|
||||
|
||||
private static ConfiguredInternalTopic fromPersistedTopicInfo(final StreamsGroupTopologyValue.TopicInfo topicInfo,
|
||||
final Map<String, Integer> decidedPartitionCountsForInternalTopics) {
|
||||
if (topicInfo.partitions() == 0 && !decidedPartitionCountsForInternalTopics.containsKey(topicInfo.name())) {
|
||||
|
|
|
|||
|
|
@ -28,6 +28,7 @@ import org.apache.kafka.common.errors.GroupMaxSizeReachedException;
|
|||
import org.apache.kafka.common.errors.IllegalGenerationException;
|
||||
import org.apache.kafka.common.errors.InconsistentGroupProtocolException;
|
||||
import org.apache.kafka.common.errors.InvalidRegularExpression;
|
||||
import org.apache.kafka.common.errors.InvalidRequestException;
|
||||
import org.apache.kafka.common.errors.NotLeaderOrFollowerException;
|
||||
import org.apache.kafka.common.errors.RebalanceInProgressException;
|
||||
import org.apache.kafka.common.errors.UnknownMemberIdException;
|
||||
|
|
@ -16014,6 +16015,67 @@ public class GroupMetadataManagerTest {
|
|||
assertEquals(100, result.response().data().memberEpoch());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStreamsOwnedTasksValidation() {
|
||||
String groupId = "fooup";
|
||||
String memberId = Uuid.randomUuid().toString();
|
||||
String subtopology1 = "subtopology1";
|
||||
String subtopologyMissing = "subtopologyMissing";
|
||||
String fooTopicName = "foo";
|
||||
Uuid fooTopicId = Uuid.randomUuid();
|
||||
Topology topology = new Topology().setSubtopologies(List.of(
|
||||
new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName))
|
||||
));
|
||||
|
||||
MockTaskAssignor assignor = new MockTaskAssignor("sticky");
|
||||
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
|
||||
.withStreamsGroupTaskAssignors(List.of(assignor))
|
||||
.withMetadataImage(new MetadataImageBuilder()
|
||||
.addTopic(fooTopicId, fooTopicName, 3)
|
||||
.build())
|
||||
.withStreamsGroup(new StreamsGroupBuilder(groupId, 10)
|
||||
.withMember(streamsGroupMemberBuilderWithDefaults(memberId)
|
||||
.setMemberEpoch(10)
|
||||
.setPreviousMemberEpoch(10)
|
||||
.setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE,
|
||||
TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2)))
|
||||
.build())
|
||||
.withTopology(StreamsTopology.fromHeartbeatRequest(topology))
|
||||
.withTargetAssignment(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE,
|
||||
TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2)))
|
||||
.withTargetAssignmentEpoch(10)
|
||||
)
|
||||
.build();
|
||||
|
||||
InvalidRequestException e1 = assertThrows(InvalidRequestException.class, () -> context.streamsGroupHeartbeat(
|
||||
new StreamsGroupHeartbeatRequestData()
|
||||
.setGroupId(groupId)
|
||||
.setMemberId(memberId)
|
||||
.setMemberEpoch(10)
|
||||
.setActiveTasks(List.of(
|
||||
new StreamsGroupHeartbeatRequestData.TaskIds()
|
||||
.setSubtopologyId(subtopologyMissing)
|
||||
.setPartitions(List.of(0))
|
||||
))
|
||||
.setStandbyTasks(List.of())
|
||||
.setWarmupTasks(List.of())));
|
||||
assertEquals(e1.getMessage(), "Subtopology subtopologyMissing does not exist in the topology.");
|
||||
|
||||
InvalidRequestException e2 = assertThrows(InvalidRequestException.class, () -> context.streamsGroupHeartbeat(
|
||||
new StreamsGroupHeartbeatRequestData()
|
||||
.setGroupId(groupId)
|
||||
.setMemberId(memberId)
|
||||
.setMemberEpoch(10)
|
||||
.setActiveTasks(List.of(
|
||||
new StreamsGroupHeartbeatRequestData.TaskIds()
|
||||
.setSubtopologyId(subtopology1)
|
||||
.setPartitions(List.of(3))
|
||||
))
|
||||
.setStandbyTasks(List.of())
|
||||
.setWarmupTasks(List.of())));
|
||||
assertEquals(e2.getMessage(), "Task 3 for subtopology subtopology1 is invalid. Number of tasks for this subtopology: 3");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStreamsNewMemberIsRejectedWithMaximumMembersIsReached() {
|
||||
String groupId = "fooup";
|
||||
|
|
|
|||
|
|
@ -713,7 +713,7 @@ public class TargetAssignmentBuilderTest {
|
|||
String subtopologyId = Uuid.randomUuid().toString();
|
||||
Uuid topicId = Uuid.randomUuid();
|
||||
topicsImageBuilder = topicsImageBuilder.addTopic(topicId, topicName, numTasks);
|
||||
subtopologies.put(subtopologyId, new ConfiguredSubtopology(Set.of(topicId.toString()), Map.of(), Set.of(), Map.of()));
|
||||
subtopologies.put(subtopologyId, new ConfiguredSubtopology(numTasks, Set.of(topicId.toString()), Map.of(), Set.of(), Map.of()));
|
||||
|
||||
return subtopologyId;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -28,7 +28,6 @@ import org.junit.jupiter.api.Test;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.Set;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
|
|
@ -82,11 +81,9 @@ class TopologyMetadataTest {
|
|||
|
||||
@Test
|
||||
void testMaxNumInputPartitions() {
|
||||
ConfiguredInternalTopic internalTopic = mock(ConfiguredInternalTopic.class);
|
||||
ConfiguredSubtopology subtopology = mock(ConfiguredSubtopology.class);
|
||||
subtopologyMap.put("subtopology1", subtopology);
|
||||
when(subtopology.sourceTopics()).thenReturn(Set.of("source_topic"));
|
||||
when(subtopology.repartitionSourceTopics()).thenReturn(Map.of("repartition_source_topic", internalTopic));
|
||||
when(subtopology.numberOfTasks()).thenReturn(4);
|
||||
|
||||
assertEquals(4, topologyMetadata.maxNumInputPartitions("subtopology1"));
|
||||
}
|
||||
|
|
@ -111,14 +108,4 @@ class TopologyMetadataTest {
|
|||
void testMaxNumInputPartitionsThrowsExceptionWhenSubtopologyIdDoesNotExist() {
|
||||
assertThrows(NoSuchElementException.class, () -> topologyMetadata.maxNumInputPartitions("non_existent_subtopology"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testMaxNumInputPartitionsThrowsExceptionWhenSubtopologyContainsNoSourceTopics() {
|
||||
ConfiguredSubtopology subtopology = mock(ConfiguredSubtopology.class);
|
||||
when(subtopology.sourceTopics()).thenReturn(Set.of());
|
||||
when(subtopology.repartitionSourceTopics()).thenReturn(Map.of());
|
||||
subtopologyMap.put("subtopology1", subtopology);
|
||||
|
||||
assertThrows(IllegalStateException.class, () -> topologyMetadata.maxNumInputPartitions("subtopology1"));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -36,6 +36,7 @@ public class ConfiguredSubtopologyTest {
|
|||
public void testConstructorWithNullSourceTopics() {
|
||||
assertThrows(NullPointerException.class,
|
||||
() -> new ConfiguredSubtopology(
|
||||
2,
|
||||
null,
|
||||
Map.of(),
|
||||
Set.of(),
|
||||
|
|
@ -48,6 +49,7 @@ public class ConfiguredSubtopologyTest {
|
|||
public void testConstructorWithNullRepartitionSourceTopics() {
|
||||
assertThrows(NullPointerException.class,
|
||||
() -> new ConfiguredSubtopology(
|
||||
2,
|
||||
Set.of(),
|
||||
null,
|
||||
Set.of(),
|
||||
|
|
@ -60,6 +62,7 @@ public class ConfiguredSubtopologyTest {
|
|||
public void testConstructorWithNullRepartitionSinkTopics() {
|
||||
assertThrows(NullPointerException.class,
|
||||
() -> new ConfiguredSubtopology(
|
||||
2,
|
||||
Set.of(),
|
||||
Map.of(),
|
||||
null,
|
||||
|
|
@ -72,6 +75,7 @@ public class ConfiguredSubtopologyTest {
|
|||
public void testConstructorWithNullStateChangelogTopics() {
|
||||
assertThrows(NullPointerException.class,
|
||||
() -> new ConfiguredSubtopology(
|
||||
2,
|
||||
Set.of(),
|
||||
Map.of(),
|
||||
Set.of(),
|
||||
|
|
@ -80,6 +84,19 @@ public class ConfiguredSubtopologyTest {
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConstructorWithNegativeTaskCount() {
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> new ConfiguredSubtopology(
|
||||
-1,
|
||||
Set.of(),
|
||||
Map.of(),
|
||||
Set.of(),
|
||||
Map.of()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAsStreamsGroupDescribeSubtopology() {
|
||||
String subtopologyId = "subtopology1";
|
||||
|
|
@ -91,7 +108,7 @@ public class ConfiguredSubtopologyTest {
|
|||
Map<String, ConfiguredInternalTopic> repartitionSourceTopics = Map.of("repartitionSourceTopic1", internalTopicMock);
|
||||
Map<String, ConfiguredInternalTopic> stateChangelogTopics = Map.of("stateChangelogTopic1", internalTopicMock);
|
||||
ConfiguredSubtopology configuredSubtopology = new ConfiguredSubtopology(
|
||||
sourceTopics, repartitionSourceTopics, repartitionSinkTopics, stateChangelogTopics);
|
||||
1, sourceTopics, repartitionSourceTopics, repartitionSinkTopics, stateChangelogTopics);
|
||||
|
||||
StreamsGroupDescribeResponseData.Subtopology subtopology = configuredSubtopology.asStreamsGroupDescribeSubtopology(subtopologyId);
|
||||
|
||||
|
|
|
|||
|
|
@ -65,9 +65,9 @@ class EndpointToPartitionsManagerTest {
|
|||
streamsGroup = mock(StreamsGroup.class);
|
||||
streamsGroupMember = mock(StreamsGroupMember.class);
|
||||
configuredTopology = mock(ConfiguredTopology.class);
|
||||
configuredSubtopologyOne = new ConfiguredSubtopology(Set.of("Topic-A"), new HashMap<>(), new HashSet<>(), new HashMap<>());
|
||||
configuredSubtopologyOne = new ConfiguredSubtopology(1, Set.of("Topic-A"), new HashMap<>(), new HashSet<>(), new HashMap<>());
|
||||
Map<String, ConfiguredInternalTopic> repartitionSourceTopics = Map.of("Topic-B", new ConfiguredInternalTopic("Topic-B", 1, Optional.of((short) 1), Collections.emptyMap()));
|
||||
configuredSubtopologyTwo = new ConfiguredSubtopology(new HashSet<>(), repartitionSourceTopics, new HashSet<>(), new HashMap<>());
|
||||
configuredSubtopologyTwo = new ConfiguredSubtopology(1, new HashSet<>(), repartitionSourceTopics, new HashSet<>(), new HashMap<>());
|
||||
SortedMap<String, ConfiguredSubtopology> configuredSubtopologyOneMap = new TreeMap<>();
|
||||
configuredSubtopologyOneMap.put("0", configuredSubtopologyOne);
|
||||
SortedMap<String, ConfiguredSubtopology> configuredSubtopologyTwoMap = new TreeMap<>();
|
||||
|
|
@ -128,7 +128,7 @@ class EndpointToPartitionsManagerTest {
|
|||
.addTopic(Uuid.randomUuid(), "Topic-A", topicAPartitions)
|
||||
.addTopic(Uuid.randomUuid(), "Topic-B", topicBPartitions)
|
||||
.build();
|
||||
configuredSubtopologyOne = new ConfiguredSubtopology(Set.of("Topic-A", "Topic-B"), new HashMap<>(), new HashSet<>(), new HashMap<>());
|
||||
configuredSubtopologyOne = new ConfiguredSubtopology(Math.max(topicAPartitions, topicBPartitions), Set.of("Topic-A", "Topic-B"), new HashMap<>(), new HashSet<>(), new HashMap<>());
|
||||
|
||||
activeTasks.put("0", Set.of(0, 1, 2, 3, 4));
|
||||
when(streamsGroupMember.assignedTasks()).thenReturn(new TasksTuple(activeTasks, Collections.emptyMap(), Collections.emptyMap()));
|
||||
|
|
@ -160,4 +160,4 @@ class EndpointToPartitionsManagerTest {
|
|||
arguments(3, 3, List.of(0, 1, 2), List.of(0, 1, 2), "Should assign correct partitions when partitions same between topics")
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -107,6 +107,7 @@ class InternalTopicManagerTest {
|
|||
return mkMap(
|
||||
mkEntry(SUBTOPOLOGY_1,
|
||||
new ConfiguredSubtopology(
|
||||
2,
|
||||
Set.of(SOURCE_TOPIC_1),
|
||||
Map.of(),
|
||||
Set.of(REPARTITION_TOPIC),
|
||||
|
|
@ -121,6 +122,7 @@ class InternalTopicManagerTest {
|
|||
),
|
||||
mkEntry(SUBTOPOLOGY_2,
|
||||
new ConfiguredSubtopology(
|
||||
2,
|
||||
Set.of(SOURCE_TOPIC_2),
|
||||
Map.of(REPARTITION_TOPIC,
|
||||
new ConfiguredInternalTopic(REPARTITION_TOPIC,
|
||||
|
|
|
|||
Loading…
Reference in New Issue