-
Notifications
You must be signed in to change notification settings - Fork 14.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Add and distribute IQv2 information in KIP-1071 #18278
Open
bbejeck
wants to merge
8
commits into
apache:kip1071
Choose a base branch
from
bbejeck:KIP_1071_add_IQv2_information
base: kip1071
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
+585
−79
Open
Changes from all commits
Commits
Show all changes
8 commits
Select commit
Hold shift + click to select a range
d0cd04d
Fix merge conflict with First pass at adding IQv2 information
bbejeck dfa9008
Added IQv2 information updated IQ related integration tests
bbejeck 088e26f
Checkstyle fixes
bbejeck 3b655a2
Start addressing comments from initial review
bbejeck 83762a2
Partial implementation for adjusting topic partitions
bbejeck 9623ffc
Added functionality to share IQ v2 information, updated integration t…
bbejeck 26530c2
Intermediate state
bbejeck cc72e4f
Get IQv2 state propogated to members and added integration test
bbejeck File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
84 changes: 84 additions & 0 deletions
84
...n/java/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManager.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,84 @@ | ||
/* | ||
* 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.StreamsGroupHeartbeatResponseData; | ||
import org.apache.kafka.coordinator.group.streams.StreamsGroup; | ||
import org.apache.kafka.coordinator.group.streams.StreamsGroupMember; | ||
import org.apache.kafka.coordinator.group.streams.TopicMetadata; | ||
|
||
import java.util.ArrayList; | ||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Set; | ||
import java.util.stream.Stream; | ||
|
||
|
||
public class EndpointToPartitionsManager { | ||
|
||
|
||
public EndpointToPartitionsManager() {} | ||
|
||
public StreamsGroupHeartbeatResponseData.EndpointToPartitions endpointToPartitions(final StreamsGroupMember streamsGroupMember, | ||
final StreamsGroupHeartbeatResponseData.Endpoint responseEndpoint, | ||
final StreamsGroup streamsGroup) { | ||
|
||
StreamsGroupHeartbeatResponseData.EndpointToPartitions endpointToPartitions = new StreamsGroupHeartbeatResponseData.EndpointToPartitions(); | ||
List<StreamsGroupHeartbeatResponseData.TopicPartition> allTopicPartitions = new ArrayList<>(); | ||
for (Map.Entry<String, ConfiguredSubtopology> entry : streamsGroup.configuredTopology().subtopologies().entrySet()) { | ||
ConfiguredSubtopology configuredSubtopology = entry.getValue(); | ||
endpointToPartitions.setUserEndpoint(responseEndpoint); | ||
final Map<String, TopicMetadata> groupTopicMetadata = streamsGroup.partitionMetadata(); | ||
getActiveTaskTopicPartitions(configuredSubtopology, groupTopicMetadata, streamsGroupMember, allTopicPartitions); | ||
} | ||
endpointToPartitions.setPartitions(allTopicPartitions); | ||
return endpointToPartitions; | ||
} | ||
|
||
private void getActiveTaskTopicPartitions(final ConfiguredSubtopology configuredSubtopology, | ||
final Map<String, TopicMetadata> groupTopicMetadata, | ||
final StreamsGroupMember streamsGroupMember, | ||
final List<StreamsGroupHeartbeatResponseData.TopicPartition> allTopicPartitions) { | ||
|
||
List<String> standbyTopicNames = configuredSubtopology.nonSourceChangelogTopics().stream().map(ConfiguredInternalTopic::name).toList(); | ||
|
||
for (Map.Entry<String, Set<Integer>> taskEntry : streamsGroupMember.assignedActiveTasks().entrySet()) { | ||
Set<Integer> taskPartitions = taskEntry.getValue(); | ||
List<StreamsGroupHeartbeatResponseData.TopicPartition> topicPartitionList = | ||
Stream.concat( | ||
configuredSubtopology.sourceTopics().stream(), | ||
configuredSubtopology.repartitionSourceTopics().keySet().stream() | ||
).map(topic -> { | ||
int numPartitionsForTopic = groupTopicMetadata.get(topic).numPartitions(); | ||
StreamsGroupHeartbeatResponseData.TopicPartition tp = new StreamsGroupHeartbeatResponseData.TopicPartition(); | ||
tp.setTopic(topic); | ||
List<Integer> tpPartitions = new ArrayList<>(taskPartitions); | ||
if (numPartitionsForTopic < taskPartitions.size()) { | ||
Collections.sort(tpPartitions); | ||
tp.setPartitions(tpPartitions.subList(0, numPartitionsForTopic)); | ||
} else { | ||
tp.setPartitions(tpPartitions); | ||
} | ||
return tp; | ||
}).toList(); | ||
allTopicPartitions.addAll(topicPartitionList); | ||
} | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
116 changes: 116 additions & 0 deletions
116
...va/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManagerTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,116 @@ | ||
/* | ||
* 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.Uuid; | ||
import org.apache.kafka.common.message.StreamsGroupHeartbeatResponseData; | ||
import org.apache.kafka.common.utils.LogContext; | ||
import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue; | ||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard; | ||
import org.apache.kafka.coordinator.group.streams.StreamsGroup; | ||
import org.apache.kafka.coordinator.group.streams.StreamsGroupMember; | ||
import org.apache.kafka.coordinator.group.streams.StreamsTopology; | ||
import org.apache.kafka.coordinator.group.streams.TopicMetadata; | ||
import org.apache.kafka.timeline.SnapshotRegistry; | ||
|
||
import org.junit.jupiter.params.ParameterizedTest; | ||
import org.junit.jupiter.params.provider.Arguments; | ||
import org.junit.jupiter.params.provider.MethodSource; | ||
|
||
import java.util.Collections; | ||
import java.util.HashMap; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Set; | ||
import java.util.TreeSet; | ||
import java.util.stream.Stream; | ||
|
||
import static org.junit.jupiter.api.Assertions.assertEquals; | ||
import static org.junit.jupiter.params.provider.Arguments.arguments; | ||
import static org.mockito.Mockito.mock; | ||
|
||
class EndpointToPartitionsManagerTest { | ||
|
||
private static final LogContext LOG_CONTEXT = new LogContext(); | ||
|
||
@ParameterizedTest(name = "{5}") | ||
@MethodSource("argsProvider") | ||
void testEndpointToPartitionsWithTwoTopicsAndDifferentPartitions(int topicAPartitions, | ||
int topicBPartitions, | ||
List<Integer> topicAExpectedPartitions, | ||
List<Integer> topicBExpectedPartitions, | ||
String testName | ||
) { | ||
Uuid topicAId = Uuid.randomUuid(); | ||
Uuid topicBId = Uuid.randomUuid(); | ||
|
||
Map<Integer, Set<String>> emptyRackMap = Collections.emptyMap(); | ||
StreamsGroupHeartbeatResponseData.Endpoint endpoint = new StreamsGroupHeartbeatResponseData.Endpoint(); | ||
endpoint.setPort(8080); | ||
endpoint.setHost("localhost"); | ||
|
||
Map<String, TopicMetadata> topicMetadata = new HashMap<>(); | ||
topicMetadata.put("Topic-A", new TopicMetadata(topicAId, "Topic-A", topicAPartitions, emptyRackMap)); | ||
topicMetadata.put("Topic-B", new TopicMetadata(topicBId, "Topic-B", topicBPartitions, emptyRackMap)); | ||
|
||
StreamsGroup streamsGroup = createStreamsGroup("streamsGroup"); | ||
streamsGroup.setPartitionMetadata(topicMetadata); | ||
streamsGroup.setGroupEpoch(1); | ||
streamsGroup.setTopology(topology()); | ||
|
||
EndpointToPartitionsManager endpointToPartitionsManager = new EndpointToPartitionsManager(); | ||
|
||
StreamsGroupHeartbeatResponseData.EndpointToPartitions result = endpointToPartitionsManager.endpointToPartitions(null, endpoint, streamsGroup); | ||
|
||
assertEquals(endpoint, result.userEndpoint()); | ||
assertEquals(2, result.partitions().size()); | ||
|
||
StreamsGroupHeartbeatResponseData.TopicPartition topicAPartition = result.partitions().get(0); | ||
assertEquals("Topic-A", topicAPartition.topic()); | ||
assertEquals(topicAExpectedPartitions, topicAPartition.partitions()); | ||
|
||
StreamsGroupHeartbeatResponseData.TopicPartition topicBPartition = result.partitions().get(1); | ||
assertEquals("Topic-B", topicBPartition.topic()); | ||
assertEquals(topicBExpectedPartitions, topicBPartition.partitions()); | ||
} | ||
|
||
static Stream<Arguments> argsProvider() { | ||
return Stream.of( | ||
arguments(2, 5, new TreeSet<>(List.of(0, 1, 2, 3, 4)), List.of(0, 1), List.of(0, 1, 2, 3, 4), "Should assign correct partitions when partitions differ between topics"), | ||
arguments(3, 3, new TreeSet<>(List.of(0, 1, 2)), List.of(0, 1, 2), List.of(0, 1, 2), "Should assign correct partitions when partitions same between topics") | ||
); | ||
} | ||
|
||
private StreamsTopology topology() { | ||
StreamsGroupTopologyValue.Subtopology subtopology = new StreamsGroupTopologyValue.Subtopology(); | ||
subtopology.setSubtopologyId("subtopology-1"); | ||
subtopology.setSourceTopics(List.of("Topic-A", "Topic-B")); | ||
return new StreamsTopology(1, Map.of("subtopology-1", subtopology)); | ||
} | ||
|
||
private StreamsGroup createStreamsGroup(String groupId) { | ||
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(LOG_CONTEXT); | ||
return new StreamsGroup( | ||
LOG_CONTEXT, | ||
snapshotRegistry, | ||
groupId, | ||
mock(GroupCoordinatorMetricsShard.class) | ||
); | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,6 +18,7 @@ | |
package org.apache.kafka.coordinator.group.taskassignor; | ||
|
||
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; | ||
|
||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. checkstyle fix |
||
import org.junit.jupiter.api.Test; | ||
|
||
import java.util.ArrayList; | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
checkstyle fix