You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2023/01/19 00:22:03 UTC

[GitHub] [kafka] hachikuji commented on a diff in pull request #13021: KAFKA-14468: Implement CommitRequestManager to manage the commit and autocommit requests

hachikuji commented on code in PR #13021:
URL: https://github.com/apache/kafka/pull/13021#discussion_r1080689650


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.ClientResponse;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.clients.consumer.RetriableCommitFailedException;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.OffsetCommitRequestData;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.requests.OffsetCommitRequest;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Timer;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+public class CommitRequestManager implements RequestManager {
+    private final Queue<StagedCommit> stagedCommits;
+    // TODO: We will need to refactor the subscriptionState
+    private final SubscriptionState subscriptionState;
+    private final Logger log;
+    private final Optional<AutoCommitState> autoCommitState;
+    private final Optional<CoordinatorRequestManager> coordinatorRequestManager;
+    private final GroupStateManager groupState;
+
+    public CommitRequestManager(
+            final Time time,
+            final LogContext logContext,
+            final SubscriptionState subscriptionState,
+            final ConsumerConfig config,
+            final CoordinatorRequestManager coordinatorRequestManager,
+            final GroupStateManager groupState) {
+        this.log = logContext.logger(getClass());
+        this.stagedCommits = new LinkedList<>();
+        if (config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) {
+            final long autoCommitInterval =
+                    Integer.toUnsignedLong(config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG));
+            this.autoCommitState = Optional.of(new AutoCommitState(time, autoCommitInterval));
+        } else {
+            this.autoCommitState = Optional.empty();
+        }
+        this.coordinatorRequestManager = Optional.ofNullable(coordinatorRequestManager);
+        this.groupState = groupState;
+        this.subscriptionState = subscriptionState;
+    }
+
+    // Visible for testing
+    CommitRequestManager(
+            final Time time,
+            final LogContext logContext,
+            final SubscriptionState subscriptionState,
+            final ConsumerConfig config,
+            final CoordinatorRequestManager coordinatorRequestManager,
+            final GroupStateManager groupState,
+            final AutoCommitState autoCommitState) {
+        this.log = logContext.logger(getClass());
+        this.subscriptionState = subscriptionState;
+        this.coordinatorRequestManager = Optional.ofNullable(coordinatorRequestManager);
+        this.groupState = groupState;
+        this.autoCommitState = Optional.ofNullable(autoCommitState);
+        this.stagedCommits = new LinkedList<>();
+    }
+
+    /**
+     * Poll for the commit request if there's any. The function will also try to autocommit, if enabled.
+     *
+     * @param currentTimeMs
+     * @return
+     */
+    @Override
+    public NetworkClientDelegate.PollResult poll(final long currentTimeMs) {
+        if (!coordinatorRequestManager.isPresent()) {
+            return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, new ArrayList<>());
+        }
+
+        maybeAutoCommit(currentTimeMs);
+
+        if (stagedCommits.isEmpty()) {
+            return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, new ArrayList<>());
+        }
+
+        List<NetworkClientDelegate.UnsentRequest> unsentCommitRequests =
+                stagedCommits.stream().map(StagedCommit::toUnsentRequest).collect(Collectors.toList());
+        return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, Collections.unmodifiableList(unsentCommitRequests));
+    }
+
+    public RequestFuture<ClientResponse> add(final Map<TopicPartition, OffsetAndMetadata> offsets,
+                                             final Optional<OffsetCommitCallback> callback) {

Review Comment:
   I think we can simplify this API by removing the callback argument. Instead, if we let this API return `CompletableFuture` as suggested in another comment, then we can chain the callback logic to the future when invoked from `KafkaConsumer`. 



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.ClientResponse;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.clients.consumer.RetriableCommitFailedException;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.OffsetCommitRequestData;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.requests.OffsetCommitRequest;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Timer;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+public class CommitRequestManager implements RequestManager {
+    private final Queue<StagedCommit> stagedCommits;
+    // TODO: We will need to refactor the subscriptionState
+    private final SubscriptionState subscriptionState;
+    private final Logger log;
+    private final Optional<AutoCommitState> autoCommitState;
+    private final Optional<CoordinatorRequestManager> coordinatorRequestManager;
+    private final GroupStateManager groupState;
+
+    public CommitRequestManager(
+            final Time time,
+            final LogContext logContext,
+            final SubscriptionState subscriptionState,
+            final ConsumerConfig config,
+            final CoordinatorRequestManager coordinatorRequestManager,
+            final GroupStateManager groupState) {
+        this.log = logContext.logger(getClass());
+        this.stagedCommits = new LinkedList<>();
+        if (config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) {
+            final long autoCommitInterval =
+                    Integer.toUnsignedLong(config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG));
+            this.autoCommitState = Optional.of(new AutoCommitState(time, autoCommitInterval));
+        } else {
+            this.autoCommitState = Optional.empty();
+        }
+        this.coordinatorRequestManager = Optional.ofNullable(coordinatorRequestManager);
+        this.groupState = groupState;
+        this.subscriptionState = subscriptionState;
+    }
+
+    // Visible for testing
+    CommitRequestManager(
+            final Time time,
+            final LogContext logContext,
+            final SubscriptionState subscriptionState,
+            final ConsumerConfig config,
+            final CoordinatorRequestManager coordinatorRequestManager,
+            final GroupStateManager groupState,
+            final AutoCommitState autoCommitState) {
+        this.log = logContext.logger(getClass());
+        this.subscriptionState = subscriptionState;
+        this.coordinatorRequestManager = Optional.ofNullable(coordinatorRequestManager);
+        this.groupState = groupState;
+        this.autoCommitState = Optional.ofNullable(autoCommitState);
+        this.stagedCommits = new LinkedList<>();
+    }
+
+    /**
+     * Poll for the commit request if there's any. The function will also try to autocommit, if enabled.
+     *
+     * @param currentTimeMs
+     * @return
+     */
+    @Override
+    public NetworkClientDelegate.PollResult poll(final long currentTimeMs) {
+        if (!coordinatorRequestManager.isPresent()) {
+            return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, new ArrayList<>());
+        }
+
+        maybeAutoCommit(currentTimeMs);
+
+        if (stagedCommits.isEmpty()) {
+            return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, new ArrayList<>());
+        }
+
+        List<NetworkClientDelegate.UnsentRequest> unsentCommitRequests =
+                stagedCommits.stream().map(StagedCommit::toUnsentRequest).collect(Collectors.toList());
+        return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, Collections.unmodifiableList(unsentCommitRequests));
+    }
+
+    public RequestFuture<ClientResponse> add(final Map<TopicPartition, OffsetAndMetadata> offsets,

Review Comment:
   Can we use a normal `CompletableFuture` instead of `RequestFuture`. The latter is bound up with `ConsumerNetworkClient` which we are not using in this implementation.
   
   Also, I'm surprised to see this return something as low level as `ClientResponse`. I think we can leave the response handling confined to this class and just expose `CompletableFuture<Void>` or something like that. 



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.ClientResponse;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.clients.consumer.RetriableCommitFailedException;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.OffsetCommitRequestData;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.requests.OffsetCommitRequest;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Timer;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+public class CommitRequestManager implements RequestManager {
+    private final Queue<StagedCommit> stagedCommits;
+    // TODO: We will need to refactor the subscriptionState
+    private final SubscriptionState subscriptionState;
+    private final Logger log;
+    private final Optional<AutoCommitState> autoCommitState;
+    private final Optional<CoordinatorRequestManager> coordinatorRequestManager;

Review Comment:
   Why would this be optional? We cannot commit offsets unless we have a coordinator.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.ClientResponse;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.clients.consumer.RetriableCommitFailedException;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.OffsetCommitRequestData;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.requests.OffsetCommitRequest;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Timer;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+public class CommitRequestManager implements RequestManager {
+    private final Queue<StagedCommit> stagedCommits;
+    // TODO: We will need to refactor the subscriptionState
+    private final SubscriptionState subscriptionState;

Review Comment:
   I think the main problem with using the current `SubscriptionState` is that it couples together the commit position and the fetch position. With the move to the background thread, I don't think we can depend on these advancing in lock-step anymore. The fetch position can advance after we have received a fetch position and the commit position will be advanced when the application calls `poll()`. With that said, it is probably simpler to build this into the current `SubscriptionState` instead of creating something new.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.ClientResponse;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.clients.consumer.RetriableCommitFailedException;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.OffsetCommitRequestData;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.requests.OffsetCommitRequest;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Timer;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+public class CommitRequestManager implements RequestManager {
+    private final Queue<StagedCommit> stagedCommits;
+    // TODO: We will need to refactor the subscriptionState
+    private final SubscriptionState subscriptionState;
+    private final Logger log;
+    private final Optional<AutoCommitState> autoCommitState;

Review Comment:
   The optional is kind of annoying. I wonder if we could treat auto-commit disabled as having an auto-commit interval of `Long.MaxVAlue`.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.ClientResponse;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.clients.consumer.RetriableCommitFailedException;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.OffsetCommitRequestData;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.requests.OffsetCommitRequest;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Timer;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+public class CommitRequestManager implements RequestManager {
+    private final Queue<StagedCommit> stagedCommits;
+    // TODO: We will need to refactor the subscriptionState
+    private final SubscriptionState subscriptionState;
+    private final Logger log;
+    private final Optional<AutoCommitState> autoCommitState;
+    private final Optional<CoordinatorRequestManager> coordinatorRequestManager;
+    private final GroupStateManager groupState;
+
+    public CommitRequestManager(
+            final Time time,
+            final LogContext logContext,
+            final SubscriptionState subscriptionState,
+            final ConsumerConfig config,
+            final CoordinatorRequestManager coordinatorRequestManager,
+            final GroupStateManager groupState) {
+        this.log = logContext.logger(getClass());
+        this.stagedCommits = new LinkedList<>();
+        if (config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) {
+            final long autoCommitInterval =
+                    Integer.toUnsignedLong(config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG));
+            this.autoCommitState = Optional.of(new AutoCommitState(time, autoCommitInterval));
+        } else {
+            this.autoCommitState = Optional.empty();
+        }
+        this.coordinatorRequestManager = Optional.ofNullable(coordinatorRequestManager);
+        this.groupState = groupState;
+        this.subscriptionState = subscriptionState;
+    }
+
+    // Visible for testing
+    CommitRequestManager(
+            final Time time,
+            final LogContext logContext,
+            final SubscriptionState subscriptionState,
+            final ConsumerConfig config,
+            final CoordinatorRequestManager coordinatorRequestManager,
+            final GroupStateManager groupState,
+            final AutoCommitState autoCommitState) {
+        this.log = logContext.logger(getClass());
+        this.subscriptionState = subscriptionState;
+        this.coordinatorRequestManager = Optional.ofNullable(coordinatorRequestManager);
+        this.groupState = groupState;
+        this.autoCommitState = Optional.ofNullable(autoCommitState);
+        this.stagedCommits = new LinkedList<>();
+    }
+
+    /**
+     * Poll for the commit request if there's any. The function will also try to autocommit, if enabled.
+     *
+     * @param currentTimeMs
+     * @return
+     */
+    @Override
+    public NetworkClientDelegate.PollResult poll(final long currentTimeMs) {
+        if (!coordinatorRequestManager.isPresent()) {
+            return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, new ArrayList<>());
+        }
+
+        maybeAutoCommit(currentTimeMs);
+
+        if (stagedCommits.isEmpty()) {
+            return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, new ArrayList<>());
+        }
+
+        List<NetworkClientDelegate.UnsentRequest> unsentCommitRequests =
+                stagedCommits.stream().map(StagedCommit::toUnsentRequest).collect(Collectors.toList());
+        return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, Collections.unmodifiableList(unsentCommitRequests));
+    }
+
+    public RequestFuture<ClientResponse> add(final Map<TopicPartition, OffsetAndMetadata> offsets,
+                                             final Optional<OffsetCommitCallback> callback) {
+        OffsetCommitRequestHandler handler = new OffsetCommitRequestHandler(offsets, callback.orElse(null));
+        this.add(offsets, handler);
+        return handler.future();
+    }
+
+    public RequestFuture<ClientResponse> add(final Map<TopicPartition, OffsetAndMetadata> offsets,
+                                             final NetworkClientDelegate.AbstractRequestFutureCompletionHandler handler) {
+        this.stagedCommits.add(new StagedCommit(
+                offsets,
+                handler,
+                groupState.groupId,
+                groupState.groupInstanceId.orElse(null),
+                groupState.generation));
+        return handler.future();
+    }
+
+    // TODO: I think we might need to return the callback because we will need to send autocommit and wait for its
+    //  completion before joinGroup. Alternatively, we could try to change the join group state upon success or
+    //  failure.
+    private void maybeAutoCommit(final long currentTimeMs) {
+        if (!autoCommitState.isPresent()) {
+            return;
+        }
+
+        AutoCommitState autocommit = autoCommitState.get();
+        if (!autocommit.canSendAutocommit(currentTimeMs)) {
+            return;
+        }
+
+        Map<TopicPartition, OffsetAndMetadata> allConsumedOffsets = subscriptionState.allConsumed();
+        log.debug("Auto-committing offsets {}", allConsumedOffsets);
+        this.add(allConsumedOffsets, new AutoCommitRequestHandler(allConsumedOffsets));
+        autocommit.reset();
+    }
+
+    public void clientPoll(final long currentTimeMs) {
+        this.autoCommitState.ifPresent(t -> t.ack(currentTimeMs));
+    }
+
+    private class DefaultOffsetCommitCallback implements OffsetCommitCallback {
+        @Override
+        public void onComplete(Map<TopicPartition, OffsetAndMetadata> offsets, Exception exception) {
+            if (exception != null)
+                log.error("Offset commit with offsets {} failed", offsets, exception);
+        }
+    }
+
+    private class AutoCommitRequestHandler extends OffsetCommitRequestHandler {
+        public AutoCommitRequestHandler(final Map<TopicPartition, OffsetAndMetadata> offsets) {
+            super(offsets, null);
+        }
+
+        @Override
+        public void handleResponse(ClientResponse r, Exception e) {
+            if (e != null) {

Review Comment:
   Stuff like this is better to just chain on top of a future.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.ClientResponse;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.clients.consumer.RetriableCommitFailedException;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.OffsetCommitRequestData;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.requests.OffsetCommitRequest;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Timer;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+public class CommitRequestManager implements RequestManager {
+    private final Queue<StagedCommit> stagedCommits;
+    // TODO: We will need to refactor the subscriptionState
+    private final SubscriptionState subscriptionState;
+    private final Logger log;
+    private final Optional<AutoCommitState> autoCommitState;
+    private final Optional<CoordinatorRequestManager> coordinatorRequestManager;
+    private final GroupStateManager groupState;
+
+    public CommitRequestManager(
+            final Time time,
+            final LogContext logContext,
+            final SubscriptionState subscriptionState,
+            final ConsumerConfig config,
+            final CoordinatorRequestManager coordinatorRequestManager,
+            final GroupStateManager groupState) {
+        this.log = logContext.logger(getClass());
+        this.stagedCommits = new LinkedList<>();
+        if (config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) {
+            final long autoCommitInterval =
+                    Integer.toUnsignedLong(config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG));
+            this.autoCommitState = Optional.of(new AutoCommitState(time, autoCommitInterval));
+        } else {
+            this.autoCommitState = Optional.empty();
+        }
+        this.coordinatorRequestManager = Optional.ofNullable(coordinatorRequestManager);
+        this.groupState = groupState;
+        this.subscriptionState = subscriptionState;
+    }
+
+    // Visible for testing
+    CommitRequestManager(
+            final Time time,

Review Comment:
   nit: this and `config` are unused



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org