Skip to content

KAFKA-17747: [7/N] Add consumer group integration test for rack aware assignment #19856

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

Merged
merged 10 commits into from
Jun 4, 2025
Merged
1 change: 1 addition & 0 deletions build.gradle
Original file line number Diff line number Diff line change
@@ -1995,6 +1995,7 @@ project(':clients:clients-integration-tests') {
implementation project(':server-common')
testImplementation project(':metadata')
implementation project(':group-coordinator')
implementation project(':group-coordinator:group-coordinator-api')
implementation project(':transaction-coordinator')

testImplementation libs.junitJupiter
1 change: 1 addition & 0 deletions checkstyle/import-control.xml
Original file line number Diff line number Diff line change
@@ -235,6 +235,7 @@
<subpackage name="clients">
<allow pkg="org.apache.kafka.common" />
<allow pkg="org.apache.kafka.clients" exact-match="true"/>
<allow pkg="org.apache.kafka.clients.consumer" exact-match="true"/>
<allow pkg="org.apache.kafka.test" />
<allow class="org.apache.logging.log4j.Level" />

Original file line number Diff line number Diff line change
@@ -16,7 +16,12 @@
*/
package org.apache.kafka.clients.consumer;

import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.admin.NewPartitionReassignment;
import org.apache.kafka.clients.admin.NewPartitions;
import org.apache.kafka.clients.admin.NewTopic;
import org.apache.kafka.clients.consumer.internals.AbstractHeartbeatRequestManager;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.KafkaException;
@@ -31,12 +36,16 @@
import org.apache.kafka.common.test.api.ClusterTest;
import org.apache.kafka.common.test.api.ClusterTests;
import org.apache.kafka.common.test.api.Type;
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;

import java.time.Duration;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ExecutionException;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -216,6 +225,116 @@ public void testLeaderEpoch(ClusterInstance clusterInstance) throws Exception {
}
}

