cadonna commented on code in PR #15000:
URL: https://github.com/apache/kafka/pull/15000#discussion_r1458727101


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.FencedInstanceIdException;
+
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+/**
+ * Utility class that helps the application thread to invoke user registered 
{@link OffsetCommitCallback} amd
+ * {@link org.apache.kafka.clients.consumer.ConsumerInterceptor}s. This is
+ * achieved by having the background thread register a {@link 
OffsetCommitCallbackTask} to the invoker upon the
+ * future completion, and execute the callbacks when user polls/commits/closes 
the consumer.
+ */
+public class OffsetCommitCallbackInvoker {
+    private final ConsumerInterceptors<?, ?> interceptors;
+    private boolean hasFencedException = false;
+
+    OffsetCommitCallbackInvoker(ConsumerInterceptors<?, ?> interceptors) {
+        this.interceptors = interceptors;
+    }
+
+    // Thread-safe queue to store user-defined callbacks and interceptors to 
be executed
+    private final BlockingQueue<OffsetCommitCallbackTask> callbackQueue = new 
LinkedBlockingQueue<>();
+
+    public void submitCommitInterceptors(final Map<TopicPartition, 
OffsetAndMetadata> offsets) {
+        if (!interceptors.isEmpty()) {
+            callbackQueue.add(new OffsetCommitCallbackTask(
+                (innerOffsets, exception) -> 
interceptors.onCommit(innerOffsets),
+                offsets,
+                null
+            ));
+        }
+    }
+
+    public void submitUserCallback(final OffsetCommitCallback callback,
+                                   final Map<TopicPartition, 
OffsetAndMetadata> offsets,
+                                   final Exception exception) {
+        callbackQueue.add(new OffsetCommitCallbackTask(callback, offsets, 
exception));
+    }
+
+    public void executeCallbacks() {
+        while (!callbackQueue.isEmpty()) {
+            OffsetCommitCallbackTask task = callbackQueue.poll();
+            if (task != null) {
+
+                if (task.exception instanceof FencedInstanceIdException)
+                    hasFencedException = true;
+
+                task.callback.onComplete(task.offsets, task.exception);
+

Review Comment:
   nit:
   ```suggestion
   ```



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.FencedInstanceIdException;
+
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+/**
+ * Utility class that helps the application thread to invoke user registered 
{@link OffsetCommitCallback} amd
+ * {@link org.apache.kafka.clients.consumer.ConsumerInterceptor}s. This is
+ * achieved by having the background thread register a {@link 
OffsetCommitCallbackTask} to the invoker upon the
+ * future completion, and execute the callbacks when user polls/commits/closes 
the consumer.
+ */
+public class OffsetCommitCallbackInvoker {
+    private final ConsumerInterceptors<?, ?> interceptors;
+    private boolean hasFencedException = false;
+
+    OffsetCommitCallbackInvoker(ConsumerInterceptors<?, ?> interceptors) {
+        this.interceptors = interceptors;
+    }
+
+    // Thread-safe queue to store user-defined callbacks and interceptors to 
be executed
+    private final BlockingQueue<OffsetCommitCallbackTask> callbackQueue = new 
LinkedBlockingQueue<>();
+
+    public void submitCommitInterceptors(final Map<TopicPartition, 
OffsetAndMetadata> offsets) {
+        if (!interceptors.isEmpty()) {
+            callbackQueue.add(new OffsetCommitCallbackTask(
+                (innerOffsets, exception) -> 
interceptors.onCommit(innerOffsets),
+                offsets,
+                null
+            ));
+        }
+    }
+
+    public void submitUserCallback(final OffsetCommitCallback callback,
+                                   final Map<TopicPartition, 
OffsetAndMetadata> offsets,
+                                   final Exception exception) {
+        callbackQueue.add(new OffsetCommitCallbackTask(callback, offsets, 
exception));
+    }
+
+    public void executeCallbacks() {
+        while (!callbackQueue.isEmpty()) {
+            OffsetCommitCallbackTask task = callbackQueue.poll();
+            if (task != null) {
+
+                if (task.exception instanceof FencedInstanceIdException)
+                    hasFencedException = true;

Review Comment:
   I think this is not completely correct. 
   The javadocs for `commitAsync()` (w/o callback) say:
   ```
   @throws org.apache.kafka.common.errors.FencedInstanceIdException if this 
consumer instance gets fenced by broker.
   ```
   If no callback is passed into `commitAsync()`, no offset commit callback 
invocation is submitted. However, we only  check for a 
`FencedInstanceIdException` when we execute a callback. It seems to me that 
with `commitAsync()` we would not throw at all when the consumer gets fenced.
   In any case, we need a unit test that verifies that the 
`FencedInstanceIdException` is thrown for each version of `commitAsync()`.  



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.FencedInstanceIdException;
+
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+/**
+ * Utility class that helps the application thread to invoke user registered 
{@link OffsetCommitCallback} amd
+ * {@link org.apache.kafka.clients.consumer.ConsumerInterceptor}s. This is
+ * achieved by having the background thread register a {@link 
OffsetCommitCallbackTask} to the invoker upon the
+ * future completion, and execute the callbacks when user polls/commits/closes 
the consumer.
+ */
+public class OffsetCommitCallbackInvoker {
+    private final ConsumerInterceptors<?, ?> interceptors;
+    private boolean hasFencedException = false;
+
+    OffsetCommitCallbackInvoker(ConsumerInterceptors<?, ?> interceptors) {
+        this.interceptors = interceptors;
+    }
+
+    // Thread-safe queue to store user-defined callbacks and interceptors to 
be executed
+    private final BlockingQueue<OffsetCommitCallbackTask> callbackQueue = new 
LinkedBlockingQueue<>();
+
+    public void submitCommitInterceptors(final Map<TopicPartition, 
OffsetAndMetadata> offsets) {
+        if (!interceptors.isEmpty()) {
+            callbackQueue.add(new OffsetCommitCallbackTask(
+                (innerOffsets, exception) -> 
interceptors.onCommit(innerOffsets),
+                offsets,

Review Comment:
   nit:
   That is also a bit confusing. What are `innerOffsets`? I think it would be 
better to rename `innerOffsets` to `offsetsParam`. Additionally, `offsets` 
could be renamed to `actualOffsets` but that is actually not needed since after 
the renaming of `innerOffsets` it should be quite clear what `offsets` 
represents.



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvokerTest.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.common.TopicPartition;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.InOrder;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+import java.util.Collections;
+import java.util.Map;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.inOrder;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+public class OffsetCommitCallbackInvokerTest {
+
+    @Mock
+    private ConsumerInterceptors<?, ?> consumerInterceptors;
+    private OffsetCommitCallbackInvoker offsetCommitCallbackInvoker;
+
+    @BeforeEach
+    public void setup() {
+        offsetCommitCallbackInvoker = new 
OffsetCommitCallbackInvoker(consumerInterceptors);
+    }
+

Review Comment:
   I think there some edge cases missing like empty interceptors and empty user 
callbacks. Also only interceptors is also not tested. 



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.FencedInstanceIdException;
+
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+/**
+ * Utility class that helps the application thread to invoke user registered 
{@link OffsetCommitCallback} amd
+ * {@link org.apache.kafka.clients.consumer.ConsumerInterceptor}s. This is
+ * achieved by having the background thread register a {@link 
OffsetCommitCallbackTask} to the invoker upon the
+ * future completion, and execute the callbacks when user polls/commits/closes 
the consumer.
+ */
+public class OffsetCommitCallbackInvoker {
+    private final ConsumerInterceptors<?, ?> interceptors;
+    private boolean hasFencedException = false;
+
+    OffsetCommitCallbackInvoker(ConsumerInterceptors<?, ?> interceptors) {
+        this.interceptors = interceptors;
+    }
+
+    // Thread-safe queue to store user-defined callbacks and interceptors to 
be executed
+    private final BlockingQueue<OffsetCommitCallbackTask> callbackQueue = new 
LinkedBlockingQueue<>();
+
+    public void submitCommitInterceptors(final Map<TopicPartition, 
OffsetAndMetadata> offsets) {

Review Comment:
   nit:
   I find the name of this method quite confusing. It says that it submits a 
commit interceptor, but actually it submits a call to the commit interceptor.
   Similar applies to `submitUserCallback()`.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##########
@@ -271,6 +283,7 @@ public CompletableFuture<Void> 
maybeAutoCommitAllConsumedNow(
         return (response, throwable) -> {
             autoCommitState.ifPresent(autoCommitState -> 
autoCommitState.setInflightCommitStatus(false));
             if (throwable == null) {
+                
offsetCommitCallbackInvoker.submitCommitInterceptors(allConsumedOffsets);

Review Comment:
   Why is this only called in the case of an auto commit? Or is the name of 
this method misleading and this method is called also in case of async commits?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to