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


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java:
##########
@@ -32,36 +36,42 @@ public enum Type {
 
     private final Type type;
 
+    /**
+     * This identifies a particular event. It is used to disambiguate events 
via {@link #hashCode()} and
+     * {@link #equals(Object)} and can be used in log messages when debugging.
+     */
+    private final Uuid id;

Review Comment:
   Have you considered using just a long counter here? A number is easier to 
read in logs than a full-fledged UUID. Plus, a counter also gives you order 
information.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java:
##########
@@ -32,36 +36,42 @@ public enum Type {
 
     private final Type type;
 
+    /**
+     * This identifies a particular event. It is used to disambiguate events 
via {@link #hashCode()} and
+     * {@link #equals(Object)} and can be used in log messages when debugging.

Review Comment:
   That is not true because the ID is not used in `equals()`.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -988,12 +988,11 @@ public List<PartitionInfo> partitionsFor(String topic, 
Duration timeout) {
                 throw new TimeoutException();
             }
 
-            final TopicMetadataApplicationEvent topicMetadataApplicationEvent =
-                    new TopicMetadataApplicationEvent(topic, 
timeout.toMillis());
-            
wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future());
+            final TopicMetadataEvent topicMetadataEvent = new 
TopicMetadataEvent(Optional.of(topic), timeout.toMillis());
+            wakeupTrigger.setActiveTask(topicMetadataEvent.future());

Review Comment:
   Wouldn't it be clearer from a code readability point of view to use two 
different events?
   Something like `TopicMetadataEvent` and `AllTopicsMetadataEvent`.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##########
@@ -142,40 +143,40 @@ private void process(final PollApplicationEvent event) {
         requestManagers.heartbeatRequestManager.ifPresent(hrm -> 
hrm.resetPollTimer(event.pollTimeMs()));
     }
 
-    private void process(final AsyncCommitApplicationEvent event) {
+    private void process(final AsyncCommitEvent event) {
         CommitRequestManager manager = 
requestManagers.commitRequestManager.get();
-        CompletableFuture<Void> commitResult = 
manager.commitAsync(event.offsets());
-        event.chain(commitResult);
+        CompletableFuture<Void> future = manager.commitAsync(event.offsets());
+        chain(future, event.future());

Review Comment:
   If we are refactoring the `chain()` function, maybe we should also give it a 
more meaningful name or just use:
   ```
   future.whenComplete(complete(event.future()));
   ```
   with
   ```
   private <T> BiConsumer<? super T, ? super Throwable> complete(final 
CompletableFuture<T> b) {
       return (value, exception) -> {
           if (exception != null) {
               b.completeExceptionally(exception);
           } else {
               b.complete(value);
           }
       };
   }
   ```
   I think this makes the intent of completing the future of the event when the 
future from the request completes much clearer than
   ```
   chain(future, event.future());
   ```



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java:
##########
@@ -32,36 +36,42 @@ public enum Type {
 
     private final Type type;
 
+    /**
+     * This identifies a particular event. It is used to disambiguate events 
via {@link #hashCode()} and
+     * {@link #equals(Object)} and can be used in log messages when debugging.
+     */
+    private final Uuid id;
+
     protected ApplicationEvent(Type type) {
         this.type = Objects.requireNonNull(type);
+        this.id = Uuid.randomUuid();
     }
 
     public Type type() {
         return type;
     }
 
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-
-        ApplicationEvent that = (ApplicationEvent) o;
+    public Uuid id() {
+        return id;
+    }
 
-        return type == that.type;
+    @Override
+    public final boolean equals(Object o) {
+        return this == o;
     }
 
     @Override
-    public int hashCode() {
-        return type.hashCode();
+    public final int hashCode() {
+        return Objects.hash(type, id);
     }
 
     protected String toStringBase() {
-        return "type=" + type;
+        return "type=" + type + ", id=" + id;

Review Comment:
   nit: I think it would be better to use `StringBuilder` in all 
`toStringBase()` methods.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java:
##########
@@ -31,36 +32,42 @@ public enum Type {
 
     private final Type type;
 
-    public BackgroundEvent(Type type) {
+    /**
+     * This identifies a particular event. It is used to disambiguate events 
via {@link #hashCode()} and
+     * {@link #equals(Object)} and can be used in log messages when debugging.

Review Comment:
   This is again not true because the ID is not used in `equals()`.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java:
##########
@@ -32,36 +36,42 @@ public enum Type {
 
     private final Type type;
 
+    /**
+     * This identifies a particular event. It is used to disambiguate events 
via {@link #hashCode()} and
+     * {@link #equals(Object)} and can be used in log messages when debugging.
+     */
+    private final Uuid id;
+
     protected ApplicationEvent(Type type) {
         this.type = Objects.requireNonNull(type);
+        this.id = Uuid.randomUuid();
     }
 
     public Type type() {
         return type;
     }
 
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-
-        ApplicationEvent that = (ApplicationEvent) o;
+    public Uuid id() {
+        return id;
+    }
 
-        return type == that.type;
+    @Override
+    public final boolean equals(Object o) {

Review Comment:
   Is there a good reason to make `equals()` that strict?



-- 
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-unsubscr...@kafka.apache.org

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

Reply via email to