@ClusterTest(
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder whether we could add another test which verifies that a rebalance is triggered when the racks of a partition has changed. Have you considered it?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, I added it to the same case. It uses alterPartitionReassignments to change partitions to different brokers and make sure consumers get new assignments.

types = {Type.KRAFT},
brokers = 3,
serverProperties = {
@ClusterConfigProperty(id = 0, key = "broker.rack", value = "rack0"),
@ClusterConfigProperty(id = 1, key = "broker.rack", value = "rack1"),
@ClusterConfigProperty(id = 2, key = "broker.rack", value = "rack2"),
@ClusterConfigProperty(key = GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, value = "org.apache.kafka.clients.consumer.RackAwareAssignor")
}
)
public void testRackAwareAssignment(ClusterInstance clusterInstance) throws ExecutionException, InterruptedException {
String topic = "test-topic";
try (Admin admin = clusterInstance.admin();
Producer<byte[], byte[]> producer = clusterInstance.producer();
Consumer<byte[], byte[]> consumer0 = clusterInstance.consumer(Map.of(
ConsumerConfig.GROUP_ID_CONFIG, "group0",
ConsumerConfig.CLIENT_RACK_CONFIG, "rack0",
ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CONSUMER.name()
));
Consumer<byte[], byte[]> consumer1 = clusterInstance.consumer(Map.of(
ConsumerConfig.GROUP_ID_CONFIG, "group0",
ConsumerConfig.CLIENT_RACK_CONFIG, "rack1",
ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CONSUMER.name()
));
Consumer<byte[], byte[]> consumer2 = clusterInstance.consumer(Map.of(
ConsumerConfig.GROUP_ID_CONFIG, "group0",
ConsumerConfig.CLIENT_RACK_CONFIG, "rack2",
ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CONSUMER.name()
))
) {
// Create a new topic with 1 partition on broker 0.
admin.createTopics(List.of(new NewTopic(topic, Map.of(0, List.of(0)))));
clusterInstance.waitForTopic(topic, 1);

producer.send(new ProducerRecord<>(topic, "key".getBytes(), "value".getBytes()));
producer.flush();

consumer0.subscribe(List.of(topic));
consumer1.subscribe(List.of(topic));
consumer2.subscribe(List.of(topic));

TestUtils.waitForCondition(() -> {
consumer0.poll(Duration.ofMillis(1000));
consumer1.poll(Duration.ofMillis(1000));
consumer2.poll(Duration.ofMillis(1000));
return consumer0.assignment().equals(Set.of(new TopicPartition(topic, 0))) &&
consumer1.assignment().isEmpty() &&
consumer2.assignment().isEmpty();
}, "Consumer 0 should be assigned to topic partition 0");

// Add a new partition 1 and 2 to broker 1.
admin.createPartitions(
Map.of(
topic,
NewPartitions.increaseTo(3, List.of(List.of(1), List.of(1)))
)
);
clusterInstance.waitForTopic(topic, 3);
TestUtils.waitForCondition(() -> {
consumer0.poll(Duration.ofMillis(1000));
consumer1.poll(Duration.ofMillis(1000));
consumer2.poll(Duration.ofMillis(1000));
return consumer0.assignment().equals(Set.of(new TopicPartition(topic, 0))) &&
consumer1.assignment().equals(Set.of(new TopicPartition(topic, 1), new TopicPartition(topic, 2))) &&
consumer2.assignment().isEmpty();
}, "Consumer 1 should be assigned to topic partition 1 and 2");

// Add a new partition 3, 4, and 5 to broker 2.
admin.createPartitions(
Map.of(
topic,
NewPartitions.increaseTo(6, List.of(List.of(2), List.of(2), List.of(2)))
)
);
clusterInstance.waitForTopic(topic, 6);
TestUtils.waitForCondition(() -> {
consumer0.poll(Duration.ofMillis(1000));
consumer1.poll(Duration.ofMillis(1000));
consumer2.poll(Duration.ofMillis(1000));
return consumer0.assignment().equals(Set.of(new TopicPartition(topic, 0))) &&
consumer1.assignment().equals(Set.of(new TopicPartition(topic, 1), new TopicPartition(topic, 2))) &&
consumer2.assignment().equals(Set.of(new TopicPartition(topic, 3), new TopicPartition(topic, 4), new TopicPartition(topic, 5)));
}, "Consumer 2 should be assigned to topic partition 3, 4, and 5");

// Change partitions to different brokers.
// partition 0 -> broker 2
// partition 1 -> broker 2
// partition 2 -> broker 2
// partition 3 -> broker 1
// partition 4 -> broker 1
// partition 5 -> broker 0
admin.alterPartitionReassignments(Map.of(
new TopicPartition(topic, 0), Optional.of(new NewPartitionReassignment(List.of(2))),
new TopicPartition(topic, 1), Optional.of(new NewPartitionReassignment(List.of(2))),
new TopicPartition(topic, 2), Optional.of(new NewPartitionReassignment(List.of(2))),
new TopicPartition(topic, 3), Optional.of(new NewPartitionReassignment(List.of(1))),
new TopicPartition(topic, 4), Optional.of(new NewPartitionReassignment(List.of(1))),
new TopicPartition(topic, 5), Optional.of(new NewPartitionReassignment(List.of(0)))
)).all().get();
TestUtils.waitForCondition(() -> {
consumer0.poll(Duration.ofMillis(1000));
consumer1.poll(Duration.ofMillis(1000));
consumer2.poll(Duration.ofMillis(1000));
return consumer0.assignment().equals(Set.of(new TopicPartition(topic, 5))) &&
consumer1.assignment().equals(Set.of(new TopicPartition(topic, 3), new TopicPartition(topic, 4))) &&
consumer2.assignment().equals(Set.of(new TopicPartition(topic, 0), new TopicPartition(topic, 1), new TopicPartition(topic, 2)));
}, "Consumer with topic partition mapping should be 0 -> 5 | 1 -> 3, 4 | 2 -> 0, 1, 2");
}
}

