chia7712 commented on code in PR #21396:
URL: https://github.com/apache/kafka/pull/21396#discussion_r2761706248
##########
coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java:
##########
@@ -710,6 +710,87 @@ public void
testScheduleUnloadingWithDeferredEventExceptions() throws ExecutionE
assertThrows(NotCoordinatorException.class, () ->
runtime.contextOrThrow(TP));
}
+ @Test
+ public void
testScheduleUnloadingWithPendingBatchWhenPartitionWriterConfigThrows() {
Review Comment:
I'm thinking about adding integration tests. For example:
```java
@ClusterTest(
brokers = 2,
types = {Type.KRAFT},
serverProperties = {
@ClusterConfigProperty(key =
GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"),
@ClusterConfigProperty(key =
"group.coordinator.append.linger.ms", value = "3000")
}
)
public void test(ClusterInstance clusterInstance) throws
InterruptedException, ExecutionException, TimeoutException {
try (var producer = clusterInstance.<byte[], byte[]>producer()) {
producer.send(new ProducerRecord<>("topic",
"value".getBytes(StandardCharsets.UTF_8)));
}
try (var admin = clusterInstance.admin()) {
admin.createTopics(List.of(new
NewTopic(Topic.GROUP_METADATA_TOPIC_NAME, Map.of(0, List.of(0))))).all().get();
}
try (var consumer =
clusterInstance.consumer(Map.of(ConsumerConfig.GROUP_ID_CONFIG, "test-group"));
var admin = clusterInstance.admin()) {
consumer.subscribe(List.of("topic"));
while (consumer.poll(Duration.ofMillis(100)).isEmpty()) {
// empty body
}
// append records to coordinator
consumer.commitSync();
// unload the coordinator by changing leader (0 -> 1)
admin.alterPartitionReassignments(Map.of(new
TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0),
Optional.of(new
NewPartitionReassignment(List.of(1))))).all().get();
}
Function<GroupCoordinator, List<TopicPartition>>
partitionsInGroupMetrics = service -> assertDoesNotThrow(() -> {
var f0 =
GroupCoordinatorService.class.getDeclaredField("groupCoordinatorMetrics");
f0.setAccessible(true);
var f1 =
GroupCoordinatorMetrics.class.getDeclaredField("shards");
f1.setAccessible(true);
return List.copyOf(((Map<TopicPartition, ?>)
f1.get(f0.get(service))).keySet());
});
// the offset partition should NOT be hosted by multiple coordinators
var tps = clusterInstance.brokers().values().stream()
.flatMap(b ->
partitionsInGroupMetrics.apply(b.groupCoordinator()).stream()).toList();
assertEquals(1, tps.size());
}
```
WDYT?
--
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]