soarez commented on code in PR #14369:
URL: https://github.com/apache/kafka/pull/14369#discussion_r1392487870


##########
core/src/main/java/kafka/server/AssignmentsManager.java:
##########
@@ -0,0 +1,384 @@
+/*
+ * 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 kafka.server;
+
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.AssignReplicasToDirsRequestData;
+import 
org.apache.kafka.common.message.AssignReplicasToDirsRequestData.DirectoryData;
+import 
org.apache.kafka.common.message.AssignReplicasToDirsRequestData.PartitionData;
+import 
org.apache.kafka.common.message.AssignReplicasToDirsRequestData.TopicData;
+import org.apache.kafka.common.message.AssignReplicasToDirsResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AssignReplicasToDirsRequest;
+import org.apache.kafka.common.requests.AssignReplicasToDirsResponse;
+import org.apache.kafka.common.utils.ExponentialBackoff;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.server.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+public class AssignmentsManager {
+
+    private static final Logger log = 
LoggerFactory.getLogger(AssignmentsManager.class);
+
+    /**
+     * Assignments are dispatched to the controller this long after
+     * being submitted to {@link AssignmentsManager}, if there
+     * is no request in flight already.
+     * The interval is reset when a new assignment is submitted.
+     */
+    private static final long DISPATCH_INTERVAL_NS = 
TimeUnit.SECONDS.toNanos(1);
+
+    private static final long MAX_BACKOFF_INTERVAL_MS = 
TimeUnit.SECONDS.toNanos(10);
+
+    private final Time time;
+    private final NodeToControllerChannelManager channelManager;
+    private final int brokerId;
+    private final Supplier<Long> brokerEpochSupplier;
+    private final KafkaEventQueue eventQueue;
+
+    // These variables should only be mutated from the event loop thread
+    private Map<TopicIdPartition, AssignmentEvent> inflight = null;
+    private Map<TopicIdPartition, AssignmentEvent> pending = new HashMap<>();
+    private final ExponentialBackoff resendExponentialBackoff =
+            new ExponentialBackoff(100, 2, MAX_BACKOFF_INTERVAL_MS, 0.02);
+    private int failedAttempts = 0;
+
+    public AssignmentsManager(Time time,
+                              NodeToControllerChannelManager channelManager,
+                              int brokerId,
+                              Supplier<Long> brokerEpochSupplier) {
+        this.time = time;
+        this.channelManager = channelManager;
+        this.brokerId = brokerId;
+        this.brokerEpochSupplier = brokerEpochSupplier;
+        this.eventQueue = new KafkaEventQueue(time,
+                new LogContext("[AssignmentsManager id=" + brokerId + "]"),
+                "broker-" + brokerId + "-directory-assignments-manager-");
+    }
+
+    public void close() throws InterruptedException {
+        eventQueue.close();
+        channelManager.shutdown();
+    }
+
+    public void onAssignment(TopicIdPartition topicPartition, Uuid dirId) {
+        eventQueue.append(new AssignmentEvent(time.nanoseconds(), 
topicPartition, dirId));
+    }
+
+    // only for testing
+    void wakeup() {
+        eventQueue.wakeup();
+    }
+
+    /**
+     * Base class for all the events handled by {@link AssignmentsManager}.
+     */
+    private abstract static class Event implements EventQueue.Event {
+        /**
+         * Override the default behavior in
+         * {@link EventQueue.Event#handleException}
+         * which swallows the exception.
+         */
+        @Override
+        public void handleException(Throwable e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Handles new generated assignments, to be propagated to the controller.
+     * Assignment events may be handled out of order, so for any two assignment
+     * events for the same topic partition, the one with the oldest timestamp 
is
+     * disregarded.
+     */
+    private class AssignmentEvent extends Event {
+        final long timestampNs;
+        final TopicIdPartition partition;

Review Comment:
   It's useful to have a single object to index pending and sent events. i.e. 
`Map<TopicIdPartition, AssignmentEvent>` is better than `Map<Uuid, Map<Integer, 
AssignmentEvent>>`. 
   
   I chose `org.apache.kafka.server.common.TopicIdPartition` rather than 
`org.apache.kafka.common.TopicIdPartition` because the latter requires a topic 
name, which isn't part of the AssignReplicasToDirs RPC and makes things a bit 
messier.
   



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