You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/02/03 22:27:45 UTC

[GitHub] [kafka] jsancio commented on a change in pull request #10030: MINOR: Add KafkaEventQueue

jsancio commented on a change in pull request #10030:
URL: https://github.com/apache/kafka/pull/10030#discussion_r569716767



##########
File path: metadata/src/main/java/org/apache/kafka/queue/EventQueue.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.queue;
+
+import org.slf4j.Logger;
+
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+
+
+public interface EventQueue extends AutoCloseable {
+    interface Event {
+        void run() throws Exception;
+        default void handleException(Throwable e) {}
+    }
+
+    abstract class FailureLoggingEvent implements Event {
+        private final Logger log;
+
+        public FailureLoggingEvent(Logger log) {
+            this.log = log;
+        }
+
+        @Override
+        public void handleException(Throwable e) {
+            if (e instanceof EventQueueClosedException) {
+                log.info("Not processing {} because the event queue is closed.",
+                    this.toString());
+            } else {
+                log.error("Unexpected error handling {}", this.toString(), e);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return this.getClass().getSimpleName();
+        }
+    }
+
+    class DeadlineFunction implements Function<Long, Long> {
+        private final long deadlineNs;
+
+        public DeadlineFunction(long deadlineNs) {
+            this.deadlineNs = deadlineNs;
+        }
+
+        @Override
+        public Long apply(Long t) {
+            return deadlineNs;
+        }
+    }
+
+    class EarliestDeadlineFunction implements Function<Long, Long> {
+        private final long newDeadlineNs;
+
+        public EarliestDeadlineFunction(long newDeadlineNs) {
+            this.newDeadlineNs = newDeadlineNs;
+        }
+
+        @Override
+        public Long apply(Long prevDeadlineNs) {
+            if (prevDeadlineNs == null) {
+                return newDeadlineNs;
+            } else if (prevDeadlineNs < newDeadlineNs) {
+                return prevDeadlineNs;
+            } else {
+                return newDeadlineNs;
+            }
+        }
+    }
+
+    class VoidEvent implements Event {
+        public final static VoidEvent INSTANCE = new VoidEvent();
+
+        @Override
+        public void run() throws Exception {
+        }
+    }
+
+    /**
+     * Add an element to the front of the queue.
+     *
+     * @param event             The mandatory event to prepend.
+     */
+    default void prepend(Event event) {
+        enqueue(EventInsertionType.PREPEND, null, null, event);
+    }
+
+    /**
+     * Add an element to the end of the queue.
+     *
+     * @param event             The event to append.
+     */
+    default void append(Event event) {
+        enqueue(EventInsertionType.APPEND, null, null, event);
+    }
+
+    /**
+     * Enqueue an event to be run in FIFO order.
+     *
+     * @param deadlineNs        The time in monotonic nanoseconds after which the future
+     *                          is completed with a
+     *                          @{org.apache.kafka.common.errors.TimeoutException},
+     *                          and the event is cancelled.

Review comment:
       Which future? Did you mean `Event`. I assume that `deadlineNs` is the deadline for scheduling/executing the `run` method. The `run` method can take longer than `deadlineNs`.

##########
File path: metadata/src/main/java/org/apache/kafka/queue/EventQueue.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.queue;
+
+import org.slf4j.Logger;
+
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+
+
+public interface EventQueue extends AutoCloseable {
+    interface Event {
+        void run() throws Exception;
+        default void handleException(Throwable e) {}
+    }
+
+    abstract class FailureLoggingEvent implements Event {
+        private final Logger log;
+
+        public FailureLoggingEvent(Logger log) {
+            this.log = log;
+        }
+
+        @Override
+        public void handleException(Throwable e) {
+            if (e instanceof EventQueueClosedException) {
+                log.info("Not processing {} because the event queue is closed.",
+                    this.toString());
+            } else {
+                log.error("Unexpected error handling {}", this.toString(), e);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return this.getClass().getSimpleName();
+        }
+    }
+
+    class DeadlineFunction implements Function<Long, Long> {
+        private final long deadlineNs;
+
+        public DeadlineFunction(long deadlineNs) {
+            this.deadlineNs = deadlineNs;
+        }
+
+        @Override
+        public Long apply(Long t) {
+            return deadlineNs;
+        }
+    }
+
+    class EarliestDeadlineFunction implements Function<Long, Long> {
+        private final long newDeadlineNs;
+
+        public EarliestDeadlineFunction(long newDeadlineNs) {
+            this.newDeadlineNs = newDeadlineNs;
+        }
+
+        @Override
+        public Long apply(Long prevDeadlineNs) {
+            if (prevDeadlineNs == null) {
+                return newDeadlineNs;
+            } else if (prevDeadlineNs < newDeadlineNs) {
+                return prevDeadlineNs;
+            } else {
+                return newDeadlineNs;
+            }
+        }
+    }
+
+    class VoidEvent implements Event {
+        public final static VoidEvent INSTANCE = new VoidEvent();
+
+        @Override
+        public void run() throws Exception {
+        }
+    }
+
+    /**
+     * Add an element to the front of the queue.
+     *
+     * @param event             The mandatory event to prepend.
+     */
+    default void prepend(Event event) {
+        enqueue(EventInsertionType.PREPEND, null, null, event);
+    }
+
+    /**
+     * Add an element to the end of the queue.
+     *
+     * @param event             The event to append.
+     */
+    default void append(Event event) {
+        enqueue(EventInsertionType.APPEND, null, null, event);
+    }
+
+    /**
+     * Enqueue an event to be run in FIFO order.
+     *
+     * @param deadlineNs        The time in monotonic nanoseconds after which the future
+     *                          is completed with a
+     *                          @{org.apache.kafka.common.errors.TimeoutException},
+     *                          and the event is cancelled.
+     * @param event             The event to append.
+     */
+    default void appendWithDeadline(long deadlineNs, Event event) {
+        enqueue(EventInsertionType.APPEND, null, __ -> deadlineNs, event);
+    }
+
+    /**
+     * Schedule an event to be run at a specific time.
+     *
+     * @param tag                   If this is non-null, the unique tag to use for this
+     *                              event.  If an event with this tag already exists, it
+     *                              will be cancelled.
+     * @param deadlineNsCalculator  A function which takes as an argument the existing
+     *                              deadline for the event with this tag (or null if the
+     *                              event has no tag, or if there is none such), and
+     *                              produces the deadline to use for this event.
+     * @param event                 The event to schedule.
+     */
+    default void scheduleDeferred(String tag,
+                                  Function<Long, Long> deadlineNsCalculator,
+                                  Event event) {
+        enqueue(EventInsertionType.DEFERRED, tag, deadlineNsCalculator, event);
+    }
+
+    /**
+     * Cancel a deferred event.
+     *
+     * @param tag                   The unique tag for the event to be cancelled.  Must be
+     *                              non-null.  If the event with the tag has not been
+     *                              scheduled, this call will be ignored.
+     */
+    void cancelDeferred(String tag);
+
+    enum EventInsertionType {
+        PREPEND,
+        APPEND,
+        DEFERRED;
+    }
+
+    /**
+     * Enqueue an event to be run in FIFO order.
+     *
+     * @param insertionType         How to insert the event.
+     *                              PREPEND means insert the event as the first thing
+     *                              to run.  APPEND means insert the event as the last
+     *                              thing to run.  DEFERRED means insert the event to
+     *                              run after a delay.

Review comment:
       The headline description says "Enqueue an event to be run in FIFO order." but looking the param description it doesn't looking like it is FIFO order since it support prepend, append and deferred.
   
   What do you mean by DEFEREED? How is the delay specified? I see the param `deadlineNsCalculator` but it looks like this describe the maximum amount of time this event is allowed to stay in the queue before it is cancelled by the queue.

##########
File path: metadata/src/main/java/org/apache/kafka/queue/EventQueue.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.queue;
+
+import org.slf4j.Logger;
+
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+
+
+public interface EventQueue extends AutoCloseable {
+    interface Event {
+        void run() throws Exception;
+        default void handleException(Throwable e) {}
+    }
+
+    abstract class FailureLoggingEvent implements Event {
+        private final Logger log;
+
+        public FailureLoggingEvent(Logger log) {
+            this.log = log;
+        }
+
+        @Override
+        public void handleException(Throwable e) {
+            if (e instanceof EventQueueClosedException) {
+                log.info("Not processing {} because the event queue is closed.",
+                    this.toString());
+            } else {
+                log.error("Unexpected error handling {}", this.toString(), e);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return this.getClass().getSimpleName();
+        }
+    }
+
+    class DeadlineFunction implements Function<Long, Long> {
+        private final long deadlineNs;
+
+        public DeadlineFunction(long deadlineNs) {
+            this.deadlineNs = deadlineNs;
+        }
+
+        @Override
+        public Long apply(Long t) {
+            return deadlineNs;
+        }
+    }
+
+    class EarliestDeadlineFunction implements Function<Long, Long> {
+        private final long newDeadlineNs;
+
+        public EarliestDeadlineFunction(long newDeadlineNs) {
+            this.newDeadlineNs = newDeadlineNs;
+        }
+
+        @Override
+        public Long apply(Long prevDeadlineNs) {
+            if (prevDeadlineNs == null) {
+                return newDeadlineNs;
+            } else if (prevDeadlineNs < newDeadlineNs) {
+                return prevDeadlineNs;
+            } else {
+                return newDeadlineNs;
+            }
+        }
+    }
+
+    class VoidEvent implements Event {
+        public final static VoidEvent INSTANCE = new VoidEvent();
+
+        @Override
+        public void run() throws Exception {
+        }
+    }
+
+    /**
+     * Add an element to the front of the queue.
+     *
+     * @param event             The mandatory event to prepend.
+     */
+    default void prepend(Event event) {
+        enqueue(EventInsertionType.PREPEND, null, null, event);
+    }
+
+    /**
+     * Add an element to the end of the queue.
+     *
+     * @param event             The event to append.
+     */
+    default void append(Event event) {
+        enqueue(EventInsertionType.APPEND, null, null, event);
+    }
+
+    /**
+     * Enqueue an event to be run in FIFO order.
+     *
+     * @param deadlineNs        The time in monotonic nanoseconds after which the future
+     *                          is completed with a
+     *                          @{org.apache.kafka.common.errors.TimeoutException},
+     *                          and the event is cancelled.
+     * @param event             The event to append.
+     */
+    default void appendWithDeadline(long deadlineNs, Event event) {
+        enqueue(EventInsertionType.APPEND, null, __ -> deadlineNs, event);
+    }
+
+    /**
+     * Schedule an event to be run at a specific time.
+     *
+     * @param tag                   If this is non-null, the unique tag to use for this
+     *                              event.  If an event with this tag already exists, it
+     *                              will be cancelled.
+     * @param deadlineNsCalculator  A function which takes as an argument the existing
+     *                              deadline for the event with this tag (or null if the
+     *                              event has no tag, or if there is none such), and
+     *                              produces the deadline to use for this event.
+     * @param event                 The event to schedule.
+     */
+    default void scheduleDeferred(String tag,
+                                  Function<Long, Long> deadlineNsCalculator,
+                                  Event event) {
+        enqueue(EventInsertionType.DEFERRED, tag, deadlineNsCalculator, event);
+    }
+
+    /**
+     * Cancel a deferred event.
+     *
+     * @param tag                   The unique tag for the event to be cancelled.  Must be
+     *                              non-null.  If the event with the tag has not been
+     *                              scheduled, this call will be ignored.
+     */
+    void cancelDeferred(String tag);
+
+    enum EventInsertionType {
+        PREPEND,
+        APPEND,
+        DEFERRED;
+    }
+
+    /**
+     * Enqueue an event to be run in FIFO order.
+     *
+     * @param insertionType         How to insert the event.
+     *                              PREPEND means insert the event as the first thing
+     *                              to run.  APPEND means insert the event as the last
+     *                              thing to run.  DEFERRED means insert the event to
+     *                              run after a delay.
+     * @param tag                   If this is non-null, the unique tag to use for
+     *                              this event.  If an event with this tag already
+     *                              exists, it will be cancelled.
+     * @param deadlineNsCalculator  If this is non-null, it is a function which takes
+     *                              as an argument the existing deadline for the
+     *                              event with this tag (or null if the event has no
+     *                              tag, or if there is none such), and produces the
+     *                              deadline to use for this event (or null to use
+     *                              none.)
+     * @param event                 The event to enqueue.
+     */
+    void enqueue(EventInsertionType insertionType,
+                 String tag,
+                 Function<Long, Long> deadlineNsCalculator,
+                 Event event);
+
+    /**
+     * Asynchronously shut down the event queue with no unnecessary delay.
+     * @see #beginShutdown(String, Event, TimeUnit, long)
+     *
+     * @param source                The source of the shutdown.
+     */
+    default void beginShutdown(String source) {
+        beginShutdown(source, new VoidEvent());
+    }
+
+    /**
+     * Asynchronously shut down the event queue with no unnecessary delay.
+     *
+     * @param source        The source of the shutdown.
+     * @param cleanupEvent  The mandatory event to invoke after all other events have
+     *                      been processed.
+     * @see #beginShutdown(String, Event, TimeUnit, long)
+     */
+    default void beginShutdown(String source, Event cleanupEvent) {
+        beginShutdown(source, cleanupEvent, TimeUnit.SECONDS, 0);
+    }
+
+    /**
+     * Asynchronously shut down the event queue.
+     *
+     * No new events will be accepted, and the timeout will be initiated
+     * for all existing events.
+     *
+     * @param source        The source of the shutdown.
+     * @param cleanupEvent  The mandatory event to invoke after all other events have
+     *                      been processed.
+     * @param timeUnit      The time unit to use for the timeout.
+     * @param timeSpan      The amount of time to use for the timeout.
+     *                      Once the timeout elapses, any remaining queued
+     *                      events will get a
+     *                      @{org.apache.kafka.common.errors.TimeoutException}.
+     */
+    void beginShutdown(String source, Event cleanupEvent, TimeUnit timeUnit, long timeSpan);

Review comment:
       In Java the unit of measure `timeUnit` and scalar `timeSpan` are declare in reverse order. For example:
   ```java
        void beginShutdown(String source, Event cleanupEvent, long timeSpan, TimeUnit timeUnit);
   ```

##########
File path: metadata/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.queue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.utils.KafkaThread;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+
+public final class KafkaEventQueue implements EventQueue {
+    /**
+     * A context object that wraps events.
+     */
+    private static class EventContext {
+        /**
+         * The caller-supplied event.
+         */
+        private final Event event;
+
+        /**
+         * How this event was inserted.
+         */
+        private final EventInsertionType insertionType;
+
+        /**
+         * The previous pointer of our circular doubly-linked list.
+         */
+        private EventContext prev = this;
+
+        /**
+         * The next pointer in our circular doubly-linked list.
+         */
+        private EventContext next = this;
+
+        /**
+         * If this event is in the delay map, this is the key it is there under.
+         * If it is not in the map, this is null.
+         */
+        private Long deadlineNs = null;
+
+        /**
+         * The tag associated with this event.
+         */
+        private String tag;
+
+        EventContext(Event event, EventInsertionType insertionType, String tag) {
+            this.event = event;
+            this.insertionType = insertionType;
+            this.tag = tag;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list after this node.
+         */
+        void insertAfter(EventContext other) {
+            this.next.prev = other;
+            other.next = this.next;
+            other.prev = this;
+            this.next = other;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list before this node.
+         */
+        void insertBefore(EventContext other) {
+            this.prev.next = other;
+            other.prev = this.prev;
+            other.next = this;
+            this.prev = other;
+        }
+
+        /**
+         * Remove this node from the circularly linked list.
+         */
+        void remove() {
+            this.prev.next = this.next;
+            this.next.prev = this.prev;
+            this.prev = this;
+            this.next = this;
+        }
+
+        /**
+         * Returns true if this node is the only element in its list.
+         */
+        boolean isSingleton() {
+            return prev == this && next == this;
+        }
+
+        /**
+         * Run the event associated with this EventContext.
+         */
+        void run() throws InterruptedException {
+            try {
+                event.run();
+            } catch (InterruptedException e) {
+                throw e;
+            } catch (Exception e) {
+                event.handleException(e);
+            }
+        }
+
+        /**
+         * Complete the event associated with this EventContext with a timeout exception.
+         */
+        void completeWithTimeout() {
+            completeWithException(new TimeoutException());
+        }
+
+        /**
+         * Complete the event associated with this EventContext with the specified
+         * exception.
+         */
+        void completeWithException(Throwable t) {
+            event.handleException(t);
+        }
+    }
+
+    private class EventHandler implements Runnable {
+        /**
+         * Event contexts indexed by tag.  Events without a tag are not included here.
+         */
+        private final Map<String, EventContext> tagToEventContext = new HashMap<>();
+
+        /**
+         * The head of the event queue.
+         */
+        private final EventContext head = new EventContext(null, null, null);
+
+        /**
+         * An ordered map of times in monotonic nanoseconds to events to time out.
+         */
+        private final TreeMap<Long, EventContext> delayMap = new TreeMap<>();
+
+        /**
+         * A condition variable for waking up the event handler thread.
+         */
+        private final Condition cond = lock.newCondition();
+
+        @Override
+        public void run() {
+            try {
+                handleEvents();
+                cleanupEvent.run();
+            } catch (Throwable e) {
+                log.warn("event handler thread exiting with exception", e);
+            }
+        }
+
+        private void remove(EventContext eventContext) {
+            eventContext.remove();
+            if (eventContext.deadlineNs != null) {
+                delayMap.remove(eventContext.deadlineNs);
+                eventContext.deadlineNs = null;
+            }
+            if (eventContext.tag != null) {
+                tagToEventContext.remove(eventContext.tag, eventContext);
+                eventContext.tag = null;
+            }
+        }
+
+        private void handleEvents() throws InterruptedException {
+            EventContext toTimeout = null;
+            EventContext toRun = null;
+            while (true) {
+                if (toTimeout != null) {
+                    toTimeout.completeWithTimeout();
+                    toTimeout = null;
+                } else if (toRun != null) {
+                    toRun.run();
+                    toRun = null;
+                }
+                lock.lock();
+                try {
+                    long awaitNs = Long.MAX_VALUE;
+                    Map.Entry<Long, EventContext> entry = delayMap.firstEntry();
+                    if (entry != null) {
+                        // Search for timed-out events or deferred events that are ready
+                        // to run.
+                        long now = time.nanoseconds();
+                        long timeoutNs = entry.getKey();
+                        EventContext eventContext = entry.getValue();
+                        if (timeoutNs <= now) {
+                            if (eventContext.insertionType == EventInsertionType.DEFERRED) {

Review comment:
       Hmm. Does this mean that `DEFERRED` events cannot timeout? How about making the two threshold described independently when the event is enqueued?

##########
File path: metadata/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.queue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.utils.KafkaThread;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+
+public final class KafkaEventQueue implements EventQueue {
+    /**
+     * A context object that wraps events.
+     */
+    private static class EventContext {
+        /**
+         * The caller-supplied event.
+         */
+        private final Event event;
+
+        /**
+         * How this event was inserted.
+         */
+        private final EventInsertionType insertionType;
+
+        /**
+         * The previous pointer of our circular doubly-linked list.
+         */
+        private EventContext prev = this;
+
+        /**
+         * The next pointer in our circular doubly-linked list.
+         */
+        private EventContext next = this;
+
+        /**
+         * If this event is in the delay map, this is the key it is there under.
+         * If it is not in the map, this is null.
+         */
+        private Long deadlineNs = null;
+
+        /**
+         * The tag associated with this event.
+         */
+        private String tag;
+
+        EventContext(Event event, EventInsertionType insertionType, String tag) {
+            this.event = event;
+            this.insertionType = insertionType;
+            this.tag = tag;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list after this node.
+         */
+        void insertAfter(EventContext other) {
+            this.next.prev = other;
+            other.next = this.next;
+            other.prev = this;
+            this.next = other;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list before this node.
+         */
+        void insertBefore(EventContext other) {
+            this.prev.next = other;
+            other.prev = this.prev;
+            other.next = this;
+            this.prev = other;
+        }
+
+        /**
+         * Remove this node from the circularly linked list.
+         */
+        void remove() {
+            this.prev.next = this.next;
+            this.next.prev = this.prev;
+            this.prev = this;
+            this.next = this;
+        }
+
+        /**
+         * Returns true if this node is the only element in its list.
+         */
+        boolean isSingleton() {
+            return prev == this && next == this;
+        }
+
+        /**
+         * Run the event associated with this EventContext.
+         */
+        void run() throws InterruptedException {
+            try {
+                event.run();
+            } catch (InterruptedException e) {
+                throw e;
+            } catch (Exception e) {
+                event.handleException(e);
+            }
+        }
+
+        /**
+         * Complete the event associated with this EventContext with a timeout exception.
+         */
+        void completeWithTimeout() {
+            completeWithException(new TimeoutException());
+        }
+
+        /**
+         * Complete the event associated with this EventContext with the specified
+         * exception.
+         */
+        void completeWithException(Throwable t) {
+            event.handleException(t);
+        }
+    }
+
+    private class EventHandler implements Runnable {
+        /**
+         * Event contexts indexed by tag.  Events without a tag are not included here.
+         */
+        private final Map<String, EventContext> tagToEventContext = new HashMap<>();
+
+        /**
+         * The head of the event queue.
+         */
+        private final EventContext head = new EventContext(null, null, null);
+
+        /**
+         * An ordered map of times in monotonic nanoseconds to events to time out.
+         */
+        private final TreeMap<Long, EventContext> delayMap = new TreeMap<>();
+
+        /**
+         * A condition variable for waking up the event handler thread.
+         */
+        private final Condition cond = lock.newCondition();
+
+        @Override
+        public void run() {
+            try {
+                handleEvents();
+                cleanupEvent.run();
+            } catch (Throwable e) {
+                log.warn("event handler thread exiting with exception", e);
+            }
+        }
+
+        private void remove(EventContext eventContext) {
+            eventContext.remove();
+            if (eventContext.deadlineNs != null) {
+                delayMap.remove(eventContext.deadlineNs);
+                eventContext.deadlineNs = null;
+            }
+            if (eventContext.tag != null) {
+                tagToEventContext.remove(eventContext.tag, eventContext);
+                eventContext.tag = null;
+            }
+        }
+
+        private void handleEvents() throws InterruptedException {
+            EventContext toTimeout = null;
+            EventContext toRun = null;
+            while (true) {
+                if (toTimeout != null) {
+                    toTimeout.completeWithTimeout();
+                    toTimeout = null;
+                } else if (toRun != null) {
+                    toRun.run();
+                    toRun = null;
+                }
+                lock.lock();
+                try {
+                    long awaitNs = Long.MAX_VALUE;
+                    Map.Entry<Long, EventContext> entry = delayMap.firstEntry();
+                    if (entry != null) {
+                        // Search for timed-out events or deferred events that are ready
+                        // to run.
+                        long now = time.nanoseconds();
+                        long timeoutNs = entry.getKey();
+                        EventContext eventContext = entry.getValue();
+                        if (timeoutNs <= now) {
+                            if (eventContext.insertionType == EventInsertionType.DEFERRED) {
+                                // The deferred event is ready to run.  Prepend it to the
+                                // queue.  (The value for deferred events is a schedule time
+                                // rather than a timeout.)
+                                remove(eventContext);
+                                head.insertAfter(eventContext);
+                            } else {
+                                // not a deferred event, so it is a deadline, and it is timed out.
+                                remove(eventContext);
+                                toTimeout = eventContext;
+                            }
+                            continue;
+                        } else if (closingTimeNs <= now) {
+                            remove(eventContext);
+                            toTimeout = eventContext;
+                            continue;
+                        }
+                        awaitNs = timeoutNs - now;
+                    }
+                    if (head.next == head) {
+                        if ((closingTimeNs != Long.MAX_VALUE) && delayMap.isEmpty()) {
+                            // If there are no more entries to process, and the queue is
+                            // closing, exit the thread.
+                            return;
+                        }
+                    } else {
+                        toRun = head.next;
+                        remove(toRun);
+                        continue;
+                    }
+                    if (closingTimeNs != Long.MAX_VALUE) {
+                        long now = time.nanoseconds();
+                        if (awaitNs > closingTimeNs - now) {
+                            awaitNs = closingTimeNs - now;
+                        }
+                    }
+                    if (awaitNs == Long.MAX_VALUE) {
+                        cond.await();
+                    } else {
+                        cond.awaitNanos(awaitNs);
+                    }
+                } finally {
+                    lock.unlock();
+                }
+            }
+        }
+
+        private void enqueue(EventContext eventContext,
+                             Function<Long, Long> deadlineNsCalculator) {
+            lock.lock();
+            try {
+                Long existingDeadlineNs = null;
+                if (eventContext.tag != null) {
+                    EventContext toRemove =
+                        tagToEventContext.put(eventContext.tag, eventContext);
+                    if (toRemove != null) {
+                        existingDeadlineNs = toRemove.deadlineNs;
+                        remove(toRemove);
+                    }
+                }
+                Long deadlineNs = deadlineNsCalculator.apply(existingDeadlineNs);
+                boolean queueWasEmpty = head.isSingleton();
+                boolean shouldSignal = false;
+                switch (eventContext.insertionType) {
+                    case APPEND:
+                        head.insertBefore(eventContext);
+                        if (queueWasEmpty) {
+                            shouldSignal = true;
+                        }
+                        break;
+                    case PREPEND:
+                        head.insertAfter(eventContext);
+                        if (queueWasEmpty) {
+                            shouldSignal = true;
+                        }
+                        break;
+                    case DEFERRED:
+                        if (deadlineNs == null) {
+                            eventContext.completeWithException(new RuntimeException(
+                                "You must specify a deadline for deferred events."));

Review comment:
       Hmm. This is surprising to me. The caller scheduling the event passed illegal arguments to the queue. Shouldn't this method throw an IllegalArgumentException instead of sending an exception to the event?

##########
File path: metadata/src/main/java/org/apache/kafka/queue/EventQueue.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.queue;
+
+import org.slf4j.Logger;
+
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+
+
+public interface EventQueue extends AutoCloseable {
+    interface Event {
+        void run() throws Exception;
+        default void handleException(Throwable e) {}
+    }
+
+    abstract class FailureLoggingEvent implements Event {
+        private final Logger log;
+
+        public FailureLoggingEvent(Logger log) {
+            this.log = log;
+        }
+
+        @Override
+        public void handleException(Throwable e) {
+            if (e instanceof EventQueueClosedException) {
+                log.info("Not processing {} because the event queue is closed.",
+                    this.toString());
+            } else {
+                log.error("Unexpected error handling {}", this.toString(), e);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return this.getClass().getSimpleName();
+        }
+    }
+
+    class DeadlineFunction implements Function<Long, Long> {
+        private final long deadlineNs;
+
+        public DeadlineFunction(long deadlineNs) {
+            this.deadlineNs = deadlineNs;
+        }
+
+        @Override
+        public Long apply(Long t) {
+            return deadlineNs;
+        }
+    }
+
+    class EarliestDeadlineFunction implements Function<Long, Long> {
+        private final long newDeadlineNs;
+
+        public EarliestDeadlineFunction(long newDeadlineNs) {
+            this.newDeadlineNs = newDeadlineNs;
+        }
+
+        @Override
+        public Long apply(Long prevDeadlineNs) {
+            if (prevDeadlineNs == null) {
+                return newDeadlineNs;
+            } else if (prevDeadlineNs < newDeadlineNs) {
+                return prevDeadlineNs;
+            } else {
+                return newDeadlineNs;
+            }
+        }
+    }
+
+    class VoidEvent implements Event {
+        public final static VoidEvent INSTANCE = new VoidEvent();
+
+        @Override
+        public void run() throws Exception {
+        }
+    }
+
+    /**
+     * Add an element to the front of the queue.
+     *
+     * @param event             The mandatory event to prepend.
+     */
+    default void prepend(Event event) {
+        enqueue(EventInsertionType.PREPEND, null, null, event);
+    }
+
+    /**
+     * Add an element to the end of the queue.
+     *
+     * @param event             The event to append.
+     */
+    default void append(Event event) {
+        enqueue(EventInsertionType.APPEND, null, null, event);
+    }
+
+    /**
+     * Enqueue an event to be run in FIFO order.
+     *
+     * @param deadlineNs        The time in monotonic nanoseconds after which the future
+     *                          is completed with a
+     *                          @{org.apache.kafka.common.errors.TimeoutException},
+     *                          and the event is cancelled.
+     * @param event             The event to append.
+     */
+    default void appendWithDeadline(long deadlineNs, Event event) {
+        enqueue(EventInsertionType.APPEND, null, __ -> deadlineNs, event);
+    }
+
+    /**
+     * Schedule an event to be run at a specific time.
+     *
+     * @param tag                   If this is non-null, the unique tag to use for this
+     *                              event.  If an event with this tag already exists, it
+     *                              will be cancelled.
+     * @param deadlineNsCalculator  A function which takes as an argument the existing
+     *                              deadline for the event with this tag (or null if the
+     *                              event has no tag, or if there is none such), and
+     *                              produces the deadline to use for this event.

Review comment:
       The headline description says "Schedule an event to be run at a specific time" yet this param's description seem to indicate a different behavior.

##########
File path: metadata/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.queue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.utils.KafkaThread;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+
+public final class KafkaEventQueue implements EventQueue {
+    /**
+     * A context object that wraps events.
+     */
+    private static class EventContext {
+        /**
+         * The caller-supplied event.
+         */
+        private final Event event;
+
+        /**
+         * How this event was inserted.
+         */
+        private final EventInsertionType insertionType;
+
+        /**
+         * The previous pointer of our circular doubly-linked list.
+         */
+        private EventContext prev = this;
+
+        /**
+         * The next pointer in our circular doubly-linked list.
+         */
+        private EventContext next = this;
+
+        /**
+         * If this event is in the delay map, this is the key it is there under.
+         * If it is not in the map, this is null.
+         */
+        private Long deadlineNs = null;
+
+        /**
+         * The tag associated with this event.
+         */
+        private String tag;
+
+        EventContext(Event event, EventInsertionType insertionType, String tag) {
+            this.event = event;
+            this.insertionType = insertionType;
+            this.tag = tag;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list after this node.
+         */
+        void insertAfter(EventContext other) {
+            this.next.prev = other;
+            other.next = this.next;
+            other.prev = this;
+            this.next = other;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list before this node.
+         */
+        void insertBefore(EventContext other) {
+            this.prev.next = other;
+            other.prev = this.prev;
+            other.next = this;
+            this.prev = other;
+        }
+
+        /**
+         * Remove this node from the circularly linked list.
+         */
+        void remove() {
+            this.prev.next = this.next;
+            this.next.prev = this.prev;
+            this.prev = this;
+            this.next = this;
+        }
+
+        /**
+         * Returns true if this node is the only element in its list.
+         */
+        boolean isSingleton() {
+            return prev == this && next == this;
+        }
+
+        /**
+         * Run the event associated with this EventContext.
+         */
+        void run() throws InterruptedException {
+            try {
+                event.run();
+            } catch (InterruptedException e) {
+                throw e;
+            } catch (Exception e) {
+                event.handleException(e);
+            }
+        }
+
+        /**
+         * Complete the event associated with this EventContext with a timeout exception.
+         */
+        void completeWithTimeout() {
+            completeWithException(new TimeoutException());
+        }
+
+        /**
+         * Complete the event associated with this EventContext with the specified
+         * exception.
+         */
+        void completeWithException(Throwable t) {
+            event.handleException(t);
+        }
+    }
+
+    private class EventHandler implements Runnable {
+        /**
+         * Event contexts indexed by tag.  Events without a tag are not included here.
+         */
+        private final Map<String, EventContext> tagToEventContext = new HashMap<>();
+
+        /**
+         * The head of the event queue.
+         */
+        private final EventContext head = new EventContext(null, null, null);
+
+        /**
+         * An ordered map of times in monotonic nanoseconds to events to time out.
+         */
+        private final TreeMap<Long, EventContext> delayMap = new TreeMap<>();
+
+        /**
+         * A condition variable for waking up the event handler thread.
+         */
+        private final Condition cond = lock.newCondition();
+
+        @Override
+        public void run() {
+            try {
+                handleEvents();
+                cleanupEvent.run();
+            } catch (Throwable e) {
+                log.warn("event handler thread exiting with exception", e);
+            }
+        }
+
+        private void remove(EventContext eventContext) {
+            eventContext.remove();
+            if (eventContext.deadlineNs != null) {
+                delayMap.remove(eventContext.deadlineNs);
+                eventContext.deadlineNs = null;
+            }
+            if (eventContext.tag != null) {
+                tagToEventContext.remove(eventContext.tag, eventContext);
+                eventContext.tag = null;
+            }
+        }
+
+        private void handleEvents() throws InterruptedException {
+            EventContext toTimeout = null;
+            EventContext toRun = null;
+            while (true) {
+                if (toTimeout != null) {
+                    toTimeout.completeWithTimeout();
+                    toTimeout = null;
+                } else if (toRun != null) {
+                    toRun.run();
+                    toRun = null;
+                }
+                lock.lock();
+                try {
+                    long awaitNs = Long.MAX_VALUE;
+                    Map.Entry<Long, EventContext> entry = delayMap.firstEntry();
+                    if (entry != null) {
+                        // Search for timed-out events or deferred events that are ready
+                        // to run.
+                        long now = time.nanoseconds();
+                        long timeoutNs = entry.getKey();
+                        EventContext eventContext = entry.getValue();
+                        if (timeoutNs <= now) {
+                            if (eventContext.insertionType == EventInsertionType.DEFERRED) {
+                                // The deferred event is ready to run.  Prepend it to the
+                                // queue.  (The value for deferred events is a schedule time
+                                // rather than a timeout.)
+                                remove(eventContext);
+                                head.insertAfter(eventContext);
+                            } else {
+                                // not a deferred event, so it is a deadline, and it is timed out.
+                                remove(eventContext);
+                                toTimeout = eventContext;
+                            }
+                            continue;
+                        } else if (closingTimeNs <= now) {
+                            remove(eventContext);
+                            toTimeout = eventContext;
+                            continue;
+                        }
+                        awaitNs = timeoutNs - now;
+                    }
+                    if (head.next == head) {
+                        if ((closingTimeNs != Long.MAX_VALUE) && delayMap.isEmpty()) {
+                            // If there are no more entries to process, and the queue is
+                            // closing, exit the thread.
+                            return;
+                        }
+                    } else {
+                        toRun = head.next;
+                        remove(toRun);
+                        continue;
+                    }
+                    if (closingTimeNs != Long.MAX_VALUE) {
+                        long now = time.nanoseconds();
+                        if (awaitNs > closingTimeNs - now) {
+                            awaitNs = closingTimeNs - now;
+                        }
+                    }
+                    if (awaitNs == Long.MAX_VALUE) {
+                        cond.await();
+                    } else {
+                        cond.awaitNanos(awaitNs);
+                    }
+                } finally {
+                    lock.unlock();
+                }
+            }
+        }
+
+        private void enqueue(EventContext eventContext,
+                             Function<Long, Long> deadlineNsCalculator) {
+            lock.lock();
+            try {
+                Long existingDeadlineNs = null;
+                if (eventContext.tag != null) {
+                    EventContext toRemove =
+                        tagToEventContext.put(eventContext.tag, eventContext);
+                    if (toRemove != null) {
+                        existingDeadlineNs = toRemove.deadlineNs;
+                        remove(toRemove);
+                    }
+                }
+                Long deadlineNs = deadlineNsCalculator.apply(existingDeadlineNs);
+                boolean queueWasEmpty = head.isSingleton();
+                boolean shouldSignal = false;
+                switch (eventContext.insertionType) {
+                    case APPEND:
+                        head.insertBefore(eventContext);
+                        if (queueWasEmpty) {
+                            shouldSignal = true;
+                        }
+                        break;
+                    case PREPEND:
+                        head.insertAfter(eventContext);
+                        if (queueWasEmpty) {
+                            shouldSignal = true;
+                        }
+                        break;
+                    case DEFERRED:
+                        if (deadlineNs == null) {
+                            eventContext.completeWithException(new RuntimeException(
+                                "You must specify a deadline for deferred events."));
+                            return;
+                        }
+                        break;
+                }
+                if (deadlineNs != null) {
+                    long insertNs =  deadlineNs;
+                    long prevStartNs = delayMap.isEmpty() ? Long.MAX_VALUE : delayMap.firstKey();
+                    // If the time in nanoseconds is already taken, take the next one.
+                    while (delayMap.putIfAbsent(insertNs, eventContext) != null) {
+                        insertNs++;
+                    }
+                    eventContext.deadlineNs = insertNs;
+                    // If the new timeout is before all the existing ones, wake up the
+                    // timeout thread.
+                    if (insertNs <= prevStartNs) {
+                        shouldSignal = true;
+                    }
+                }
+                if (shouldSignal) {
+                    cond.signal();
+                }
+            } finally {
+                lock.unlock();
+            }
+        }
+
+        public void cancelDeferred(String tag) {
+            EventContext eventContext = tagToEventContext.get(tag);
+            if (eventContext != null) {
+                remove(eventContext);
+            }
+        }
+    }
+
+    private final Time time;
+    private final ReentrantLock lock;
+    private final Logger log;
+    private final EventHandler eventHandler;
+    private final Thread eventHandlerThread;
+
+    /**
+     * The time in monotonic nanoseconds when the queue is closing, or Long.MAX_VALUE if
+     * the queue is not currently closing.
+     */
+    private long closingTimeNs;
+
+    private Event cleanupEvent;
+
+    public KafkaEventQueue(Time time,
+                           LogContext logContext,
+                           String threadNamePrefix) {
+        this.time = time;
+        this.lock = new ReentrantLock();
+        this.log = logContext.logger(KafkaEventQueue.class);
+        this.eventHandler = new EventHandler();
+        this.eventHandlerThread = new KafkaThread(threadNamePrefix + "EventHandler",
+            this.eventHandler, false);
+        this.closingTimeNs = Long.MAX_VALUE;
+        this.cleanupEvent = null;
+        this.eventHandlerThread.start();
+    }
+
+    @Override
+    public void enqueue(EventInsertionType insertionType,
+                        String tag,
+                        Function<Long, Long> deadlineNsCalculator,
+                        Event event) {
+        lock.lock();
+        try {
+            EventContext eventContext = new EventContext(event, insertionType, tag);
+            if (closingTimeNs != Long.MAX_VALUE) {
+                eventContext.completeWithException(new EventQueueClosedException());
+            } else {
+                eventHandler.enqueue(eventContext,
+                    deadlineNsCalculator == null ? __ -> null : deadlineNsCalculator);
+            }
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    @Override
+    public void cancelDeferred(String tag) {
+        lock.lock();
+        try {
+            eventHandler.cancelDeferred(tag);
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    @Override
+    public void beginShutdown(String source, Event newCleanupEvent,
+                              TimeUnit timeUnit, long timeSpan) {
+        if (timeSpan < 0) {
+            throw new IllegalArgumentException("beginShutdown must be called with a " +
+                "non-negative timeout.");
+        }
+        Objects.requireNonNull(newCleanupEvent);
+        lock.lock();
+        try {
+            if (cleanupEvent != null) {
+                log.debug("{}: Event queue is already shut down.", source);
+                return;
+            }
+            log.info("{}: shutting down event queue.", source);
+            cleanupEvent = newCleanupEvent;
+            long newClosingTimeNs = time.nanoseconds() + timeUnit.toNanos(timeSpan);
+            if (closingTimeNs >= newClosingTimeNs)
+                closingTimeNs = newClosingTimeNs;
+            eventHandler.cond.signal();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    @Override
+    public void wakeup() {
+        lock.lock();
+        try {
+            eventHandler.cond.signal();
+        } finally {
+            lock.unlock();
+        }
+    }

Review comment:
       How about moving this concurrency complexity to `EventHandler` by adding a method called `EventHandler::wakeup`?

##########
File path: metadata/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.queue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.utils.KafkaThread;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+
+public final class KafkaEventQueue implements EventQueue {
+    /**
+     * A context object that wraps events.
+     */
+    private static class EventContext {
+        /**
+         * The caller-supplied event.
+         */
+        private final Event event;
+
+        /**
+         * How this event was inserted.
+         */
+        private final EventInsertionType insertionType;
+
+        /**
+         * The previous pointer of our circular doubly-linked list.
+         */
+        private EventContext prev = this;
+
+        /**
+         * The next pointer in our circular doubly-linked list.
+         */
+        private EventContext next = this;
+
+        /**
+         * If this event is in the delay map, this is the key it is there under.
+         * If it is not in the map, this is null.
+         */
+        private Long deadlineNs = null;
+
+        /**
+         * The tag associated with this event.
+         */
+        private String tag;
+
+        EventContext(Event event, EventInsertionType insertionType, String tag) {
+            this.event = event;
+            this.insertionType = insertionType;
+            this.tag = tag;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list after this node.

Review comment:
       How about "Insert the EventContext other in the circularly linked list after this node." I would also mention that this requires that `other.isSingleton()` is true. Should we check for that in this implementation?

##########
File path: metadata/src/main/java/org/apache/kafka/queue/EventQueueClosedException.java
##########
@@ -0,0 +1,27 @@
+/*
+ * 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.queue;
+
+
+public class EventQueueClosedException extends RuntimeException {

Review comment:
       Can we use Java's existing type `java.util.concurrent.RejectedExecutionException`?

##########
File path: metadata/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.queue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.utils.KafkaThread;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+
+public final class KafkaEventQueue implements EventQueue {
+    /**
+     * A context object that wraps events.
+     */
+    private static class EventContext {
+        /**
+         * The caller-supplied event.
+         */
+        private final Event event;
+
+        /**
+         * How this event was inserted.
+         */
+        private final EventInsertionType insertionType;
+
+        /**
+         * The previous pointer of our circular doubly-linked list.
+         */
+        private EventContext prev = this;
+
+        /**
+         * The next pointer in our circular doubly-linked list.
+         */
+        private EventContext next = this;
+
+        /**
+         * If this event is in the delay map, this is the key it is there under.
+         * If it is not in the map, this is null.
+         */
+        private Long deadlineNs = null;
+
+        /**
+         * The tag associated with this event.
+         */
+        private String tag;
+
+        EventContext(Event event, EventInsertionType insertionType, String tag) {
+            this.event = event;
+            this.insertionType = insertionType;
+            this.tag = tag;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list after this node.
+         */
+        void insertAfter(EventContext other) {
+            this.next.prev = other;
+            other.next = this.next;
+            other.prev = this;
+            this.next = other;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list before this node.
+         */
+        void insertBefore(EventContext other) {
+            this.prev.next = other;
+            other.prev = this.prev;
+            other.next = this;
+            this.prev = other;
+        }
+
+        /**
+         * Remove this node from the circularly linked list.
+         */
+        void remove() {
+            this.prev.next = this.next;
+            this.next.prev = this.prev;
+            this.prev = this;
+            this.next = this;
+        }
+
+        /**
+         * Returns true if this node is the only element in its list.
+         */
+        boolean isSingleton() {
+            return prev == this && next == this;
+        }
+
+        /**
+         * Run the event associated with this EventContext.
+         */
+        void run() throws InterruptedException {
+            try {
+                event.run();
+            } catch (InterruptedException e) {
+                throw e;
+            } catch (Exception e) {
+                event.handleException(e);

Review comment:
       Do we want to handle the case where `Event::handleException` throws?

##########
File path: metadata/src/main/java/org/apache/kafka/queue/EventQueue.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.queue;
+
+import org.slf4j.Logger;
+
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+
+
+public interface EventQueue extends AutoCloseable {
+    interface Event {
+        void run() throws Exception;
+        default void handleException(Throwable e) {}
+    }

Review comment:
       Let's document these two methods. For example, looking at the implementation it looks like `handleException` needs to handle:
   1. Exceptions thrown by `Event::run`
   2. Exception passed by the `EventQueue` like `EventQueueClosedException`, `TimeoutException`, anything else?
   
   Any thoughts on why this design instead of not allowing `run` to throw a checked exception? For example:
   ```java
   interface Event extends Runnable {
       default void cancel(Throwable e) {}
   }
   ```

##########
File path: metadata/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.queue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.utils.KafkaThread;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+
+public final class KafkaEventQueue implements EventQueue {
+    /**
+     * A context object that wraps events.
+     */
+    private static class EventContext {
+        /**
+         * The caller-supplied event.
+         */
+        private final Event event;
+
+        /**
+         * How this event was inserted.
+         */
+        private final EventInsertionType insertionType;
+
+        /**
+         * The previous pointer of our circular doubly-linked list.
+         */
+        private EventContext prev = this;
+
+        /**
+         * The next pointer in our circular doubly-linked list.
+         */
+        private EventContext next = this;
+
+        /**
+         * If this event is in the delay map, this is the key it is there under.
+         * If it is not in the map, this is null.
+         */
+        private Long deadlineNs = null;

Review comment:
       We can use `OptionalLong`.

##########
File path: metadata/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.queue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.utils.KafkaThread;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+
+public final class KafkaEventQueue implements EventQueue {
+    /**
+     * A context object that wraps events.
+     */
+    private static class EventContext {
+        /**
+         * The caller-supplied event.
+         */
+        private final Event event;
+
+        /**
+         * How this event was inserted.
+         */
+        private final EventInsertionType insertionType;
+
+        /**
+         * The previous pointer of our circular doubly-linked list.
+         */
+        private EventContext prev = this;
+
+        /**
+         * The next pointer in our circular doubly-linked list.
+         */
+        private EventContext next = this;
+
+        /**
+         * If this event is in the delay map, this is the key it is there under.
+         * If it is not in the map, this is null.
+         */
+        private Long deadlineNs = null;
+
+        /**
+         * The tag associated with this event.
+         */
+        private String tag;
+
+        EventContext(Event event, EventInsertionType insertionType, String tag) {
+            this.event = event;
+            this.insertionType = insertionType;
+            this.tag = tag;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list after this node.
+         */
+        void insertAfter(EventContext other) {
+            this.next.prev = other;
+            other.next = this.next;
+            other.prev = this;
+            this.next = other;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list before this node.
+         */
+        void insertBefore(EventContext other) {
+            this.prev.next = other;
+            other.prev = this.prev;
+            other.next = this;
+            this.prev = other;
+        }
+
+        /**
+         * Remove this node from the circularly linked list.
+         */
+        void remove() {
+            this.prev.next = this.next;
+            this.next.prev = this.prev;
+            this.prev = this;
+            this.next = this;
+        }
+
+        /**
+         * Returns true if this node is the only element in its list.
+         */
+        boolean isSingleton() {
+            return prev == this && next == this;
+        }
+
+        /**
+         * Run the event associated with this EventContext.
+         */
+        void run() throws InterruptedException {
+            try {
+                event.run();
+            } catch (InterruptedException e) {
+                throw e;
+            } catch (Exception e) {
+                event.handleException(e);
+            }
+        }
+
+        /**
+         * Complete the event associated with this EventContext with a timeout exception.
+         */
+        void completeWithTimeout() {
+            completeWithException(new TimeoutException());
+        }
+
+        /**
+         * Complete the event associated with this EventContext with the specified
+         * exception.
+         */
+        void completeWithException(Throwable t) {
+            event.handleException(t);
+        }
+    }
+
+    private class EventHandler implements Runnable {
+        /**
+         * Event contexts indexed by tag.  Events without a tag are not included here.
+         */
+        private final Map<String, EventContext> tagToEventContext = new HashMap<>();
+
+        /**
+         * The head of the event queue.
+         */
+        private final EventContext head = new EventContext(null, null, null);
+
+        /**
+         * An ordered map of times in monotonic nanoseconds to events to time out.
+         */
+        private final TreeMap<Long, EventContext> delayMap = new TreeMap<>();
+
+        /**
+         * A condition variable for waking up the event handler thread.
+         */
+        private final Condition cond = lock.newCondition();
+
+        @Override
+        public void run() {
+            try {
+                handleEvents();
+                cleanupEvent.run();
+            } catch (Throwable e) {
+                log.warn("event handler thread exiting with exception", e);
+            }
+        }
+
+        private void remove(EventContext eventContext) {
+            eventContext.remove();
+            if (eventContext.deadlineNs != null) {
+                delayMap.remove(eventContext.deadlineNs);
+                eventContext.deadlineNs = null;
+            }
+            if (eventContext.tag != null) {
+                tagToEventContext.remove(eventContext.tag, eventContext);
+                eventContext.tag = null;
+            }
+        }
+
+        private void handleEvents() throws InterruptedException {
+            EventContext toTimeout = null;
+            EventContext toRun = null;
+            while (true) {
+                if (toTimeout != null) {
+                    toTimeout.completeWithTimeout();
+                    toTimeout = null;
+                } else if (toRun != null) {
+                    toRun.run();
+                    toRun = null;
+                }
+                lock.lock();
+                try {
+                    long awaitNs = Long.MAX_VALUE;
+                    Map.Entry<Long, EventContext> entry = delayMap.firstEntry();
+                    if (entry != null) {
+                        // Search for timed-out events or deferred events that are ready
+                        // to run.
+                        long now = time.nanoseconds();
+                        long timeoutNs = entry.getKey();
+                        EventContext eventContext = entry.getValue();
+                        if (timeoutNs <= now) {
+                            if (eventContext.insertionType == EventInsertionType.DEFERRED) {
+                                // The deferred event is ready to run.  Prepend it to the
+                                // queue.  (The value for deferred events is a schedule time
+                                // rather than a timeout.)
+                                remove(eventContext);
+                                head.insertAfter(eventContext);
+                            } else {
+                                // not a deferred event, so it is a deadline, and it is timed out.
+                                remove(eventContext);
+                                toTimeout = eventContext;
+                            }
+                            continue;
+                        } else if (closingTimeNs <= now) {
+                            remove(eventContext);
+                            toTimeout = eventContext;
+                            continue;
+                        }
+                        awaitNs = timeoutNs - now;
+                    }
+                    if (head.next == head) {
+                        if ((closingTimeNs != Long.MAX_VALUE) && delayMap.isEmpty()) {
+                            // If there are no more entries to process, and the queue is
+                            // closing, exit the thread.
+                            return;
+                        }
+                    } else {
+                        toRun = head.next;
+                        remove(toRun);
+                        continue;
+                    }
+                    if (closingTimeNs != Long.MAX_VALUE) {
+                        long now = time.nanoseconds();
+                        if (awaitNs > closingTimeNs - now) {
+                            awaitNs = closingTimeNs - now;
+                        }
+                    }
+                    if (awaitNs == Long.MAX_VALUE) {
+                        cond.await();
+                    } else {
+                        cond.awaitNanos(awaitNs);
+                    }
+                } finally {
+                    lock.unlock();
+                }
+            }
+        }
+
+        private void enqueue(EventContext eventContext,
+                             Function<Long, Long> deadlineNsCalculator) {
+            lock.lock();
+            try {
+                Long existingDeadlineNs = null;
+                if (eventContext.tag != null) {
+                    EventContext toRemove =
+                        tagToEventContext.put(eventContext.tag, eventContext);
+                    if (toRemove != null) {
+                        existingDeadlineNs = toRemove.deadlineNs;
+                        remove(toRemove);
+                    }
+                }
+                Long deadlineNs = deadlineNsCalculator.apply(existingDeadlineNs);
+                boolean queueWasEmpty = head.isSingleton();
+                boolean shouldSignal = false;
+                switch (eventContext.insertionType) {
+                    case APPEND:
+                        head.insertBefore(eventContext);
+                        if (queueWasEmpty) {
+                            shouldSignal = true;
+                        }
+                        break;
+                    case PREPEND:
+                        head.insertAfter(eventContext);
+                        if (queueWasEmpty) {
+                            shouldSignal = true;
+                        }
+                        break;
+                    case DEFERRED:
+                        if (deadlineNs == null) {
+                            eventContext.completeWithException(new RuntimeException(
+                                "You must specify a deadline for deferred events."));
+                            return;
+                        }
+                        break;
+                }
+                if (deadlineNs != null) {
+                    long insertNs =  deadlineNs;
+                    long prevStartNs = delayMap.isEmpty() ? Long.MAX_VALUE : delayMap.firstKey();
+                    // If the time in nanoseconds is already taken, take the next one.
+                    while (delayMap.putIfAbsent(insertNs, eventContext) != null) {
+                        insertNs++;
+                    }
+                    eventContext.deadlineNs = insertNs;
+                    // If the new timeout is before all the existing ones, wake up the
+                    // timeout thread.
+                    if (insertNs <= prevStartNs) {
+                        shouldSignal = true;
+                    }
+                }
+                if (shouldSignal) {
+                    cond.signal();
+                }
+            } finally {
+                lock.unlock();
+            }
+        }
+
+        public void cancelDeferred(String tag) {
+            EventContext eventContext = tagToEventContext.get(tag);
+            if (eventContext != null) {
+                remove(eventContext);
+            }
+        }
+    }
+
+    private final Time time;
+    private final ReentrantLock lock;
+    private final Logger log;
+    private final EventHandler eventHandler;
+    private final Thread eventHandlerThread;
+
+    /**
+     * The time in monotonic nanoseconds when the queue is closing, or Long.MAX_VALUE if
+     * the queue is not currently closing.
+     */
+    private long closingTimeNs;
+
+    private Event cleanupEvent;
+
+    public KafkaEventQueue(Time time,
+                           LogContext logContext,
+                           String threadNamePrefix) {
+        this.time = time;
+        this.lock = new ReentrantLock();
+        this.log = logContext.logger(KafkaEventQueue.class);
+        this.eventHandler = new EventHandler();
+        this.eventHandlerThread = new KafkaThread(threadNamePrefix + "EventHandler",
+            this.eventHandler, false);
+        this.closingTimeNs = Long.MAX_VALUE;
+        this.cleanupEvent = null;
+        this.eventHandlerThread.start();
+    }
+
+    @Override
+    public void enqueue(EventInsertionType insertionType,
+                        String tag,
+                        Function<Long, Long> deadlineNsCalculator,
+                        Event event) {
+        lock.lock();
+        try {
+            EventContext eventContext = new EventContext(event, insertionType, tag);
+            if (closingTimeNs != Long.MAX_VALUE) {
+                eventContext.completeWithException(new EventQueueClosedException());
+            } else {
+                eventHandler.enqueue(eventContext,
+                    deadlineNsCalculator == null ? __ -> null : deadlineNsCalculator);
+            }
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    @Override
+    public void cancelDeferred(String tag) {
+        lock.lock();

Review comment:
       We can move this lock to `EventHandler::cancelDeferred`. `EventHandler` is already handling concurrent access to the fields in this object.

##########
File path: metadata/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.queue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.utils.KafkaThread;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+
+public final class KafkaEventQueue implements EventQueue {
+    /**
+     * A context object that wraps events.
+     */
+    private static class EventContext {
+        /**
+         * The caller-supplied event.
+         */
+        private final Event event;
+
+        /**
+         * How this event was inserted.
+         */
+        private final EventInsertionType insertionType;
+
+        /**
+         * The previous pointer of our circular doubly-linked list.
+         */
+        private EventContext prev = this;
+
+        /**
+         * The next pointer in our circular doubly-linked list.
+         */
+        private EventContext next = this;
+
+        /**
+         * If this event is in the delay map, this is the key it is there under.
+         * If it is not in the map, this is null.
+         */
+        private Long deadlineNs = null;
+
+        /**
+         * The tag associated with this event.
+         */
+        private String tag;

Review comment:
       We can use `Optional<String>`.

##########
File path: metadata/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.queue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.utils.KafkaThread;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+
+public final class KafkaEventQueue implements EventQueue {
+    /**
+     * A context object that wraps events.
+     */
+    private static class EventContext {
+        /**
+         * The caller-supplied event.
+         */
+        private final Event event;
+
+        /**
+         * How this event was inserted.
+         */
+        private final EventInsertionType insertionType;
+
+        /**
+         * The previous pointer of our circular doubly-linked list.
+         */
+        private EventContext prev = this;
+
+        /**
+         * The next pointer in our circular doubly-linked list.
+         */
+        private EventContext next = this;
+
+        /**
+         * If this event is in the delay map, this is the key it is there under.
+         * If it is not in the map, this is null.
+         */
+        private Long deadlineNs = null;
+
+        /**
+         * The tag associated with this event.
+         */
+        private String tag;
+
+        EventContext(Event event, EventInsertionType insertionType, String tag) {
+            this.event = event;
+            this.insertionType = insertionType;
+            this.tag = tag;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list after this node.
+         */
+        void insertAfter(EventContext other) {
+            this.next.prev = other;
+            other.next = this.next;
+            other.prev = this;
+            this.next = other;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list before this node.
+         */
+        void insertBefore(EventContext other) {
+            this.prev.next = other;
+            other.prev = this.prev;
+            other.next = this;
+            this.prev = other;
+        }
+
+        /**
+         * Remove this node from the circularly linked list.
+         */
+        void remove() {
+            this.prev.next = this.next;
+            this.next.prev = this.prev;
+            this.prev = this;
+            this.next = this;
+        }
+
+        /**
+         * Returns true if this node is the only element in its list.
+         */
+        boolean isSingleton() {
+            return prev == this && next == this;
+        }
+
+        /**
+         * Run the event associated with this EventContext.
+         */
+        void run() throws InterruptedException {
+            try {
+                event.run();
+            } catch (InterruptedException e) {
+                throw e;
+            } catch (Exception e) {
+                event.handleException(e);
+            }
+        }
+
+        /**
+         * Complete the event associated with this EventContext with a timeout exception.
+         */
+        void completeWithTimeout() {
+            completeWithException(new TimeoutException());
+        }
+
+        /**
+         * Complete the event associated with this EventContext with the specified
+         * exception.
+         */
+        void completeWithException(Throwable t) {
+            event.handleException(t);
+        }
+    }
+
+    private class EventHandler implements Runnable {
+        /**
+         * Event contexts indexed by tag.  Events without a tag are not included here.
+         */
+        private final Map<String, EventContext> tagToEventContext = new HashMap<>();
+
+        /**
+         * The head of the event queue.
+         */
+        private final EventContext head = new EventContext(null, null, null);
+
+        /**
+         * An ordered map of times in monotonic nanoseconds to events to time out.
+         */
+        private final TreeMap<Long, EventContext> delayMap = new TreeMap<>();
+
+        /**
+         * A condition variable for waking up the event handler thread.
+         */
+        private final Condition cond = lock.newCondition();
+
+        @Override
+        public void run() {
+            try {
+                handleEvents();
+                cleanupEvent.run();
+            } catch (Throwable e) {
+                log.warn("event handler thread exiting with exception", e);
+            }
+        }
+
+        private void remove(EventContext eventContext) {
+            eventContext.remove();
+            if (eventContext.deadlineNs != null) {
+                delayMap.remove(eventContext.deadlineNs);
+                eventContext.deadlineNs = null;
+            }
+            if (eventContext.tag != null) {
+                tagToEventContext.remove(eventContext.tag, eventContext);
+                eventContext.tag = null;
+            }
+        }
+
+        private void handleEvents() throws InterruptedException {
+            EventContext toTimeout = null;
+            EventContext toRun = null;
+            while (true) {
+                if (toTimeout != null) {
+                    toTimeout.completeWithTimeout();
+                    toTimeout = null;
+                } else if (toRun != null) {
+                    toRun.run();
+                    toRun = null;
+                }
+                lock.lock();
+                try {
+                    long awaitNs = Long.MAX_VALUE;
+                    Map.Entry<Long, EventContext> entry = delayMap.firstEntry();
+                    if (entry != null) {
+                        // Search for timed-out events or deferred events that are ready
+                        // to run.
+                        long now = time.nanoseconds();
+                        long timeoutNs = entry.getKey();
+                        EventContext eventContext = entry.getValue();
+                        if (timeoutNs <= now) {
+                            if (eventContext.insertionType == EventInsertionType.DEFERRED) {
+                                // The deferred event is ready to run.  Prepend it to the
+                                // queue.  (The value for deferred events is a schedule time
+                                // rather than a timeout.)
+                                remove(eventContext);
+                                head.insertAfter(eventContext);
+                            } else {
+                                // not a deferred event, so it is a deadline, and it is timed out.
+                                remove(eventContext);
+                                toTimeout = eventContext;
+                            }
+                            continue;
+                        } else if (closingTimeNs <= now) {
+                            remove(eventContext);
+                            toTimeout = eventContext;
+                            continue;
+                        }
+                        awaitNs = timeoutNs - now;
+                    }
+                    if (head.next == head) {
+                        if ((closingTimeNs != Long.MAX_VALUE) && delayMap.isEmpty()) {
+                            // If there are no more entries to process, and the queue is
+                            // closing, exit the thread.
+                            return;
+                        }
+                    } else {
+                        toRun = head.next;
+                        remove(toRun);
+                        continue;
+                    }
+                    if (closingTimeNs != Long.MAX_VALUE) {
+                        long now = time.nanoseconds();
+                        if (awaitNs > closingTimeNs - now) {
+                            awaitNs = closingTimeNs - now;
+                        }
+                    }
+                    if (awaitNs == Long.MAX_VALUE) {
+                        cond.await();
+                    } else {
+                        cond.awaitNanos(awaitNs);
+                    }
+                } finally {
+                    lock.unlock();
+                }
+            }
+        }
+
+        private void enqueue(EventContext eventContext,
+                             Function<Long, Long> deadlineNsCalculator) {
+            lock.lock();
+            try {
+                Long existingDeadlineNs = null;
+                if (eventContext.tag != null) {
+                    EventContext toRemove =
+                        tagToEventContext.put(eventContext.tag, eventContext);
+                    if (toRemove != null) {
+                        existingDeadlineNs = toRemove.deadlineNs;
+                        remove(toRemove);
+                    }
+                }
+                Long deadlineNs = deadlineNsCalculator.apply(existingDeadlineNs);
+                boolean queueWasEmpty = head.isSingleton();
+                boolean shouldSignal = false;
+                switch (eventContext.insertionType) {
+                    case APPEND:
+                        head.insertBefore(eventContext);
+                        if (queueWasEmpty) {
+                            shouldSignal = true;
+                        }
+                        break;
+                    case PREPEND:
+                        head.insertAfter(eventContext);
+                        if (queueWasEmpty) {
+                            shouldSignal = true;
+                        }
+                        break;
+                    case DEFERRED:
+                        if (deadlineNs == null) {
+                            eventContext.completeWithException(new RuntimeException(
+                                "You must specify a deadline for deferred events."));
+                            return;
+                        }
+                        break;
+                }
+                if (deadlineNs != null) {
+                    long insertNs =  deadlineNs;
+                    long prevStartNs = delayMap.isEmpty() ? Long.MAX_VALUE : delayMap.firstKey();
+                    // If the time in nanoseconds is already taken, take the next one.
+                    while (delayMap.putIfAbsent(insertNs, eventContext) != null) {
+                        insertNs++;
+                    }

Review comment:
       Hmm. This can cause an event with delay X to be schedule after an event with delay Y even if X < Y.

##########
File path: metadata/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.queue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.utils.KafkaThread;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+
+public final class KafkaEventQueue implements EventQueue {
+    /**
+     * A context object that wraps events.
+     */
+    private static class EventContext {
+        /**
+         * The caller-supplied event.
+         */
+        private final Event event;
+
+        /**
+         * How this event was inserted.
+         */
+        private final EventInsertionType insertionType;
+
+        /**
+         * The previous pointer of our circular doubly-linked list.
+         */
+        private EventContext prev = this;
+
+        /**
+         * The next pointer in our circular doubly-linked list.
+         */
+        private EventContext next = this;
+
+        /**
+         * If this event is in the delay map, this is the key it is there under.
+         * If it is not in the map, this is null.
+         */
+        private Long deadlineNs = null;
+
+        /**
+         * The tag associated with this event.
+         */
+        private String tag;
+
+        EventContext(Event event, EventInsertionType insertionType, String tag) {
+            this.event = event;
+            this.insertionType = insertionType;
+            this.tag = tag;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list after this node.
+         */
+        void insertAfter(EventContext other) {
+            this.next.prev = other;
+            other.next = this.next;
+            other.prev = this;
+            this.next = other;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list before this node.
+         */
+        void insertBefore(EventContext other) {
+            this.prev.next = other;
+            other.prev = this.prev;
+            other.next = this;
+            this.prev = other;
+        }
+
+        /**
+         * Remove this node from the circularly linked list.
+         */
+        void remove() {
+            this.prev.next = this.next;
+            this.next.prev = this.prev;
+            this.prev = this;
+            this.next = this;
+        }
+
+        /**
+         * Returns true if this node is the only element in its list.
+         */
+        boolean isSingleton() {
+            return prev == this && next == this;
+        }
+
+        /**
+         * Run the event associated with this EventContext.
+         */
+        void run() throws InterruptedException {
+            try {
+                event.run();
+            } catch (InterruptedException e) {
+                throw e;
+            } catch (Exception e) {
+                event.handleException(e);
+            }
+        }
+
+        /**
+         * Complete the event associated with this EventContext with a timeout exception.
+         */
+        void completeWithTimeout() {
+            completeWithException(new TimeoutException());
+        }
+
+        /**
+         * Complete the event associated with this EventContext with the specified
+         * exception.
+         */
+        void completeWithException(Throwable t) {
+            event.handleException(t);
+        }
+    }
+
+    private class EventHandler implements Runnable {
+        /**
+         * Event contexts indexed by tag.  Events without a tag are not included here.
+         */
+        private final Map<String, EventContext> tagToEventContext = new HashMap<>();
+
+        /**
+         * The head of the event queue.
+         */
+        private final EventContext head = new EventContext(null, null, null);
+
+        /**
+         * An ordered map of times in monotonic nanoseconds to events to time out.
+         */
+        private final TreeMap<Long, EventContext> delayMap = new TreeMap<>();
+
+        /**
+         * A condition variable for waking up the event handler thread.
+         */
+        private final Condition cond = lock.newCondition();
+
+        @Override
+        public void run() {
+            try {
+                handleEvents();
+                cleanupEvent.run();
+            } catch (Throwable e) {
+                log.warn("event handler thread exiting with exception", e);
+            }
+        }
+
+        private void remove(EventContext eventContext) {
+            eventContext.remove();
+            if (eventContext.deadlineNs != null) {
+                delayMap.remove(eventContext.deadlineNs);
+                eventContext.deadlineNs = null;
+            }
+            if (eventContext.tag != null) {
+                tagToEventContext.remove(eventContext.tag, eventContext);
+                eventContext.tag = null;
+            }
+        }
+
+        private void handleEvents() throws InterruptedException {
+            EventContext toTimeout = null;
+            EventContext toRun = null;
+            while (true) {
+                if (toTimeout != null) {
+                    toTimeout.completeWithTimeout();
+                    toTimeout = null;
+                } else if (toRun != null) {
+                    toRun.run();
+                    toRun = null;
+                }
+                lock.lock();
+                try {
+                    long awaitNs = Long.MAX_VALUE;
+                    Map.Entry<Long, EventContext> entry = delayMap.firstEntry();
+                    if (entry != null) {
+                        // Search for timed-out events or deferred events that are ready
+                        // to run.
+                        long now = time.nanoseconds();
+                        long timeoutNs = entry.getKey();
+                        EventContext eventContext = entry.getValue();
+                        if (timeoutNs <= now) {
+                            if (eventContext.insertionType == EventInsertionType.DEFERRED) {
+                                // The deferred event is ready to run.  Prepend it to the
+                                // queue.  (The value for deferred events is a schedule time
+                                // rather than a timeout.)
+                                remove(eventContext);
+                                head.insertAfter(eventContext);
+                            } else {
+                                // not a deferred event, so it is a deadline, and it is timed out.
+                                remove(eventContext);
+                                toTimeout = eventContext;
+                            }
+                            continue;
+                        } else if (closingTimeNs <= now) {
+                            remove(eventContext);
+                            toTimeout = eventContext;
+                            continue;
+                        }
+                        awaitNs = timeoutNs - now;
+                    }
+                    if (head.next == head) {
+                        if ((closingTimeNs != Long.MAX_VALUE) && delayMap.isEmpty()) {
+                            // If there are no more entries to process, and the queue is
+                            // closing, exit the thread.
+                            return;
+                        }
+                    } else {
+                        toRun = head.next;
+                        remove(toRun);
+                        continue;
+                    }
+                    if (closingTimeNs != Long.MAX_VALUE) {
+                        long now = time.nanoseconds();
+                        if (awaitNs > closingTimeNs - now) {
+                            awaitNs = closingTimeNs - now;
+                        }
+                    }
+                    if (awaitNs == Long.MAX_VALUE) {
+                        cond.await();
+                    } else {
+                        cond.awaitNanos(awaitNs);
+                    }
+                } finally {
+                    lock.unlock();
+                }
+            }
+        }
+
+        private void enqueue(EventContext eventContext,
+                             Function<Long, Long> deadlineNsCalculator) {
+            lock.lock();
+            try {
+                Long existingDeadlineNs = null;
+                if (eventContext.tag != null) {
+                    EventContext toRemove =
+                        tagToEventContext.put(eventContext.tag, eventContext);
+                    if (toRemove != null) {
+                        existingDeadlineNs = toRemove.deadlineNs;
+                        remove(toRemove);
+                    }
+                }
+                Long deadlineNs = deadlineNsCalculator.apply(existingDeadlineNs);
+                boolean queueWasEmpty = head.isSingleton();
+                boolean shouldSignal = false;
+                switch (eventContext.insertionType) {
+                    case APPEND:
+                        head.insertBefore(eventContext);
+                        if (queueWasEmpty) {
+                            shouldSignal = true;
+                        }
+                        break;
+                    case PREPEND:
+                        head.insertAfter(eventContext);
+                        if (queueWasEmpty) {
+                            shouldSignal = true;
+                        }
+                        break;
+                    case DEFERRED:
+                        if (deadlineNs == null) {
+                            eventContext.completeWithException(new RuntimeException(
+                                "You must specify a deadline for deferred events."));
+                            return;
+                        }
+                        break;
+                }
+                if (deadlineNs != null) {
+                    long insertNs =  deadlineNs;
+                    long prevStartNs = delayMap.isEmpty() ? Long.MAX_VALUE : delayMap.firstKey();
+                    // If the time in nanoseconds is already taken, take the next one.
+                    while (delayMap.putIfAbsent(insertNs, eventContext) != null) {
+                        insertNs++;
+                    }
+                    eventContext.deadlineNs = insertNs;
+                    // If the new timeout is before all the existing ones, wake up the
+                    // timeout thread.
+                    if (insertNs <= prevStartNs) {
+                        shouldSignal = true;
+                    }
+                }
+                if (shouldSignal) {
+                    cond.signal();
+                }
+            } finally {
+                lock.unlock();
+            }
+        }
+
+        public void cancelDeferred(String tag) {
+            EventContext eventContext = tagToEventContext.get(tag);
+            if (eventContext != null) {
+                remove(eventContext);
+            }
+        }
+    }
+
+    private final Time time;
+    private final ReentrantLock lock;
+    private final Logger log;
+    private final EventHandler eventHandler;
+    private final Thread eventHandlerThread;
+
+    /**
+     * The time in monotonic nanoseconds when the queue is closing, or Long.MAX_VALUE if
+     * the queue is not currently closing.
+     */
+    private long closingTimeNs;
+
+    private Event cleanupEvent;
+
+    public KafkaEventQueue(Time time,
+                           LogContext logContext,
+                           String threadNamePrefix) {
+        this.time = time;
+        this.lock = new ReentrantLock();
+        this.log = logContext.logger(KafkaEventQueue.class);
+        this.eventHandler = new EventHandler();
+        this.eventHandlerThread = new KafkaThread(threadNamePrefix + "EventHandler",
+            this.eventHandler, false);
+        this.closingTimeNs = Long.MAX_VALUE;
+        this.cleanupEvent = null;
+        this.eventHandlerThread.start();
+    }
+
+    @Override
+    public void enqueue(EventInsertionType insertionType,
+                        String tag,
+                        Function<Long, Long> deadlineNsCalculator,
+                        Event event) {
+        lock.lock();
+        try {
+            EventContext eventContext = new EventContext(event, insertionType, tag);
+            if (closingTimeNs != Long.MAX_VALUE) {
+                eventContext.completeWithException(new EventQueueClosedException());

Review comment:
       Looks like we are calling unknown code while holding a lock.

##########
File path: metadata/src/main/java/org/apache/kafka/queue/EventQueue.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.queue;
+
+import org.slf4j.Logger;
+
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+
+
+public interface EventQueue extends AutoCloseable {
+    interface Event {
+        void run() throws Exception;
+        default void handleException(Throwable e) {}
+    }
+
+    abstract class FailureLoggingEvent implements Event {
+        private final Logger log;
+
+        public FailureLoggingEvent(Logger log) {
+            this.log = log;
+        }
+
+        @Override
+        public void handleException(Throwable e) {
+            if (e instanceof EventQueueClosedException) {
+                log.info("Not processing {} because the event queue is closed.",
+                    this.toString());
+            } else {
+                log.error("Unexpected error handling {}", this.toString(), e);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return this.getClass().getSimpleName();
+        }
+    }
+
+    class DeadlineFunction implements Function<Long, Long> {
+        private final long deadlineNs;
+
+        public DeadlineFunction(long deadlineNs) {
+            this.deadlineNs = deadlineNs;
+        }
+
+        @Override
+        public Long apply(Long t) {
+            return deadlineNs;
+        }
+    }
+
+    class EarliestDeadlineFunction implements Function<Long, Long> {
+        private final long newDeadlineNs;
+
+        public EarliestDeadlineFunction(long newDeadlineNs) {
+            this.newDeadlineNs = newDeadlineNs;
+        }
+
+        @Override
+        public Long apply(Long prevDeadlineNs) {
+            if (prevDeadlineNs == null) {
+                return newDeadlineNs;
+            } else if (prevDeadlineNs < newDeadlineNs) {
+                return prevDeadlineNs;
+            } else {
+                return newDeadlineNs;
+            }
+        }
+    }
+
+    class VoidEvent implements Event {
+        public final static VoidEvent INSTANCE = new VoidEvent();
+
+        @Override
+        public void run() throws Exception {
+        }
+    }
+
+    /**
+     * Add an element to the front of the queue.
+     *
+     * @param event             The mandatory event to prepend.
+     */
+    default void prepend(Event event) {
+        enqueue(EventInsertionType.PREPEND, null, null, event);
+    }
+
+    /**
+     * Add an element to the end of the queue.
+     *
+     * @param event             The event to append.
+     */
+    default void append(Event event) {
+        enqueue(EventInsertionType.APPEND, null, null, event);
+    }
+
+    /**
+     * Enqueue an event to be run in FIFO order.
+     *
+     * @param deadlineNs        The time in monotonic nanoseconds after which the future
+     *                          is completed with a
+     *                          @{org.apache.kafka.common.errors.TimeoutException},
+     *                          and the event is cancelled.
+     * @param event             The event to append.
+     */
+    default void appendWithDeadline(long deadlineNs, Event event) {
+        enqueue(EventInsertionType.APPEND, null, __ -> deadlineNs, event);
+    }
+
+    /**
+     * Schedule an event to be run at a specific time.
+     *
+     * @param tag                   If this is non-null, the unique tag to use for this
+     *                              event.  If an event with this tag already exists, it
+     *                              will be cancelled.

Review comment:
       It would be interesting to explain this design. I get the impression that we are interesting in rescheduling events that haven't triggered.
   
   One way to change the deadline is to specified the previous event that is being rescheduled. E.g.
   ```java
   rescheduleDeferred(Event oldEvent, Function<Long, Long> deadlineNsCalculator, Event newEvent);
   ```
   
   This API allows the deadline to either increase or decrease. Are both cases used by the Controller?

##########
File path: metadata/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.queue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.utils.KafkaThread;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+
+public final class KafkaEventQueue implements EventQueue {
+    /**
+     * A context object that wraps events.
+     */
+    private static class EventContext {
+        /**
+         * The caller-supplied event.
+         */
+        private final Event event;
+
+        /**
+         * How this event was inserted.
+         */
+        private final EventInsertionType insertionType;
+
+        /**
+         * The previous pointer of our circular doubly-linked list.
+         */
+        private EventContext prev = this;
+
+        /**
+         * The next pointer in our circular doubly-linked list.
+         */
+        private EventContext next = this;
+
+        /**
+         * If this event is in the delay map, this is the key it is there under.
+         * If it is not in the map, this is null.
+         */
+        private Long deadlineNs = null;
+
+        /**
+         * The tag associated with this event.
+         */
+        private String tag;
+
+        EventContext(Event event, EventInsertionType insertionType, String tag) {
+            this.event = event;
+            this.insertionType = insertionType;
+            this.tag = tag;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list after this node.
+         */
+        void insertAfter(EventContext other) {
+            this.next.prev = other;
+            other.next = this.next;
+            other.prev = this;
+            this.next = other;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list before this node.
+         */
+        void insertBefore(EventContext other) {
+            this.prev.next = other;
+            other.prev = this.prev;
+            other.next = this;
+            this.prev = other;
+        }
+
+        /**
+         * Remove this node from the circularly linked list.
+         */
+        void remove() {
+            this.prev.next = this.next;
+            this.next.prev = this.prev;
+            this.prev = this;
+            this.next = this;
+        }
+
+        /**
+         * Returns true if this node is the only element in its list.
+         */
+        boolean isSingleton() {
+            return prev == this && next == this;
+        }
+
+        /**
+         * Run the event associated with this EventContext.
+         */
+        void run() throws InterruptedException {
+            try {
+                event.run();
+            } catch (InterruptedException e) {
+                throw e;
+            } catch (Exception e) {
+                event.handleException(e);
+            }
+        }
+
+        /**
+         * Complete the event associated with this EventContext with a timeout exception.
+         */
+        void completeWithTimeout() {
+            completeWithException(new TimeoutException());
+        }
+
+        /**
+         * Complete the event associated with this EventContext with the specified
+         * exception.
+         */
+        void completeWithException(Throwable t) {
+            event.handleException(t);
+        }
+    }
+
+    private class EventHandler implements Runnable {
+        /**
+         * Event contexts indexed by tag.  Events without a tag are not included here.
+         */
+        private final Map<String, EventContext> tagToEventContext = new HashMap<>();
+
+        /**
+         * The head of the event queue.
+         */
+        private final EventContext head = new EventContext(null, null, null);
+
+        /**
+         * An ordered map of times in monotonic nanoseconds to events to time out.
+         */
+        private final TreeMap<Long, EventContext> delayMap = new TreeMap<>();

Review comment:
       Hmm. The word "delay" suggests relative time but it looks like the keys are absolute times. How about `scheduleMap`?

##########
File path: metadata/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.queue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.utils.KafkaThread;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+
+public final class KafkaEventQueue implements EventQueue {
+    /**
+     * A context object that wraps events.
+     */
+    private static class EventContext {
+        /**
+         * The caller-supplied event.
+         */
+        private final Event event;
+
+        /**
+         * How this event was inserted.
+         */
+        private final EventInsertionType insertionType;
+
+        /**
+         * The previous pointer of our circular doubly-linked list.
+         */
+        private EventContext prev = this;
+
+        /**
+         * The next pointer in our circular doubly-linked list.
+         */
+        private EventContext next = this;
+
+        /**
+         * If this event is in the delay map, this is the key it is there under.
+         * If it is not in the map, this is null.
+         */
+        private Long deadlineNs = null;
+
+        /**
+         * The tag associated with this event.
+         */
+        private String tag;
+
+        EventContext(Event event, EventInsertionType insertionType, String tag) {
+            this.event = event;
+            this.insertionType = insertionType;
+            this.tag = tag;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list after this node.
+         */
+        void insertAfter(EventContext other) {
+            this.next.prev = other;
+            other.next = this.next;
+            other.prev = this;
+            this.next = other;
+        }
+
+        /**
+         * Insert a new node in the circularly linked list before this node.
+         */
+        void insertBefore(EventContext other) {
+            this.prev.next = other;
+            other.prev = this.prev;
+            other.next = this;
+            this.prev = other;
+        }
+
+        /**
+         * Remove this node from the circularly linked list.
+         */
+        void remove() {
+            this.prev.next = this.next;
+            this.next.prev = this.prev;
+            this.prev = this;
+            this.next = this;
+        }
+
+        /**
+         * Returns true if this node is the only element in its list.
+         */
+        boolean isSingleton() {
+            return prev == this && next == this;
+        }
+
+        /**
+         * Run the event associated with this EventContext.
+         */
+        void run() throws InterruptedException {
+            try {
+                event.run();
+            } catch (InterruptedException e) {
+                throw e;
+            } catch (Exception e) {
+                event.handleException(e);
+            }
+        }
+
+        /**
+         * Complete the event associated with this EventContext with a timeout exception.
+         */
+        void completeWithTimeout() {
+            completeWithException(new TimeoutException());
+        }
+
+        /**
+         * Complete the event associated with this EventContext with the specified
+         * exception.
+         */
+        void completeWithException(Throwable t) {
+            event.handleException(t);
+        }
+    }
+
+    private class EventHandler implements Runnable {
+        /**
+         * Event contexts indexed by tag.  Events without a tag are not included here.
+         */
+        private final Map<String, EventContext> tagToEventContext = new HashMap<>();
+
+        /**
+         * The head of the event queue.
+         */
+        private final EventContext head = new EventContext(null, null, null);
+
+        /**
+         * An ordered map of times in monotonic nanoseconds to events to time out.
+         */
+        private final TreeMap<Long, EventContext> delayMap = new TreeMap<>();
+
+        /**
+         * A condition variable for waking up the event handler thread.
+         */
+        private final Condition cond = lock.newCondition();
+
+        @Override
+        public void run() {
+            try {
+                handleEvents();
+                cleanupEvent.run();
+            } catch (Throwable e) {
+                log.warn("event handler thread exiting with exception", e);
+            }
+        }
+
+        private void remove(EventContext eventContext) {
+            eventContext.remove();
+            if (eventContext.deadlineNs != null) {
+                delayMap.remove(eventContext.deadlineNs);
+                eventContext.deadlineNs = null;
+            }
+            if (eventContext.tag != null) {
+                tagToEventContext.remove(eventContext.tag, eventContext);
+                eventContext.tag = null;
+            }
+        }
+
+        private void handleEvents() throws InterruptedException {
+            EventContext toTimeout = null;
+            EventContext toRun = null;
+            while (true) {
+                if (toTimeout != null) {
+                    toTimeout.completeWithTimeout();
+                    toTimeout = null;
+                } else if (toRun != null) {
+                    toRun.run();
+                    toRun = null;
+                }
+                lock.lock();
+                try {
+                    long awaitNs = Long.MAX_VALUE;
+                    Map.Entry<Long, EventContext> entry = delayMap.firstEntry();
+                    if (entry != null) {
+                        // Search for timed-out events or deferred events that are ready
+                        // to run.
+                        long now = time.nanoseconds();
+                        long timeoutNs = entry.getKey();
+                        EventContext eventContext = entry.getValue();
+                        if (timeoutNs <= now) {
+                            if (eventContext.insertionType == EventInsertionType.DEFERRED) {
+                                // The deferred event is ready to run.  Prepend it to the
+                                // queue.  (The value for deferred events is a schedule time
+                                // rather than a timeout.)
+                                remove(eventContext);
+                                head.insertAfter(eventContext);
+                            } else {
+                                // not a deferred event, so it is a deadline, and it is timed out.
+                                remove(eventContext);
+                                toTimeout = eventContext;
+                            }
+                            continue;
+                        } else if (closingTimeNs <= now) {
+                            remove(eventContext);
+                            toTimeout = eventContext;
+                            continue;
+                        }
+                        awaitNs = timeoutNs - now;
+                    }
+                    if (head.next == head) {
+                        if ((closingTimeNs != Long.MAX_VALUE) && delayMap.isEmpty()) {
+                            // If there are no more entries to process, and the queue is
+                            // closing, exit the thread.
+                            return;
+                        }
+                    } else {
+                        toRun = head.next;
+                        remove(toRun);
+                        continue;
+                    }
+                    if (closingTimeNs != Long.MAX_VALUE) {
+                        long now = time.nanoseconds();
+                        if (awaitNs > closingTimeNs - now) {
+                            awaitNs = closingTimeNs - now;
+                        }
+                    }
+                    if (awaitNs == Long.MAX_VALUE) {
+                        cond.await();
+                    } else {
+                        cond.awaitNanos(awaitNs);
+                    }
+                } finally {
+                    lock.unlock();
+                }
+            }
+        }
+
+        private void enqueue(EventContext eventContext,
+                             Function<Long, Long> deadlineNsCalculator) {
+            lock.lock();
+            try {
+                Long existingDeadlineNs = null;
+                if (eventContext.tag != null) {
+                    EventContext toRemove =
+                        tagToEventContext.put(eventContext.tag, eventContext);
+                    if (toRemove != null) {
+                        existingDeadlineNs = toRemove.deadlineNs;
+                        remove(toRemove);
+                    }
+                }
+                Long deadlineNs = deadlineNsCalculator.apply(existingDeadlineNs);
+                boolean queueWasEmpty = head.isSingleton();
+                boolean shouldSignal = false;
+                switch (eventContext.insertionType) {
+                    case APPEND:
+                        head.insertBefore(eventContext);
+                        if (queueWasEmpty) {
+                            shouldSignal = true;
+                        }
+                        break;
+                    case PREPEND:
+                        head.insertAfter(eventContext);
+                        if (queueWasEmpty) {
+                            shouldSignal = true;
+                        }
+                        break;
+                    case DEFERRED:
+                        if (deadlineNs == null) {
+                            eventContext.completeWithException(new RuntimeException(
+                                "You must specify a deadline for deferred events."));
+                            return;
+                        }
+                        break;
+                }
+                if (deadlineNs != null) {
+                    long insertNs =  deadlineNs;
+                    long prevStartNs = delayMap.isEmpty() ? Long.MAX_VALUE : delayMap.firstKey();
+                    // If the time in nanoseconds is already taken, take the next one.
+                    while (delayMap.putIfAbsent(insertNs, eventContext) != null) {
+                        insertNs++;
+                    }
+                    eventContext.deadlineNs = insertNs;
+                    // If the new timeout is before all the existing ones, wake up the
+                    // timeout thread.
+                    if (insertNs <= prevStartNs) {
+                        shouldSignal = true;
+                    }
+                }
+                if (shouldSignal) {
+                    cond.signal();
+                }
+            } finally {
+                lock.unlock();
+            }
+        }
+
+        public void cancelDeferred(String tag) {
+            EventContext eventContext = tagToEventContext.get(tag);
+            if (eventContext != null) {
+                remove(eventContext);
+            }
+        }
+    }
+
+    private final Time time;
+    private final ReentrantLock lock;
+    private final Logger log;
+    private final EventHandler eventHandler;
+    private final Thread eventHandlerThread;
+
+    /**
+     * The time in monotonic nanoseconds when the queue is closing, or Long.MAX_VALUE if
+     * the queue is not currently closing.
+     */
+    private long closingTimeNs;
+
+    private Event cleanupEvent;
+
+    public KafkaEventQueue(Time time,
+                           LogContext logContext,
+                           String threadNamePrefix) {
+        this.time = time;
+        this.lock = new ReentrantLock();
+        this.log = logContext.logger(KafkaEventQueue.class);
+        this.eventHandler = new EventHandler();
+        this.eventHandlerThread = new KafkaThread(threadNamePrefix + "EventHandler",
+            this.eventHandler, false);
+        this.closingTimeNs = Long.MAX_VALUE;
+        this.cleanupEvent = null;
+        this.eventHandlerThread.start();
+    }
+
+    @Override
+    public void enqueue(EventInsertionType insertionType,
+                        String tag,
+                        Function<Long, Long> deadlineNsCalculator,
+                        Event event) {
+        lock.lock();
+        try {
+            EventContext eventContext = new EventContext(event, insertionType, tag);
+            if (closingTimeNs != Long.MAX_VALUE) {
+                eventContext.completeWithException(new EventQueueClosedException());
+            } else {
+                eventHandler.enqueue(eventContext,
+                    deadlineNsCalculator == null ? __ -> null : deadlineNsCalculator);
+            }
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    @Override
+    public void cancelDeferred(String tag) {
+        lock.lock();
+        try {
+            eventHandler.cancelDeferred(tag);
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    @Override
+    public void beginShutdown(String source, Event newCleanupEvent,
+                              TimeUnit timeUnit, long timeSpan) {
+        if (timeSpan < 0) {
+            throw new IllegalArgumentException("beginShutdown must be called with a " +
+                "non-negative timeout.");
+        }
+        Objects.requireNonNull(newCleanupEvent);
+        lock.lock();
+        try {
+            if (cleanupEvent != null) {
+                log.debug("{}: Event queue is already shut down.", source);

Review comment:
       ".. shutting down"

##########
File path: metadata/src/main/java/org/apache/kafka/queue/EventQueue.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.queue;
+
+import org.slf4j.Logger;
+
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+
+
+public interface EventQueue extends AutoCloseable {
+    interface Event {
+        void run() throws Exception;
+        default void handleException(Throwable e) {}
+    }
+
+    abstract class FailureLoggingEvent implements Event {
+        private final Logger log;
+
+        public FailureLoggingEvent(Logger log) {
+            this.log = log;
+        }
+
+        @Override
+        public void handleException(Throwable e) {
+            if (e instanceof EventQueueClosedException) {
+                log.info("Not processing {} because the event queue is closed.",
+                    this.toString());
+            } else {
+                log.error("Unexpected error handling {}", this.toString(), e);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return this.getClass().getSimpleName();
+        }
+    }
+
+    class DeadlineFunction implements Function<Long, Long> {
+        private final long deadlineNs;
+
+        public DeadlineFunction(long deadlineNs) {
+            this.deadlineNs = deadlineNs;
+        }
+
+        @Override
+        public Long apply(Long t) {
+            return deadlineNs;
+        }
+    }
+
+    class EarliestDeadlineFunction implements Function<Long, Long> {
+        private final long newDeadlineNs;
+
+        public EarliestDeadlineFunction(long newDeadlineNs) {
+            this.newDeadlineNs = newDeadlineNs;
+        }
+
+        @Override
+        public Long apply(Long prevDeadlineNs) {
+            if (prevDeadlineNs == null) {
+                return newDeadlineNs;
+            } else if (prevDeadlineNs < newDeadlineNs) {
+                return prevDeadlineNs;
+            } else {
+                return newDeadlineNs;
+            }
+        }
+    }
+
+    class VoidEvent implements Event {
+        public final static VoidEvent INSTANCE = new VoidEvent();
+
+        @Override
+        public void run() throws Exception {
+        }
+    }
+
+    /**
+     * Add an element to the front of the queue.
+     *
+     * @param event             The mandatory event to prepend.
+     */
+    default void prepend(Event event) {
+        enqueue(EventInsertionType.PREPEND, null, null, event);
+    }
+
+    /**
+     * Add an element to the end of the queue.
+     *
+     * @param event             The event to append.
+     */
+    default void append(Event event) {
+        enqueue(EventInsertionType.APPEND, null, null, event);
+    }
+
+    /**
+     * Enqueue an event to be run in FIFO order.
+     *
+     * @param deadlineNs        The time in monotonic nanoseconds after which the future
+     *                          is completed with a
+     *                          @{org.apache.kafka.common.errors.TimeoutException},
+     *                          and the event is cancelled.
+     * @param event             The event to append.
+     */
+    default void appendWithDeadline(long deadlineNs, Event event) {
+        enqueue(EventInsertionType.APPEND, null, __ -> deadlineNs, event);
+    }
+
+    /**
+     * Schedule an event to be run at a specific time.
+     *
+     * @param tag                   If this is non-null, the unique tag to use for this
+     *                              event.  If an event with this tag already exists, it
+     *                              will be cancelled.
+     * @param deadlineNsCalculator  A function which takes as an argument the existing
+     *                              deadline for the event with this tag (or null if the
+     *                              event has no tag, or if there is none such), and
+     *                              produces the deadline to use for this event.

Review comment:
       Looking at the implementation, it looks like this Long is an absolute time. Is there a reason why absolute times are used instead of relative times like delay and timeout?




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

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