guozhangwang commented on code in PR #13303: URL: https://github.com/apache/kafka/pull/13303#discussion_r1127185285
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java: ########## @@ -213,7 +215,7 @@ public UnsentRequest( Objects.requireNonNull(requestBuilder); this.requestBuilder = requestBuilder; this.node = node; - this.callback = new FutureCompletionHandler(); Review Comment: nit: just call the member field `handler` as well? ########## clients/src/test/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumerTest.java: ########## @@ -16,90 +16,77 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.OffsetResetStrategy; -import org.apache.kafka.clients.consumer.internals.events.EventHandler; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.internals.ClusterResourceListeners; -import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.mockito.Mockito; +import java.time.Duration; import java.util.HashMap; import java.util.Map; -import java.util.Optional; -import static java.util.Collections.singleton; -import static org.apache.kafka.clients.consumer.ConsumerConfig.CLIENT_ID_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; -import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; public class PrototypeAsyncConsumerTest { - private Map<String, Object> properties; - private SubscriptionState subscriptionState; - private MockTime time; - private LogContext logContext; - private Metrics metrics; - private ClusterResourceListeners clusterResourceListeners; - private Optional<String> groupId; - private String clientId; - private EventHandler eventHandler; + + private Consumer<?, ?> consumer; + private Map<String, Object> consumerProps = new HashMap<>(); + + private final Time time = new MockTime(); @BeforeEach public void setup() { - this.subscriptionState = Mockito.mock(SubscriptionState.class); - this.eventHandler = Mockito.mock(DefaultEventHandler.class); - this.logContext = new LogContext(); - this.time = new MockTime(); - this.metrics = new Metrics(time); - this.groupId = Optional.empty(); - this.clientId = "client-1"; - this.clusterResourceListeners = new ClusterResourceListeners(); - this.properties = new HashMap<>(); - this.properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, - "localhost" + - ":9999"); - this.properties.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - this.properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - this.properties.put(CLIENT_ID_CONFIG, "test-client"); + injectConsumerConfigs(); + } + + @AfterEach + public void cleanup() { + if (consumer != null) { + consumer.close(Duration.ZERO); + } } + @Test - public void testSubscription() { - this.subscriptionState = - new SubscriptionState(new LogContext(), OffsetResetStrategy.EARLIEST); - PrototypeAsyncConsumer<String, String> consumer = - setupConsumerWithDefault(); - subscriptionState.subscribe(singleton("t1"), - new NoOpConsumerRebalanceListener()); - assertEquals(1, consumer.subscription().size()); + public void testBackgroundThreadRunning() { + consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); Review Comment: +1 ########## core/src/test/scala/integration/kafka/api/BaseAsyncConsumerTest.scala: ########## @@ -0,0 +1,37 @@ +/* + * 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 kafka.api + +import kafka.utils.TestUtils.waitUntilTrue +import org.junit.jupiter.api.Test + +class BaseAsyncConsumerTest extends AbstractConsumerTest { + @Test + def testCommitAPI(): Unit = { + val consumer = createAsyncConsumer() + val producer = createProducer() + val numRecords = 10000 + val startingTimestamp = System.currentTimeMillis() + val cb = new CountConsumerCommitCallback + sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) + consumer.commitAsync(cb) + waitUntilTrue(() => { + cb.successCount == 1 + }, "wait until commit is completed successfully", 5000) + consumer.commitSync(); Review Comment: Why call `commitSync` here again? If the goal is to make sure the second commit also goes through shall we verify that on the broker's side? -- 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