Skip to content

Commit

Permalink
ARTEMIS-5038 Mirrored ACKs are broken if using multiple priorities on…
Browse files Browse the repository at this point in the history
… producers

PriorityLinkedList has multiple sub-lists, before this commit PriorityLinkedList::setNodeStore would set the same node store between all the lists.
When a removeWithID was called for an item on list[0] the remove from list[4] would always succeed first. This operation would work correctly most of the time except
when tail and head is being used. Many NullPointerExceptions would be seen while iterating on the list for remove operations, and the navigation would be completely broken.

A test was added to PriorityLinkedListTest to make sure the correct lists were used however I was not able to reproduce the NPE condition in that test.
AccumulatedInPageSoakTest reproduced the exact condition for the NPE when significant load is used.
  • Loading branch information
clebertsuconic committed Sep 23, 2024
1 parent a70b053 commit f92a846
Show file tree
Hide file tree
Showing 19 changed files with 270 additions and 109 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -16,8 +16,11 @@
*/
package org.apache.activemq.artemis.utils;

import java.io.BufferedReader;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.lang.invoke.MethodHandles;
import java.nio.file.FileVisitResult;
import java.nio.file.Files;
Expand All @@ -27,6 +30,10 @@
import java.nio.file.attribute.BasicFileAttributes;
import java.util.Arrays;
import java.util.HashSet;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;

import org.apache.activemq.artemis.logs.ActiveMQUtilLogger;
import org.slf4j.Logger;
Expand Down Expand Up @@ -126,4 +133,21 @@ public static boolean findReplace(File file, String find, String replace) throws
}
}

public static String readFile(InputStream inputStream) throws Exception {
BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(inputStream));
String fileOutput = bufferedReader.lines().collect(Collectors.joining(System.lineSeparator()));
return fileOutput;
}

