You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "philipnee (via GitHub)" <gi...@apache.org> on 2023/02/06 21:10:36 UTC

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

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