private void sendMsg(ClusterInstance clusterInstance, String topic, int sendMsgNum) {
try (var producer = clusterInstance.producer(Map.of(
ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class,
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
/*
* 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.clients.consumer;

import org.apache.kafka.common.Uuid;
import org.apache.kafka.coordinator.group.api.assignor.ConsumerGroupPartitionAssignor;
import org.apache.kafka.coordinator.group.api.assignor.GroupAssignment;
import org.apache.kafka.coordinator.group.api.assignor.GroupSpec;
import org.apache.kafka.coordinator.group.api.assignor.MemberAssignment;
import org.apache.kafka.coordinator.group.api.assignor.PartitionAssignorException;
import org.apache.kafka.coordinator.group.api.assignor.SubscribedTopicDescriber;
import org.apache.kafka.coordinator.group.modern.MemberAssignmentImpl;

import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;

/**
* The RackAwareAssignor is a consumer group partition assignor that takes into account the rack
* information of the members when assigning partitions to them.
* It needs all brokers and members to have rack information available.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It also requires all member to subscribe same topics, right?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, this only checks subscription of first member and assume all members use same subscription. This is a sample assignor under test module. Do we want to make it support heterogeneous group?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we want to make it support heterogeneous group?

no, it is used by testing only so it should be fine.

*/
public class RackAwareAssignor implements ConsumerGroupPartitionAssignor {
@Override
public String name() {
return "rack-aware-assignor";
}

@Override
public GroupAssignment assign(GroupSpec groupSpec, SubscribedTopicDescriber subscribedTopicDescriber) throws PartitionAssignorException {
Map<String, String> rackIdToMemberId = new HashMap<>();
List<String> memberIds = new ArrayList<>(groupSpec.memberIds());
for (String memberId : memberIds) {
if (groupSpec.memberSubscription(memberId).rackId().isEmpty()) {
throw new PartitionAssignorException("Member " + memberId + " does not have rack information available.");
}
rackIdToMemberId.put(
groupSpec.memberSubscription(memberId).rackId().get(),
memberId
);
}

Map<String, Map<Uuid, Set<Integer>>> assignments = new HashMap<>();
for (Uuid topicId : groupSpec.memberSubscription(memberIds.get(0)).subscribedTopicIds()) {
int numPartitions = subscribedTopicDescriber.numPartitions(topicId);
if (numPartitions == -1) {
throw new PartitionAssignorException("Member is subscribed to a non-existent topic");
}

for (int partitionId = 0; partitionId < numPartitions; partitionId++) {
Set<String> racks = subscribedTopicDescriber.racksForPartition(topicId, partitionId);
if (racks.isEmpty()) {
throw new PartitionAssignorException("No racks available for partition " + partitionId + " of topic " + topicId);
}

String assignedRack = null;
for (String rack : racks) {
String memberId = rackIdToMemberId.get(rack);
if (memberId == null) {
continue;
}
assignedRack = rack;
break;
}

if (assignedRack == null) {
throw new PartitionAssignorException("No member found for racks " + racks + " for partition " + partitionId + " of topic " + topicId);
}

Map<Uuid, Set<Integer>> assignment = assignments.computeIfAbsent(
rackIdToMemberId.get(assignedRack),
k -> new HashMap<>()
);
Set<Integer> partitions = assignment.computeIfAbsent(
topicId,
k -> new java.util.HashSet<>()
);
partitions.add(partitionId);
}
}

Map<String, MemberAssignment> memberAssignments = new HashMap<>();
for (Map.Entry<String, Map<Uuid, Set<Integer>>> entry : assignments.entrySet()) {
memberAssignments.put(entry.getKey(), new MemberAssignmentImpl(entry.getValue()));
}
return new GroupAssignment(memberAssignments);
}
}
Original file line number Diff line number Diff line change
@@ -43,6 +43,7 @@ public String toString() {
public final int heartbeatIntervalMs;
public final String groupId;
public final Optional<String> groupInstanceId;
public final Optional<String> rackId;
public final long retryBackoffMs;
public final long retryBackoffMaxMs;
public final boolean leaveGroupOnClose;
@@ -53,8 +54,12 @@ public GroupRebalanceConfig(AbstractConfig config, ProtocolType protocolType) {
// Consumer and Connect use different config names for defining rebalance timeout
if ((protocolType == ProtocolType.CONSUMER) || (protocolType == ProtocolType.SHARE)) {
this.rebalanceTimeoutMs = config.getInt(CommonClientConfigs.MAX_POLL_INTERVAL_MS_CONFIG);

String rackId = config.getString(CommonClientConfigs.CLIENT_RACK_CONFIG);
this.rackId = rackId == null || rackId.isEmpty() ? Optional.empty() : Optional.of(rackId);
} else {
this.rebalanceTimeoutMs = config.getInt(CommonClientConfigs.REBALANCE_TIMEOUT_MS_CONFIG);
this.rackId = Optional.empty();
}

this.heartbeatIntervalMs = config.getInt(CommonClientConfigs.HEARTBEAT_INTERVAL_MS_CONFIG);
@@ -90,6 +95,7 @@ public GroupRebalanceConfig(final int sessionTimeoutMs,
final int heartbeatIntervalMs,
String groupId,
Optional<String> groupInstanceId,
String rackId,
long retryBackoffMs,
long retryBackoffMaxMs,
boolean leaveGroupOnClose) {
@@ -98,6 +104,7 @@ public GroupRebalanceConfig(final int sessionTimeoutMs,
this.heartbeatIntervalMs = heartbeatIntervalMs;
this.groupId = groupId;
this.groupInstanceId = groupInstanceId;
this.rackId = rackId == null || rackId.isEmpty() ? Optional.empty() : Optional.of(rackId);
this.retryBackoffMs = retryBackoffMs;
this.retryBackoffMaxMs = retryBackoffMaxMs;
this.leaveGroupOnClose = leaveGroupOnClose;
Original file line number Diff line number Diff line change
@@ -230,7 +230,6 @@ public class ClassicKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG),
this.interceptors,
config.getBoolean(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED),
config.getString(ConsumerConfig.CLIENT_RACK_CONFIG),
clientTelemetryReporter);
}
this.fetcher = new Fetcher<>(
@@ -330,6 +329,7 @@ public class ClassicKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
heartbeatIntervalMs,
groupId.get(),
groupInstanceId,
rackId,
retryBackoffMs,
retryBackoffMaxMs,
true
@@ -348,7 +348,6 @@ public class ClassicKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
autoCommitIntervalMs,
interceptors,
throwOnStableOffsetNotSupported,
rackId,
clientTelemetryReporter
);
} else {
Original file line number Diff line number Diff line change
@@ -178,7 +178,6 @@ public ConsumerCoordinator(GroupRebalanceConfig rebalanceConfig,
int autoCommitIntervalMs,
ConsumerInterceptors<?, ?> interceptors,
boolean throwOnFetchStableOffsetsUnsupported,
String rackId,
Optional<ClientTelemetryReporter> clientTelemetryReporter) {
this(rebalanceConfig,
logContext,
@@ -193,7 +192,6 @@ public ConsumerCoordinator(GroupRebalanceConfig rebalanceConfig,
autoCommitIntervalMs,
interceptors,
throwOnFetchStableOffsetsUnsupported,
rackId,
clientTelemetryReporter,
Optional.empty());
}
@@ -214,7 +212,6 @@ public ConsumerCoordinator(GroupRebalanceConfig rebalanceConfig,
int autoCommitIntervalMs,
ConsumerInterceptors<?, ?> interceptors,
boolean throwOnFetchStableOffsetsUnsupported,
String rackId,
Optional<ClientTelemetryReporter> clientTelemetryReporter,
Optional<Supplier<BaseHeartbeatThread>> heartbeatThreadSupplier) {
super(rebalanceConfig,
@@ -228,7 +225,7 @@ public ConsumerCoordinator(GroupRebalanceConfig rebalanceConfig,
this.rebalanceConfig = rebalanceConfig;
this.log = logContext.logger(ConsumerCoordinator.class);
this.metadata = metadata;
this.rackId = rackId == null || rackId.isEmpty() ? Optional.empty() : Optional.of(rackId);
this.rackId = rebalanceConfig.rackId;
this.metadataSnapshot = new MetadataSnapshot(this.rackId, subscriptions, metadata.fetch(), metadata.updateVersion());
this.subscriptions = subscriptions;
this.defaultOffsetCommitCallback = new DefaultOffsetCommitCallback();
Original file line number Diff line number Diff line change
@@ -247,6 +247,7 @@ public void reset() {
sentFields.reset();
}

@SuppressWarnings("NPathComplexity")
public ConsumerGroupHeartbeatRequestData buildRequestData() {
ConsumerGroupHeartbeatRequestData data = new ConsumerGroupHeartbeatRequestData();

@@ -306,6 +307,12 @@ public ConsumerGroupHeartbeatRequestData buildRequestData() {
sentFields.localAssignment = local;
}

// RackId - sent when joining
String rackId = membershipManager.rackId().orElse(null);
if (sendAllFields) {
data.setRackId(rackId);
}

return data;
}

Loading
Oops, something went wrong.