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 21:25:46 UTC

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

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



##########
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();

Review comment:
       Do we need the lock here again since the caller already locks?

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

Review comment:
       Since this is unexpected, should we just throw the exception directly back to the caller?

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

Review comment:
       Perhaps we could document the ordering guarantee with the deferred event compared with other types of events?

##########
File path: metadata/src/test/java/org/apache/kafka/queue/KafkaEventQueueTest.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Supplier;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+
+@Timeout(value = 60)
+public class KafkaEventQueueTest {
+    private static class FutureEvent<T> implements EventQueue.Event {
+        private final CompletableFuture<T> future;
+        private final Supplier<T> supplier;
+
+        FutureEvent(CompletableFuture<T> future, Supplier<T> supplier) {
+            this.future = future;
+            this.supplier = supplier;
+        }
+
+        @Override
+        public void run() throws Exception {
+            T value = supplier.get();
+            future.complete(value);
+        }
+
+        @Override
+        public void handleException(Throwable e) {
+            future.completeExceptionally(e);
+        }
+    }
+
+    @Test
+    public void testCreateAndClose() throws Exception {
+        KafkaEventQueue queue =
+            new KafkaEventQueue(Time.SYSTEM, new LogContext(), "testCreateAndClose");
+        queue.close();
+    }
+
+    @Test
+    public void testHandleEvents() throws Exception {
+        KafkaEventQueue queue =
+            new KafkaEventQueue(Time.SYSTEM, new LogContext(), "testHandleEvents");
+        AtomicInteger numEventsExecuted = new AtomicInteger(0);
+        CompletableFuture<Integer> future1 = new CompletableFuture<>();
+        queue.prepend(new FutureEvent<>(future1, () -> {
+            assertEquals(1, numEventsExecuted.incrementAndGet());
+            return 1;
+        }));
+        CompletableFuture<Integer> future2 = new CompletableFuture<>();
+        queue.appendWithDeadline(Time.SYSTEM.nanoseconds() + TimeUnit.SECONDS.toNanos(30),

Review comment:
       Strictly speaking, with system time, this event could time out before the first future is executed. Should we use mock time?

##########
File path: metadata/src/test/java/org/apache/kafka/queue/KafkaEventQueueTest.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Supplier;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+
+@Timeout(value = 60)
+public class KafkaEventQueueTest {
+    private static class FutureEvent<T> implements EventQueue.Event {
+        private final CompletableFuture<T> future;
+        private final Supplier<T> supplier;
+
+        FutureEvent(CompletableFuture<T> future, Supplier<T> supplier) {
+            this.future = future;
+            this.supplier = supplier;
+        }
+
+        @Override
+        public void run() throws Exception {
+            T value = supplier.get();
+            future.complete(value);
+        }
+
+        @Override
+        public void handleException(Throwable e) {
+            future.completeExceptionally(e);
+        }
+    }
+
+    @Test
+    public void testCreateAndClose() throws Exception {
+        KafkaEventQueue queue =
+            new KafkaEventQueue(Time.SYSTEM, new LogContext(), "testCreateAndClose");
+        queue.close();
+    }
+
+    @Test
+    public void testHandleEvents() throws Exception {
+        KafkaEventQueue queue =
+            new KafkaEventQueue(Time.SYSTEM, new LogContext(), "testHandleEvents");
+        AtomicInteger numEventsExecuted = new AtomicInteger(0);
+        CompletableFuture<Integer> future1 = new CompletableFuture<>();
+        queue.prepend(new FutureEvent<>(future1, () -> {
+            assertEquals(1, numEventsExecuted.incrementAndGet());
+            return 1;
+        }));
+        CompletableFuture<Integer> future2 = new CompletableFuture<>();
+        queue.appendWithDeadline(Time.SYSTEM.nanoseconds() + TimeUnit.SECONDS.toNanos(30),
+            new FutureEvent<>(future2, () -> {
+                assertEquals(2, numEventsExecuted.incrementAndGet());
+                return 2;
+            }));
+        CompletableFuture<Integer> future3 = new CompletableFuture<>();
+        queue.append(new FutureEvent<>(future3, () -> {
+            assertEquals(3, numEventsExecuted.incrementAndGet());
+            return 3;
+        }));
+        assertEquals(Integer.valueOf(1), future1.get());
+        assertEquals(Integer.valueOf(3), future3.get());
+        assertEquals(Integer.valueOf(2), future2.get());
+        CompletableFuture<Integer> future4 = new CompletableFuture<>();
+        queue.appendWithDeadline(Time.SYSTEM.nanoseconds() + TimeUnit.SECONDS.toNanos(30),
+            new FutureEvent<>(future4, () -> {
+                assertEquals(4, numEventsExecuted.incrementAndGet());
+                return 4;
+            }));
+        future4.get();
+        queue.beginShutdown("testHandleEvents");
+        queue.close();
+    }
+
+    @Test
+    public void testTimeouts() throws Exception {
+        KafkaEventQueue queue =
+            new KafkaEventQueue(Time.SYSTEM, new LogContext(), "testTimeouts");
+        AtomicInteger numEventsExecuted = new AtomicInteger(0);
+        CompletableFuture<Integer> future1 = new CompletableFuture<>();
+        queue.append(new FutureEvent<>(future1, () -> {
+            assertEquals(1, numEventsExecuted.incrementAndGet());
+            return 1;
+        }));
+        CompletableFuture<Integer> future2 = new CompletableFuture<>();
+        queue.append(new FutureEvent<>(future2, () -> {
+            assertEquals(2, numEventsExecuted.incrementAndGet());
+            Time.SYSTEM.sleep(1);
+            return 2;
+        }));
+        CompletableFuture<Integer> future3 = new CompletableFuture<>();
+        queue.appendWithDeadline(Time.SYSTEM.nanoseconds() + 1,
+            new FutureEvent<>(future3, () -> {
+                numEventsExecuted.incrementAndGet();
+                return 3;
+            }));
+        CompletableFuture<Integer> future4 = new CompletableFuture<>();
+        queue.append(new FutureEvent<>(future4, () -> {
+            numEventsExecuted.incrementAndGet();
+            return 4;
+        }));
+        assertEquals(Integer.valueOf(1), future1.get());
+        assertEquals(Integer.valueOf(2), future2.get());
+        assertEquals(Integer.valueOf(4), future4.get());
+        assertEquals(TimeoutException.class,
+            assertThrows(ExecutionException.class,
+                () -> future3.get()).getCause().getClass());
+        queue.close();
+        assertEquals(3, numEventsExecuted.get());
+    }
+
+    @Test
+    public void testScheduleDeferred() throws Exception {
+        KafkaEventQueue queue =
+            new KafkaEventQueue(Time.SYSTEM, new LogContext(), "testAppendDeferred");
+
+        // Wait for the deferred event to happen after the non-deferred event.
+        // It may not happpen every time, so we keep trying until it does.

Review comment:
       typo happpen

##########
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();
+        }
+    }
+
+    @Override
+    public void close() throws InterruptedException {
+        beginShutdown("KafkaEventQueue#close");
+        eventHandlerThread.join();
+        log.info("closed event queue.");

Review comment:
       Is this necessary since we already have an info logging in beginShutdown().

##########
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);

Review comment:
       Since now can change after each while loop, it's possible that deferred events are prepended to the queue in reverse order. Does that matter?




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