AndrewJSchofield commented on code in PR #20324:
URL: https://github.com/apache/kafka/pull/20324#discussion_r2309952955


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -1847,11 +1871,21 @@ private Fetch<K, V> collectFetch() {
      *                                       defined
      */
     private boolean updateFetchPositions(final Timer timer) {
-        cachedSubscriptionHasAllFetchPositions = false;
+        // Fetch position validation is in the hot path for poll() and the 
cost of thread interaction for
+        // event processing is *very* heavy, CPU-wise. In a stable system, the 
positions are valid; having the
+        // network thread check the validity yields the same answer 99%+ of 
the time. But calling the
+        // network thread to determine that is very expensive.
+        //
+        // Instead, let the *application thread* determine if any partitions 
need their positions updated. If not,
+        // the application thread can skip sending an event to the network 
thread that will simply end up coming
+        // to the same conclusion, albeit much slower.
+        if (sharedConsumerState.canSkipUpdateFetchPositions())
+            return true;
+
         try {
             CheckAndUpdatePositionsEvent checkAndUpdatePositionsEvent = new 
CheckAndUpdatePositionsEvent(calculateDeadlineMs(timer));
             wakeupTrigger.setActiveTask(checkAndUpdatePositionsEvent.future());
-            cachedSubscriptionHasAllFetchPositions = 
applicationEventHandler.addAndGet(checkAndUpdatePositionsEvent);
+            applicationEventHandler.addAndGet(checkAndUpdatePositionsEvent);

Review Comment:
   We are discarding the result of this event. Does it need a result any more? 
If not, do we want the result?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.ApiVersions;
+import org.apache.kafka.clients.consumer.Consumer;
+import 
org.apache.kafka.clients.consumer.internals.events.CheckAndUpdatePositionsEvent;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+
+import java.time.Duration;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class stores shared state needed by both the application thread 
({@link AsyncKafkaConsumer}) and the
+ * network thread ({@link ConsumerNetworkThread}) to avoid costly inter-thread 
communication, where possible.
+ * This class compromises on the ideal of keeping state only in the network 
thread. However, this class only
+ * relies on classes which are designed to be thread-safe, thus they can be 
used in both the application
+ * and network threads.
+ *
+ * <p/>
+ *
+ * The following thread-safe classes are used by this class:
+ *
+ * <ul>
+ *     <li>{@link ApiVersions}</li>
+ *     <li>{@link ConsumerMetadata}</li>
+ *     <li>{@link OffsetFetcherUtils}</li>
+ *     <li>{@link SharedExceptionReference}</li>
+ *     <li>{@link SubscriptionState}</li>
+ *     <li>{@link Time}</li>
+ * </ul>
+ *
+ * <p/>
+ *
+ * In general, callers from the application thread should not mutate any of 
the state contained within this class.
+ * It should be considered as <em>read-only</em>, and only the network thread 
should mutate the state.
+ */
+public class SharedConsumerState {
+
+    private final SubscriptionState subscriptions;
+    private final OffsetFetcherUtils offsetFetcherUtils;
+    private final SharedExceptionReference updatePositionsError;
+    private final SharedExceptionReference metadataError;
+
+    public SharedConsumerState(LogContext logContext,
+                               ConsumerMetadata metadata,
+                               SubscriptionState subscriptions,
+                               Time time,
+                               long retryBackoffMs) {
+        this(logContext, metadata, subscriptions, time, retryBackoffMs, new 
ApiVersions());
+    }
+
+    public SharedConsumerState(LogContext logContext,
+                               ConsumerMetadata metadata,
+                               SubscriptionState subscriptions,
+                               Time time,
+                               long retryBackoffMs,
+                               ApiVersions apiVersions) {
+        requireNonNull(logContext);
+        requireNonNull(metadata);
+        requireNonNull(subscriptions);
+        requireNonNull(time);
+        requireNonNull(apiVersions);
+
+        this.subscriptions = subscriptions;
+        this.offsetFetcherUtils = new OffsetFetcherUtils(
+            logContext,
+            metadata,
+            subscriptions,
+            time,
+            retryBackoffMs,
+            apiVersions
+        );
+        this.updatePositionsError = new SharedExceptionReference();
+        this.metadataError = new SharedExceptionReference();
+    }
+
+    OffsetFetcherUtils offsetFetcherUtils() {
+        return offsetFetcherUtils;
+    }
+
+    public SharedExceptionReference updatePositionsError() {
+        return updatePositionsError;
+    }
+
+    public SharedExceptionReference metadataError() {
+        return metadataError;
+    }
+
+    /**
+     * This method is used by {@code 
AsyncKafkaConsumer#updateFetchPositions()} to determine if it can skip
+     * the step of sending (and waiting for) a {@link 
CheckAndUpdatePositionsEvent}. {@code updateFetchPositions()}
+     * is in the critical path for the {@link Consumer#poll(Duration)}, and if 
the application thread can determine
+     * that it doesn't need to perform the {@link 
OffsetsRequestManager#updateFetchPositions(long)} call (via the
+     * {@link CheckAndUpdatePositionsEvent}), that is a big performance 
savings.
+     *
+     * <p/>
+     *
+     * This method performs similar checks to the start of {@link 
OffsetsRequestManager#updateFetchPositions(long)}:
+     *
+     * <ol>
+     *     <li>
+     *         Checks for previous exceptions during update positions
+     *         ({@code OffsetsRequestManager#cacheExceptionIfEventExpired()})
+     *     </li>
+     *     <li>
+     *         Checks that there are no positions in the {@link 
SubscriptionState.FetchStates#AWAIT_VALIDATION}
+     *         state ({@link OffsetFetcherUtils#getPartitionsToValidate()})
+     *     </li>
+     *     <li>
+     *         Checks that all positions are in the {@link 
SubscriptionState.FetchStates#FETCHING} state
+     *         ({@link SubscriptionState#hasAllFetchPositions()})
+     *     </li>
+     * </ol>
+     *
+     * If the first check fails, an exception will be thrown. If any of the 
second, third, or fourth checks fail, this
+     * method will return {@code false}. Otherwise, this method will return 
{@code true}, which signals to the
+     * application thread that the {@link CheckAndUpdatePositionsEvent} can be 
skipped.
+     *
+     * @return true if all checks pass, false if either of the latter two 
checks fail

Review Comment:
   I think an important part of this method's behaviour is that it will throw 
exceptions in several situations. That's probably relevant in the javadoc 
comment on this method.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.ApiVersions;
+import org.apache.kafka.clients.consumer.Consumer;
+import 
org.apache.kafka.clients.consumer.internals.events.CheckAndUpdatePositionsEvent;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+
+import java.time.Duration;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class stores shared state needed by both the application thread 
({@link AsyncKafkaConsumer}) and the
+ * network thread ({@link ConsumerNetworkThread}) to avoid costly inter-thread 
communication, where possible.
+ * This class compromises on the ideal of keeping state only in the network 
thread. However, this class only
+ * relies on classes which are designed to be thread-safe, thus they can be 
used in both the application
+ * and network threads.
+ *
+ * <p/>
+ *
+ * The following thread-safe classes are used by this class:
+ *
+ * <ul>
+ *     <li>{@link ApiVersions}</li>
+ *     <li>{@link ConsumerMetadata}</li>
+ *     <li>{@link OffsetFetcherUtils}</li>
+ *     <li>{@link SharedExceptionReference}</li>
+ *     <li>{@link SubscriptionState}</li>
+ *     <li>{@link Time}</li>
+ * </ul>
+ *
+ * <p/>
+ *
+ * In general, callers from the application thread should not mutate any of 
the state contained within this class.
+ * It should be considered as <em>read-only</em>, and only the network thread 
should mutate the state.
+ */
+public class SharedConsumerState {
+
+    private final SubscriptionState subscriptions;
+    private final OffsetFetcherUtils offsetFetcherUtils;
+    private final SharedExceptionReference updatePositionsError;
+    private final SharedExceptionReference metadataError;
+
+    public SharedConsumerState(LogContext logContext,
+                               ConsumerMetadata metadata,
+                               SubscriptionState subscriptions,
+                               Time time,
+                               long retryBackoffMs) {
+        this(logContext, metadata, subscriptions, time, retryBackoffMs, new 
ApiVersions());
+    }
+
+    public SharedConsumerState(LogContext logContext,
+                               ConsumerMetadata metadata,
+                               SubscriptionState subscriptions,
+                               Time time,
+                               long retryBackoffMs,
+                               ApiVersions apiVersions) {
+        requireNonNull(logContext);
+        requireNonNull(metadata);
+        requireNonNull(subscriptions);
+        requireNonNull(time);
+        requireNonNull(apiVersions);
+
+        this.subscriptions = subscriptions;
+        this.offsetFetcherUtils = new OffsetFetcherUtils(
+            logContext,
+            metadata,
+            subscriptions,
+            time,
+            retryBackoffMs,
+            apiVersions
+        );
+        this.updatePositionsError = new SharedExceptionReference();
+        this.metadataError = new SharedExceptionReference();
+    }
+
+    OffsetFetcherUtils offsetFetcherUtils() {
+        return offsetFetcherUtils;
+    }
+
+    public SharedExceptionReference updatePositionsError() {

Review Comment:
   So, this sounds a bit odd because it sounds like a mutating method name. 
Does it update the positions error? Nope. Would positionsUpdateError be 
acceptable?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/SharedConsumerState.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.ApiVersions;
+import org.apache.kafka.clients.consumer.Consumer;
+import 
org.apache.kafka.clients.consumer.internals.events.CheckAndUpdatePositionsEvent;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+
+import java.time.Duration;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class stores shared state needed by both the application thread 
({@link AsyncKafkaConsumer}) and the
+ * network thread ({@link ConsumerNetworkThread}) to avoid costly inter-thread 
communication, where possible.

Review Comment:
   nit: We have used "background thread" rather than "network thread" 
everywhere else. And I know the class name is different, but I would go for 
consistency in the comments so people do not get confused about a new kind of 
thread.



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