kirktrue commented on code in PR #17075:
URL: https://github.com/apache/kafka/pull/17075#discussion_r1742328712


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##########
@@ -409,4 +413,13 @@ protected ApplicationEventProcessor create() {
             }
         };
     }
+
+    private void process(final SeekUnvalidatedEvent event) {
+        try {
+            subscriptions.seekUnvalidated(event.partition(), event.position());

Review Comment:
   If we update the event class, we can then move more of the logic here:
   
   ```suggestion
               SubscriptionState.FetchPosition newPosition = new 
SubscriptionState.FetchPosition(
                   event.offset(),
                   event.offsetEpoch(),
                   metadata.currentLeader(event.partition())
               );
               subscriptions.seekUnvalidated(event.partition(), newPosition);
   ```



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SeekUnvalidatedEvent.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.internals.SubscriptionState;
+import org.apache.kafka.common.TopicPartition;
+
+public class SeekUnvalidatedEvent extends CompletableApplicationEvent<Void> {
+    private final TopicPartition partition;
+    private final SubscriptionState.FetchPosition position;

Review Comment:
   It's just my preference, but I'd rather not pass the `FetchPosition` around 
in the event. Could we instead include the following directly in the event:
   
   * `TopicPartition partition`
   * `long offset`
   * `Optional<Integer> offsetEpoch`
   
   The `ApplicationEventProcessor` has access to the `ConsumerMetadata` object, 
so we could fetch the current leader there, in one place, instead of having it 
twice in the `AsyncKafkaConsumer`.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -820,7 +823,10 @@ public void seek(TopicPartition partition, 
OffsetAndMetadata offsetAndMetadata)
                 offsetAndMetadata.leaderEpoch(),
                 currentLeaderAndEpoch);
             updateLastSeenEpochIfNewer(partition, offsetAndMetadata);
-            subscriptions.seekUnvalidated(partition, newPosition);
+
+            Timer timer = time.timer(defaultApiTimeoutMs);
+            SeekUnvalidatedEvent seekUnvalidatedEventEvent = new 
SeekUnvalidatedEvent(calculateDeadlineMs(timer), partition, newPosition);
+            applicationEventHandler.addAndGet(seekUnvalidatedEventEvent);

Review Comment:
   Per the other suggestions, we could make this a bit more succinct:
   
   ```suggestion
               SeekUnvalidatedEvent event = new SeekUnvalidatedEvent(
                   calculateDeadlineMs(timer),
                   partition,
                   offsetAndMetadata.offset(),
                   offsetAndMetadata.leaderEpoch()
               );
               applicationEventHandler.addAndGet(event);
   ```



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -793,7 +794,9 @@ public void seek(TopicPartition partition, long offset) {
                 offset,
                 Optional.empty(), // This will ensure we skip validation
                 metadata.currentLeader(partition));
-            subscriptions.seekUnvalidated(partition, newPosition);
+            Timer timer = time.timer(defaultApiTimeoutMs);
+            SeekUnvalidatedEvent seekUnvalidatedEventEvent = new 
SeekUnvalidatedEvent(calculateDeadlineMs(timer), partition, newPosition);
+            applicationEventHandler.addAndGet(seekUnvalidatedEventEvent);

Review Comment:
   Per the other suggestions, we could defer the creation of the 
`SubscriptionState.FetchPosition ` until the `ApplicationEventProcessor` 
processor method to make this call a bit more succinct:
   
   ```suggestion
               SeekUnvalidatedEvent event = new SeekUnvalidatedEvent(
                   calculateDeadlineMs(timer),
                   partition,
                   offset,
                   Optional.empty(), // This will ensure we skip validation
               );
               applicationEventHandler.addAndGet(event);
   ```



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