lucasbru commented on code in PR #15426: URL: https://github.com/apache/kafka/pull/15426#discussion_r1504029285
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ########## @@ -293,6 +276,23 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { private final AtomicLong currentThread = new AtomicLong(NO_CURRENT_THREAD); private final AtomicInteger refCount = new AtomicInteger(0); + private final InvalidGroupIdException invalidGroupIdException = + new InvalidGroupIdException("To use the group management or offset commit APIs, you must " + + "provide a valid " + ConsumerConfig.GROUP_ID_CONFIG + " in the consumer configuration."); + + private void updateGroupMetadata(final Optional<Integer> memberEpoch, final Optional<String> memberId) { Review Comment: Please move this method. We shouldn't put private methods before the constructor. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ########## @@ -293,6 +276,23 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { private final AtomicLong currentThread = new AtomicLong(NO_CURRENT_THREAD); private final AtomicInteger refCount = new AtomicInteger(0); + private final InvalidGroupIdException invalidGroupIdException = + new InvalidGroupIdException("To use the group management or offset commit APIs, you must " + + "provide a valid " + ConsumerConfig.GROUP_ID_CONFIG + " in the consumer configuration."); + + private void updateGroupMetadata(final Optional<Integer> memberEpoch, final Optional<String> memberId) { + groupMetadata.updateAndGet( Review Comment: Can we not just use a `volatile` variable here? We only have the background thread updating the epoch and the ID, with the exception of the shutdown procedure in the application thread. A race between background thread and application thread however doesn't really seem to be solved by using `updateAndGet` (that would require checking if the consumer group metadata is "closed" here). ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ########## @@ -1186,8 +1181,7 @@ public OptionalLong currentLag(TopicPartition topicPartition) { public ConsumerGroupMetadata groupMetadata() { acquireAndEnsureOpen(); try { - maybeThrowInvalidGroupIdException(); - return groupMetadata.get(); + return groupMetadata.get().orElseThrow(() -> invalidGroupIdException); Review Comment: I'd prefer calling `maybeThrowInvalidGroupIdException` here. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java: ########## @@ -1372,4 +1373,8 @@ public PollResult poll(final long currentTimeMs) { } return PollResult.EMPTY; } + + List<MemberStateListener> stateListeners() { Review Comment: // for testing only ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ########## @@ -293,6 +276,23 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { private final AtomicLong currentThread = new AtomicLong(NO_CURRENT_THREAD); private final AtomicInteger refCount = new AtomicInteger(0); + private final InvalidGroupIdException invalidGroupIdException = Review Comment: We will have one such instance for each consumer in memory at all times. Not a biggie, but I'd avoid it. ########## clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java: ########## @@ -0,0 +1,71 @@ +/* + * 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.internals; + +import org.apache.kafka.clients.ApiVersions; +import org.apache.kafka.clients.GroupRebalanceConfig; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.junit.jupiter.api.Test; + +import java.util.Optional; +import java.util.Properties; + +import static org.apache.kafka.test.TestUtils.requiredConsumerConfig; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; + +public class RequestManagersTest { + + @Test + public void testMemberStateListenerRegistered() { + + final MemberStateListener listener = (memberEpoch, memberId) -> { Review Comment: nit: I would not put a new-line after this -- 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: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org