public static boolean find(File file, Predicate<String> search) throws Exception {
AtomicBoolean found = new AtomicBoolean(false);
try (Stream<String> lines = Files.lines(file.toPath())) {
lines.filter(search::test).findFirst().ifPresent(line -> {
logger.info("pattern found at {}", line);
found.set(true);
});
}
return found.get();
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,11 @@ public E poll() {
public void repeat() {
}

@Override
public E removeLastElement() {
return null;
}

@Override
public void close() {
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -599,22 +599,31 @@ public E next() {

@Override
public void remove() {
removeLastElement();
}

@Override
public E removeLastElement() {
synchronized (LinkedListImpl.this) {
if (last == null) {
throw new NoSuchElementException();
}

if (current == null) {
return;
return null;
}

E returningElement = current.val();

Node<E> prev = current.prev;

if (prev != null) {
LinkedListImpl.this.removeAfter(prev);

last = null;
}

return returningElement;
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,9 @@ public interface LinkedListIterator<E> extends Iterator<E>, AutoCloseable {

void repeat();

/** This method is doing exactly what {@link Iterator#remove()} would do, however it will return the removed element being removed. */
E removeLastElement();

@Override
void close();
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,14 @@ public interface NodeStore<E> {

void removeNode(E element, LinkedListImpl.Node<E> node);

default NodeStore<E> setName(String name) {
return this;
}

default String getName() {
return null;
}

/** this is meant to be a quick help to Garbage Collection.
* Whenever the IDSupplier list is being cleared, you should first call the clear method and
* empty every list before you let the instance go. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@
*/
package org.apache.activemq.artemis.utils.collections;

import java.util.function.Supplier;

/**
* A type of linked list which maintains items according to a priority
* and allows adding and removing of elements at both ends, and peeking.<br>
Expand All @@ -40,7 +42,7 @@ public interface PriorityLinkedList<E> {
* @see LinkedList#setNodeStore(NodeStore)
* @param supplier
*/
void setNodeStore(NodeStore<E> supplier);
void setNodeStore(Supplier<NodeStore<E>> supplier);

E removeWithID(String listID, long id);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import java.util.Comparator;
import java.util.NoSuchElementException;
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
import java.util.function.Supplier;

/**
* A priority linked list implementation
Expand All @@ -40,6 +41,10 @@ public class PriorityLinkedListImpl<E> implements PriorityLinkedList<E> {

private int lastPriority = -1;

protected void removed(final int level, final E element) {
exclusiveIncrementSize(-1);
}

public PriorityLinkedListImpl(final int priorities) {
this(priorities, null);
}
Expand Down Expand Up @@ -96,9 +101,10 @@ public void addSorted(E e, int priority) {
}

@Override
public void setNodeStore(NodeStore<E> supplier) {
public void setNodeStore(Supplier<NodeStore<E>> supplier) {
for (LinkedList<E> list : levels) {
list.setNodeStore(supplier);
NodeStore<E> nodeStore = supplier.get();
list.setNodeStore(nodeStore);
}
}

Expand All @@ -109,7 +115,7 @@ public E removeWithID(String listID, long id) {
for (int l = 4; l < levels.length; l++) {
E removed = levels[l].removeWithID(listID, id);
if (removed != null) {
exclusiveIncrementSize(-1);
removed(l, removed);
return removed;
}
}
Expand All @@ -118,7 +124,7 @@ public E removeWithID(String listID, long id) {
for (int l = Math.min(3, levels.length); l >= 0; l--) {
E removed = levels[l].removeWithID(listID, id);
if (removed != null) {
exclusiveIncrementSize(-1);
removed(l, removed);
return removed;
}
}
Expand Down Expand Up @@ -155,7 +161,7 @@ public E poll() {
e = ll.poll();

if (e != null) {
exclusiveIncrementSize(-1);
removed(i, e);

if (ll.size() == 0) {
if (highestPriority == i) {
Expand Down Expand Up @@ -211,6 +217,8 @@ private class PriorityLinkedListIterator implements LinkedListIterator<E> {

private LinkedListIterator<E> lastIter;

private int lastLevel = -1;

private int resetCount = lastReset;

volatile boolean closed = false;
Expand All @@ -233,6 +241,7 @@ public void close() {
if (!closed) {
closed = true;
lastIter = null;
lastLevel = -1;

for (LinkedListIterator<E> iter : cachedIters) {
if (iter != null) {
Expand All @@ -256,6 +265,7 @@ public boolean hasNext() {

while (index >= 0) {
lastIter = cachedIters[index];
lastLevel = index;

if (lastIter == null) {
lastIter = cachedIters[index] = levels[index].iterator();
Expand Down Expand Up @@ -289,18 +299,25 @@ public E next() {

@Override
public void remove() {
removeLastElement();
}

@Override
public E removeLastElement() {
if (lastIter == null) {
throw new NoSuchElementException();
}

lastIter.remove();
E returningElement = lastIter.removeLastElement();

// If the last message in the current priority is removed then find the next highest
for (int i = index; i >= 0 && levels[i].size() == 0; i--) {
highestPriority = i;
}

exclusiveIncrementSize(-1);
removed(lastLevel, returningElement);

return returningElement;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import io.netty.util.collection.LongObjectHashMap;
import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.core.server.MessageReference;
import org.apache.activemq.artemis.protocol.amqp.logger.ActiveMQAMQPProtocolLogger;
import org.apache.activemq.artemis.utils.collections.NodeStore;
import org.apache.activemq.artemis.utils.collections.LinkedListImpl;

Expand All @@ -35,9 +36,26 @@ public ReferenceNodeStore(ReferenceIDSupplier idSupplier) {
// This is where the messages are stored by server id...
HashMap<String, LongObjectHashMap<LinkedListImpl.Node<MessageReference>>> lists;

String name;

String lruListID;
LongObjectHashMap<LinkedListImpl.Node<MessageReference>> lruMap;

@Override
public String toString() {
return "ReferenceNodeStore{" + "name='" + name + "'}" + "@" + Integer.toHexString(System.identityHashCode(ReferenceNodeStore.this));
}

@Override
public NodeStore<MessageReference> setName(String name) {
this.name = name;
return this;
}

@Override
public String getName() {
return name;
}

@Override
public void storeNode(MessageReference element, LinkedListImpl.Node<MessageReference> node) {
Expand All @@ -50,7 +68,10 @@ private void storeNode(String serverID, long id, LinkedListImpl.Node<MessageRefe
LongObjectHashMap<LinkedListImpl.Node<MessageReference>> nodesMap = getMap(serverID);
if (nodesMap != null) {
synchronized (nodesMap) {
nodesMap.put(id, node);
LinkedListImpl.Node<MessageReference> previousNode = nodesMap.put(id, node);
if (previousNode != null) {
ActiveMQAMQPProtocolLogger.LOGGER.duplicateNodeStoreID(name, serverID, id, new Exception("trace"));
}
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,4 +64,7 @@ public interface ActiveMQAMQPProtocolLogger {

@LogMessage(id = 111009, value = "The AckManager was interrupt. timeout = {} milliseconds", level = LogMessage.Level.WARN)
void interruptedAckManager(Exception e);

@LogMessage(id = 111010, value = "Duplicate AckManager node detected. Queue={}, ServerID={}, recordID={}", level = LogMessage.Level.WARN)
void duplicateNodeStoreID(String queue, String serverId, long recordID, Exception trace);
}
Original file line number Diff line number Diff line change
Expand Up @@ -1452,13 +1452,19 @@ public synchronized boolean hasNext() {

@Override
public void remove() {
removeLastElement();
}

@Override
public PagedReference removeLastElement() {
PagedReference delivery = currentDelivery;
if (delivery != null) {
PageCursorInfo info = PageSubscriptionImpl.this.getPageInfo(delivery.getPagedMessage().getPageNumber());
if (info != null) {
info.remove(delivery.getPagedMessage().getMessageNumber());
}
}
return delivery;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -315,8 +315,6 @@ default void flushOnIntermediate(Runnable runnable) {

MessageReference removeReferenceWithID(long id) throws Exception;

MessageReference getReference(long id) throws ActiveMQException;

int deleteAllReferences() throws Exception;

int deleteAllReferences(int flushLimit) throws Exception;
Expand Down
Loading

0 comments on commit f92a846

Please sign in to comment.