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 2022/12/19 18:08:53 UTC

[GitHub] [kafka] philipnee opened a new pull request, #13021: KAFKA-14468: Implement CommitRequestManager to manage the commit and autocommit requests

philipnee opened a new pull request, #13021:
URL: https://github.com/apache/kafka/pull/13021

   In this issue, we implemented a CommitRequestManager to handle the commit request from the client and the autocommit state.  When the client commits offsets, a "staged" commit will be added to a queue in the manager.  When the background thread polls the manager, it will check the queue and possibly generate a list of UnsentRequest, which will later be handled by the NetworkClientDelegate.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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


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

Posted by GitBox <gi...@apache.org>.
philipnee commented on code in PR #13021:
URL: https://github.com/apache/kafka/pull/13021#discussion_r1052895650


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java:
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.MockTime;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+
+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.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class CommitRequestManagerTest {

Review Comment:
   we should test the order commit when calling poll.  The commit sequence order should be guarantee.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
philipnee commented on PR #13021:
URL: https://github.com/apache/kafka/pull/13021#issuecomment-1358815133

   @kirktrue - Could you take a quick pass?


-- 
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


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

Posted by GitBox <gi...@apache.org>.
philipnee commented on code in PR #13021:
URL: https://github.com/apache/kafka/pull/13021#discussion_r1052890658


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.events;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.clients.consumer.internals.RequestFuture;
+import org.apache.kafka.common.TopicPartition;
+
+import java.util.Map;
+import java.util.Optional;
+
+class CommitApplicationEvent extends ApplicationEvent {
+    final public RequestFuture<Void> future;
+    final public Map<TopicPartition, OffsetAndMetadata> offsets;
+    final public Optional<OffsetCommitCallback> callback;
+
+    protected CommitApplicationEvent(final Map<TopicPartition, OffsetAndMetadata> offsets,
+                                     final OffsetCommitCallback callback,
+                                     final RequestFuture<Void> future) {
+        super(Type.COMMIT);
+        this.offsets = offsets;
+        this.callback = Optional.ofNullable(callback);
+        Optional<Exception> exception = isValid(offsets);
+        if (exception.isPresent()) {
+            throw new RuntimeException(exception.get());
+        }
+        this.future = future;
+    }
+
+    public Optional<Exception> isValid(final Map<TopicPartition, OffsetAndMetadata> offsets) {
+        for (Map.Entry<TopicPartition, OffsetAndMetadata> entry : offsets.entrySet()) {
+            TopicPartition topicPartition = entry.getKey();
+            OffsetAndMetadata offsetAndMetadata = entry.getValue();
+            if (offsetAndMetadata.offset() < 0) {
+                return Optional.of(new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset()));
+            }
+        }
+        return Optional.empty();
+    }
+
+    @Override
+    public String toString() {
+        return getClass() + "_" + this.offsets;

Review Comment:
   need to refactor this sad toString method.



-- 
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


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

Posted by "philipnee (via GitHub)" <gi...@apache.org>.
philipnee commented on PR #13021:
URL: https://github.com/apache/kafka/pull/13021#issuecomment-1421340357

   much thanks @guozhangwang 


-- 
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


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

Posted by "philipnee (via GitHub)" <gi...@apache.org>.
philipnee commented on code in PR #13021:
URL: https://github.com/apache/kafka/pull/13021#discussion_r1097801059


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/GroupStateManager.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.GroupRebalanceConfig;
+import org.apache.kafka.common.requests.JoinGroupRequest;
+import org.apache.kafka.common.requests.OffsetCommitRequest;
+
+import java.util.Objects;
+import java.util.Optional;
+
+public class GroupStateManager {

Review Comment:
   thanks for suggestions, changed.



##########
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;

Review Comment:
   Agreed. makes sense. thanks.



##########
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:
   Hmm, I see your point. Let's track the inflight status in the autocommit state and reset them upon completing the future (in the callback).



##########
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()) {

Review Comment:
   thanks, addressed.



##########
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);
+        autocommit.reset();
+    }
+
+    // Visible for testing
+    CompletableFuture<ClientResponse> sendAutoCommit(final Map<TopicPartition, OffsetAndMetadata> allConsumedOffsets) {
+        CompletableFuture<ClientResponse> future = this.add(allConsumedOffsets);
+        future.whenComplete((response, throwable) -> {
+            if (throwable == null) {
+                log.debug("Completed asynchronous auto-commit of offsets {}", allConsumedOffsets);
+            }
+
+            if (throwable instanceof RetriableCommitFailedException) {

Review Comment:
   I moved it to the exceptionally block, it think it would be clearer this way.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
philipnee commented on code in PR #13021:
URL: https://github.com/apache/kafka/pull/13021#discussion_r1080743797


##########
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:
   It makes sense to use a MaxValue as well, my counter argument is, i think explicitly disabling autoCommitState makes the logic more straightforward.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
philipnee commented on code in PR #13021:
URL: https://github.com/apache/kafka/pull/13021#discussion_r1062809110


##########
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:
   We should finalize the plan for subscriptionState, whether we will reuse the same class or implement a new one (and deprecate the old one)



-- 
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


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

Posted by "guozhangwang (via GitHub)" <gi...@apache.org>.
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-unsubscribe@kafka.apache.org

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


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

Posted by "guozhangwang (via GitHub)" <gi...@apache.org>.
guozhangwang commented on code in PR #13021:
URL: https://github.com/apache/kafka/pull/13021#discussion_r1096346374


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/GroupStateManager.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.GroupRebalanceConfig;
+import org.apache.kafka.common.requests.JoinGroupRequest;
+import org.apache.kafka.common.requests.OffsetCommitRequest;
+
+import java.util.Objects;
+import java.util.Optional;
+
+public class GroupStateManager {

Review Comment:
   Maybe just call it `GroupState` since it does not do any management, but more of a grouping of multiple primitive states?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThread.java:
##########
@@ -73,7 +76,9 @@ public class DefaultBackgroundThread extends KafkaThread {
                             final ApplicationEventProcessor processor,
                             final ConsumerMetadata metadata,
                             final NetworkClientDelegate networkClient,
-                            final CoordinatorRequestManager coordinatorManager) {
+                            final GroupStateManager groupState,
+                            final CoordinatorRequestManager coordinatorManager,
+                            CommitRequestManager commitRequestManager) {

Review Comment:
   Why this is not final?



##########
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;

Review Comment:
   I'd suggest we either change the class name to just `GroupState` or name this field `groupStateManager`? Personally I prefer the former.



##########
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:
   +1. I checked the current SusbscriptionState class and feel it's still resuable.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##########
@@ -46,4 +59,36 @@ public boolean process(final ApplicationEvent event) {
     private boolean process(final NoopApplicationEvent event) {
         return backgroundEventQueue.add(new NoopBackgroundEvent(event.message));
     }
+
+    private boolean process(final PollApplicationEvent event) {
+        Optional<RequestManager> commitRequestManger = registry.get(RequestManager.Type.COMMIT);
+        if (!commitRequestManger.isPresent()) {
+            return false;

Review Comment:
   Why return `false` here? If the user did not set `group.id` and never use auto or manual commits, then here we should just skip right?



##########
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);
+        autocommit.reset();
+    }
+
+    // Visible for testing
+    CompletableFuture<ClientResponse> sendAutoCommit(final Map<TopicPartition, OffsetAndMetadata> allConsumedOffsets) {
+        CompletableFuture<ClientResponse> future = this.add(allConsumedOffsets);
+        future.whenComplete((response, throwable) -> {
+            if (throwable == null) {
+                log.debug("Completed asynchronous auto-commit of offsets {}", allConsumedOffsets);
+            }
+
+            if (throwable instanceof RetriableCommitFailedException) {

Review Comment:
   merge this as `else if` along with line 124?



##########
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()) {

Review Comment:
   The `stagedCommits` seems never cleared?



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

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


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

Posted by "guozhangwang (via GitHub)" <gi...@apache.org>.
guozhangwang commented on PR #13021:
URL: https://github.com/apache/kafka/pull/13021#issuecomment-1421334205

   Made another look, LGTM. Leaving for @hachikuji for a final look.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
philipnee commented on code in PR #13021:
URL: https://github.com/apache/kafka/pull/13021#discussion_r1064949778


##########
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;
+    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) {

Review Comment:
   this method is still WIP: because we need to finalize on the subscriptions state changes. my initial thought was using the consumed and fetch positions to determine if the previous fetch has been consumed or not. 



-- 
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


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

Posted by GitBox <gi...@apache.org>.
vvcephei commented on PR #13021:
URL: https://github.com/apache/kafka/pull/13021#issuecomment-1367452673

   Thanks @philipnee !
   
   I just made a quick pass. I'm sorry; I haven't been keeping up with the progress of your project, so I'm a little hazy on the details.
   
   It looks like the new CommitRequestManager is a component that lives in the background part of the system, basically to help the background thread keep track of commit requests that have been/need to be sent to the network?
   
   I think that makes sense, but in that case, it is mildly surprising to see autocommit living where it lives. I initially would have expected it to originate from the foreground, but on second though, maybe it's because the background thread runs on a reliable timer? The caller is required to call `poll` on a regular basis, but maybe it's not frequent enough for your purposes?
   
   It would be good to understand the general principle for the division of responsibilities between the foreground and background. Do you have this documented somewhere that we can use for a north star?


-- 
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


[GitHub] [kafka] guozhangwang merged pull request #13021: KAFKA-14468: Implement CommitRequestManager to manage the commit and autocommit requests

Posted by "guozhangwang (via GitHub)" <gi...@apache.org>.
guozhangwang merged PR #13021:
URL: https://github.com/apache/kafka/pull/13021


-- 
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


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

Posted by GitBox <gi...@apache.org>.
philipnee commented on code in PR #13021:
URL: https://github.com/apache/kafka/pull/13021#discussion_r1052892406


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.HashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+
+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(

Review Comment:
   should add another constructor to inject the autocommitstate for testing



-- 
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


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

Posted by GitBox <gi...@apache.org>.
philipnee commented on code in PR #13021:
URL: https://github.com/apache/kafka/pull/13021#discussion_r1053661970


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThread.java:
##########
@@ -111,21 +120,36 @@ public DefaultBackgroundThread(final Time time,
                     networkClient);
             this.running = true;
             this.errorEventHandler = new ErrorEventHandler(this.backgroundEventQueue);
-            String groupId = rebalanceConfig.groupId;
-            this.coordinatorManager = groupId == null ?
-                    Optional.empty() :
-                    Optional.of(new CoordinatorRequestManager(
-                            logContext,
-                            config,
-                            errorEventHandler,
-                            groupId));
-            this.applicationEventProcessor = new ApplicationEventProcessor(backgroundEventQueue);
+            this.groupState = new GroupStateManager(rebalanceConfig);
+            this.requestManagerRegistry = Collections.unmodifiableMap(buildRequestManagerRegistry(logContext));
+            this.applicationEventProcessor = new ApplicationEventProcessor(backgroundEventQueue, requestManagerRegistry);
         } catch (final Exception e) {
             close();
             throw new KafkaException("Failed to construct background processor", e.getCause());
         }
     }
 
+    private Map<RequestManager.Type, Optional<RequestManager>> buildRequestManagerRegistry(final LogContext logContext) {

Review Comment:
   Not sure if it is worth writing a builder.



-- 
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


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

Posted by "philipnee (via GitHub)" <gi...@apache.org>.
philipnee commented on PR #13021:
URL: https://github.com/apache/kafka/pull/13021#issuecomment-1426265591

   There's still a couple things to be hashed out in the future, in particular
   1. how do we plan to structure the subscription state and 
   2. what are the API calls for the subscription state in the background thread
   
   For this specific PR, I think let's use the existing API, and change it later. There's really not a lot of interaction to the subscription state for this model for now, so I think it's fine.  In the new model, we might need to update the commit positions in the callback, but we can add that after this.
   
   @hachikuji @kirktrue 


-- 
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