You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "dajac (via GitHub)" <gi...@apache.org> on 2023/04/05 16:55:50 UTC

[GitHub] [kafka] dajac opened a new pull request, #13505: KAFKA-14462; [5/N] Add ConcurrentEventQueue

dajac opened a new pull request, #13505:
URL: https://github.com/apache/kafka/pull/13505

   This patch adds the `ConcurrentEventQueue` which will be used in the runtime of the new group coordinator. The aim of this queue is to basically have a queue per __consumer_group partitions and to ensure that events addressed to the same partitions are not processed concurrently. The queue is generic so we could reuse it in different context.
   
   I am not really happy with the name but I could not come up with something better. I am open to suggestions.
   
   This patch is part of a series which will bring the new group coordinator runtime. I will open them more or less class by class in order to facilitate the reviews.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add EventAccumulator

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1163207183


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/ConcurrentEventQueue.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class ConcurrentEventQueue<K, T extends ConcurrentEventQueue.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private Set<K> inflightKeys;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private Condition condition;
+
+    public ConcurrentEventQueue() {
+        this(new Random());
+    }
+
+    public ConcurrentEventQueue(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();
+        try {
+            if (closed) throw new IllegalStateException("Can't accept an event because the queue is close.");
+
+            K key = event.key();
+            Queue<T> queue = queues.get(key);
+            if (queue == null) {
+                queue = new LinkedList<>();
+                queues.put(key, queue);
+                if (!inflightKeys.contains(key)) {
+                    addAvailableKey(key);
+                }
+            }
+            queue.add(event);
+            size++;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method block indefinitely until
+     * one event is ready or the queue is closed.
+     *
+     * @return The next event.
+     */
+    public T poll() {
+        return poll(Long.MAX_VALUE, TimeUnit.SECONDS);
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method blocks for the provided

Review Comment:
   We can also just pass in a seed for random to make testing deterministic right?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add EventDispatcher

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1160336099


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventDispatcher.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class EventDispatcher<K, T extends EventDispatcher.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private final Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private final Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private final List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private final Set<K> inflightKeys;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private final ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private final Condition condition;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    public EventDispatcher() {
+        this(new Random());
+    }
+
+    public EventDispatcher(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();
+        try {
+            if (closed) throw new IllegalStateException("Can't accept an event because the queue is closed.");
+
+            K key = event.key();
+            Queue<T> queue = queues.get(key);
+            if (queue == null) {
+                queue = new LinkedList<>();
+                queues.put(key, queue);
+                if (!inflightKeys.contains(key)) {
+                    addAvailableKey(key);
+                }
+            }
+            queue.add(event);
+            size++;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method block indefinitely until
+     * one event is ready or the queue is closed.
+     *
+     * @return The next event.
+     */
+    public T poll() {
+        return poll(Long.MAX_VALUE, TimeUnit.SECONDS);
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method blocks for the provided
+     * time and returns null of not event is available.
+     *
+     * @param timeout   The timeout.
+     * @param unit      The timeout unit.
+     * @return The next event available or null.
+     */
+    public T poll(long timeout, TimeUnit unit) {
+        lock.lock();
+        try {
+            K key = randomKey();
+            long nanos = unit.toNanos(timeout);
+            while (key == null && !closed && nanos > 0) {
+                try {
+                    nanos = condition.awaitNanos(nanos);
+                } catch (InterruptedException e) {
+                    // Ignore.
+                }
+                key = randomKey();
+            }
+
+            if (key == null) return null;
+
+            Queue<T> queue = queues.get(key);
+            T event = queue.poll();
+
+            if (queue.isEmpty()) queues.remove(key);
+            inflightKeys.add(key);
+            size--;
+
+            return event;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Marks the event as processed and releases the next event
+     * with the same key. This unblocks waiting threads.
+     *
+     * @param event The event that was processed.
+     */
+    public void done(T event) {
+        lock.lock();
+        try {
+            K key = event.key();
+            inflightKeys.remove(key);
+            if (queues.containsKey(key)) {
+                addAvailableKey(key);
+            }
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Returns the size of the queue.
+     *
+     * @return The size.
+     */
+    public int size() {
+        lock.lock();
+        try {
+            return size;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Closes the queue. This unblocks all the waiting threads.
+     */
+    @Override
+    public void close() {
+        lock.lock();
+        try {
+            closed = true;
+            condition.signalAll();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Adds the key to the available keys set.
+     *
+     * This method must be called while holding the lock.
+     */
+    private void addAvailableKey(K key) {
+        availableKeys.add(key);
+        condition.signalAll();
+    }
+
+    /**
+     * Returns the next available key. The key is selected randomly
+     * from the available keys set.
+     *
+     * This method must be called while holding the lock.
+     */
+    private K randomKey() {
+        if (availableKeys.isEmpty()) return null;
+
+        int lastIndex = availableKeys.size() - 1;
+        int randomIndex = random.nextInt(availableKeys.size());
+        K randomKey = availableKeys.get(randomIndex);
+        Collections.swap(availableKeys, randomIndex, lastIndex);

Review Comment:
   Seems like the swapping here was done because this is an array list. What was the rationale for choosing this type?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add EventDispatcher

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1162920767


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/ConcurrentEventQueue.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class ConcurrentEventQueue<K, T extends ConcurrentEventQueue.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private Set<K> inflightKeys;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private Condition condition;
+
+    public ConcurrentEventQueue() {
+        this(new Random());
+    }
+
+    public ConcurrentEventQueue(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();

Review Comment:
   I am actually tempted to directly do what I suggested. This would be way more efficient, I think.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on pull request #13505: KAFKA-14462; [5/N] Add EventDispatcher

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on PR #13505:
URL: https://github.com/apache/kafka/pull/13505#issuecomment-1499757999

   I am a fan of the rename -- I think it makes the usage clearer and removes confusion about what is the queue here.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac merged pull request #13505: KAFKA-14462; [5/N] Add EventAccumulator

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac merged PR #13505:
URL: https://github.com/apache/kafka/pull/13505


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add ConcurrentEventQueue

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1159387714


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/ConcurrentEventQueue.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class ConcurrentEventQueue<K, T extends ConcurrentEventQueue.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private Set<K> inflightKeys;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private Condition condition;
+
+    public ConcurrentEventQueue() {
+        this(new Random());
+    }
+
+    public ConcurrentEventQueue(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();

Review Comment:
   Basically, all the requests for the group coordinator will end up in this queue. Therefore, the throughput could be high. As we don't have numbers at the moment, I went with the simplest implementation for now. I think that we can easily improve it when we start to run benchmarks. I don't want to do premature optimizations.
   
   If locking becomes a concern, my idea would be to deliver events in batch instead of delivering them individually. This would have other benefits as well (e.g. write bigger batches to the log). The batch size could be configurable.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add EventDispatcher

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1160011880


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/ConcurrentEventQueue.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class ConcurrentEventQueue<K, T extends ConcurrentEventQueue.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private Set<K> inflightKeys;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private Condition condition;
+
+    public ConcurrentEventQueue() {
+        this(new Random());
+    }
+
+    public ConcurrentEventQueue(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();
+        try {
+            if (closed) throw new IllegalStateException("Can't accept an event because the queue is close.");
+
+            K key = event.key();
+            Queue<T> queue = queues.get(key);
+            if (queue == null) {
+                queue = new LinkedList<>();
+                queues.put(key, queue);
+                if (!inflightKeys.contains(key)) {
+                    addAvailableKey(key);
+                }
+            }
+            queue.add(event);
+            size++;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method block indefinitely until
+     * one event is ready or the queue is closed.
+     *
+     * @return The next event.
+     */
+    public T poll() {
+        return poll(Long.MAX_VALUE, TimeUnit.SECONDS);
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method blocks for the provided
+     * time and returns null of not event is available.
+     *
+     * @param timeout   The timeout.
+     * @param unit      The timeout unit.
+     * @return The next event available or null.
+     */
+    public T poll(long timeout, TimeUnit unit) {
+        lock.lock();
+        try {
+            K key = randomKey();
+            long nanos = unit.toNanos(timeout);
+            while (key == null && !closed && nanos > 0) {
+                try {
+                    nanos = condition.awaitNanos(nanos);
+                } catch (InterruptedException e) {
+                    // Ignore.
+                }
+                key = randomKey();
+            }
+
+            if (key == null) return null;
+
+            Queue<T> queue = queues.get(key);
+            T event = queue.poll();
+
+            if (queue.isEmpty()) queues.remove(key);
+            inflightKeys.add(key);
+            size--;
+
+            return event;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Marks the event as processed and releases the next event
+     * with the same key. This unblocks waiting threads.
+     *
+     * @param event The event that was processed.
+     */
+    public void done(T event) {
+        lock.lock();
+        try {
+            K key = event.key();
+            inflightKeys.remove(key);
+            if (queues.containsKey(key)) {

Review Comment:
   That is correct.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add EventDispatcher

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1160006940


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/ConcurrentEventQueue.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class ConcurrentEventQueue<K, T extends ConcurrentEventQueue.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private Set<K> inflightKeys;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private Condition condition;
+
+    public ConcurrentEventQueue() {
+        this(new Random());
+    }
+
+    public ConcurrentEventQueue(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();
+        try {
+            if (closed) throw new IllegalStateException("Can't accept an event because the queue is close.");
+
+            K key = event.key();
+            Queue<T> queue = queues.get(key);
+            if (queue == null) {
+                queue = new LinkedList<>();
+                queues.put(key, queue);
+                if (!inflightKeys.contains(key)) {
+                    addAvailableKey(key);
+                }
+            }
+            queue.add(event);
+            size++;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method block indefinitely until
+     * one event is ready or the queue is closed.
+     *
+     * @return The next event.
+     */
+    public T poll() {
+        return poll(Long.MAX_VALUE, TimeUnit.SECONDS);
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method blocks for the provided
+     * time and returns null of not event is available.
+     *
+     * @param timeout   The timeout.
+     * @param unit      The timeout unit.
+     * @return The next event available or null.
+     */
+    public T poll(long timeout, TimeUnit unit) {
+        lock.lock();
+        try {
+            K key = randomKey();
+            long nanos = unit.toNanos(timeout);
+            while (key == null && !closed && nanos > 0) {
+                try {
+                    nanos = condition.awaitNanos(nanos);
+                } catch (InterruptedException e) {
+                    // Ignore.
+                }
+                key = randomKey();
+            }
+
+            if (key == null) return null;
+
+            Queue<T> queue = queues.get(key);
+            T event = queue.poll();
+
+            if (queue.isEmpty()) queues.remove(key);
+            inflightKeys.add(key);
+            size--;
+
+            return event;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Marks the event as processed and releases the next event
+     * with the same key. This unblocks waiting threads.
+     *
+     * @param event The event that was processed.
+     */
+    public void done(T event) {
+        lock.lock();
+        try {
+            K key = event.key();
+            inflightKeys.remove(key);
+            if (queues.containsKey(key)) {

Review Comment:
   ah ok -- so the semantics here are we only have the key in the map if it has items in the queue -- we readd to the map if an event is added. Is this correct?
   
   I guess I could also imagine an approach where we have keys with empty queues, but the above approach makes sense.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add ConcurrentEventQueue

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1159388206


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/ConcurrentEventQueue.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class ConcurrentEventQueue<K, T extends ConcurrentEventQueue.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private Set<K> inflightKeys;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private Condition condition;
+
+    public ConcurrentEventQueue() {
+        this(new Random());
+    }
+
+    public ConcurrentEventQueue(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();
+        try {
+            if (closed) throw new IllegalStateException("Can't accept an event because the queue is close.");
+
+            K key = event.key();
+            Queue<T> queue = queues.get(key);
+            if (queue == null) {
+                queue = new LinkedList<>();
+                queues.put(key, queue);
+                if (!inflightKeys.contains(key)) {
+                    addAvailableKey(key);
+                }
+            }
+            queue.add(event);
+            size++;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method block indefinitely until
+     * one event is ready or the queue is closed.
+     *
+     * @return The next event.
+     */
+    public T poll() {
+        return poll(Long.MAX_VALUE, TimeUnit.SECONDS);
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method blocks for the provided

Review Comment:
   Yeah, I agree with you. I don't like this name either. How about `EventDispatcher`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add EventAccumulator

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1163201442


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventAccumulator.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event accumulator which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class EventAccumulator<K, T extends EventAccumulator.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private final Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private final Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private final List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private final Set<K> inflightKeys;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private final ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private final Condition condition;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.

Review Comment:
   This is for the whole accumulator right? Not an individual queue? There's a few other instances of queue used both for size and for closed throughout the comments.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add ConcurrentEventQueue

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1158787361


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/ConcurrentEventQueue.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class ConcurrentEventQueue<K, T extends ConcurrentEventQueue.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private Set<K> inflightKeys;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private Condition condition;
+
+    public ConcurrentEventQueue() {
+        this(new Random());
+    }
+
+    public ConcurrentEventQueue(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();
+        try {
+            if (closed) throw new IllegalStateException("Can't accept an event because the queue is close.");

Review Comment:
   nit: "closed"



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add EventAccumulator

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1163756580


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/ConcurrentEventQueue.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class ConcurrentEventQueue<K, T extends ConcurrentEventQueue.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private Set<K> inflightKeys;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private Condition condition;
+
+    public ConcurrentEventQueue() {
+        this(new Random());
+    }
+
+    public ConcurrentEventQueue(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();
+        try {
+            if (closed) throw new IllegalStateException("Can't accept an event because the queue is close.");
+
+            K key = event.key();
+            Queue<T> queue = queues.get(key);
+            if (queue == null) {
+                queue = new LinkedList<>();
+                queues.put(key, queue);
+                if (!inflightKeys.contains(key)) {
+                    addAvailableKey(key);
+                }
+            }
+            queue.add(event);
+            size++;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method block indefinitely until
+     * one event is ready or the queue is closed.
+     *
+     * @return The next event.
+     */
+    public T poll() {
+        return poll(Long.MAX_VALUE, TimeUnit.SECONDS);
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method blocks for the provided

Review Comment:
   `EventAccumulator` sounds good. @Justine, yes, you are right, we could also provide a seed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add EventDispatcher

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1162780120


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/ConcurrentEventQueue.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class ConcurrentEventQueue<K, T extends ConcurrentEventQueue.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private Set<K> inflightKeys;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private Condition condition;
+
+    public ConcurrentEventQueue() {
+        this(new Random());
+    }
+
+    public ConcurrentEventQueue(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();

Review Comment:
   Makes sense, thanks for explaining.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add EventAccumulator

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1163723563


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventAccumulator.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event accumulator which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class EventAccumulator<K, T extends EventAccumulator.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private final Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private final Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private final List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private final Set<K> inflightKeys;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private final ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private final Condition condition;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add EventAccumulator

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1163208120


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/ConcurrentEventQueue.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class ConcurrentEventQueue<K, T extends ConcurrentEventQueue.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private Set<K> inflightKeys;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private Condition condition;
+
+    public ConcurrentEventQueue() {
+        this(new Random());
+    }
+
+    public ConcurrentEventQueue(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();

Review Comment:
   batching sounds like it could be pretty good here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add ConcurrentEventQueue

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1158951727


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/ConcurrentEventQueue.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class ConcurrentEventQueue<K, T extends ConcurrentEventQueue.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private Set<K> inflightKeys;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private Condition condition;
+
+    public ConcurrentEventQueue() {
+        this(new Random());
+    }
+
+    public ConcurrentEventQueue(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();
+        try {
+            if (closed) throw new IllegalStateException("Can't accept an event because the queue is close.");
+
+            K key = event.key();
+            Queue<T> queue = queues.get(key);
+            if (queue == null) {
+                queue = new LinkedList<>();
+                queues.put(key, queue);
+                if (!inflightKeys.contains(key)) {
+                    addAvailableKey(key);
+                }
+            }
+            queue.add(event);
+            size++;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method block indefinitely until
+     * one event is ready or the queue is closed.
+     *
+     * @return The next event.
+     */
+    public T poll() {
+        return poll(Long.MAX_VALUE, TimeUnit.SECONDS);
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method blocks for the provided
+     * time and returns null of not event is available.
+     *
+     * @param timeout   The timeout.
+     * @param unit      The timeout unit.
+     * @return The next event available or null.
+     */
+    public T poll(long timeout, TimeUnit unit) {
+        lock.lock();
+        try {
+            K key = randomKey();
+            long nanos = unit.toNanos(timeout);
+            while (key == null && !closed && nanos > 0) {
+                try {
+                    nanos = condition.awaitNanos(nanos);
+                } catch (InterruptedException e) {
+                    // Ignore.
+                }
+                key = randomKey();
+            }
+
+            if (key == null) return null;
+
+            Queue<T> queue = queues.get(key);
+            T event = queue.poll();
+
+            if (queue.isEmpty()) queues.remove(key);
+            inflightKeys.add(key);
+            size--;
+
+            return event;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Marks the event as processed and releases the next event
+     * with the same key. This unblocks waiting threads.
+     *
+     * @param event The event that was processed.
+     */
+    public void done(T event) {
+        lock.lock();
+        try {
+            K key = event.key();
+            inflightKeys.remove(key);
+            if (queues.containsKey(key)) {

Review Comment:
   Do we have a method that allows us to remove keys from queues?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add EventDispatcher

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1160571805


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventDispatcher.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class EventDispatcher<K, T extends EventDispatcher.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private final Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private final Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private final List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private final Set<K> inflightKeys;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private final ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private final Condition condition;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    public EventDispatcher() {
+        this(new Random());
+    }
+
+    public EventDispatcher(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();
+        try {
+            if (closed) throw new IllegalStateException("Can't accept an event because the queue is closed.");
+
+            K key = event.key();
+            Queue<T> queue = queues.get(key);
+            if (queue == null) {
+                queue = new LinkedList<>();
+                queues.put(key, queue);
+                if (!inflightKeys.contains(key)) {
+                    addAvailableKey(key);
+                }
+            }
+            queue.add(event);
+            size++;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method block indefinitely until
+     * one event is ready or the queue is closed.
+     *
+     * @return The next event.
+     */
+    public T poll() {
+        return poll(Long.MAX_VALUE, TimeUnit.SECONDS);
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method blocks for the provided
+     * time and returns null of not event is available.
+     *
+     * @param timeout   The timeout.
+     * @param unit      The timeout unit.
+     * @return The next event available or null.
+     */
+    public T poll(long timeout, TimeUnit unit) {
+        lock.lock();
+        try {
+            K key = randomKey();
+            long nanos = unit.toNanos(timeout);
+            while (key == null && !closed && nanos > 0) {
+                try {
+                    nanos = condition.awaitNanos(nanos);
+                } catch (InterruptedException e) {
+                    // Ignore.
+                }
+                key = randomKey();
+            }
+
+            if (key == null) return null;
+
+            Queue<T> queue = queues.get(key);
+            T event = queue.poll();
+
+            if (queue.isEmpty()) queues.remove(key);
+            inflightKeys.add(key);
+            size--;
+
+            return event;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Marks the event as processed and releases the next event
+     * with the same key. This unblocks waiting threads.
+     *
+     * @param event The event that was processed.
+     */
+    public void done(T event) {
+        lock.lock();
+        try {
+            K key = event.key();
+            inflightKeys.remove(key);
+            if (queues.containsKey(key)) {
+                addAvailableKey(key);
+            }
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Returns the size of the queue.
+     *
+     * @return The size.
+     */
+    public int size() {
+        lock.lock();
+        try {
+            return size;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Closes the queue. This unblocks all the waiting threads.
+     */
+    @Override
+    public void close() {
+        lock.lock();
+        try {
+            closed = true;
+            condition.signalAll();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Adds the key to the available keys set.
+     *
+     * This method must be called while holding the lock.
+     */
+    private void addAvailableKey(K key) {
+        availableKeys.add(key);
+        condition.signalAll();
+    }
+
+    /**
+     * Returns the next available key. The key is selected randomly
+     * from the available keys set.
+     *
+     * This method must be called while holding the lock.
+     */
+    private K randomKey() {
+        if (availableKeys.isEmpty()) return null;
+
+        int lastIndex = availableKeys.size() - 1;
+        int randomIndex = random.nextInt(availableKeys.size());
+        K randomKey = availableKeys.get(randomIndex);
+        Collections.swap(availableKeys, randomIndex, lastIndex);

Review Comment:
   Yeah, I swap with the last in order to moving all elements in remove. I use an ArrayList because I need random access.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add ConcurrentEventQueue

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1158880294


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/ConcurrentEventQueue.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class ConcurrentEventQueue<K, T extends ConcurrentEventQueue.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private Set<K> inflightKeys;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private Condition condition;
+
+    public ConcurrentEventQueue() {
+        this(new Random());
+    }
+
+    public ConcurrentEventQueue(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();

Review Comment:
   What is the expected throughput on this data structure? If "high" (to be quantified) throughput is expected, is there any risk of contention and then would there be any benefit in locking be performed more granularly (e.g. at the level of a queue)? Although I understand a global lock is also needed to perform signalling between poll and add/release operations.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/ConcurrentEventQueue.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class ConcurrentEventQueue<K, T extends ConcurrentEventQueue.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private Set<K> inflightKeys;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private Condition condition;
+
+    public ConcurrentEventQueue() {
+        this(new Random());
+    }
+
+    public ConcurrentEventQueue(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();
+        try {
+            if (closed) throw new IllegalStateException("Can't accept an event because the queue is close.");
+
+            K key = event.key();
+            Queue<T> queue = queues.get(key);
+            if (queue == null) {
+                queue = new LinkedList<>();
+                queues.put(key, queue);
+                if (!inflightKeys.contains(key)) {
+                    addAvailableKey(key);
+                }
+            }
+            queue.add(event);
+            size++;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method block indefinitely until
+     * one event is ready or the queue is closed.
+     *
+     * @return The next event.
+     */
+    public T poll() {
+        return poll(Long.MAX_VALUE, TimeUnit.SECONDS);
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method blocks for the provided

Review Comment:
   nit about naming - in this case, the `poll` method can return any event which key is not currently leased. This makes the semantic of the data structure not strictly aligned with a queue, what do you think? From a user perspective, the returned value from the poll is not deterministic. I wonder if there are naming conventions which could reflect that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add ConcurrentEventQueue

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1159388918


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/ConcurrentEventQueue.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class ConcurrentEventQueue<K, T extends ConcurrentEventQueue.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private Set<K> inflightKeys;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private Condition condition;
+
+    public ConcurrentEventQueue() {
+        this(new Random());
+    }
+
+    public ConcurrentEventQueue(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();
+        try {
+            if (closed) throw new IllegalStateException("Can't accept an event because the queue is close.");
+
+            K key = event.key();
+            Queue<T> queue = queues.get(key);
+            if (queue == null) {
+                queue = new LinkedList<>();
+                queues.put(key, queue);
+                if (!inflightKeys.contains(key)) {
+                    addAvailableKey(key);
+                }
+            }
+            queue.add(event);
+            size++;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method block indefinitely until
+     * one event is ready or the queue is closed.
+     *
+     * @return The next event.
+     */
+    public T poll() {
+        return poll(Long.MAX_VALUE, TimeUnit.SECONDS);
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method blocks for the provided
+     * time and returns null of not event is available.
+     *
+     * @param timeout   The timeout.
+     * @param unit      The timeout unit.
+     * @return The next event available or null.
+     */
+    public T poll(long timeout, TimeUnit unit) {
+        lock.lock();
+        try {
+            K key = randomKey();
+            long nanos = unit.toNanos(timeout);
+            while (key == null && !closed && nanos > 0) {
+                try {
+                    nanos = condition.awaitNanos(nanos);
+                } catch (InterruptedException e) {
+                    // Ignore.
+                }
+                key = randomKey();
+            }
+
+            if (key == null) return null;
+
+            Queue<T> queue = queues.get(key);
+            T event = queue.poll();
+
+            if (queue.isEmpty()) queues.remove(key);
+            inflightKeys.add(key);
+            size--;
+
+            return event;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Marks the event as processed and releases the next event
+     * with the same key. This unblocks waiting threads.
+     *
+     * @param event The event that was processed.
+     */
+    public void done(T event) {
+        lock.lock();
+        try {
+            K key = event.key();
+            inflightKeys.remove(key);
+            if (queues.containsKey(key)) {

Review Comment:
   They are removed [here](https://github.com/apache/kafka/pull/13505/files#diff-bf202b87edf0eb78086b24213f97883a62c57a7a5c966f891d52055aeff01f27R175) in `poll`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add EventDispatcher

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1160571805


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventDispatcher.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class EventDispatcher<K, T extends EventDispatcher.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private final Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private final Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private final List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private final Set<K> inflightKeys;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private final ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private final Condition condition;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    public EventDispatcher() {
+        this(new Random());
+    }
+
+    public EventDispatcher(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();
+        try {
+            if (closed) throw new IllegalStateException("Can't accept an event because the queue is closed.");
+
+            K key = event.key();
+            Queue<T> queue = queues.get(key);
+            if (queue == null) {
+                queue = new LinkedList<>();
+                queues.put(key, queue);
+                if (!inflightKeys.contains(key)) {
+                    addAvailableKey(key);
+                }
+            }
+            queue.add(event);
+            size++;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method block indefinitely until
+     * one event is ready or the queue is closed.
+     *
+     * @return The next event.
+     */
+    public T poll() {
+        return poll(Long.MAX_VALUE, TimeUnit.SECONDS);
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method blocks for the provided
+     * time and returns null of not event is available.
+     *
+     * @param timeout   The timeout.
+     * @param unit      The timeout unit.
+     * @return The next event available or null.
+     */
+    public T poll(long timeout, TimeUnit unit) {
+        lock.lock();
+        try {
+            K key = randomKey();
+            long nanos = unit.toNanos(timeout);
+            while (key == null && !closed && nanos > 0) {
+                try {
+                    nanos = condition.awaitNanos(nanos);
+                } catch (InterruptedException e) {
+                    // Ignore.
+                }
+                key = randomKey();
+            }
+
+            if (key == null) return null;
+
+            Queue<T> queue = queues.get(key);
+            T event = queue.poll();
+
+            if (queue.isEmpty()) queues.remove(key);
+            inflightKeys.add(key);
+            size--;
+
+            return event;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Marks the event as processed and releases the next event
+     * with the same key. This unblocks waiting threads.
+     *
+     * @param event The event that was processed.
+     */
+    public void done(T event) {
+        lock.lock();
+        try {
+            K key = event.key();
+            inflightKeys.remove(key);
+            if (queues.containsKey(key)) {
+                addAvailableKey(key);
+            }
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Returns the size of the queue.
+     *
+     * @return The size.
+     */
+    public int size() {
+        lock.lock();
+        try {
+            return size;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Closes the queue. This unblocks all the waiting threads.
+     */
+    @Override
+    public void close() {
+        lock.lock();
+        try {
+            closed = true;
+            condition.signalAll();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Adds the key to the available keys set.
+     *
+     * This method must be called while holding the lock.
+     */
+    private void addAvailableKey(K key) {
+        availableKeys.add(key);
+        condition.signalAll();
+    }
+
+    /**
+     * Returns the next available key. The key is selected randomly
+     * from the available keys set.
+     *
+     * This method must be called while holding the lock.
+     */
+    private K randomKey() {
+        if (availableKeys.isEmpty()) return null;
+
+        int lastIndex = availableKeys.size() - 1;
+        int randomIndex = random.nextInt(availableKeys.size());
+        K randomKey = availableKeys.get(randomIndex);
+        Collections.swap(availableKeys, randomIndex, lastIndex);

Review Comment:
   Yeah, I swap with the last in order to avoid moving all elements in remove. I use an ArrayList because I need random access.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add EventDispatcher

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1162913636


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/ConcurrentEventQueue.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class ConcurrentEventQueue<K, T extends ConcurrentEventQueue.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private Set<K> inflightKeys;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private Condition condition;
+
+    public ConcurrentEventQueue() {
+        this(new Random());
+    }
+
+    public ConcurrentEventQueue(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();
+        try {
+            if (closed) throw new IllegalStateException("Can't accept an event because the queue is close.");
+
+            K key = event.key();
+            Queue<T> queue = queues.get(key);
+            if (queue == null) {
+                queue = new LinkedList<>();
+                queues.put(key, queue);
+                if (!inflightKeys.contains(key)) {
+                    addAvailableKey(key);
+                }
+            }
+            queue.add(event);
+            size++;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method block indefinitely until
+     * one event is ready or the queue is closed.
+     *
+     * @return The next event.
+     */
+    public T poll() {
+        return poll(Long.MAX_VALUE, TimeUnit.SECONDS);
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method blocks for the provided

Review Comment:
   I am not a fan of `KeyedEventSet` because `Set` is a bit inaccurate here. How about `EventAccumulator`? `Accumulator` is generic so we can define it as we like.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13505: KAFKA-14462; [5/N] Add EventDispatcher

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13505:
URL: https://github.com/apache/kafka/pull/13505#discussion_r1162819684


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/ConcurrentEventQueue.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.coordinator.group.runtime;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A concurrent event queue which group events per key and ensure that only one
+ * event with a given key can't be processed concurrently.
+ *
+ * This class is threadsafe.
+ *
+ * @param <K> The type of the key of the event.
+ * @param <T> The type of the event itself. It implements the {{@link Event}} interface.
+ *
+ * There are a few examples about how to use it in the unit tests.
+ */
+public class ConcurrentEventQueue<K, T extends ConcurrentEventQueue.Event<K>> implements AutoCloseable {
+
+    /**
+     * The interface which must be implemented by all events.
+     *
+     * @param <K> The type of the key of the event.
+     */
+    public interface Event<K> {
+        K key();
+    }
+
+    /**
+     * The random generator used by this class.
+     */
+    private Random random;
+
+    /**
+     * The map of queues keyed by K.
+     */
+    private Map<K, Queue<T>> queues;
+
+    /**
+     * The list of available keys. Keys in this list can
+     * be delivered to pollers.
+     */
+    private List<K> availableKeys;
+
+    /**
+     * The set of keys that are being processed.
+     */
+    private Set<K> inflightKeys;
+
+    /**
+     * The number of events in the queue.
+     */
+    private int size;
+
+    /**
+     * A boolean indicated whether the queue is closed.
+     */
+    private boolean closed;
+
+    /**
+     * The lock for protecting access to the resources.
+     */
+    private ReentrantLock lock;
+
+    /**
+     * The condition variable for waking up poller threads.
+     */
+    private Condition condition;
+
+    public ConcurrentEventQueue() {
+        this(new Random());
+    }
+
+    public ConcurrentEventQueue(
+        Random random
+    ) {
+        this.random = random;
+        this.queues = new HashMap<>();
+        this.availableKeys = new ArrayList<>();
+        this.inflightKeys = new HashSet<>();
+        this.closed = false;
+        this.lock = new ReentrantLock();
+        this.condition = lock.newCondition();
+    }
+
+    /**
+     * Adds an {{@link Event}} to the queue.
+     *
+     * @param event An {{@link Event}}.
+     */
+    public void add(T event) {
+        lock.lock();
+        try {
+            if (closed) throw new IllegalStateException("Can't accept an event because the queue is close.");
+
+            K key = event.key();
+            Queue<T> queue = queues.get(key);
+            if (queue == null) {
+                queue = new LinkedList<>();
+                queues.put(key, queue);
+                if (!inflightKeys.contains(key)) {
+                    addAvailableKey(key);
+                }
+            }
+            queue.add(event);
+            size++;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method block indefinitely until
+     * one event is ready or the queue is closed.
+     *
+     * @return The next event.
+     */
+    public T poll() {
+        return poll(Long.MAX_VALUE, TimeUnit.SECONDS);
+    }
+
+    /**
+     * Returns the next {{@link Event}} available. This method blocks for the provided

Review Comment:
   I was thinking something along `KeyedEventSet` with a `pollAny` method to highlight an element of any given key not already in-use can be taken. One of the reason it is difficult to name is that the class combines a data structure with a semaphore semantic over one attribute (the key) of the elements in that data structure. If round-robin selection could be used (I don't know) instead of random selection, an actual FIFO or LIFO queue of key -> [events] augmented with the set of keys could also be used. But I cannot think about another approach which is as efficient as the bespoke implementation here.
   
   On a separate note, I wonder if the random selection of keys could be extracted and provided to the component to make testing deterministic?
   
   On the `EventDispatcher` name: isn't dispatcher typically used for components which actually performs a dispatch/scheduling action? (e.g. a dispatcher thread)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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