dajac commented on code in PR #21396:
URL: https://github.com/apache/kafka/pull/21396#discussion_r2775605420
##########
clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java:
##########
@@ -335,6 +343,50 @@ public void testRackAwareAssignment(ClusterInstance
clusterInstance) throws Exec
}
}
+ @SuppressWarnings("unchecked")
+ @ClusterTest(
+ brokers = 2,
+ types = {Type.KRAFT},
+ serverProperties = {
+ @ClusterConfigProperty(key =
GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"),
+ @ClusterConfigProperty(key =
GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG, value =
"3000")
+ }
+ )
+ public void
testSingleCoordinatorOwnershipAfterPartitionReassignment(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"));
+ TestUtils.waitForCondition(() ->
consumer.poll(Duration.ofMillis(100)).isEmpty(), "polling to join group");
+ // 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();
+ }
Review Comment:
Should we wait until this operation completes?
##########
coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java:
##########
@@ -879,15 +880,23 @@ private void maybeFlushCurrentBatch(long currentTimeMs) {
}
}
+ private void failCurrentBatch(Throwable t) {
+ failCurrentBatch(t, true);
+ }
+
+ private void failCurrentBatchWithoutRelease(Throwable t) {
+ failCurrentBatch(t, false);
+ }
+
/**
* Fails the current batch, reverts to the snapshot to the base/start
offset of the
* batch, fails all the associated events.
*/
- private void failCurrentBatch(Throwable t) {
+ private void failCurrentBatch(Throwable t, boolean freeCurrentBatch) {
if (currentBatch != null) {
coordinator.revertLastWrittenOffset(currentBatch.baseOffset);
currentBatch.deferredEvents.complete(t);
- freeCurrentBatch();
+ if (freeCurrentBatch) freeCurrentBatch();
Review Comment:
I was discussing with @squah-confluent offline and we thought that caching
`maxBatchSize` when the batch is allocated may be easier as it would avoid
calling `partitionWriter.config(tp)` in `freeCurrentBatch`. It would also be
more aligned with trunk. What do you guys think?
##########
clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java:
##########
@@ -335,6 +343,50 @@ public void testRackAwareAssignment(ClusterInstance
clusterInstance) throws Exec
}
}
+ @SuppressWarnings("unchecked")
+ @ClusterTest(
+ brokers = 2,
+ types = {Type.KRAFT},
+ serverProperties = {
+ @ClusterConfigProperty(key =
GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"),
+ @ClusterConfigProperty(key =
GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG, value =
"3000")
+ }
+ )
+ public void
testSingleCoordinatorOwnershipAfterPartitionReassignment(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"));
+ TestUtils.waitForCondition(() ->
consumer.poll(Duration.ofMillis(100)).isEmpty(), "polling to join group");
+ // 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);
Review Comment:
I am not a fan of this. Could we rather query the actual exposed metrics?
##########
clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java:
##########
@@ -335,6 +343,50 @@ public void testRackAwareAssignment(ClusterInstance
clusterInstance) throws Exec
}
}
+ @SuppressWarnings("unchecked")
+ @ClusterTest(
+ brokers = 2,
+ types = {Type.KRAFT},
+ serverProperties = {
+ @ClusterConfigProperty(key =
GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"),
+ @ClusterConfigProperty(key =
GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG, value =
"3000")
+ }
+ )
+ public void
testSingleCoordinatorOwnershipAfterPartitionReassignment(ClusterInstance
clusterInstance) throws InterruptedException, ExecutionException,
TimeoutException {
Review Comment:
Does this test fail before the fix? I guess that it does not because the
unloading of the shard actually goes through. It does not hurt to have it
though.
--
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]