guozhangwang commented on code in PR #13021: URL: https://github.com/apache/kafka/pull/13021#discussion_r1096350392
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java: ########## @@ -0,0 +1,224 @@ +/* + * 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.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.Objects; +import java.util.Optional; +import java.util.Queue; +import java.util.concurrent.CompletableFuture; +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 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) { + Objects.requireNonNull(coordinatorRequestManager, "Coordinator is needed upon committing offsets"); + 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 = coordinatorRequestManager; + this.groupState = groupState; + this.subscriptionState = subscriptionState; + } + + /** + * 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) { + 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 CompletableFuture<ClientResponse> add(final Map<TopicPartition, OffsetAndMetadata> offsets) { + StagedCommit commit = new StagedCommit( + offsets, + groupState.groupId, + groupState.groupInstanceId.orElse(null), + groupState.generation); + this.stagedCommits.add(commit); + return commit.future(); + } + + 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); + sendAutoCommit(allConsumedOffsets); Review Comment: I'm wondering if we should reset immediately or should we have a flag of `autoCommitInFlight` and only reset to re-enable `canSendAutocommit`, because I vaguely remember in the past, we have seen issues where auto commit keeps being triggered while there are still auto commits inflight due to network partition, causing OOM and other issues. And that's why we ended up adding this flag in the old code. If we want to go very fancy, we can potentially just update the unsent auto commit request inside the `stagedCommits` when we want to send another.. -- 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