ARTEMIS-3850 Eliminate Page Soft Cache. Add Option to read messages based on sizing.

This commit is contained in:
Clebert Suconic 2022-05-31 21:45:58 -04:00 committed by clebertsuconic
parent 64de8f8f4e
commit 342565d4c1
102 changed files with 3020 additions and 3143 deletions

View File

@ -61,6 +61,8 @@ import org.apache.activemq.artemis.spi.core.protocol.MessagePersister;
import org.apache.activemq.artemis.utils.ActiveMQThreadFactory;
import org.apache.activemq.artemis.utils.ExecutorFactory;
import org.apache.activemq.artemis.utils.actors.ArtemisExecutor;
import org.apache.activemq.artemis.utils.collections.LinkedList;
import org.apache.activemq.artemis.utils.collections.LinkedListIterator;
@Command(name = "print", description = "Print data records information (WARNING: don't use while a production server is running)")
public class PrintData extends DBOption {
@ -285,80 +287,84 @@ public class PrintData extends DBOption {
out.println("******* Giving up at Page " + pgid + ", System has a total of " + pgStore.getNumberOfPages() + " pages");
break;
}
Page page = pgStore.createPage(pgid);
Page page = pgStore.newPageObject(pgid);
while (!page.getFile().exists() && pgid < pgStore.getCurrentWritingPage()) {
pgid++;
page = pgStore.createPage(pgid);
page = pgStore.newPageObject(pgid);
}
out.println("******* Page " + pgid);
page.open(false);
List<PagedMessage> msgs = page.read(sm);
LinkedList<PagedMessage> msgs = page.read(sm);
page.close(false, false);
int msgID = 0;
for (PagedMessage msg : msgs) {
msg.initMessage(sm);
if (safe) {
try {
out.print("pg=" + pgid + ", msg=" + msgID + ",pgTX=" + msg.getTransactionID() + ", msg=" + msg.getMessage().getClass().getSimpleName() + "(safe data, size=" + msg.getMessage().getPersistentSize() + ")");
} catch (Exception e) {
out.print("pg=" + pgid + ", msg=" + msgID + ",pgTX=" + msg.getTransactionID() + ", msg=" + msg.getMessage().getClass().getSimpleName() + "(safe data)");
}
} else {
out.print("pg=" + pgid + ", msg=" + msgID + ",pgTX=" + msg.getTransactionID() + ",userMessageID=" + (msg.getMessage().getUserID() != null ? msg.getMessage().getUserID() : "") + ", msg=" + msg.getMessage());
}
out.print(",Queues = ");
long[] q = msg.getQueueIDs();
int ackCount = 0;
for (int i = 0; i < q.length; i++) {
out.print(q[i]);
PagePosition posCheck = new PagePositionImpl(pgid, msgID);
boolean acked = false;
Set<PagePosition> positions = cursorACKs.getCursorRecords().get(q[i]);
if (positions != null) {
acked = positions.contains(posCheck);
}
if (acked) {
out.print(" (ACK)");
}
if (cursorACKs.getCompletePages(q[i]).contains(Long.valueOf(pgid))) {
acked = true;
out.print(" (PG-COMPLETE)");
}
if (!existingQueues.contains(q[i])) {
out.print(" (N/A) ");
acked = true;
}
if (acked) {
ackCount++;
try (LinkedListIterator<PagedMessage> iter = msgs.iterator()) {
while (iter.hasNext()) {
PagedMessage msg = iter.next();
msg.initMessage(sm);
if (safe) {
try {
out.print("pg=" + pgid + ", msg=" + msgID + ",pgTX=" + msg.getTransactionID() + ", msg=" + msg.getMessage().getClass().getSimpleName() + "(safe data, size=" + msg.getMessage().getPersistentSize() + ")");
} catch (Exception e) {
out.print("pg=" + pgid + ", msg=" + msgID + ",pgTX=" + msg.getTransactionID() + ", msg=" + msg.getMessage().getClass().getSimpleName() + "(safe data)");
}
} else {
out.print(" (OK) ");
out.print("pg=" + pgid + ", msg=" + msgID + ",pgTX=" + msg.getTransactionID() + ",userMessageID=" + (msg.getMessage().getUserID() != null ? msg.getMessage().getUserID() : "") + ", msg=" + msg.getMessage());
}
out.print(",Queues = ");
long[] q = msg.getQueueIDs();
int ackCount = 0;
for (int i = 0; i < q.length; i++) {
out.print(q[i]);
if (i + 1 < q.length) {
out.print(",");
PagePosition posCheck = new PagePositionImpl(pgid, msgID);
boolean acked = false;
Set<PagePosition> positions = cursorACKs.getCursorRecords().get(q[i]);
if (positions != null) {
acked = positions.contains(posCheck);
}
if (acked) {
out.print(" (ACK)");
}
if (cursorACKs.getCompletePages(q[i]).contains(Long.valueOf(pgid))) {
acked = true;
out.print(" (PG-COMPLETE)");
}
if (!existingQueues.contains(q[i])) {
out.print(" (N/A) ");
acked = true;
}
if (acked) {
ackCount++;
} else {
out.print(" (OK) ");
}
if (i + 1 < q.length) {
out.print(",");
}
}
if (msg.getTransactionID() >= 0 && !pgTXs.contains(msg.getTransactionID())) {
out.print(", **PG_TX_NOT_FOUND**");
}
}
if (msg.getTransactionID() >= 0 && !pgTXs.contains(msg.getTransactionID())) {
out.print(", **PG_TX_NOT_FOUND**");
}
out.println();
if (ackCount != q.length) {
out.println("^^^ Previous record has " + ackCount + " acked queues and " + q.length + " queues routed");
out.println();
if (ackCount != q.length) {
out.println("^^^ Previous record has " + ackCount + " acked queues and " + q.length + " queues routed");
out.println();
}
msgID++;
}
msgID++;
pgid++;
}
pgid++;
}
}
}

View File

@ -64,6 +64,7 @@ import org.apache.activemq.artemis.core.persistence.impl.journal.codec.Persisten
import org.apache.activemq.artemis.core.persistence.impl.journal.codec.PersistentQueueBindingEncoding;
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
import org.apache.activemq.artemis.core.server.JournalType;
import org.apache.activemq.artemis.utils.collections.LinkedListIterator;
@Command(name = "exp", description = "Export all message-data using an XML that could be interpreted by any system.")
public final class XmlDataExporter extends DBOption {
@ -397,47 +398,51 @@ public final class XmlDataExporter extends DBOption {
File folder = pageStore.getFolder();
ActiveMQServerLogger.LOGGER.debug("Reading page store " + store + " folder = " + folder);
int pageId = (int) pageStore.getFirstPage();
for (int i = 0; i < pageStore.getNumberOfPages(); i++) {
long pageId = pageStore.getFirstPage();
for (long i = 0; i < pageStore.getNumberOfPages(); i++) {
ActiveMQServerLogger.LOGGER.debug("Reading page " + pageId);
Page page = pageStore.createPage(pageId);
Page page = pageStore.newPageObject(pageId);
page.open(false);
List<PagedMessage> messages = page.read(storageManager);
org.apache.activemq.artemis.utils.collections.LinkedList<PagedMessage> messages = page.read(storageManager);
page.close(false, false);
int messageId = 0;
for (PagedMessage message : messages) {
message.initMessage(storageManager);
long[] queueIDs = message.getQueueIDs();
List<String> queueNames = new ArrayList<>();
for (long queueID : queueIDs) {
PagePosition posCheck = new PagePositionImpl(pageId, messageId);
try (LinkedListIterator<PagedMessage> iter = messages.iterator()) {
boolean acked = false;
while (iter.hasNext()) {
PagedMessage message = iter.next();
message.initMessage(storageManager);
long[] queueIDs = message.getQueueIDs();
List<String> queueNames = new ArrayList<>();
for (long queueID : queueIDs) {
PagePosition posCheck = new PagePositionImpl(pageId, messageId);
Set<PagePosition> positions = cursorRecords.get(queueID);
if (positions != null) {
acked = positions.contains(posCheck);
}
boolean acked = false;
if (!acked) {
PersistentQueueBindingEncoding queueBinding = queueBindings.get(queueID);
if (queueBinding != null) {
SimpleString queueName = queueBinding.getQueueName();
queueNames.add(queueName.toString());
Set<PagePosition> positions = cursorRecords.get(queueID);
if (positions != null) {
acked = positions.contains(posCheck);
}
if (!acked) {
PersistentQueueBindingEncoding queueBinding = queueBindings.get(queueID);
if (queueBinding != null) {
SimpleString queueName = queueBinding.getQueueName();
queueNames.add(queueName.toString());
}
}
}
if (queueNames.size() > 0 && (message.getTransactionID() == -1 || pgTXs.contains(message.getTransactionID()))) {
printSingleMessageAsXML(message.getMessage().toCore(), queueNames);
}
messageId++;
}
if (queueNames.size() > 0 && (message.getTransactionID() == -1 || pgTXs.contains(message.getTransactionID()))) {
printSingleMessageAsXML(message.getMessage().toCore(), queueNames);
}
messageId++;
pageId++;
}
pageId++;
}
} else {
ActiveMQServerLogger.LOGGER.debug("Page store was null");

View File

@ -141,6 +141,13 @@ ${cluster-security.settings}${cluster.settings}${replicated.settings}${shared-st
<max-size-bytes>-1</max-size-bytes>
<!-- limit for the address in messages, -1 means unlimited -->
<max-size-messages>-1</max-size-messages>
<!-- the size of each file on paging. Notice we keep files in memory while they are in use.
Lower this setting if you have too many queues in memory. -->
<page-size-bytes>10M</page-size-bytes>
<!-- how many messages are kept in memory from paging. The system will stop reading whenever this or max-read-page-bytes hits the max first. -->
<max-read-page-messages>1000</max-read-page-messages>
<!-- how many bytes equivalent of messages are kept in memory from paging (based on memory estimate). The system will stop reading whenever this or max-read-page-messages hits the max first. -->
<max-read-page-bytes>1M</max-read-page-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>${full-policy}</address-full-policy>
<auto-create-queues>${auto-create}</auto-create-queues>

View File

@ -17,9 +17,9 @@
package org.apache.activemq.artemis.utils;
import java.util.concurrent.Executor;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
public class ReferenceCounterUtil implements ReferenceCounter {
public class ReferenceCounterUtil implements ReferenceCounter, AutoCloseable {
private Runnable task;
@ -28,7 +28,9 @@ public class ReferenceCounterUtil implements ReferenceCounter {
*/
private final Executor executor;
private final AtomicInteger uses = new AtomicInteger(0);
private volatile int use = 0;
private static final AtomicIntegerFieldUpdater<ReferenceCounterUtil> useUpdater = AtomicIntegerFieldUpdater.newUpdater(ReferenceCounterUtil.class, "use");
public ReferenceCounterUtil() {
this.executor = null;
@ -61,12 +63,12 @@ public class ReferenceCounterUtil implements ReferenceCounter {
@Override
public int increment() {
return uses.incrementAndGet();
return useUpdater.incrementAndGet(this);
}
@Override
public int decrement() {
int value = uses.decrementAndGet();
int value = useUpdater.decrementAndGet(this);
if (value == 0) {
execute();
}
@ -74,11 +76,19 @@ public class ReferenceCounterUtil implements ReferenceCounter {
return value;
}
/** it will set the value all the way to 0, and execute the task meant for when the value was 0. */
public void exhaust() {
execute();
useUpdater.set(this, 0);
}
private void execute() {
if (executor != null) {
executor.execute(task);
} else {
task.run();
if (task != null) {
if (executor != null) {
executor.execute(task);
} else {
task.run();
}
}
}
@ -91,6 +101,11 @@ public class ReferenceCounterUtil implements ReferenceCounter {
@Override
public int getCount() {
return uses.get();
return useUpdater.get(this);
}
@Override
public void close() {
decrement();
}
}

View File

@ -59,6 +59,13 @@ public class SizeAwareMetric {
private Runnable underCallback;
/** To be used in a case where we just measure elements */
public SizeAwareMetric() {
this.sizeEnabled = false;
this.elementsEnabled = false;
}
public SizeAwareMetric(long maxSize, long lowerMarkSize, long maxElements, long lowerMarkElements) {
if (lowerMarkSize > maxSize) {
throw new IllegalArgumentException("lowerMark must be <= maxSize");

View File

@ -0,0 +1,108 @@
/*
* 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.activemq.artemis.utils.collections;
import java.util.NoSuchElementException;
import java.util.function.Consumer;
public class EmptyList<E> implements LinkedList<E> {
private static final LinkedList EMPTY_LIST = new EmptyList();
public static final <T> LinkedList<T> getEmptyList() {
return (LinkedList<T>) EMPTY_LIST;
}
private EmptyList() {
}
@Override
public void addHead(E e) {
throw new UnsupportedOperationException("method not supported");
}
@Override
public void addTail(E e) {
throw new UnsupportedOperationException("method not supported");
}
@Override
public E get(int position) {
throw new IndexOutOfBoundsException("position = " + position);
}
@Override
public E poll() {
return null;
}
LinkedListIterator<E> emptyIterator = new LinkedListIterator<E>() {
@Override
public void repeat() {
}
@Override
public void close() {
}
@Override
public boolean hasNext() {
return false;
}
@Override
public E next() {
throw new NoSuchElementException();
}
};
@Override
public LinkedListIterator<E> iterator() {
return emptyIterator;
}
@Override
public void clear() {
throw new UnsupportedOperationException("method not supported");
}
@Override
public int size() {
return 0;
}
@Override
public void clearID() {
throw new UnsupportedOperationException("method not supported");
}
@Override
public void setNodeStore(NodeStore<E> store) {
throw new UnsupportedOperationException("method not supported");
}
@Override
public E removeWithID(String listID, long id) {
throw new UnsupportedOperationException("method not supported");
}
@Override
public void forEach(Consumer<E> consumer) {
}
}

View File

@ -16,12 +16,16 @@
*/
package org.apache.activemq.artemis.utils.collections;
import java.util.function.Consumer;
public interface LinkedList<E> {
void addHead(E e);
void addTail(E e);
E get(int position);
E poll();
LinkedListIterator<E> iterator();
@ -37,4 +41,6 @@ public interface LinkedList<E> {
/** you need to call {@link #setNodeStore(NodeStore)} before you are able to call this method. */
E removeWithID(String listID, long id);
void forEach(Consumer<E> consumer);
}

View File

@ -20,6 +20,7 @@ import java.lang.reflect.Array;
import java.util.Comparator;
import java.util.NoSuchElementException;
import java.util.Objects;
import java.util.function.Consumer;
/**
* A linked list implementation which allows multiple iterators to exist at the same time on the queue, and which see any
@ -102,6 +103,21 @@ public class LinkedListImpl<E> implements LinkedList<E> {
size++;
}
@Override
public E get(int position) {
Node<E> current = head.next;
for (int i = 0; i < position && current != null; i++) {
current = current.next;
}
if (current == null) {
throw new IndexOutOfBoundsException(position + " > " + size());
}
return current.val();
}
@Override
public synchronized E removeWithID(String listID, long id) {
assert nodeStore != null; // it is assumed the code will call setNodeStore before callin removeWithID
@ -117,6 +133,17 @@ public class LinkedListImpl<E> implements LinkedList<E> {
return node.val();
}
@Override
public void forEach(Consumer<E> consumer) {
try (LinkedListIterator<E> iter = iterator()) {
while (iter.hasNext()) {
E nextValue = iter.next();
consumer.accept(nextValue);
}
}
}
private void itemAdded(Node<E> node, E item) {
if (nodeStore != null) {
putID(item, node);

View File

@ -18,6 +18,7 @@ package org.apache.activemq.artemis.utils;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.LockSupport;
import java.util.function.Supplier;
import org.junit.Assert;
@ -142,13 +143,16 @@ public class Wait {
assertTrue(DEFAULT_FAILURE_MESSAGE, condition, duration, sleep);
}
public static void assertTrue(String failureMessage, Condition condition, final long duration, final long sleep) {
assertTrue(() -> failureMessage, condition, duration, sleep);
}
public static void assertTrue(Supplier<String> failureMessage, Condition condition, final long duration, final long sleep) {
boolean result = waitFor(condition, duration, sleep);
if (!result) {
Assert.fail(failureMessage);
Assert.fail(failureMessage.get());
}
}

View File

@ -98,7 +98,7 @@ public interface AddressControl {
* Returns the number of pages used by this address.
*/
@Attribute(desc = NUMBER_OF_PAGES_DESCRIPTION)
int getNumberOfPages();
long getNumberOfPages();
/**
* Returns whether this address is paging.

View File

@ -66,6 +66,11 @@ public interface CoreRemotingConnection extends RemotingConnection {
return version >= PacketImpl.ARTEMIS_2_21_0_VERSION;
}
default boolean isVersionUsingLongOnPageReplication() {
int version = getChannelVersion();
return version >= PacketImpl.ARTEMIS_2_24_0_VERSION;
}
/**
* Sets the client protocol used on the communication. This will determine if the client has
* support for certain packet types

View File

@ -44,6 +44,9 @@ public class PacketImpl implements Packet {
// 2.21.0
public static final int ARTEMIS_2_21_0_VERSION = 132;
// 2.24.0
public static final int ARTEMIS_2_24_0_VERSION = 133;
public static final SimpleString OLD_QUEUE_PREFIX = new SimpleString("jms.queue.");
public static final SimpleString OLD_TEMP_QUEUE_PREFIX = new SimpleString("jms.tempqueue.");
public static final SimpleString OLD_TOPIC_PREFIX = new SimpleString("jms.topic.");

View File

@ -1,399 +0,0 @@
/*
* 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.activemq.artemis.utils;
import java.lang.ref.ReferenceQueue;
import java.lang.ref.SoftReference;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import io.netty.util.collection.LongObjectHashMap;
import io.netty.util.collection.LongObjectMap;
import org.jboss.logging.Logger;
public class SoftValueLongObjectHashMap<V extends SoftValueLongObjectHashMap.ValueCache> implements LongObjectMap<V> {
private static final Logger logger = Logger.getLogger(SoftValueLongObjectHashMap.class);
// The soft references that are already good.
// too bad there's no way to override the queue method on ReferenceQueue, so I wouldn't need this
private final ReferenceQueue<V> refQueue = new ReferenceQueue<>();
private final LongObjectMap<AggregatedSoftReference<V>> mapDelegate = new LongObjectHashMap<>();
private long usedCounter = 0;
private int maxElements;
public interface ValueCache {
boolean isLive();
}
public SoftValueLongObjectHashMap(final int maxElements) {
this.maxElements = maxElements;
}
public void setMaxElements(final int maxElements) {
this.maxElements = maxElements;
checkCacheSize();
}
public int getMaxEelements() {
return this.maxElements;
}
/**
* @see java.util.Map#size()
*/
@Override
public int size() {
processQueue();
return mapDelegate.size();
}
/**
* @see java.util.Map#isEmpty()
*/
@Override
public boolean isEmpty() {
processQueue();
return mapDelegate.isEmpty();
}
/**
* @param key
* @see java.util.Map#containsKey(java.lang.Object)
*/
@Override
public boolean containsKey(final Object key) {
return containsKey(objectToKey(key));
}
/**
* @param value
* @see java.util.Map#containsValue(java.lang.Object)
*/
@Override
public boolean containsValue(final Object value) {
processQueue();
for (AggregatedSoftReference<V> valueIter : mapDelegate.values()) {
V valueElement = valueIter.get();
if (valueElement != null && value.equals(valueElement)) {
return true;
}
}
return false;
}
private static long objectToKey(Object key) {
return ((Long) key).longValue();
}
/**
* @param key
* @see java.util.Map#get(java.lang.Object)
*/
@Override
public V get(final Object key) {
return get(objectToKey(key));
}
@Override
public V put(Long key, V value) {
return put(objectToKey(key), value);
}
@Override
public V get(long key) {
processQueue();
AggregatedSoftReference<V> value = mapDelegate.get(key);
if (value != null) {
usedCounter++;
value.used(usedCounter);
return value.get();
} else {
return null;
}
}
/**
* @param key
* @param value
* @see java.util.Map#put(java.lang.Object, java.lang.Object)
*/
@Override
public V put(final long key, final V value) {
processQueue();
AggregatedSoftReference<V> newRef = createReference(key, value);
AggregatedSoftReference<V> oldRef = mapDelegate.put(key, newRef);
checkCacheSize();
usedCounter++;
newRef.used(usedCounter);
if (oldRef != null) {
return oldRef.get();
} else {
return null;
}
}
@Override
public V remove(long key) {
processQueue();
AggregatedSoftReference<V> ref = mapDelegate.remove(key);
if (ref != null) {
return ref.get();
} else {
return null;
}
}
private void checkCacheSize() {
if (maxElements > 0 && mapDelegate.size() > maxElements) {
TreeSet<AggregatedSoftReference> usedReferences = new TreeSet<>(new ComparatorAgregated());
for (AggregatedSoftReference<V> ref : mapDelegate.values()) {
V v = ref.get();
if (v != null && !v.isLive()) {
usedReferences.add(ref);
}
}
for (AggregatedSoftReference ref : usedReferences) {
if (ref.used > 0) {
Object removed = mapDelegate.remove(ref.key);
if (logger.isTraceEnabled()) {
logger.trace("Removing " + removed + " with id = " + ref.key + " from SoftValueLongObjectHashMap");
}
if (mapDelegate.size() <= maxElements) {
break;
}
}
}
}
}
class ComparatorAgregated implements Comparator<AggregatedSoftReference> {
@Override
public int compare(AggregatedSoftReference o1, AggregatedSoftReference o2) {
long k = o1.used - o2.used;
if (k > 0) {
return 1;
} else if (k < 0) {
return -1;
}
k = o1.hashCode() - o2.hashCode();
if (k > 0) {
return 1;
} else if (k < 0) {
return -1;
} else {
return 0;
}
}
}
/**
* @param key
* @see java.util.Map#remove(java.lang.Object)
*/
@Override
public V remove(final Object key) {
return remove(objectToKey(key));
}
/**
* @param m
* @see java.util.Map#putAll(java.util.Map)
*/
@Override
public void putAll(final Map<? extends Long, ? extends V> m) {
processQueue();
if (m instanceof LongObjectMap) {
final LongObjectMap<? extends V> primitiveMap = (LongObjectMap<? extends V>) m;
for (PrimitiveEntry<? extends V> entry : primitiveMap.entries()) {
put(entry.key(), entry.value());
}
} else {
for (Map.Entry<? extends Long, ? extends V> e : m.entrySet()) {
put(e.getKey(), e.getValue());
}
}
}
/**
* @see java.util.Map#clear()
*/
@Override
public void clear() {
mapDelegate.clear();
}
/**
* @see java.util.Map#keySet()
*/
@Override
public Set<Long> keySet() {
processQueue();
return mapDelegate.keySet();
}
/**
* @see java.util.Map#values()
*/
@Override
public Collection<V> values() {
processQueue();
ArrayList<V> list = new ArrayList<>();
for (AggregatedSoftReference<V> refs : mapDelegate.values()) {
V value = refs.get();
if (value != null) {
list.add(value);
}
}
return list;
}
@Override
public Set<Entry<Long, V>> entrySet() {
return null;
}
@Override
public Iterable<PrimitiveEntry<V>> entries() {
processQueue();
final int size = mapDelegate.size();
final List<PrimitiveEntry<V>> entries = new ArrayList<>(size);
for (PrimitiveEntry<AggregatedSoftReference<V>> pair : mapDelegate.entries()) {
V value = pair.value().get();
if (value != null) {
entries.add(new EntryElement<>(pair.key(), value));
}
}
return entries;
}
@Override
public boolean containsKey(long key) {
processQueue();
return mapDelegate.containsKey(key);
}
/**
* @param o
* @see java.util.Map#equals(java.lang.Object)
*/
@Override
public boolean equals(final Object o) {
processQueue();
return mapDelegate.equals(o);
}
/**
* @see java.util.Map#hashCode()
*/
@Override
public int hashCode() {
return mapDelegate.hashCode();
}
@SuppressWarnings("unchecked")
private void processQueue() {
AggregatedSoftReference ref;
while ((ref = (AggregatedSoftReference) this.refQueue.poll()) != null) {
logger.tracef("Removing reference through processQueue:: %s", ref.get());
mapDelegate.remove(ref.key);
}
}
private AggregatedSoftReference createReference(final long key, final V value) {
return new AggregatedSoftReference(key, value, refQueue);
}
static final class AggregatedSoftReference<V> extends SoftReference<V> {
final long key;
long used = 0;
public long getUsed() {
return used;
}
public void used(long value) {
this.used = value;
}
AggregatedSoftReference(final long key, final V referent, ReferenceQueue<V> refQueue) {
super(referent, refQueue);
this.key = key;
}
@Override
public String toString() {
return "AggregatedSoftReference [key=" + key + ", used=" + used + "]";
}
}
static final class EntryElement<V> implements LongObjectMap.PrimitiveEntry<V> {
final long key;
final V value;
EntryElement(final long key, final V value) {
this.key = key;
this.value = value;
}
@Override
public long key() {
return key;
}
@Override
public V value() {
return value;
}
@Override
@Deprecated
public void setValue(V value) {
throw new UnsupportedOperationException();
}
}
}

View File

@ -20,4 +20,4 @@ activemq.version.minorVersion=${activemq.version.minorVersion}
activemq.version.microVersion=${activemq.version.microVersion}
activemq.version.incrementingVersion=${activemq.version.incrementingVersion}
activemq.version.versionTag=${activemq.version.versionTag}
activemq.version.compatibleVersionList=121,122,123,124,125,126,127,128,129,130,131,132
activemq.version.compatibleVersionList=121,122,123,124,125,126,127,128,129,130,131,132,133

View File

@ -480,7 +480,7 @@ public class AMQPLargeMessage extends AMQPMessage implements LargeServerMessage
@Override
protected void releaseComplete() {
largeBody.deleteFile();
largeBody.releaseComplete();
}
@Override

View File

@ -218,6 +218,10 @@ public final class FileConfigurationParser extends XMLConfigurationUtil {
private static final String ADDRESS_FULL_MESSAGE_POLICY_NODE_NAME = "address-full-policy";
private static final String MAX_READ_PAGE_BYTES_NODE_NAME = "max-read-page-bytes";
private static final String MAX_READ_PAGE_MESSAGES_NODE_NAME = "max-read-page-messages";
private static final String PAGE_SIZE_BYTES_NODE_NAME = "page-size-bytes";
private static final String PAGE_MAX_CACHE_SIZE_NODE_NAME = "page-max-cache-size";
@ -330,6 +334,8 @@ public final class FileConfigurationParser extends XMLConfigurationUtil {
private boolean validateAIO = false;
private boolean printPageMaxSizeUsed = false;
/**
* @return the validateAIO
*/
@ -1261,7 +1267,19 @@ public final class FileConfigurationParser extends XMLConfigurationUtil {
long pageSizeLong = ByteUtil.convertTextBytes(getTrimmedTextContent(child));
Validators.POSITIVE_INT.validate(PAGE_SIZE_BYTES_NODE_NAME, pageSizeLong);
addressSettings.setPageSizeBytes((int) pageSizeLong);
} else if (MAX_READ_PAGE_MESSAGES_NODE_NAME.equalsIgnoreCase(name)) {
long maxReadPageMessages = Long.parseLong(getTrimmedTextContent(child));
Validators.POSITIVE_INT.validate(MAX_READ_PAGE_MESSAGES_NODE_NAME, maxReadPageMessages);
addressSettings.setMaxReadPageMessages((int)maxReadPageMessages);
} else if (MAX_READ_PAGE_BYTES_NODE_NAME.equalsIgnoreCase(name)) {
long maxReadPageBytes = ByteUtil.convertTextBytes(getTrimmedTextContent(child));
Validators.POSITIVE_INT.validate(MAX_READ_PAGE_BYTES_NODE_NAME, maxReadPageBytes);
addressSettings.setMaxReadPageBytes((int)maxReadPageBytes);
} else if (PAGE_MAX_CACHE_SIZE_NODE_NAME.equalsIgnoreCase(name)) {
if (!printPageMaxSizeUsed) {
printPageMaxSizeUsed = true;
ActiveMQServerLogger.LOGGER.pageMaxSizeUsed();
}
addressSettings.setPageCacheMaxSize(XMLUtil.parseInt(child));
} else if (MESSAGE_COUNTER_HISTORY_DAY_LIMIT_NODE_NAME.equalsIgnoreCase(name)) {
addressSettings.setMessageCounterHistoryDayLimit(XMLUtil.parseInt(child));

View File

@ -402,7 +402,7 @@ public class AddressControlImpl extends AbstractControl implements AddressContro
}
@Override
public int getNumberOfPages() {
public long getNumberOfPages() {
if (AuditLogger.isBaseLoggingEnabled()) {
AuditLogger.getNumberOfPages(this.addressInfo);
}

View File

@ -18,8 +18,8 @@ package org.apache.activemq.artemis.core.paging;
import org.apache.activemq.artemis.core.journal.EncodingSupport;
import org.apache.activemq.artemis.core.paging.cursor.PageIterator;
import org.apache.activemq.artemis.core.paging.cursor.PagePosition;
import org.apache.activemq.artemis.core.paging.cursor.PageSubscription;
import org.apache.activemq.artemis.core.paging.cursor.PagedReference;
import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.core.transaction.Transaction;
@ -65,6 +65,6 @@ public interface PageTransactionInfo extends EncodingSupport {
*
* @return true if the message will be delivered later, false if it should be delivered right away
*/
boolean deliverAfterCommit(PageIterator pageIterator, PageSubscription cursor, PagePosition cursorPos);
boolean deliverAfterCommit(PageIterator pageIterator, PageSubscription cursor, PagedReference pagedMessage);
}

View File

@ -19,6 +19,7 @@ package org.apache.activemq.artemis.core.paging;
import org.apache.activemq.artemis.api.core.ActiveMQException;
import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.core.journal.EncodingSupport;
import org.apache.activemq.artemis.core.paging.cursor.PagePosition;
import org.apache.activemq.artemis.core.persistence.StorageManager;
/**
@ -31,6 +32,8 @@ public interface PagedMessage extends EncodingSupport {
Message getMessage();
PagePosition newPositionObject();
/**
* The queues that were routed during paging
*/
@ -55,4 +58,13 @@ public interface PagedMessage extends EncodingSupport {
* at the same amount of bytes it used. In some cases it may need to add headers in AMQP
* or extra data that may affect the outcome of getEncodeSize() */
int getStoredSize();
long getPageNumber();
PagedMessage setPageNumber(long pageNr);
int getMessageNumber();
PagedMessage setMessageNumber(int messageNr);
}

View File

@ -30,6 +30,7 @@ import org.apache.activemq.artemis.core.server.RouteContextList;
import org.apache.activemq.artemis.core.settings.impl.AddressFullMessagePolicy;
import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
import org.apache.activemq.artemis.core.transaction.Transaction;
import org.apache.activemq.artemis.utils.actors.ArtemisExecutor;
/**
* <p>
@ -45,12 +46,12 @@ public interface PagingStore extends ActiveMQComponent, RefCountMessageListener
SimpleString getAddress();
int getNumberOfPages();
long getNumberOfPages();
/**
* Returns the page id of the current page in which the system is writing files.
*/
int getCurrentWritingPage();
long getCurrentWritingPage();
SimpleString getStoreName();
@ -66,6 +67,10 @@ public interface PagingStore extends ActiveMQComponent, RefCountMessageListener
long getMaxSize();
int getMaxPageReadBytes();
int getMaxPageReadMessages();
void applySetting(AddressSettings addressSettings);
/** This method will look if the current state of paging is not paging,
@ -94,9 +99,14 @@ public interface PagingStore extends ActiveMQComponent, RefCountMessageListener
*/
boolean page(Message message, Transaction tx, RouteContextList listCtx) throws Exception;
Page createPage(int page) throws Exception;
Page usePage(long page);
boolean checkPageFileExists(int page) throws Exception;
/** Use this method when you want to use the cache of used pages. If you are just using offline (e.g. print-data), use the newPageObject method.*/
Page usePage(long page, boolean create);
Page newPageObject(long page) throws Exception;
boolean checkPageFileExists(long page) throws Exception;
PagingManager getPagingManager();
@ -170,6 +180,10 @@ public interface PagingStore extends ActiveMQComponent, RefCountMessageListener
*/
void flushExecutors();
void execute(Runnable runnable);
ArtemisExecutor getExecutor();
/**
* Files to synchronize with a remote backup.
*

View File

@ -24,16 +24,7 @@ import org.apache.activemq.artemis.core.paging.PagedMessage;
*/
public interface PageCursorProvider {
/**
* Used on tests, to simulate a scenario where the VM cleared space
*/
void clearCache();
PageCache getPageCache(long pageNr);
PagedReference newReference(PagePosition pos, PagedMessage msg, PageSubscription sub);
void addLivePageCache(LivePageCache cache);
PagedReference newReference(PagedMessage msg, PageSubscription sub);
/**
* @param queueId The cursorID should be the same as the queueId associated for persistence
@ -43,8 +34,6 @@ public interface PageCursorProvider {
PageSubscription createSubscription(long queueId, Filter filter, boolean durable);
PagedMessage getMessage(PagePosition pos);
void processReload() throws Exception;
void stop();
@ -62,21 +51,9 @@ public interface PageCursorProvider {
*/
void onPageModeCleared();
/**
* Perform the cleanup at the caller's thread (for startup and recovery)
*/
void cleanup();
void setCacheMaxSize(int size);
/**
* @param pageCursorImpl
*/
void close(PageSubscription pageCursorImpl);
// to be used on tests -------------------------------------------
int getCacheSize();
void printDebug();
}

View File

@ -21,8 +21,12 @@ import org.apache.activemq.artemis.utils.collections.LinkedListIterator;
public interface PageIterator extends LinkedListIterator<PagedReference> {
void redeliver(PagePosition reference);
enum NextResult {
noElements,
hasElements,
retry
}
void redeliver(PagedReference reference);
// return 0 if no elements, 1 if having more elements, 2 if taking too long to find
int tryNext();
NextResult tryNext();
}

View File

@ -28,12 +28,8 @@ public interface PagePosition extends Comparable<PagePosition> {
int getMessageNr();
int getFileOffset();
long getPersistentSize();
void setPersistentSize(long persistentSize);
PagePosition nextPage();
}

View File

@ -25,8 +25,6 @@ import org.apache.activemq.artemis.core.paging.PagingStore;
import org.apache.activemq.artemis.core.paging.impl.Page;
import org.apache.activemq.artemis.core.server.Queue;
import org.apache.activemq.artemis.core.transaction.Transaction;
import org.apache.activemq.artemis.utils.actors.ArtemisExecutor;
import org.apache.activemq.artemis.utils.collections.LinkedListIterator;
public interface PageSubscription {
@ -61,7 +59,7 @@ public interface PageSubscription {
PageIterator iterator();
LinkedListIterator<PagedReference> iterator(boolean jumpRemoves);
PageIterator iterator(boolean browsing);
// To be called when the cursor is closed for good. Most likely when the queue is deleted
@ -133,14 +131,9 @@ public interface PageSubscription {
void processReload() throws Exception;
void addPendingDelivery(PagePosition position);
void addPendingDelivery(PagedMessage pagedMessage);
/**
* To be used on redeliveries
*
* @param position
*/
void redeliver(PageIterator iterator, PagePosition position);
void redeliver(PageIterator iterator, PagedReference reference);
void printDebug();
@ -153,26 +146,16 @@ public interface PageSubscription {
void forEachConsumedPage(Consumer<ConsumedPage> pageCleaner);
/**
* wait all the scheduled runnables to finish their current execution
*/
void flushExecutors();
void setQueue(Queue queue);
Queue getQueue();
/**
* To be used to requery the reference case the Garbage Collection removed it from the PagedReference as it's using WeakReferences
* To be used to requery the reference
*
* @param pos
* @return
*/
PagedMessage queryMessage(PagePosition pos);
/**
* @return executor used by the PageSubscription
*/
ArtemisExecutor getExecutor();
void setQueue(Queue queue);
Queue getQueue();
/**
* @param deletedPage
@ -186,5 +169,5 @@ public interface PageSubscription {
void incrementDeliveredSize(long size);
void removePendingDelivery(PagePosition position);
void removePendingDelivery(PagedMessage pagedMessage);
}

View File

@ -29,19 +29,7 @@ public interface PagedReference extends MessageReference {
long getTransactionID();
/** this method affects paging clean up
It adds to the flag that prevents its page from cleanup.
It's a helper method to call the proper {@link PageSubscription#addPendingDelivery(PagePosition)}
@see PageSubscription#addPendingDelivery(PagePosition)
*/
void addPendingFlag();
/** this method affects paging clean up
It adds to the flag that prevents its page from cleanup.
It's a helper method to call the proper {@link PageSubscription#addPendingDelivery(PagePosition)}
@see PageSubscription#addPendingDelivery(PagePosition)
*/
void removePendingFlag();
}

View File

@ -16,7 +16,6 @@
*/
package org.apache.activemq.artemis.core.paging.cursor;
import java.lang.ref.WeakReference;
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
import java.util.function.Consumer;
@ -39,9 +38,7 @@ public class PagedReferenceImpl extends LinkedListImpl.Node<PagedReferenceImpl>
private static final AtomicIntegerFieldUpdater<PagedReferenceImpl> DELIVERY_COUNT_UPDATER = AtomicIntegerFieldUpdater
.newUpdater(PagedReferenceImpl.class, "deliveryCount");
private final PagePosition position;
private WeakReference<PagedMessage> message;
protected PagedMessage message;
private static final long UNDEFINED_DELIVERY_TIME = Long.MIN_VALUE;
private long deliveryTime = UNDEFINED_DELIVERY_TIME;
@ -50,6 +47,21 @@ public class PagedReferenceImpl extends LinkedListImpl.Node<PagedReferenceImpl>
private int messageEstimate = -1;
// this is a cached position returned on getPosition.
// just to avoid creating on object on each call
PagePosition cachedPositionObject;
/** This will create a new PagePosition, or return one previously created.
* This method is used to avoid repetitions on browsing iteration only.
*/
@Override
public PagePosition getPosition() {
if (cachedPositionObject == null) {
cachedPositionObject = getPagedMessage().newPositionObject();
}
return cachedPositionObject;
}
private long consumerID;
private boolean hasConsumerID = false;
@ -57,7 +69,7 @@ public class PagedReferenceImpl extends LinkedListImpl.Node<PagedReferenceImpl>
@SuppressWarnings("unused")
private volatile int deliveryCount = 0;
private final PageSubscription subscription;
protected final PageSubscription subscription;
private boolean alreadyAcked;
@ -120,30 +132,12 @@ public class PagedReferenceImpl extends LinkedListImpl.Node<PagedReferenceImpl>
@Override
public synchronized PagedMessage getPagedMessage() {
PagedMessage returnMessage = message != null ? message.get() : null;
// We only keep a few references on the Queue from paging...
// Besides those references are SoftReferenced on page cache...
// So, this will unlikely be null,
// unless the Queue has stalled for some time after paging
if (returnMessage == null) {
// reference is gone, we will reconstruct it
returnMessage = subscription.queryMessage(position);
message = new WeakReference<>(returnMessage);
}
return returnMessage;
return message;
}
@Override
public PagePosition getPosition() {
return position;
}
public PagedReferenceImpl(final PagePosition position,
final PagedMessage message,
public PagedReferenceImpl(final PagedMessage message,
final PageSubscription subscription) {
this.position = position;
this.message = new WeakReference<>(message);
this.message = message;
this.subscription = subscription;
if (message != null) {
this.largeMessage = message.getMessage().isLargeMessage() ? IS_LARGE_MESSAGE : IS_NOT_LARGE_MESSAGE;
@ -192,7 +186,7 @@ public class PagedReferenceImpl extends LinkedListImpl.Node<PagedReferenceImpl>
@Override
public MessageReference copy(final Queue queue) {
return new PagedReferenceImpl(this.position, this.getPagedMessage(), this.subscription);
return new PagedReferenceImpl(this.getPagedMessage(), this.subscription);
}
@Override
@ -307,8 +301,8 @@ public class PagedReferenceImpl extends LinkedListImpl.Node<PagedReferenceImpl>
// in case of an exception because of a missing page, we just want toString to return null
msgToString = "error:" + e.getMessage();
}
return "PagedReferenceImpl [position=" + position +
", message=" +
return "PagedReferenceImpl [" +
"message=" +
msgToString +
", deliveryTime=" +
(deliveryTime == UNDEFINED_DELIVERY_TIME ? null : deliveryTime) +
@ -368,12 +362,12 @@ public class PagedReferenceImpl extends LinkedListImpl.Node<PagedReferenceImpl>
@Override
public void addPendingFlag() {
subscription.addPendingDelivery(position);
subscription.addPendingDelivery(this.getPagedMessage());
}
@Override
public void removePendingFlag() {
subscription.removePendingDelivery(position);
subscription.removePendingDelivery(this.getPagedMessage());
}
@Override

View File

@ -17,21 +17,22 @@
package org.apache.activemq.artemis.core.paging.cursor;
import org.apache.activemq.artemis.core.paging.PagedMessage;
import org.apache.activemq.artemis.utils.SoftValueLongObjectHashMap;
public interface PageCache extends SoftValueLongObjectHashMap.ValueCache, AutoCloseable {
public class QueryPagedReferenceImpl extends PagedReferenceImpl {
long getPageId();
final PagePosition position;
int getNumberOfMessages();
/**
* @param pagePosition page position
* @return
*/
PagedMessage getMessage(PagePosition pagePosition);
public QueryPagedReferenceImpl(PagePosition position, PagedMessage message, PageSubscription subscription) {
super(message, subscription);
this.position = position;
}
@Override
void close();
public synchronized PagedMessage getPagedMessage() {
if (message == null) {
message = subscription.queryMessage(position);
}
return message;
}
}

View File

@ -1,114 +0,0 @@
/*
* 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.activemq.artemis.core.paging.cursor.impl;
import org.apache.activemq.artemis.core.paging.PagedMessage;
import org.apache.activemq.artemis.core.paging.cursor.LivePageCache;
import org.apache.activemq.artemis.core.paging.cursor.PagePosition;
import org.apache.activemq.artemis.utils.collections.ConcurrentAppendOnlyChunkedList;
import org.jboss.logging.Logger;
/**
* This is the same as PageCache, however this is for the page that's being currently written.
*/
public final class LivePageCacheImpl implements LivePageCache {
private static final Logger logger = Logger.getLogger(LivePageCacheImpl.class);
private static final int CHUNK_SIZE = 32;
private final PagedMessage[] initialMessages;
private final ConcurrentAppendOnlyChunkedList<PagedMessage> liveMessages;
private final long pageId;
private volatile boolean isLive = true;
public LivePageCacheImpl(final long pageId) {
this.pageId = pageId;
this.liveMessages = new ConcurrentAppendOnlyChunkedList<>(CHUNK_SIZE);
this.initialMessages = null;
}
public LivePageCacheImpl(final long pageId, PagedMessage[] initialMessages) {
this.pageId = pageId;
this.liveMessages = new ConcurrentAppendOnlyChunkedList<>(CHUNK_SIZE);
this.initialMessages = initialMessages;
}
private int initialMessagesSize() {
final PagedMessage[] initialMessages = this.initialMessages;
return initialMessages == null ? 0 : initialMessages.length;
}
@Override
public long getPageId() {
return pageId;
}
@Override
public int getNumberOfMessages() {
return initialMessagesSize() + liveMessages.size();
}
@Override
public PagedMessage getMessage(PagePosition pagePosition) {
final int messageNr = pagePosition.getMessageNr();
if (messageNr < 0) {
return null;
}
final int initialOffset = initialMessagesSize();
if (messageNr < initialOffset) {
return initialMessages[messageNr];
}
final int index = messageNr - initialOffset;
return liveMessages.get(index);
}
@Override
public boolean isLive() {
return isLive;
}
@Override
public void addLiveMessage(PagedMessage message) {
message.getMessage().usageUp();
liveMessages.add(message);
}
@Override
public void close() {
logger.tracef("Closing %s", this);
this.isLive = false;
}
@Override
public PagedMessage[] getMessages() {
final PagedMessage[] pagedMessages = liveMessages.toArray(size -> new PagedMessage[initialMessagesSize() + size], initialMessagesSize());
final PagedMessage[] initialMessages = this.initialMessages;
if (initialMessages != null) {
System.arraycopy(initialMessages, 0, pagedMessages, 0, initialMessages.length);
}
return pagedMessages;
}
@Override
public String toString() {
return "LivePacheCacheImpl::page=" + pageId + " number of messages=" + getNumberOfMessages() + " isLive = " + isLive;
}
}

View File

@ -1,74 +0,0 @@
/*
* 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.activemq.artemis.core.paging.cursor.impl;
import org.apache.activemq.artemis.core.paging.PagedMessage;
import org.apache.activemq.artemis.core.paging.cursor.BulkPageCache;
import org.apache.activemq.artemis.core.paging.cursor.PagePosition;
/**
* The caching associated to a single page.
*/
class PageCacheImpl implements BulkPageCache {
private final PagedMessage[] messages;
private final long pageId;
PageCacheImpl(final long pageId, PagedMessage[] messages) {
this.pageId = pageId;
this.messages = messages;
}
@Override
public PagedMessage getMessage(PagePosition pagePosition) {
if (pagePosition.getMessageNr() < messages.length) {
return messages[pagePosition.getMessageNr()];
} else {
return null;
}
}
@Override
public long getPageId() {
return pageId;
}
@Override
public int getNumberOfMessages() {
return messages.length;
}
@Override
public void close() {
}
@Override
public boolean isLive() {
return false;
}
@Override
public String toString() {
return "PageCacheImpl::page=" + pageId + " numberOfMessages = " + messages.length;
}
@Override
public PagedMessage[] getMessages() {
return messages;
}
}

View File

@ -19,21 +19,14 @@ package org.apache.activemq.artemis.core.paging.cursor.impl;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import io.netty.util.collection.LongObjectHashMap;
import org.apache.activemq.artemis.core.filter.Filter;
import org.apache.activemq.artemis.core.paging.PagedMessage;
import org.apache.activemq.artemis.core.paging.PagingStore;
import org.apache.activemq.artemis.core.paging.cursor.LivePageCache;
import org.apache.activemq.artemis.core.paging.cursor.NonExistentPage;
import org.apache.activemq.artemis.core.paging.cursor.PageCache;
import org.apache.activemq.artemis.core.paging.cursor.BulkPageCache;
import org.apache.activemq.artemis.core.paging.cursor.PageCursorProvider;
import org.apache.activemq.artemis.core.paging.cursor.PagePosition;
import org.apache.activemq.artemis.core.paging.cursor.PageSubscription;
import org.apache.activemq.artemis.core.paging.cursor.PagedReference;
import org.apache.activemq.artemis.core.paging.cursor.PagedReferenceImpl;
@ -43,10 +36,9 @@ import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
import org.apache.activemq.artemis.core.transaction.Transaction;
import org.apache.activemq.artemis.core.transaction.impl.TransactionImpl;
import org.apache.activemq.artemis.utils.ArtemisCloseable;
import org.apache.activemq.artemis.utils.SoftValueLongObjectHashMap;
import org.apache.activemq.artemis.utils.ThreadDumpUtil;
import org.apache.activemq.artemis.utils.actors.ArtemisExecutor;
import org.apache.activemq.artemis.utils.collections.ConcurrentLongHashMap;
import org.apache.activemq.artemis.utils.collections.LinkedList;
import org.apache.activemq.artemis.utils.collections.LongHashSet;
import org.jboss.logging.Logger;
@ -67,15 +59,6 @@ public class PageCursorProviderImpl implements PageCursorProvider {
protected final StorageManager storageManager;
// This is the same executor used at the PageStoreImpl. One Executor per pageStore
private final ArtemisExecutor executor;
private final SoftValueLongObjectHashMap<BulkPageCache> softCache;
private LongObjectHashMap<Integer> numberOfMessages = null;
private final LongObjectHashMap<CompletableFuture<BulkPageCache>> inProgressReadPages;
private final ConcurrentLongHashMap<PageSubscription> activeCursors = new ConcurrentLongHashMap<>();
private static final long PAGE_READ_TIMEOUT_NS = TimeUnit.SECONDS.toNanos(30);
@ -102,12 +85,6 @@ public class PageCursorProviderImpl implements PageCursorProvider {
final boolean readWholePage) {
this.pagingStore = pagingStore;
this.storageManager = storageManager;
this.executor = executor;
this.softCache = new SoftValueLongObjectHashMap<>(maxCacheSize);
if (!readWholePage) {
this.numberOfMessages = new LongObjectHashMap<>();
}
this.inProgressReadPages = new LongObjectHashMap<>();
}
@ -121,7 +98,7 @@ public class PageCursorProviderImpl implements PageCursorProvider {
throw new IllegalStateException("Cursor " + cursorID + " had already been created");
}
PageSubscription activeCursor = new PageSubscriptionImpl(this, pagingStore, storageManager, executor, filter, cursorID, persistent);
PageSubscription activeCursor = new PageSubscriptionImpl(this, pagingStore, storageManager, filter, cursorID, persistent);
activeCursors.put(cursorID, activeCursor);
return activeCursor;
}
@ -132,161 +109,11 @@ public class PageCursorProviderImpl implements PageCursorProvider {
}
@Override
public PagedMessage getMessage(final PagePosition pos) {
PageCache cache = getPageCache(pos.getPageNr());
if (cache == null || pos.getMessageNr() >= cache.getNumberOfMessages()) {
// sanity check, this should never happen unless there's a bug
throw new NonExistentPage("Invalid messageNumber passed = " + pos + " on " + cache);
}
return cache.getMessage(pos);
}
@Override
public PagedReference newReference(final PagePosition pos,
final PagedMessage msg,
public PagedReference newReference(final PagedMessage msg,
final PageSubscription subscription) {
return new PagedReferenceImpl(pos, msg, subscription);
return new PagedReferenceImpl(msg, subscription);
}
@Override
public PageCache getPageCache(final long pageId) {
try {
if (pageId > pagingStore.getCurrentWritingPage()) {
return null;
}
boolean createPage = false;
CompletableFuture<BulkPageCache> inProgressReadPage;
BulkPageCache cache;
Page page = null;
synchronized (softCache) {
cache = softCache.get(pageId);
if (cache != null) {
return cache;
}
if (!pagingStore.checkPageFileExists((int) pageId)) {
return null;
}
Page currentPage = pagingStore.getCurrentPage();
// Live page cache might be cleared by gc, we need to retrieve it otherwise partially written page cache is being returned
if (currentPage != null && currentPage.getPageId() == pageId && (cache = currentPage.getLiveCache()) != null) {
softCache.put(cache.getPageId(), cache);
return cache;
}
inProgressReadPage = inProgressReadPages.get(pageId);
if (inProgressReadPage == null) {
if (numberOfMessages != null && numberOfMessages.containsKey(pageId)) {
return new PageReader(pagingStore.createPage((int) pageId), numberOfMessages.get(pageId));
}
final CompletableFuture<BulkPageCache> readPage = new CompletableFuture<>();
page = pagingStore.createPage((int) pageId);
createPage = true;
inProgressReadPage = readPage;
inProgressReadPages.put(pageId, readPage);
}
}
if (createPage) {
return readPage(pageId, page, inProgressReadPage);
} else {
final long startedWait = System.nanoTime();
while (true) {
try {
return inProgressReadPage.get(CONCURRENT_PAGE_READ_TIMEOUT_NS, TimeUnit.NANOSECONDS);
} catch (TimeoutException e) {
final long elapsed = System.nanoTime() - startedWait;
final long elapsedMillis = TimeUnit.NANOSECONDS.toMillis(elapsed);
logger.warnf("Waiting a concurrent Page::read for pageNr=%d on cursor %s by %d ms",
pageId, pagingStore.getAddress(), elapsedMillis);
}
}
}
} catch (Exception e) {
throw new RuntimeException(e.getMessage(), e);
}
}
private PageCache readPage(long pageId,
Page page,
CompletableFuture<BulkPageCache> inProgressReadPage) throws Exception {
logger.tracef("adding pageCache pageNr=%d into cursor = %s", pageId, this.pagingStore.getAddress());
boolean acquiredPageReadPermission = false;
int num = -1;
final PageCacheImpl cache;
try {
final long startedRequest = System.nanoTime();
while (!acquiredPageReadPermission) {
acquiredPageReadPermission = storageManager.beforePageRead(PAGE_READ_PERMISSION_TIMEOUT_NS, TimeUnit.NANOSECONDS);
if (!acquiredPageReadPermission) {
final long elapsedMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startedRequest);
logger.warnf("Cannot acquire page read permission of pageNr=%d on cursor %s after %d ms: consider increasing page-max-concurrent-io or use a faster disk",
pageId, pagingStore.getAddress(), elapsedMillis);
}
}
page.open(false);
final long startedReadPage = System.nanoTime();
List<PagedMessage> pgdMessages = page.read(storageManager);
final long elapsedReadPage = System.nanoTime() - startedReadPage;
if (elapsedReadPage > PAGE_READ_TIMEOUT_NS) {
logger.warnf("Page::read for pageNr=%d on cursor %s tooks %d ms to read %d bytes", pageId,
pagingStore.getAddress(), TimeUnit.NANOSECONDS.toMillis(elapsedReadPage), page.getSize());
}
num = pgdMessages.size();
cache = new PageCacheImpl(pageId, pgdMessages.toArray(new PagedMessage[num]));
} catch (Throwable t) {
inProgressReadPage.completeExceptionally(t);
synchronized (softCache) {
inProgressReadPages.remove(pageId);
}
throw t;
} finally {
try {
if (page != null) {
page.close(false, false);
}
} catch (Throwable ignored) {
}
if (acquiredPageReadPermission) {
storageManager.afterPageRead();
}
}
inProgressReadPage.complete(cache);
synchronized (softCache) {
inProgressReadPages.remove(pageId);
softCache.put(pageId, cache);
if (numberOfMessages != null && num != -1) {
numberOfMessages.put(pageId, Integer.valueOf(num));
}
}
return cache;
}
@Override
public void addLivePageCache(LivePageCache cache) {
logger.tracef("Add live page cache %s", cache);
synchronized (softCache) {
softCache.put(cache.getPageId(), cache);
}
}
@Override
public void setCacheMaxSize(final int size) {
softCache.setMaxElements(size);
}
@Override
public int getCacheSize() {
synchronized (softCache) {
return softCache.size();
}
}
@Override
public void clearCache() {
synchronized (softCache) {
softCache.clear();
}
}
@Override
public void processReload() throws Exception {
@ -326,19 +153,9 @@ public class PageCursorProviderImpl implements PageCursorProvider {
}
}
private void waitForFuture() {
if (!executor.flush(10, TimeUnit.SECONDS)) {
ActiveMQServerLogger.LOGGER.timedOutStoppingPagingCursor(executor);
ActiveMQServerLogger.LOGGER.threadDump(ThreadDumpUtil.threadDump(""));
}
}
@Override
public void flushExecutors() {
for (PageSubscription cursor : activeCursors.values()) {
cursor.flushExecutors();
}
waitForFuture();
pagingStore.flushExecutors();
}
@Override
@ -350,7 +167,6 @@ public class PageCursorProviderImpl implements PageCursorProvider {
@Override
public void scheduleCleanup() {
if (!cleanupEnabled || scheduledCleanup.intValue() > 2) {
// Scheduled cleanup was already scheduled before.. never mind!
// or we have cleanup disabled
@ -359,7 +175,7 @@ public class PageCursorProviderImpl implements PageCursorProvider {
scheduledCleanup.incrementAndGet();
executor.execute(new Runnable() {
pagingStore.execute(new Runnable() {
@Override
public void run() {
storageManager.setContext(storageManager.newSingleThreadContext());
@ -412,8 +228,7 @@ public class PageCursorProviderImpl implements PageCursorProvider {
scheduleCleanup();
}
@Override
public void cleanup() {
protected void cleanup() {
ArrayList<Page> depagedPages = new ArrayList<>();
LongHashSet depagedPagesSet = new LongHashSet();
@ -441,17 +256,19 @@ public class PageCursorProviderImpl implements PageCursorProvider {
synchronized (this) {
try {
if (!pagingStore.isStarted()) {
logger.trace("Paging store is not started");
return;
}
if (pagingStore.getNumberOfPages() == 0) {
if (!pagingStore.isPaging()) {
logger.trace("Paging Store was not paging, so no reason to retry the cleanup");
return;
}
ArrayList<PageSubscription> cursorList = cloneSubscriptions();
long minPage = checkMinPage(cursorList);
final long firstPage = pagingStore.getFirstPage();
final long firstPage = pagingStore.getFirstPage();
deliverIfNecessary(cursorList, minPage);
logger.tracef("firstPage=%s, minPage=%s, currentWritingPage=%s", firstPage, minPage, pagingStore.getCurrentWritingPage());
@ -462,7 +279,10 @@ public class PageCursorProviderImpl implements PageCursorProvider {
// Then we do some check on eventual pages that can be already removed but they are away from the streaming
cleanupMiddleStream(depagedPages, depagedPagesSet, cursorList, minPage, firstPage);
if (pagingStore.getNumberOfPages() == 0 || pagingStore.getNumberOfPages() == 1 && pagingStore.getCurrentPage().getNumberOfMessages() == 0) {
assert pagingStore.getNumberOfPages() >= 0;
if (pagingStore.getNumberOfPages() == 0 || pagingStore.getNumberOfPages() == 1 && (pagingStore.getCurrentPage() == null || pagingStore.getCurrentPage().getNumberOfMessages() == 0)) {
logger.tracef("StopPaging being called on %s", pagingStore);
pagingStore.stopPaging();
} else {
if (logger.isTraceEnabled()) {
@ -600,48 +420,24 @@ public class PageCursorProviderImpl implements PageCursorProvider {
logger.tracef("this(%s) finishing cleanup on %s", this, depagedPages);
try {
for (Page depagedPage : depagedPages) {
BulkPageCache cache;
PagedMessage[] pgdMessages;
synchronized (softCache) {
cache = softCache.get((long) depagedPage.getPageId());
}
if (logger.isTraceEnabled()) {
logger.trace("Removing pageNr=" + depagedPage.getPageId() + " from page-cache");
}
storageManager.beforePageRead();
if (cache == null) {
// The page is not on cache any more
// We need to read the page-file before deleting it
// to make sure we remove any large-messages pending
storageManager.beforePageRead();
List<PagedMessage> pgdMessagesList = null;
LinkedList<PagedMessage> pgdMessagesList = null;
try {
depagedPage.open(false);
pgdMessagesList = depagedPage.read(storageManager, true);
} finally {
try {
depagedPage.open(false);
pgdMessagesList = depagedPage.read(storageManager, true);
} finally {
try {
depagedPage.close(false, false);
} catch (Exception e) {
}
storageManager.afterPageRead();
depagedPage.close(false, false);
} catch (Exception e) {
}
pgdMessages = pgdMessagesList.isEmpty() ? null :
pgdMessagesList.toArray(new PagedMessage[pgdMessagesList.size()]);
} else {
pgdMessages = cache.getMessages();
storageManager.afterPageRead();
}
depagedPage.delete(pgdMessages);
synchronized (softCache) {
long pageId = (long) depagedPage.getPageId();
softCache.remove(pageId);
if (numberOfMessages != null) {
numberOfMessages.remove(pageId);
}
}
depagedPage.delete(pgdMessagesList);
onDeletePage(depagedPage);
}
} catch (Exception ex) {
@ -657,8 +453,7 @@ public class PageCursorProviderImpl implements PageCursorProvider {
boolean complete = true;
Page page = pagingStore.createPage((int)minPage);
if (!page.getFile().exists()) {
if (!pagingStore.checkPageFileExists(minPage)) {
logger.tracef("store %s did not have an existing file, considering it a complete file then", pagingStore.getAddress());
return true;
}
@ -713,14 +508,6 @@ public class PageCursorProviderImpl implements PageCursorProvider {
}
}
@Override
public void printDebug() {
System.out.println("Debug information for PageCursorProviderImpl:");
for (PageCache cache : softCache.values()) {
System.out.println("Cache " + cache);
}
}
@Override
public String toString() {
return "PageCursorProviderImpl{" +
@ -769,6 +556,4 @@ public class PageCursorProviderImpl implements PageCursorProvider {
}
}
}
}
}

View File

@ -20,7 +20,7 @@ import org.apache.activemq.artemis.core.paging.cursor.PagePosition;
public class PagePositionImpl implements PagePosition {
private long pageNr;
private final long pageNr;
/**
* The index of the message on the page file.
@ -29,9 +29,7 @@ public class PagePositionImpl implements PagePosition {
* for instance when a cursor is storing the next message to be received
* or when a page is marked as fully complete (as the ACKs are removed)
*/
private int messageNr;
private int fileOffset = -1;
private final int messageNr;
/**
* ID used for storage
@ -44,25 +42,11 @@ public class PagePositionImpl implements PagePosition {
*/
private long persistentSize;
/**
* @param pageNr
* @param messageNr
* @param fileOffset
*/
public PagePositionImpl(long pageNr, int messageNr, int fileOffset) {
this();
public PagePositionImpl(long pageNr, int messageNr) {
this.pageNr = pageNr;
this.messageNr = messageNr;
this.fileOffset = fileOffset;
}
public PagePositionImpl(long pageNr, int messageNr) {
this(pageNr, messageNr, -1);
}
public PagePositionImpl() {
super();
}
/**
* @return the recordID
@ -96,11 +80,6 @@ public class PagePositionImpl implements PagePosition {
return messageNr;
}
@Override
public int getFileOffset() {
return fileOffset;
}
/**
* @return the persistentSize
*/
@ -126,11 +105,6 @@ public class PagePositionImpl implements PagePosition {
} else return Long.compare(recordID, o.getRecordID());
}
@Override
public PagePosition nextPage() {
return new PagePositionImpl(this.pageNr + 1, 0, 0);
}
@Override
public int hashCode() {
final int prime = 31;
@ -158,8 +132,7 @@ public class PagePositionImpl implements PagePosition {
@Override
public String toString() {
return "PagePositionImpl [pageNr=" + pageNr + ", messageNr=" + messageNr + ", recordID=" + recordID +
", fileOffset=" + fileOffset + "]";
return "PagePositionImpl [pageNr=" + pageNr + ", messageNr=" + messageNr + ", recordID=" + recordID + "]";
}
}

View File

@ -1,137 +0,0 @@
/**
* 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.activemq.artemis.core.paging.cursor.impl;
import org.apache.activemq.artemis.core.paging.PagedMessage;
import org.apache.activemq.artemis.core.paging.cursor.NonExistentPage;
import org.apache.activemq.artemis.core.paging.cursor.PageCache;
import org.apache.activemq.artemis.core.paging.cursor.PagePosition;
import org.apache.activemq.artemis.core.paging.impl.Page;
import org.jboss.logging.Logger;
import static org.apache.activemq.artemis.utils.Preconditions.checkNotNull;
public class PageReader implements PageCache {
private static final Logger logger = Logger.getLogger(PageReader.class);
private final Page page;
private final int numberOfMessages;
public PageReader(Page page, int numberOfMessages) {
checkNotNull(page);
this.page = page;
this.numberOfMessages = numberOfMessages;
}
@Override
public long getPageId() {
return page.getPageId();
}
@Override
public int getNumberOfMessages() {
return numberOfMessages;
}
/**
* Used just for testing purposes.
*/
protected synchronized PagedMessage[] readMessages() {
try {
openPage();
return page.read().toArray(new PagedMessage[numberOfMessages]);
} catch (Exception e) {
throw new RuntimeException(e.getMessage(), e);
} finally {
close();
}
}
@Override
public boolean isLive() {
return false;
}
/**
* @param pagePosition page position
* @param throwException if {@code true} exception will be thrown when message number is beyond the page
* @param keepOpen if {@code true} page file would keep open after reading message
* @return the paged message
*/
public synchronized PagedMessage getMessage(PagePosition pagePosition, boolean throwException, boolean keepOpen) {
if (pagePosition.getMessageNr() >= getNumberOfMessages()) {
if (throwException) {
throw new NonExistentPage("Invalid messageNumber passed = " + pagePosition + " on " + this);
}
return null;
}
boolean previouslyClosed = true;
try {
previouslyClosed = openPage();
PagedMessage msg;
if (pagePosition.getFileOffset() != -1) {
msg = page.readMessage(pagePosition.getFileOffset(), pagePosition.getMessageNr(), pagePosition.getMessageNr());
} else {
if (logger.isTraceEnabled()) {
logger.trace("get message from pos " + pagePosition, new Exception("trace get message without file offset"));
}
msg = page.readMessage(0, 0, pagePosition.getMessageNr());
}
return msg;
} catch (Exception e) {
throw new RuntimeException(e.getMessage(), e);
} finally {
if (!keepOpen && previouslyClosed) {
close();
}
}
}
@Override
public synchronized PagedMessage getMessage(PagePosition pagePosition) {
return getMessage(pagePosition, false, false);
}
/**
* @return true if file was previously closed
* @throws Exception
*/
boolean openPage() throws Exception {
if (!page.getFile().isOpen()) {
page.getFile().open();
return true;
}
return false;
}
@Override
public synchronized void close() {
try {
page.close(false, false);
} catch (Exception e) {
logger.warn("Closing page " + page.getPageId() + " occurs exception:", e);
}
}
@Override
public String toString() {
return "PageReader::page=" + getPageId() + " numberOfMessages = " + numberOfMessages;
}
}

View File

@ -20,7 +20,6 @@ import java.util.ArrayList;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.Map;
import java.util.concurrent.Executor;
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicLongFieldUpdater;
@ -58,8 +57,6 @@ public class PageSubscriptionCounterImpl implements PageSubscriptionCounter {
private final StorageManager storage;
private final Executor executor;
private final AtomicLong value = new AtomicLong(0);
private final AtomicLong persistentSize = new AtomicLong(0);
@ -79,20 +76,11 @@ public class PageSubscriptionCounterImpl implements PageSubscriptionCounter {
private LinkedList<PendingCounter> loadList;
private final Runnable cleanupCheck = new Runnable() {
@Override
public void run() {
cleanup();
}
};
public PageSubscriptionCounterImpl(final StorageManager storage,
final PageSubscription subscription,
final Executor executor,
final boolean persistent,
final long subscriptionID) {
this.subscriptionID = subscriptionID;
this.executor = executor;
this.storage = storage;
this.persistent = persistent;
this.subscription = subscription;
@ -131,6 +119,8 @@ public class PageSubscriptionCounterImpl implements PageSubscriptionCounter {
return; // nothing to be done
}
assert page != null;
PendingCounter pendingInfo = pendingCounters.get((long) page.getPageId());
if (pendingInfo == null) {
// We have to make sure this is sync here
@ -238,7 +228,7 @@ public class PageSubscriptionCounterImpl implements PageSubscriptionCounter {
public synchronized void incrementProcessed(long id, int add, long size) {
addInc(id, add, size);
if (incrementRecords.size() > FLUSH_COUNTER) {
executor.execute(cleanupCheck);
this.subscription.getPagingStore().execute(this::cleanup);
}
}
@ -452,4 +442,4 @@ public class PageSubscriptionCounterImpl implements PageSubscriptionCounter {
SIZE_UPDATER.addAndGet(this, persistentSize);
}
}
}
}

View File

@ -17,57 +17,68 @@
package org.apache.activemq.artemis.core.paging.impl;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Consumer;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.core.buffers.impl.ChannelBufferWrapper;
import org.apache.activemq.artemis.core.io.IOCallback;
import org.apache.activemq.artemis.core.io.SequentialFile;
import org.apache.activemq.artemis.core.io.SequentialFileFactory;
import org.apache.activemq.artemis.core.paging.PagedMessage;
import org.apache.activemq.artemis.core.paging.cursor.LivePageCache;
import org.apache.activemq.artemis.core.paging.cursor.PageSubscriptionCounter;
import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.core.server.ActiveMQMessageBundle;
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
import org.apache.activemq.artemis.core.server.LargeServerMessage;
import org.apache.activemq.artemis.utils.DataConstants;
import org.apache.activemq.artemis.utils.Env;
import org.apache.activemq.artemis.utils.ReferenceCounterUtil;
import org.apache.activemq.artemis.utils.collections.ConcurrentHashSet;
import org.apache.activemq.artemis.utils.collections.EmptyList;
import org.apache.activemq.artemis.utils.collections.LinkedList;
import org.apache.activemq.artemis.utils.collections.LinkedListImpl;
import org.apache.activemq.artemis.utils.collections.LinkedListIterator;
import org.jboss.logging.Logger;
public final class Page implements Comparable<Page> {
public final class Page {
private static final Logger logger = Logger.getLogger(Page.class);
public static final int SIZE_RECORD = DataConstants.SIZE_BYTE + DataConstants.SIZE_INT + DataConstants.SIZE_BYTE;
private static final AtomicInteger factory = new AtomicInteger(0);
private static final byte START_BYTE = (byte) '{';
private final int seqInt = factory.incrementAndGet();
private static final byte END_BYTE = (byte) '}';
private final ReferenceCounterUtil referenceCounter = new ReferenceCounterUtil();
public void usageExhaust() {
referenceCounter.exhaust();
}
private final int pageId;
public int usageUp() {
return referenceCounter.increment();
}
public int usageDown() {
return referenceCounter.decrement();
}
/** to be called when the page is supposed to be released */
public void releaseTask(Consumer<Page> releaseTask) {
referenceCounter.setTask(() -> releaseTask.accept(this));
}
private final long pageId;
private boolean suspiciousRecords = false;
private final AtomicInteger numberOfMessages = new AtomicInteger(0);
private volatile int numberOfMessages;
private final SequentialFile file;
private final SequentialFileFactory fileFactory;
/**
* The page cache that will be filled with data as we write more data
*/
private volatile LivePageCache pageCache;
private volatile LinkedList<PagedMessage> messages;
private final AtomicInteger size = new AtomicInteger(0);
private volatile long size;
private final StorageManager storageManager;
@ -78,181 +89,65 @@ public final class Page implements Comparable<Page> {
*/
private Set<PageSubscriptionCounter> pendingCounters;
private int lastReadMessageNumber;
private ByteBuffer readFileBuffer;
private final ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
private ChannelBufferWrapper readFileBufferWrapper;
private int readProcessedBytes;
public Page(final SimpleString storeName,
final StorageManager storageManager,
final SequentialFileFactory factory,
final SequentialFile file,
final int pageId) throws Exception {
final long pageId) throws Exception {
this.pageId = pageId;
this.file = file;
fileFactory = factory;
this.storageManager = storageManager;
this.storeName = storeName;
resetReadMessageStatus();
}
public int getPageId() {
public long getPageId() {
return pageId;
}
public void setLiveCache(LivePageCache pageCache) {
this.pageCache = pageCache;
public LinkedListIterator<PagedMessage> iterator() throws Exception {
LinkedList<PagedMessage> messages = getMessages();
return messages.iterator();
}
public LivePageCache getLiveCache() {
return pageCache;
}
private synchronized void resetReadMessageStatus() {
lastReadMessageNumber = -3;
readProcessedBytes = 0;
}
public synchronized PagedMessage readMessage(int startOffset,
int startMessageNumber,
int targetMessageNumber) throws Exception {
assert startMessageNumber <= targetMessageNumber;
if (!file.isOpen()) {
if (!file.exists()) {
return null;
public synchronized LinkedList<PagedMessage> getMessages() throws Exception {
if (messages == null) {
boolean wasOpen = file.isOpen();
if (!wasOpen) {
if (!file.exists()) {
return EmptyList.getEmptyList();
}
file.open();
}
messages = read(storageManager);
if (!wasOpen) {
file.close();
}
throw ActiveMQMessageBundle.BUNDLE.invalidPageIO();
}
final int fileSize = (int) file.size();
try {
if (readFileBuffer == null) {
readProcessedBytes = startOffset;
if (startOffset > fileSize) {
return readMessage(0, 0, targetMessageNumber);
}
file.position(readProcessedBytes);
readFileBuffer = fileFactory.allocateDirectBuffer(Math.min(fileSize - readProcessedBytes, MIN_CHUNK_SIZE));
//the wrapper is reused to avoid unnecessary allocations
readFileBufferWrapper = wrapWhole(readFileBuffer);
readFileBuffer.limit(0);
} else if (lastReadMessageNumber + 1 != targetMessageNumber) {
readProcessedBytes = startOffset;
file.position(readProcessedBytes);
readFileBuffer.limit(0);
} else {
startMessageNumber = targetMessageNumber;
}
int remainingBytes = fileSize - readProcessedBytes;
int currentMessageNumber = startMessageNumber;
// First we search forward for the file position of the target number message
while (remainingBytes >= MINIMUM_MSG_PERSISTENT_SIZE && currentMessageNumber < targetMessageNumber) {
headerBuffer.clear();
file.read(headerBuffer);
headerBuffer.position(0);
if (headerBuffer.remaining() >= HEADER_SIZE && headerBuffer.get() == START_BYTE) {
final int encodedSize = headerBuffer.getInt();
final int nextPosition = readProcessedBytes + HEADER_AND_TRAILER_SIZE + encodedSize;
if (nextPosition <= fileSize) {
final int endPosition = nextPosition - 1;
file.position(endPosition);
headerBuffer.rewind();
headerBuffer.limit(1);
file.read(headerBuffer);
headerBuffer.position(0);
if (headerBuffer.remaining() >= 1 && headerBuffer.get() == END_BYTE) {
readProcessedBytes = nextPosition;
currentMessageNumber++;
} else {
markFileAsSuspect(file.getFileName(), readProcessedBytes, currentMessageNumber);
break;
}
} else {
markFileAsSuspect(file.getFileName(), readProcessedBytes, currentMessageNumber);
break;
}
} else {
markFileAsSuspect(file.getFileName(), readProcessedBytes, currentMessageNumber);
break;
}
remainingBytes = fileSize - readProcessedBytes;
}
// Then we read the target message
if (currentMessageNumber == targetMessageNumber && remainingBytes >= MINIMUM_MSG_PERSISTENT_SIZE) {
final ByteBuffer oldFileBuffer = readFileBuffer;
readFileBuffer = readIntoFileBufferIfNecessary(readFileBuffer, MINIMUM_MSG_PERSISTENT_SIZE, true);
//change wrapper if fileBuffer has changed
if (readFileBuffer != oldFileBuffer) {
readFileBufferWrapper = wrapWhole(readFileBuffer);
}
final byte startByte = readFileBuffer.get();
if (startByte == Page.START_BYTE) {
final int encodedSize = readFileBuffer.getInt();
final int nextPosition = readProcessedBytes + HEADER_AND_TRAILER_SIZE + encodedSize;
if (nextPosition <= fileSize) {
final ByteBuffer currentFileBuffer = readFileBuffer;
readFileBuffer = readIntoFileBufferIfNecessary(readFileBuffer, encodedSize + 1, true);
//change wrapper if fileBuffer has changed
if (readFileBuffer != currentFileBuffer) {
readFileBufferWrapper = wrapWhole(readFileBuffer);
}
final int endPosition = readFileBuffer.position() + encodedSize;
//this check must be performed upfront decoding
if (readFileBuffer.remaining() >= (encodedSize + 1) && readFileBuffer.get(endPosition) == Page.END_BYTE) {
final PagedMessageImpl msg = new PagedMessageImpl(encodedSize, storageManager);
readFileBufferWrapper.setIndex(readFileBuffer.position(), endPosition);
msg.decode(readFileBufferWrapper);
readFileBuffer.position(endPosition + 1);
assert readFileBuffer.get(endPosition) == Page.END_BYTE : "decoding cannot change end byte";
msg.initMessage(storageManager);
assert validateLargeMessageStorageManager(msg);
if (logger.isTraceEnabled()) {
logger.tracef("Reading message %s on pageId=%d for address=%s", msg, pageId, storeName);
}
readProcessedBytes = nextPosition;
lastReadMessageNumber = targetMessageNumber;
return msg;
} else {
markFileAsSuspect(file.getFileName(), readProcessedBytes, currentMessageNumber);
}
} else {
markFileAsSuspect(file.getFileName(), readProcessedBytes, currentMessageNumber);
}
} else {
markFileAsSuspect(file.getFileName(), readProcessedBytes, currentMessageNumber);
}
}
} catch (Exception e) {
resetReadMessageStatus();
throw e;
}
resetReadMessageStatus();
ActiveMQServerLogger.LOGGER.pageLookupError(this.pageId, targetMessageNumber, startOffset, startMessageNumber);
if (startOffset > 0) {
return readMessage(0, 0, targetMessageNumber);
} else {
return null;
}
return messages;
}
public synchronized List<PagedMessage> read() throws Exception {
private void addMessage(PagedMessage message) {
if (messages == null) {
messages = new LinkedListImpl<>();
}
message.setMessageNumber(messages.size());
message.setPageNumber(this.pageId);
messages.addTail(message);
}
public synchronized LinkedList<PagedMessage> read() throws Exception {
return read(storageManager);
}
public synchronized List<PagedMessage> read(StorageManager storage) throws Exception {
public synchronized LinkedList<PagedMessage> read(StorageManager storage) throws Exception {
return read(storage, false);
}
public synchronized List<PagedMessage> read(StorageManager storage, boolean onlyLargeMessages) throws Exception {
public synchronized LinkedList<PagedMessage> read(StorageManager storage, boolean onlyLargeMessages) throws Exception {
if (logger.isDebugEnabled()) {
logger.debugf("reading page %d on address = %s onlyLargeMessages = %b",
new Object[] {pageId, storeName, onlyLargeMessages});
@ -260,196 +155,29 @@ public final class Page implements Comparable<Page> {
if (!file.isOpen()) {
if (!file.exists()) {
return Collections.emptyList();
return EmptyList.getEmptyList();
}
throw ActiveMQMessageBundle.BUNDLE.invalidPageIO();
}
size.lazySet((int) file.size());
size = file.size();
final List<PagedMessage> messages = new ArrayList<>();
final LinkedList<PagedMessage> messages = new LinkedListImpl<>();
final int totalMessageCount = readFromSequentialFile(storage, messages, onlyLargeMessages);
numberOfMessages.lazySet(totalMessageCount);
numberOfMessages = PageReadWriter.readFromSequentialFile(storage, storeName, fileFactory, file, this.pageId, messages::addTail, onlyLargeMessages ? PageReadWriter.ONLY_LARGE : PageReadWriter.NO_SKIP, this::markFileAsSuspect, this::setSize);
return messages;
}
private ByteBuffer allocateAndReadIntoFileBuffer(ByteBuffer fileBuffer, int requiredBytes, boolean direct) throws Exception {
ByteBuffer newFileBuffer;
if (direct) {
newFileBuffer = fileFactory.allocateDirectBuffer(Math.max(requiredBytes, MIN_CHUNK_SIZE));
newFileBuffer.put(fileBuffer);
fileFactory.releaseDirectBuffer(fileBuffer);
} else {
newFileBuffer = fileFactory.newBuffer(Math.max(requiredBytes, MIN_CHUNK_SIZE));
newFileBuffer.put(fileBuffer);
fileFactory.releaseBuffer(fileBuffer);
}
fileBuffer = newFileBuffer;
//move the limit to allow reading as much as possible from the file
fileBuffer.limit(fileBuffer.capacity());
file.read(fileBuffer);
fileBuffer.position(0);
return fileBuffer;
}
/**
* It returns a {@link ByteBuffer} that has {@link ByteBuffer#remaining()} bytes >= {@code requiredBytes}
* of valid data from {@link #file}.
*/
private ByteBuffer readIntoFileBufferIfNecessary(ByteBuffer fileBuffer, int requiredBytes, boolean direct) throws Exception {
final int remaining = fileBuffer.remaining();
//fileBuffer::remaining is the current size of valid data
final int bytesToBeRead = requiredBytes - remaining;
if (bytesToBeRead > 0) {
final int capacity = fileBuffer.capacity();
//fileBuffer has enough overall capacity to hold all the required bytes?
if (capacity >= requiredBytes) {
//we do not care to use the free space between
//fileBuffer::limit and fileBuffer::capacity
//to save compactions, because fileBuffer
//is very unlikely to not be completely full
//after each file::read
if (fileBuffer.limit() > 0) {
//the previous check avoid compact
//to attempt a copy of 0 bytes
fileBuffer.compact();
} else {
//compact already set the limit == capacity
fileBuffer.limit(capacity);
}
file.read(fileBuffer);
fileBuffer.position(0);
} else {
fileBuffer = allocateAndReadIntoFileBuffer(fileBuffer, requiredBytes, direct);
}
}
return fileBuffer;
}
private static boolean validateLargeMessageStorageManager(PagedMessage msg) {
if (!(msg.getMessage() instanceof LargeServerMessage)) {
return true;
}
LargeServerMessage largeServerMessage = ((LargeServerMessage) msg.getMessage());
return largeServerMessage.getStorageManager() != null;
}
private static ChannelBufferWrapper wrapWhole(ByteBuffer fileBuffer) {
final int position = fileBuffer.position();
final int limit = fileBuffer.limit();
final int capacity = fileBuffer.capacity();
try {
fileBuffer.clear();
final ByteBuf wrappedBuffer = Unpooled.wrappedBuffer(fileBuffer);
//this check is important to avoid next ByteBuf::setIndex
//to fail due to ByteBuf::capacity == ByteBuffer::remaining bytes
assert wrappedBuffer.readableBytes() == capacity;
final ChannelBufferWrapper fileBufferWrapper = new ChannelBufferWrapper(wrappedBuffer);
return fileBufferWrapper;
} finally {
fileBuffer.position(position);
fileBuffer.limit(limit);
}
}
//sizeOf(START_BYTE) + sizeOf(MESSAGE LENGTH) + sizeOf(END_BYTE)
private static final int HEADER_AND_TRAILER_SIZE = DataConstants.SIZE_INT + 2;
private static final int MINIMUM_MSG_PERSISTENT_SIZE = HEADER_AND_TRAILER_SIZE;
private static final int HEADER_SIZE = HEADER_AND_TRAILER_SIZE - 1;
private static final int MIN_CHUNK_SIZE = Env.osPageSize();
private int readFromSequentialFile(StorageManager storage,
List<PagedMessage> messages,
boolean onlyLargeMessages) throws Exception {
final int fileSize = (int) file.size();
file.position(0);
int processedBytes = 0;
ByteBuffer fileBuffer = null;
ChannelBufferWrapper fileBufferWrapper;
int totalMessageCount = 0;
try {
int remainingBytes = fileSize - processedBytes;
if (remainingBytes >= MINIMUM_MSG_PERSISTENT_SIZE) {
fileBuffer = fileFactory.newBuffer(Math.min(remainingBytes, MIN_CHUNK_SIZE));
//the wrapper is reused to avoid unnecessary allocations
fileBufferWrapper = wrapWhole(fileBuffer);
//no content is being added yet
fileBuffer.limit(0);
do {
final ByteBuffer oldFileBuffer = fileBuffer;
fileBuffer = readIntoFileBufferIfNecessary(fileBuffer, MINIMUM_MSG_PERSISTENT_SIZE, false);
//change wrapper if fileBuffer has changed
if (fileBuffer != oldFileBuffer) {
fileBufferWrapper = wrapWhole(fileBuffer);
}
final byte startByte = fileBuffer.get();
if (startByte == Page.START_BYTE) {
final int encodedSize = fileBuffer.getInt();
final int nextPosition = processedBytes + HEADER_AND_TRAILER_SIZE + encodedSize;
if (nextPosition <= fileSize) {
final ByteBuffer currentFileBuffer = fileBuffer;
fileBuffer = readIntoFileBufferIfNecessary(fileBuffer, encodedSize + 1, false);
//change wrapper if fileBuffer has changed
if (fileBuffer != currentFileBuffer) {
fileBufferWrapper = wrapWhole(fileBuffer);
}
final int endPosition = fileBuffer.position() + encodedSize;
//this check must be performed upfront decoding
if (fileBuffer.remaining() >= (encodedSize + 1) && fileBuffer.get(endPosition) == Page.END_BYTE) {
fileBufferWrapper.setIndex(fileBuffer.position(), endPosition);
final boolean skipMessage;
if (onlyLargeMessages) {
skipMessage = !PagedMessageImpl.isLargeMessage(fileBufferWrapper);
} else {
skipMessage = false;
}
if (!skipMessage) {
final PagedMessageImpl msg = new PagedMessageImpl(encodedSize, storageManager);
msg.decode(fileBufferWrapper);
assert fileBuffer.get(endPosition) == Page.END_BYTE : "decoding cannot change end byte";
msg.initMessage(storage);
assert validateLargeMessageStorageManager(msg);
if (logger.isTraceEnabled()) {
logger.tracef("Reading message %s on pageId=%d for address=%s", msg, pageId, storeName);
}
messages.add(msg);
}
totalMessageCount++;
fileBuffer.position(endPosition + 1);
processedBytes = nextPosition;
} else {
markFileAsSuspect(file.getFileName(), processedBytes, totalMessageCount + 1);
return totalMessageCount;
}
} else {
markFileAsSuspect(file.getFileName(), processedBytes, totalMessageCount + 1);
return totalMessageCount;
}
} else {
markFileAsSuspect(file.getFileName(), processedBytes, totalMessageCount + 1);
return totalMessageCount;
}
remainingBytes = fileSize - processedBytes;
}
while (remainingBytes >= MINIMUM_MSG_PERSISTENT_SIZE);
}
//ignore incomplete messages at the end of the file
if (logger.isTraceEnabled()) {
logger.tracef("%s has %d bytes of unknown data at position = %d", file.getFileName(), remainingBytes, processedBytes);
}
return totalMessageCount;
} finally {
if (fileBuffer != null) {
fileFactory.releaseBuffer(fileBuffer);
}
size.lazySet(processedBytes);
if (file.position() != processedBytes) {
file.position(processedBytes);
}
public String debugMessages() throws Exception {
StringBuffer buffer = new StringBuffer();
LinkedListIterator<PagedMessage> iter = getMessages().iterator();
while (iter.hasNext()) {
PagedMessage message = iter.next();
buffer.append(message.toString() + "\n");
}
iter.close();
return buffer.toString();
}
public synchronized void write(final PagedMessage message) throws Exception {
@ -459,43 +187,36 @@ public final class Page implements Comparable<Page> {
/** This write will not interact back with the storage manager.
* To avoid ping pongs with Journal retaining events and any other stuff. */
public void writeDirect(PagedMessage message) throws Exception {
public synchronized void writeDirect(PagedMessage message) throws Exception {
if (!file.isOpen()) {
throw ActiveMQMessageBundle.BUNDLE.cannotWriteToClosedFile(file);
}
final int messageEncodedSize = message.getEncodeSize();
final int bufferSize = messageEncodedSize + Page.SIZE_RECORD;
final ByteBuffer buffer = fileFactory.newBuffer(bufferSize);
ChannelBufferWrapper activeMQBuffer = new ChannelBufferWrapper(Unpooled.wrappedBuffer(buffer));
activeMQBuffer.clear();
activeMQBuffer.writeByte(Page.START_BYTE);
activeMQBuffer.writeInt(messageEncodedSize);
message.encode(activeMQBuffer);
activeMQBuffer.writeByte(Page.END_BYTE);
assert (activeMQBuffer.readableBytes() == bufferSize) : "messageEncodedSize is different from expected";
//buffer limit and position are the same
assert (buffer.remaining() == bufferSize) : "buffer position or limit are changed";
file.writeDirect(buffer, false);
if (pageCache != null) {
pageCache.addLiveMessage(message);
}
//lighter than addAndGet when single writer
numberOfMessages.lazySet(numberOfMessages.get() + 1);
size.lazySet(size.get() + bufferSize);
addMessage(message);
this.size += PageReadWriter.writeMessage(message, fileFactory, file);
numberOfMessages++;
}
public void sync() throws Exception {
file.sync();
}
public void open(boolean createFile) throws Exception {
public boolean isOpen() {
return file != null && file.isOpen();
}
public boolean open(boolean createFile) throws Exception {
boolean isOpen = false;
if (!file.isOpen() && (createFile || file.exists())) {
file.open();
isOpen = true;
}
if (file.isOpen()) {
size.set((int) file.size());
isOpen = true;
size = file.size();
file.position(0);
}
return isOpen;
}
public void close(boolean sendEvent) throws Exception {
@ -515,11 +236,6 @@ public final class Page implements Comparable<Page> {
if (sendEvent && storageManager != null) {
storageManager.pageClosed(storeName, pageId);
}
if (pageCache != null) {
pageCache.close();
// leave it to the soft cache to decide when to release it now
pageCache = null;
}
file.close(waitSync, waitSync);
Set<PageSubscriptionCounter> counters = getPendingCounters();
@ -530,11 +246,7 @@ public final class Page implements Comparable<Page> {
}
}
public boolean isLive() {
return pageCache != null;
}
public boolean delete(final PagedMessage[] messages) throws Exception {
public boolean delete(final LinkedList<PagedMessage> messages) throws Exception {
if (storageManager != null) {
storageManager.pageDeleted(storeName, pageId);
}
@ -543,77 +255,105 @@ public final class Page implements Comparable<Page> {
logger.debugf("Deleting pageNr=%d on store %s", pageId, storeName);
}
final List<Long> largeMessageIds;
if (messages != null && messages.length > 0) {
largeMessageIds = new ArrayList<>();
for (PagedMessage msg : messages) {
if ((msg.getMessage()).isLargeMessage()) {
// this will trigger large message delete: no need to do it
// for non-large messages!
msg.getMessage().usageDown();
largeMessageIds.add(msg.getMessage().getMessageID());
if (messages != null) {
try (LinkedListIterator<PagedMessage> iter = messages.iterator()) {
while (iter.hasNext()) {
PagedMessage msg = iter.next();
if ((msg.getMessage()).isLargeMessage()) {
((LargeServerMessage)(msg.getMessage())).deleteFile();
msg.getMessage().usageDown();
}
}
}
} else {
largeMessageIds = Collections.emptyList();
}
storageManager.afterCompleteOperations(new IOCallback() {
@Override
public void done() {
try {
if (suspiciousRecords) {
ActiveMQServerLogger.LOGGER.pageInvalid(file.getFileName(), file.getFileName());
file.renameTo(file.getFileName() + ".invalidPage");
} else {
file.delete();
}
referenceCounter.exhaust();
} catch (Exception e) {
ActiveMQServerLogger.LOGGER.pageDeleteError(e);
}
}
@Override
public void onError(int errorCode, String errorMessage) {
}
});
return true;
}
public int readNumberOfMessages() throws Exception {
boolean wasOpen = isOpen();
if (!wasOpen) {
if (!open(false)) {
return 0;
}
}
try {
if (!storageManager.waitOnOperations(5000)) {
ActiveMQServerLogger.LOGGER.timedOutWaitingForLargeMessagesDeletion(largeMessageIds);
int numberOfMessages = PageReadWriter.readFromSequentialFile(this.storageManager,
this.storeName,
this.fileFactory,
this.file,
this.pageId,
null,
PageReadWriter.SKIP_ALL,
null,
null);
if (logger.isDebugEnabled()) {
logger.debug(">>> Reading numberOfMessages page " + this.pageId + ", returning " + numberOfMessages);
}
if (suspiciousRecords) {
ActiveMQServerLogger.LOGGER.pageInvalid(file.getFileName(), file.getFileName());
file.renameTo(file.getFileName() + ".invalidPage");
} else {
file.delete();
return numberOfMessages;
} finally {
if (!wasOpen) {
close(false);
}
return true;
} catch (Exception e) {
ActiveMQServerLogger.LOGGER.pageDeleteError(e);
return false;
}
}
public int getNumberOfMessages() {
return numberOfMessages.intValue();
return numberOfMessages;
}
public int getSize() {
return size.intValue();
public long getSize() {
return size;
}
private void setSize(long size) {
this.size = size;
}
@Override
public String toString() {
return "Page::pageNr=" + this.pageId + ", file=" + this.file;
return "Page::seqCreation=" + seqInt + ", pageNr=" + this.pageId + ", file=" + this.file;
}
@Override
public int compareTo(Page otherPage) {
return otherPage.getPageId() - this.pageId;
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
Page page = (Page) o;
return pageId == page.pageId;
}
@Override
public int hashCode() {
final int prime = 31;
int result = 1;
result = prime * result + pageId;
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj)
return true;
if (obj == null)
return false;
if (getClass() != obj.getClass())
return false;
Page other = (Page) obj;
if (pageId != other.pageId)
return false;
return true;
return (int) (pageId ^ (pageId >>> 32));
}
/**

View File

@ -0,0 +1,69 @@
/*
* 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.activemq.artemis.core.paging.impl;
import java.util.function.Consumer;
import io.netty.util.collection.LongObjectHashMap;
import org.apache.activemq.artemis.core.paging.PagingStore;
import org.jboss.logging.Logger;
/**
* This is a simple cache where we keep Page objects only while they are being used. */
public class PageCache {
private static final Logger logger = Logger.getLogger(PageCache.class);
private final PagingStore owner;
public PageCache(PagingStore owner) {
this.owner = owner;
}
private final LongObjectHashMap<Page> usedPages = new LongObjectHashMap<>();
public synchronized Page get(long pageID) {
return usedPages.get(pageID);
}
public synchronized void forEachUsedPage(Consumer<Page> consumerPage) {
usedPages.values().forEach(consumerPage);
}
public int size() {
return usedPages.size();
}
public synchronized void injectPage(Page page) {
if (logger.isDebugEnabled()) {
logger.debug("+++ Injecting page " + page.getPageId() + " on UsedPages for destination " + owner.getAddress());
}
page.releaseTask(this::removePage);
usedPages.put(page.getPageId(), page);
}
public synchronized void removePage(Page page) {
if (usedPages.remove(page.getPageId()) != null) {
if (logger.isDebugEnabled()) {
logger.debug("--- Releasing page " + page.getPageId() + " on UsedPages for destination " + owner.getAddress());
}
}
}
}

View File

@ -0,0 +1,325 @@
/*
* 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.activemq.artemis.core.paging.impl;
import java.nio.ByteBuffer;
import java.util.function.Consumer;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.core.buffers.impl.ChannelBufferWrapper;
import org.apache.activemq.artemis.core.io.SequentialFile;
import org.apache.activemq.artemis.core.io.SequentialFileFactory;
import org.apache.activemq.artemis.core.paging.PagedMessage;
import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.core.server.LargeServerMessage;
import org.apache.activemq.artemis.utils.DataConstants;
import org.apache.activemq.artemis.utils.Env;
import org.jboss.logging.Logger;
public class PageReadWriter {
private static Logger logger = Logger.getLogger(PageReadWriter.class);
public static final int SIZE_RECORD = DataConstants.SIZE_BYTE + DataConstants.SIZE_INT + DataConstants.SIZE_BYTE;
private static final byte START_BYTE = (byte) '{';
private static final byte END_BYTE = (byte) '}';
//sizeOf(START_BYTE) + sizeOf(MESSAGE LENGTH) + sizeOf(END_BYTE)
private static final int HEADER_AND_TRAILER_SIZE = DataConstants.SIZE_INT + 2;
private static final int MINIMUM_MSG_PERSISTENT_SIZE = HEADER_AND_TRAILER_SIZE;
private static final int HEADER_SIZE = HEADER_AND_TRAILER_SIZE - 1;
private static final int MIN_CHUNK_SIZE = Env.osPageSize();
public interface SuspectFileCallback {
void onSuspect(String fileName, int position, int msgNumber);
}
public interface PageRecordFilter {
boolean skip(ActiveMQBuffer buffer);
}
public interface ReadCallback {
void readComple(int size);
}
public static final PageRecordFilter ONLY_LARGE = (buffer) -> !PagedMessageImpl.isLargeMessage(buffer);
public static final PageRecordFilter NO_SKIP = (buffer) -> false;
public static final PageRecordFilter SKIP_ALL = (buffer) -> true;
public static int writeMessage(PagedMessage message, SequentialFileFactory fileFactory, SequentialFile file) throws Exception {
final int messageEncodedSize = message.getEncodeSize();
final int bufferSize = messageEncodedSize + SIZE_RECORD;
final ByteBuffer buffer = fileFactory.newBuffer(bufferSize);
ChannelBufferWrapper activeMQBuffer = new ChannelBufferWrapper(Unpooled.wrappedBuffer(buffer));
activeMQBuffer.clear();
activeMQBuffer.writeByte(START_BYTE);
activeMQBuffer.writeInt(messageEncodedSize);
message.encode(activeMQBuffer);
activeMQBuffer.writeByte(END_BYTE);
assert (activeMQBuffer.readableBytes() == bufferSize) : "messageEncodedSize is different from expected";
//buffer limit and position are the same
assert (buffer.remaining() == bufferSize) : "buffer position or limit are changed";
file.writeDirect(buffer, false);
return bufferSize;
}
private static ChannelBufferWrapper wrapWhole(ByteBuffer fileBuffer) {
final int position = fileBuffer.position();
final int limit = fileBuffer.limit();
final int capacity = fileBuffer.capacity();
try {
fileBuffer.clear();
final ByteBuf wrappedBuffer = Unpooled.wrappedBuffer(fileBuffer);
//this check is important to avoid next ByteBuf::setIndex
//to fail due to ByteBuf::capacity == ByteBuffer::remaining bytes
assert wrappedBuffer.readableBytes() == capacity;
final ChannelBufferWrapper fileBufferWrapper = new ChannelBufferWrapper(wrappedBuffer);
return fileBufferWrapper;
} finally {
fileBuffer.position(position);
fileBuffer.limit(limit);
}
}
public static int readFromSequentialFile(StorageManager storage,
SimpleString storeName,
SequentialFileFactory fileFactory,
SequentialFile file,
long pageId,
Consumer<PagedMessage> messages,
PageRecordFilter skipRecord,
SuspectFileCallback suspectFileCallback,
ReadCallback readCallback) throws Exception {
final int fileSize = (int) file.size();
file.position(0);
int processedBytes = 0;
ByteBuffer fileBuffer = null;
ChannelBufferWrapper fileBufferWrapper;
int totalMessageCount = 0;
try {
int remainingBytes = fileSize - processedBytes;
if (remainingBytes >= MINIMUM_MSG_PERSISTENT_SIZE) {
fileBuffer = fileFactory.newBuffer(Math.min(remainingBytes, MIN_CHUNK_SIZE));
//the wrapper is reused to avoid unnecessary allocations
fileBufferWrapper = wrapWhole(fileBuffer);
//no content is being added yet
fileBuffer.limit(0);
do {
final ByteBuffer oldFileBuffer = fileBuffer;
fileBuffer = readIntoFileBufferIfNecessary(fileFactory, file, fileBuffer, MINIMUM_MSG_PERSISTENT_SIZE, false);
//change wrapper if fileBuffer has changed
if (fileBuffer != oldFileBuffer) {
fileBufferWrapper = wrapWhole(fileBuffer);
}
final byte startByte = fileBuffer.get();
if (startByte == START_BYTE) {
final int encodedSize = fileBuffer.getInt();
final int nextPosition = processedBytes + HEADER_AND_TRAILER_SIZE + encodedSize;
if (nextPosition <= fileSize) {
final ByteBuffer currentFileBuffer = fileBuffer;
fileBuffer = readIntoFileBufferIfNecessary(fileFactory, file, fileBuffer, encodedSize + 1, false);
//change wrapper if fileBuffer has changed
if (fileBuffer != currentFileBuffer) {
fileBufferWrapper = wrapWhole(fileBuffer);
}
final int endPosition = fileBuffer.position() + encodedSize;
//this check must be performed upfront decoding
if (fileBuffer.remaining() >= (encodedSize + 1) && fileBuffer.get(endPosition) == END_BYTE) {
fileBufferWrapper.setIndex(fileBuffer.position(), endPosition);
final boolean skipMessage = skipRecord.skip(fileBufferWrapper);
if (!skipMessage) {
final PagedMessageImpl msg = new PagedMessageImpl(encodedSize, storage);
msg.decode(fileBufferWrapper);
assert fileBuffer.get(endPosition) == END_BYTE : "decoding cannot change end byte";
msg.initMessage(storage);
assert validateLargeMessageStorageManager(msg);
if (logger.isTraceEnabled()) {
logger.tracef("Reading message %s on pageId=%d for address=%s", msg, pageId, storeName);
}
if (messages != null) {
messages.accept(msg);
}
msg.setPageNumber(pageId).setMessageNumber(totalMessageCount);
}
totalMessageCount++;
fileBuffer.position(endPosition + 1);
processedBytes = nextPosition;
} else {
if (suspectFileCallback != null) {
suspectFileCallback.onSuspect(file.getFileName(), processedBytes, totalMessageCount + 1);
}
return totalMessageCount;
}
} else {
if (suspectFileCallback != null) {
suspectFileCallback.onSuspect(file.getFileName(), processedBytes, totalMessageCount + 1);
}
return totalMessageCount;
}
} else {
if (suspectFileCallback != null) {
suspectFileCallback.onSuspect(file.getFileName(), processedBytes, totalMessageCount + 1);
}
return totalMessageCount;
}
remainingBytes = fileSize - processedBytes;
}
while (remainingBytes >= MINIMUM_MSG_PERSISTENT_SIZE);
}
//ignore incomplete messages at the end of the file
if (logger.isTraceEnabled()) {
logger.tracef("%s has %d bytes of unknown data at position = %d", file.getFileName(), remainingBytes, processedBytes);
}
return totalMessageCount;
} finally {
if (fileBuffer != null) {
fileFactory.releaseBuffer(fileBuffer);
}
if (readCallback != null) {
readCallback.readComple(processedBytes);
}
if (file.position() != processedBytes) {
file.position(processedBytes);
}
}
}
private static ByteBuffer readIntoFileBufferIfNecessary(SequentialFileFactory fileFactory, SequentialFile file, ByteBuffer fileBuffer, int requiredBytes, boolean direct) throws Exception {
final int remaining = fileBuffer.remaining();
//fileBuffer::remaining is the current size of valid data
final int bytesToBeRead = requiredBytes - remaining;
if (bytesToBeRead > 0) {
final int capacity = fileBuffer.capacity();
//fileBuffer has enough overall capacity to hold all the required bytes?
if (capacity >= requiredBytes) {
//we do not care to use the free space between
//fileBuffer::limit and fileBuffer::capacity
//to save compactions, because fileBuffer
//is very unlikely to not be completely full
//after each file::read
if (fileBuffer.limit() > 0) {
//the previous check avoid compact
//to attempt a copy of 0 bytes
fileBuffer.compact();
} else {
//compact already set the limit == capacity
fileBuffer.limit(capacity);
}
file.read(fileBuffer);
fileBuffer.position(0);
} else {
fileBuffer = allocateAndReadIntoFileBuffer(fileFactory, file, fileBuffer, requiredBytes, direct);
}
}
return fileBuffer;
}
private static ByteBuffer allocateAndReadIntoFileBuffer(SequentialFileFactory fileFactory, SequentialFile file, ByteBuffer fileBuffer, int requiredBytes, boolean direct) throws Exception {
ByteBuffer newFileBuffer;
if (direct) {
newFileBuffer = fileFactory.allocateDirectBuffer(Math.max(requiredBytes, MIN_CHUNK_SIZE));
newFileBuffer.put(fileBuffer);
fileFactory.releaseDirectBuffer(fileBuffer);
} else {
newFileBuffer = fileFactory.newBuffer(Math.max(requiredBytes, MIN_CHUNK_SIZE));
newFileBuffer.put(fileBuffer);
fileFactory.releaseBuffer(fileBuffer);
}
fileBuffer = newFileBuffer;
//move the limit to allow reading as much as possible from the file
fileBuffer.limit(fileBuffer.capacity());
file.read(fileBuffer);
fileBuffer.position(0);
return fileBuffer;
}
private static boolean validateLargeMessageStorageManager(PagedMessage msg) {
if (!(msg.getMessage() instanceof LargeServerMessage)) {
return true;
}
LargeServerMessage largeServerMessage = ((LargeServerMessage) msg.getMessage());
boolean storageManager = largeServerMessage.getStorageManager() != null;
if (!storageManager) {
logger.warn("storage manager is null at " + msg);
}
return storageManager;
}
}

View File

@ -27,8 +27,8 @@ import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.core.paging.PageTransactionInfo;
import org.apache.activemq.artemis.core.paging.PagingManager;
import org.apache.activemq.artemis.core.paging.cursor.PageIterator;
import org.apache.activemq.artemis.core.paging.cursor.PagePosition;
import org.apache.activemq.artemis.core.paging.cursor.PageSubscription;
import org.apache.activemq.artemis.core.paging.cursor.PagedReference;
import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
import org.apache.activemq.artemis.core.transaction.Transaction;
@ -153,7 +153,7 @@ public final class PageTransactionInfoImpl implements PageTransactionInfo {
if (lateDeliveries != null) {
// This is to make sure deliveries that were touched before the commit arrived will be delivered
for (LateDelivery pos : lateDeliveries) {
pos.getSubscription().redeliver(pos.getIterator(), pos.getPagePosition());
pos.getSubscription().redeliver(pos.getIterator(), pos.getPageReference());
}
lateDeliveries.clear();
}
@ -235,7 +235,7 @@ public final class PageTransactionInfoImpl implements PageTransactionInfo {
if (lateDeliveries != null) {
for (LateDelivery pos : lateDeliveries) {
pos.getSubscription().lateDeliveryRollback(pos.getPagePosition());
pos.getSubscription().lateDeliveryRollback(pos.getPageReference().getPagedMessage().newPositionObject());
onUpdate(1, null, pos.getSubscription().getPagingStore().getPagingManager());
}
lateDeliveries = null;
@ -255,41 +255,41 @@ public final class PageTransactionInfoImpl implements PageTransactionInfo {
@Override
public synchronized boolean deliverAfterCommit(PageIterator iterator,
PageSubscription cursor,
PagePosition cursorPos) {
PagedReference pagedReference) {
if (logger.isTraceEnabled()) {
logger.trace("deliver after commit on " + cursor + ", position=" + cursorPos);
logger.trace("deliver after commit on " + cursor + ", pagedReference=" + pagedReference);
}
if (committed && useRedelivery) {
if (logger.isTraceEnabled()) {
logger.trace("commit & useRedelivery on " + cursor + ", position=" + cursorPos);
logger.trace("commit & useRedelivery on " + cursor + ", pagedReference=" + pagedReference);
}
cursor.addPendingDelivery(cursorPos);
cursor.redeliver(iterator, cursorPos);
cursor.addPendingDelivery(pagedReference.getPagedMessage());
cursor.redeliver(iterator, pagedReference);
return true;
} else if (committed) {
if (logger.isTraceEnabled()) {
logger.trace("committed on " + cursor + ", position=" + cursorPos + ", ignoring position");
logger.trace("committed on " + cursor + ", position=" + pagedReference + ", ignoring position");
}
return false;
} else if (rolledback) {
if (logger.isTraceEnabled()) {
logger.trace("rolled back, position ignored on " + cursor + ", position=" + cursorPos);
logger.trace("rolled back, position ignored on " + cursor + ", position=" + pagedReference);
}
cursor.positionIgnored(cursorPos);
cursor.positionIgnored(pagedReference.getPagedMessage().newPositionObject());
onUpdate(1, null, cursor.getPagingStore().getPagingManager());
return true;
} else {
if (logger.isTraceEnabled()) {
logger.trace("deliverAftercommit/else, marking useRedelivery on " + cursor + ", position " + cursorPos);
logger.trace("deliverAftercommit/else, marking useRedelivery on " + cursor + ", position " + pagedReference);
}
useRedelivery = true;
if (lateDeliveries == null) {
lateDeliveries = new LinkedList<>();
}
cursor.addPendingDelivery(cursorPos);
lateDeliveries.add(new LateDelivery(cursor, cursorPos, iterator));
cursor.addPendingDelivery(pagedReference.getPagedMessage());
lateDeliveries.add(new LateDelivery(cursor, pagedReference, iterator));
return true;
}
}
@ -304,12 +304,12 @@ public final class PageTransactionInfoImpl implements PageTransactionInfo {
private static class LateDelivery {
final PageSubscription subscription;
final PagePosition pagePosition;
final PagedReference pagedReference;
final PageIterator iterator;
private LateDelivery(PageSubscription subscription, PagePosition pagePosition, PageIterator iterator) {
private LateDelivery(PageSubscription subscription, PagedReference pagedReference, PageIterator iterator) {
this.subscription = subscription;
this.pagePosition = pagePosition;
this.pagedReference = pagedReference;
this.iterator = iterator;
}
@ -317,8 +317,8 @@ public final class PageTransactionInfoImpl implements PageTransactionInfo {
return subscription;
}
public PagePosition getPagePosition() {
return pagePosition;
public PagedReference getPageReference() {
return pagedReference;
}
public PageIterator getIterator() {

View File

@ -24,6 +24,8 @@ import org.apache.activemq.artemis.api.core.ActiveMQException;
import org.apache.activemq.artemis.api.core.ICoreMessage;
import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.core.paging.PagedMessage;
import org.apache.activemq.artemis.core.paging.cursor.PagePosition;
import org.apache.activemq.artemis.core.paging.cursor.impl.PagePositionImpl;
import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.core.persistence.impl.journal.codec.LargeMessagePersister;
import org.apache.activemq.artemis.core.server.LargeServerMessage;
@ -91,6 +93,10 @@ public class PagedMessageImpl implements PagedMessage {
private final StorageManager storageManager;
private long pageNumber;
private int messageNumber;
public PagedMessageImpl(final Message message, final long[] queueIDs, final long transactionID) {
this(message, queueIDs);
this.transactionID = transactionID;
@ -108,6 +114,28 @@ public class PagedMessageImpl implements PagedMessage {
this.storedSize = storedSize;
}
@Override
public long getPageNumber() {
return pageNumber;
}
@Override
public PagedMessageImpl setPageNumber(long pageNr) {
this.pageNumber = pageNr;
return this;
}
@Override
public int getMessageNumber() {
return this.messageNumber;
}
@Override
public PagedMessageImpl setMessageNumber(int messageNr) {
this.messageNumber = messageNr;
return this;
}
@Override
public int getStoredSize() {
@ -123,6 +151,11 @@ public class PagedMessageImpl implements PagedMessage {
return message;
}
@Override
public PagePosition newPositionObject() {
return new PagePositionImpl(pageNumber, messageNumber);
}
@Override
public void initMessage(StorageManager storage) {
if (largeMessageLazyData != null) {
@ -237,7 +270,7 @@ public class PagedMessageImpl implements PagedMessage {
return "PagedMessageImpl [queueIDs=" + Arrays.toString(queueIDs) +
", transactionID=" +
transactionID +
", message=" +
", page=" + pageNumber + ", message=" +
message +
"]";
}

View File

@ -30,6 +30,7 @@ import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Consumer;
import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.api.core.SimpleString;
@ -40,9 +41,7 @@ import org.apache.activemq.artemis.core.paging.PagedMessage;
import org.apache.activemq.artemis.core.paging.PagingManager;
import org.apache.activemq.artemis.core.paging.PagingStore;
import org.apache.activemq.artemis.core.paging.PagingStoreFactory;
import org.apache.activemq.artemis.core.paging.cursor.LivePageCache;
import org.apache.activemq.artemis.core.paging.cursor.PageCursorProvider;
import org.apache.activemq.artemis.core.paging.cursor.impl.LivePageCacheImpl;
import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.core.replication.ReplicationManager;
import org.apache.activemq.artemis.core.server.ActiveMQMessageBundle;
@ -75,8 +74,10 @@ public class PagingStoreImpl implements PagingStore {
private final DecimalFormat format = new DecimalFormat("000000000");
private final PageCache usedPages = new PageCache(this);
//it's being guarded by lock.writeLock().lock() and never read concurrently
private int currentPageSize = 0;
private long currentPageSize = 0;
private final SimpleString storeName;
@ -90,6 +91,10 @@ public class PagingStoreImpl implements PagingStore {
private long maxSize;
private int maxPageReadBytes = -1;
private int maxPageReadMessages = -1;
private long maxMessages;
private int pageSize;
@ -109,11 +114,11 @@ public class PagingStoreImpl implements PagingStore {
private volatile boolean full;
private int numberOfPages;
private long numberOfPages;
private int firstPageId;
private long firstPageId;
private volatile int currentPageId;
private volatile long currentPageId;
private volatile Page currentPage;
@ -219,6 +224,10 @@ public class PagingStoreImpl implements PagingStore {
public void applySetting(final AddressSettings addressSettings) {
maxSize = addressSettings.getMaxSizeBytes();
maxPageReadMessages = addressSettings.getMaxReadPageMessages();
maxPageReadBytes = addressSettings.getMaxReadPageBytes();
maxMessages = addressSettings.getMaxSizeMessages();
configureSizeMetric();
@ -228,10 +237,6 @@ public class PagingStoreImpl implements PagingStore {
addressFullMessagePolicy = addressSettings.getAddressFullMessagePolicy();
rejectThreshold = addressSettings.getMaxSizeBytesRejectThreshold();
if (cursorProvider != null) {
cursorProvider.setCacheMaxSize(addressSettings.getPageCacheMaxSize());
}
}
@Override
@ -287,6 +292,20 @@ public class PagingStoreImpl implements PagingStore {
}
}
@Override
public int getMaxPageReadBytes() {
if (maxPageReadBytes <= 0) {
return pageSize * 2;
} else {
return maxPageReadBytes;
}
}
@Override
public int getMaxPageReadMessages() {
return maxPageReadMessages;
}
@Override
public AddressFullMessagePolicy getAddressFullMessagePolicy() {
return addressFullMessagePolicy;
@ -323,12 +342,12 @@ public class PagingStoreImpl implements PagingStore {
}
@Override
public int getNumberOfPages() {
public long getNumberOfPages() {
return numberOfPages;
}
@Override
public int getCurrentWritingPage() {
public long getCurrentWritingPage() {
return currentPageId;
}
@ -408,9 +427,17 @@ public class PagingStoreImpl implements PagingStore {
}
@Override
public void flushExecutors() {
cursorProvider.flushExecutors();
public ArtemisExecutor getExecutor() {
return executor;
}
@Override
public void execute(Runnable run) {
executor.execute(run);
}
@Override
public void flushExecutors() {
FutureLatch future = new FutureLatch();
executor.execute(future);
@ -445,7 +472,7 @@ public class PagingStoreImpl implements PagingStore {
return;
} else {
running = true;
firstPageId = Integer.MAX_VALUE;
firstPageId = Long.MAX_VALUE;
// There are no files yet on this Storage. We will just return it empty
final SequentialFileFactory fileFactory = this.fileFactory;
@ -453,10 +480,7 @@ public class PagingStoreImpl implements PagingStore {
int pageId = 0;
currentPageId = pageId;
final Page oldPage = currentPage;
if (oldPage != null) {
oldPage.close(false);
}
assert currentPage == null;
currentPage = null;
List<String> files = fileFactory.listFiles("page");
@ -494,23 +518,15 @@ public class PagingStoreImpl implements PagingStore {
}
}
protected void reloadLivePage(int pageId) throws Exception {
Page page = createPage(pageId);
protected void reloadLivePage(long pageId) throws Exception {
Page page = newPageObject(pageId);
page.open(true);
final List<PagedMessage> messages = page.read(storageManager);
final PagedMessage[] initialMessages = messages.toArray(new PagedMessage[messages.size()]);
final LivePageCache pageCache = new LivePageCacheImpl(pageId, initialMessages);
page.setLiveCache(pageCache);
currentPageSize = page.getSize();
currentPage = page;
page.getMessages();
cursorProvider.addLivePageCache(pageCache);
resetCurrentPage(page);
/**
* The page file might be incomplete in the cases: 1) last message incomplete 2) disk damaged.
@ -522,8 +538,25 @@ public class PagingStoreImpl implements PagingStore {
}
}
private void resetCurrentPage(Page newCurrentPage) {
Page theCurrentPage = this.currentPage;
if (theCurrentPage != null) {
theCurrentPage.usageDown();
}
if (newCurrentPage != null) {
newCurrentPage.usageUp();
injectPage(newCurrentPage);
}
this.currentPage = newCurrentPage;
}
@Override
public void stopPaging() {
logger.debugf("stopPaging being called, while isPaging=%s on %s", this.paging, this.storeName);
lock.writeLock().lock();
try {
final boolean isPaging = this.paging;
@ -597,7 +630,7 @@ public class PagingStoreImpl implements PagingStore {
}
@Override
public boolean checkPageFileExists(final int pageNumber) {
public boolean checkPageFileExists(final long pageNumber) {
String fileName = createFileName(pageNumber);
SequentialFileFactory factory = null;
@ -614,7 +647,7 @@ public class PagingStoreImpl implements PagingStore {
}
@Override
public Page createPage(final int pageNumber) throws Exception {
public Page newPageObject(final long pageNumber) throws Exception {
String fileName = createFileName(pageNumber);
SequentialFileFactory factory = checkFileFactory();
@ -626,6 +659,41 @@ public class PagingStoreImpl implements PagingStore {
return page;
}
@Override
public final Page usePage(final long pageId) {
return usePage(pageId, true);
}
@Override
public Page usePage(final long pageId, final boolean create) {
synchronized (usedPages) {
try {
Page page = usedPages.get(pageId);
if (create && page == null) {
page = newPageObject(pageId);
if (page.getFile().exists()) {
page.getMessages();
injectPage(page);
}
}
if (page != null) {
page.usageUp();
}
return page;
} catch (Exception e) {
logger.warn(e.getMessage(), e);
if (fileFactory != null) {
SequentialFile file = fileFactory.createSequentialFile(createFileName(pageId));
fileFactory.onIOError(e, e.getMessage(), file);
}
// in most cases this exception will not happen since the onIOError should halt the VM
// it could eventually happen in tests though
throw new RuntimeException(e.getMessage(), e);
}
}
}
protected SequentialFileFactory getFileFactory() throws Exception {
checkFileFactory();
return fileFactory;
@ -673,15 +741,24 @@ public class PagingStoreImpl implements PagingStore {
return null;
}
Page page = createPage(pageId);
Page page = usePage(pageId);
if (page.getFile().exists()) {
if (page != null && page.getFile().exists()) {
page.usageDown();
// we only decrement numberOfPages if the file existed
// it could have been removed by a previous delete
// on this case we just need to ignore this and move on
numberOfPages--;
}
logger.tracef("Removing page %s, now containing numberOfPages=%s", pageId, numberOfPages);
if (numberOfPages == 0) {
if (logger.isTraceEnabled()) {
logger.tracef("Page has no pages after removing last page %s", pageId, new Exception("Trace"));
}
}
assert numberOfPages >= 0 : "numberOfPages should never be negative. on removePage(" + pageId + "). numberOfPages=" + numberOfPages;
return page;
@ -739,7 +816,7 @@ public class PagingStoreImpl implements PagingStore {
returnPage = currentPage;
returnPage.close(false);
currentPage = null;
resetCurrentPage(null);
// The current page is empty... which means we reached the end of the pages
if (returnPage.getNumberOfMessages() == 0) {
@ -756,7 +833,7 @@ public class PagingStoreImpl implements PagingStore {
}
} else {
logger.tracef("firstPageId++ = beforeIncrement=%d", firstPageId);
returnPage = createPage(firstPageId++);
returnPage = usePage(firstPageId++);
}
if (!returnPage.getFile().exists()) {
@ -939,6 +1016,12 @@ public class PagingStoreImpl implements PagingStore {
lock.readLock().unlock();
}
return writePage(message, tx, listCtx);
}
private boolean writePage(Message message,
Transaction tx,
RouteContextList listCtx) throws Exception {
lock.writeLock().lock();
try {
@ -953,7 +1036,7 @@ public class PagingStoreImpl implements PagingStore {
((LargeServerMessage) message).setPaged();
}
int bytesToWrite = pagedMessage.getEncodeSize() + Page.SIZE_RECORD;
int bytesToWrite = pagedMessage.getEncodeSize() + PageReadWriter.SIZE_RECORD;
currentPageSize += bytesToWrite;
if (currentPageSize > pageSize && currentPage.getNumberOfMessages() > 0) {
@ -1193,7 +1276,7 @@ public class PagingStoreImpl implements PagingStore {
try {
numberOfPages++;
final int newPageId = currentPageId + 1;
final long newPageId = currentPageId + 1;
if (logger.isTraceEnabled()) {
logger.trace("new pageNr=" + newPageId);
@ -1202,17 +1285,13 @@ public class PagingStoreImpl implements PagingStore {
final Page oldPage = currentPage;
if (oldPage != null) {
oldPage.close(true);
oldPage.usageDown();
currentPage = null;
}
final Page newPage = createPage(newPageId);
final Page newPage = newPageObject(newPageId);
currentPage = newPage;
final LivePageCache pageCache = new LivePageCacheImpl(newPageId);
newPage.setLiveCache(pageCache);
cursorProvider.addLivePageCache(pageCache);
resetCurrentPage(newPage);
currentPageSize = 0;
@ -1229,11 +1308,7 @@ public class PagingStoreImpl implements PagingStore {
}
}
/**
* @param pageID
* @return
*/
public String createFileName(final int pageID) {
public String createFileName(final long pageID) {
/** {@link DecimalFormat} is not thread safe. */
synchronized (format) {
return format.format(pageID) + ".page";
@ -1317,4 +1392,17 @@ public class PagingStoreImpl implements PagingStore {
}
}
private void injectPage(Page page) {
usedPages.injectPage(page);
}
protected int getUsedPagesSize() {
return usedPages.size();
}
protected void forEachUsedPage(Consumer<Page> consumerPage) {
usedPages.forEachUsedPage(consumerPage);
}
}

View File

@ -114,11 +114,11 @@ public interface StorageManager extends IDGenerator, ActiveMQComponent {
// Message related operations
void pageClosed(SimpleString storeName, int pageNumber);
void pageClosed(SimpleString storeName, long pageNumber);
void pageDeleted(SimpleString storeName, int pageNumber);
void pageDeleted(SimpleString storeName, long pageNumber);
void pageWrite(PagedMessage message, int pageNumber);
void pageWrite(PagedMessage message, long pageNumber);
void afterCompleteOperations(IOCallback run);

View File

@ -60,13 +60,13 @@ import org.apache.activemq.artemis.core.journal.Journal;
import org.apache.activemq.artemis.core.journal.JournalLoadInformation;
import org.apache.activemq.artemis.core.journal.PreparedTransactionInfo;
import org.apache.activemq.artemis.core.journal.RecordInfo;
import org.apache.activemq.artemis.core.paging.cursor.QueryPagedReferenceImpl;
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
import org.apache.activemq.artemis.core.paging.PageTransactionInfo;
import org.apache.activemq.artemis.core.paging.PagingManager;
import org.apache.activemq.artemis.core.paging.PagingStore;
import org.apache.activemq.artemis.core.paging.cursor.PagePosition;
import org.apache.activemq.artemis.core.paging.cursor.PageSubscription;
import org.apache.activemq.artemis.core.paging.cursor.PagedReferenceImpl;
import org.apache.activemq.artemis.core.paging.impl.PageTransactionInfoImpl;
import org.apache.activemq.artemis.core.persistence.AddressBindingInfo;
import org.apache.activemq.artemis.core.persistence.GroupingInfo;
@ -1912,7 +1912,7 @@ public abstract class AbstractJournalStorageManager extends CriticalComponentImp
if (sub != null) {
sub.reloadPreparedACK(tx, encoding.position);
referencesToAck.add(new PagedReferenceImpl(encoding.position, null, sub));
referencesToAck.add(new QueryPagedReferenceImpl(encoding.position, null, sub));
} else {
ActiveMQServerLogger.LOGGER.journalCannotFindQueueReloadingACK(encoding.queueID);
}

View File

@ -533,7 +533,7 @@ public final class DescribeJournal {
PageSubscriptionCounterImpl subsCounter;
subsCounter = counters.get(queueIDForCounter);
if (subsCounter == null) {
subsCounter = new PageSubscriptionCounterImpl(null, null, null, false, -1);
subsCounter = new PageSubscriptionCounterImpl(null, null, false, -1);
counters.put(queueIDForCounter, subsCounter);
}
return subsCounter;

View File

@ -388,7 +388,7 @@ public class JournalStorageManager extends AbstractJournalStorageManager {
}
@Override
public void pageClosed(final SimpleString storeName, final int pageNumber) {
public void pageClosed(final SimpleString storeName, final long pageNumber) {
if (isReplicated()) {
try (ArtemisCloseable lock = closeableReadLock()) {
if (isReplicated())
@ -398,7 +398,7 @@ public class JournalStorageManager extends AbstractJournalStorageManager {
}
@Override
public void pageDeleted(final SimpleString storeName, final int pageNumber) {
public void pageDeleted(final SimpleString storeName, final long pageNumber) {
if (isReplicated()) {
try (ArtemisCloseable lock = closeableReadLock()) {
if (isReplicated())
@ -408,7 +408,7 @@ public class JournalStorageManager extends AbstractJournalStorageManager {
}
@Override
public void pageWrite(final PagedMessage message, final int pageNumber) {
public void pageWrite(final PagedMessage message, final long pageNumber) {
if (messageJournal.isHistory()) {
try (ArtemisCloseable lock = closeableReadLock()) {

View File

@ -102,6 +102,13 @@ public class LargeBody {
file = null;
}
public void releaseComplete() {
if (!paged) {
deleteFile();
}
}
public synchronized void deleteFile() {
try {
validateFile();

View File

@ -217,7 +217,7 @@ public final class LargeServerMessageImpl extends CoreMessage implements CoreLar
@Override
protected void releaseComplete() {
largeBody.deleteFile();
largeBody.releaseComplete();
}
// Even though not recommended, in certain instances

View File

@ -30,7 +30,6 @@ public class CursorAckRecordEncoding implements EncodingSupport {
}
public CursorAckRecordEncoding() {
this.position = new PagePositionImpl();
}
@Override

View File

@ -29,6 +29,7 @@ import org.apache.activemq.artemis.core.server.CoreLargeServerMessage;
class NullStorageLargeServerMessage extends CoreMessage implements CoreLargeServerMessage {
StorageManager storageManager;
NullStorageLargeServerMessage() {
super();
}
@ -48,7 +49,7 @@ class NullStorageLargeServerMessage extends CoreMessage implements CoreLargeServ
@Override
public StorageManager getStorageManager() {
return null;
return storageManager;
}
@Override
@ -73,7 +74,7 @@ class NullStorageLargeServerMessage extends CoreMessage implements CoreLargeServ
@Override
public void setStorageManager(StorageManager storageManager) {
this.storageManager = storageManager;
}
@Override

View File

@ -379,15 +379,15 @@ public class NullStorageManager implements StorageManager {
}
@Override
public void pageClosed(final SimpleString storeName, final int pageNumber) {
public void pageClosed(final SimpleString storeName, final long pageNumber) {
}
@Override
public void pageDeleted(final SimpleString storeName, final int pageNumber) {
public void pageDeleted(final SimpleString storeName, final long pageNumber) {
}
@Override
public void pageWrite(final PagedMessage message, final int pageNumber) {
public void pageWrite(final PagedMessage message, final long pageNumber) {
}
@Override

View File

@ -194,11 +194,11 @@ public class ServerPacketDecoder extends ClientPacketDecoder {
break;
}
case REPLICATION_PAGE_WRITE: {
packet = new ReplicationPageWriteMessage();
packet = new ReplicationPageWriteMessage(connection.isVersionUsingLongOnPageReplication());
break;
}
case REPLICATION_PAGE_EVENT: {
packet = new ReplicationPageEventMessage();
packet = new ReplicationPageEventMessage(connection.isVersionUsingLongOnPageReplication());
break;
}
case REPLICATION_LARGE_MESSAGE_BEGIN: {

View File

@ -23,7 +23,7 @@ import org.apache.activemq.artemis.utils.DataConstants;
public class ReplicationPageEventMessage extends PacketImpl {
private int pageNumber;
protected long pageNumber;
private SimpleString storeName;
@ -32,12 +32,15 @@ public class ReplicationPageEventMessage extends PacketImpl {
*/
private boolean isDelete;
public ReplicationPageEventMessage() {
private final boolean useLong;
public ReplicationPageEventMessage(boolean useLong) {
super(PacketImpl.REPLICATION_PAGE_EVENT);
this.useLong = useLong;
}
public ReplicationPageEventMessage(final SimpleString storeName, final int pageNumber, final boolean isDelete) {
this();
public ReplicationPageEventMessage(final SimpleString storeName, final long pageNumber, final boolean isDelete, final boolean useLong) {
this(useLong);
this.pageNumber = pageNumber;
this.isDelete = isDelete;
this.storeName = storeName;
@ -45,30 +48,46 @@ public class ReplicationPageEventMessage extends PacketImpl {
@Override
public int expectedEncodeSize() {
return PACKET_HEADERS_SIZE +
SimpleString.sizeofString(storeName) + // buffer.writeSimpleString(storeName);
DataConstants.SIZE_INT + // buffer.writeInt(pageNumber);
DataConstants.SIZE_BOOLEAN; // buffer.writeBoolean(isDelete);
if (useLong) {
return PACKET_HEADERS_SIZE + SimpleString.sizeofString(storeName) + // buffer.writeSimpleString(storeName);
DataConstants.SIZE_LONG + // buffer.writeLong(pageNumber);
DataConstants.SIZE_BOOLEAN; // buffer.writeBoolean(isDelete);
} else {
return PACKET_HEADERS_SIZE + SimpleString.sizeofString(storeName) + // buffer.writeSimpleString(storeName);
DataConstants.SIZE_INT + // buffer.writeInt(pageNumber);
DataConstants.SIZE_BOOLEAN; // buffer.writeBoolean(isDelete);
}
}
@Override
public void encodeRest(final ActiveMQBuffer buffer) {
buffer.writeSimpleString(storeName);
buffer.writeInt(pageNumber);
if (useLong) {
buffer.writeLong(pageNumber);
} else {
if (pageNumber > Integer.MAX_VALUE) {
throw new IllegalStateException("pageNumber=" + pageNumber + " is too large to be used with older broker version on replication");
}
buffer.writeInt((int) pageNumber);
}
buffer.writeBoolean(isDelete);
}
@Override
public void decodeRest(final ActiveMQBuffer buffer) {
storeName = buffer.readSimpleString();
pageNumber = buffer.readInt();
if (useLong) {
pageNumber = buffer.readLong();
} else {
pageNumber = buffer.readInt();
}
isDelete = buffer.readBoolean();
}
/**
* @return the pageNumber
*/
public int getPageNumber() {
public long getPageNumber() {
return pageNumber;
}
@ -98,7 +117,7 @@ public class ReplicationPageEventMessage extends PacketImpl {
final int prime = 31;
int result = super.hashCode();
result = prime * result + (isDelete ? 1231 : 1237);
result = prime * result + pageNumber;
result = prime * result + (int)pageNumber;
result = prime * result + ((storeName == null) ? 0 : storeName.hashCode());
return result;
}

View File

@ -25,16 +25,19 @@ import org.apache.activemq.artemis.utils.DataConstants;
public class ReplicationPageWriteMessage extends PacketImpl implements MessagePacketI {
private int pageNumber;
protected long pageNumber;
private PagedMessage pagedMessage;
protected PagedMessage pagedMessage;
public ReplicationPageWriteMessage() {
final boolean useLong;
public ReplicationPageWriteMessage(final boolean useLong) {
super(PacketImpl.REPLICATION_PAGE_WRITE);
this.useLong = useLong;
}
public ReplicationPageWriteMessage(final PagedMessage pagedMessage, final int pageNumber) {
this();
public ReplicationPageWriteMessage(final PagedMessage pagedMessage, final long pageNumber, final boolean useLong) {
this(useLong);
this.pageNumber = pageNumber;
this.pagedMessage = pagedMessage;
}
@ -42,20 +45,35 @@ public class ReplicationPageWriteMessage extends PacketImpl implements MessagePa
@Override
public int expectedEncodeSize() {
return PACKET_HEADERS_SIZE +
DataConstants.SIZE_INT + // buffer.writeInt(pageNumber);
pagedMessage.getEncodeSize(); // pagedMessage.encode(buffer);
if (useLong) {
return PACKET_HEADERS_SIZE + DataConstants.SIZE_LONG + // buffer.writeLong(pageNumber);
pagedMessage.getEncodeSize(); // pagedMessage.encode(buffer);
} else {
return PACKET_HEADERS_SIZE + DataConstants.SIZE_INT + // buffer.writeInt(pageNumber);
pagedMessage.getEncodeSize(); // pagedMessage.encode(buffer);
}
}
@Override
public void encodeRest(final ActiveMQBuffer buffer) {
buffer.writeInt(pageNumber);
if (useLong) {
buffer.writeLong(pageNumber);
} else {
if (pageNumber > Integer.MAX_VALUE) {
throw new IllegalStateException("pageNumber=" + pageNumber + " is too large to be used with older broker version on replication");
}
buffer.writeInt((int) pageNumber);
}
pagedMessage.encode(buffer);
}
@Override
public void decodeRest(final ActiveMQBuffer buffer) {
pageNumber = buffer.readInt();
if (useLong) {
pageNumber = buffer.readLong();
} else {
pageNumber = buffer.readInt();
}
pagedMessage = new PagedMessageImpl(0, null);
pagedMessage.decode(buffer);
}
@ -63,7 +81,7 @@ public class ReplicationPageWriteMessage extends PacketImpl implements MessagePa
/**
* @return the pageNumber
*/
public int getPageNumber() {
public long getPageNumber() {
return pageNumber;
}
@ -78,7 +96,7 @@ public class ReplicationPageWriteMessage extends PacketImpl implements MessagePa
public int hashCode() {
final int prime = 31;
int result = super.hashCode();
result = prime * result + pageNumber;
result = prime * result + (int)pageNumber;
result = prime * result + ((pagedMessage == null) ? 0 : pagedMessage.hashCode());
return result;
}

View File

@ -128,7 +128,7 @@ public final class ReplicationEndpoint implements ChannelHandler, ActiveMQCompon
private PagingManager pageManager;
private final ConcurrentMap<SimpleString, ConcurrentMap<Integer, Page>> pageIndex = new ConcurrentHashMap<>();
private final ConcurrentMap<SimpleString, ConcurrentMap<Long, Page>> pageIndex = new ConcurrentHashMap<>();
private final ConcurrentMap<Long, ReplicatedLargeMessage> largeMessages = new ConcurrentHashMap<>();
// Used on tests, to simulate failures on delete pages
@ -362,7 +362,7 @@ public final class ReplicationEndpoint implements ChannelHandler, ActiveMQCompon
}
}
for (ConcurrentMap<Integer, Page> map : pageIndex.values()) {
for (ConcurrentMap<Long, Page> map : pageIndex.values()) {
for (Page page : map.values()) {
try {
page.close(false);
@ -784,7 +784,7 @@ public final class ReplicationEndpoint implements ChannelHandler, ActiveMQCompon
* @param packet
*/
private void handlePageEvent(final ReplicationPageEventMessage packet) throws Exception {
ConcurrentMap<Integer, Page> pages = getPageMap(packet.getStoreName());
ConcurrentMap<Long, Page> pages = getPageMap(packet.getStoreName());
Page page = pages.remove(packet.getPageNumber());
@ -820,12 +820,12 @@ public final class ReplicationEndpoint implements ChannelHandler, ActiveMQCompon
page.writeDirect(pgdMessage);
}
private ConcurrentMap<Integer, Page> getPageMap(final SimpleString storeName) {
ConcurrentMap<Integer, Page> resultIndex = pageIndex.get(storeName);
private ConcurrentMap<Long, Page> getPageMap(final SimpleString storeName) {
ConcurrentMap<Long, Page> resultIndex = pageIndex.get(storeName);
if (resultIndex == null) {
resultIndex = new ConcurrentHashMap<>();
ConcurrentMap<Integer, Page> mapResult = pageIndex.putIfAbsent(storeName, resultIndex);
ConcurrentMap<Long, Page> mapResult = pageIndex.putIfAbsent(storeName, resultIndex);
if (mapResult != null) {
resultIndex = mapResult;
}
@ -834,8 +834,8 @@ public final class ReplicationEndpoint implements ChannelHandler, ActiveMQCompon
return resultIndex;
}
private Page getPage(final SimpleString storeName, final int pageId) throws Exception {
ConcurrentMap<Integer, Page> map = getPageMap(storeName);
private Page getPage(final SimpleString storeName, final long pageId) throws Exception {
ConcurrentMap<Long, Page> map = getPageMap(storeName);
Page page = map.get(pageId);
@ -851,13 +851,13 @@ public final class ReplicationEndpoint implements ChannelHandler, ActiveMQCompon
* @param map
* @return
*/
private synchronized Page newPage(final int pageId,
private synchronized Page newPage(final long pageId,
final SimpleString storeName,
final ConcurrentMap<Integer, Page> map) throws Exception {
final ConcurrentMap<Long, Page> map) throws Exception {
Page page = map.get(pageId);
if (page == null) {
page = pageManager.getPageStore(storeName).createPage(pageId);
page = pageManager.getPageStore(storeName).newPageObject(pageId);
page.open(true);
map.put(pageId, page);
}

View File

@ -288,21 +288,21 @@ public final class ReplicationManager implements ActiveMQComponent {
* @param storeName
* @param pageNumber
*/
public void pageClosed(final SimpleString storeName, final int pageNumber) {
public void pageClosed(final SimpleString storeName, final long pageNumber) {
if (enabled) {
sendReplicatePacket(new ReplicationPageEventMessage(storeName, pageNumber, false));
sendReplicatePacket(new ReplicationPageEventMessage(storeName, pageNumber, false, remotingConnection.isVersionUsingLongOnPageReplication()));
}
}
public void pageDeleted(final SimpleString storeName, final int pageNumber) {
public void pageDeleted(final SimpleString storeName, final long pageNumber) {
if (enabled) {
sendReplicatePacket(new ReplicationPageEventMessage(storeName, pageNumber, true));
sendReplicatePacket(new ReplicationPageEventMessage(storeName, pageNumber, true, remotingConnection.isVersionUsingLongOnPageReplication()));
}
}
public void pageWrite(final PagedMessage message, final int pageNumber) {
public void pageWrite(final PagedMessage message, final long pageNumber) {
if (enabled) {
sendReplicatePacket(new ReplicationPageWriteMessage(message, pageNumber));
sendReplicatePacket(new ReplicationPageWriteMessage(message, pageNumber, remotingConnection.isVersionUsingLongOnPageReplication()));
}
}

View File

@ -614,7 +614,7 @@ public interface ActiveMQServerLogger extends BasicLogger {
@Message(id = 222037, value = "IO Error, impossible to start paging", format = Message.Format.MESSAGE_FORMAT)
void pageStoreStartIOError(@Cause Exception e);
@LogMessage(level = Logger.Level.WARN)
@LogMessage(level = Logger.Level.INFO)
@Message(id = 222038, value = "Starting paging on address ''{0}''; {1}", format = Message.Format.MESSAGE_FORMAT)
void pageStoreStart(SimpleString storeName, String sizeInfo);
@ -1690,7 +1690,7 @@ public interface ActiveMQServerLogger extends BasicLogger {
@LogMessage(level = Logger.Level.WARN)
@Message(id = 222288, value = "Page {0}, message {1} could not be found on offset {2}, with starting message {3}. This represents a logic error or inconsistency on the data, and the system will try once again from the beggining of the page file.",
format = Message.Format.MESSAGE_FORMAT)
void pageLookupError(int pageNr, int messageNr, int offset, int startNr);
void pageLookupError(long pageNr, int messageNr, int offset, int startNr);
@LogMessage(level = Logger.Level.WARN)
@Message(id = 222289, value = "Did not route to any matching bindings on dead-letter-address {0} and auto-create-dead-letter-resources is true; dropping message: {1}",
@ -2184,7 +2184,7 @@ public interface ActiveMQServerLogger extends BasicLogger {
@Message(id = 224107, value = "The Critical Analyzer detected slow paths on the broker. It is recommended that you enable trace logs on org.apache.activemq.artemis.utils.critical while you troubleshoot this issue. You should disable the trace logs when you have finished troubleshooting.", format = Message.Format.MESSAGE_FORMAT)
void enableTraceForCriticalAnalyzer();
@LogMessage(level = Logger.Level.WARN)
@LogMessage(level = Logger.Level.INFO)
@Message(id = 224108, value = "Stopped paging on address ''{0}''; {1}", format = Message.Format.MESSAGE_FORMAT)
void pageStoreStop(SimpleString storeName, String pageInfo);
@ -2220,4 +2220,9 @@ public interface ActiveMQServerLogger extends BasicLogger {
@Message(id = 224116, value = "The component {0} is not responsive during start up. The Server may be taking too long to start", format = Message.Format.MESSAGE_FORMAT)
void tooLongToStart(Object component);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 224117, value = "\"page-max-cache-size\" being used on broker.xml. This configuration attribute is no longer used and it will be ignored.", format = Message.Format.MESSAGE_FORMAT)
void pageMaxSizeUsed();
}

View File

@ -62,6 +62,7 @@ import org.apache.activemq.artemis.core.server.management.ManagementService;
import org.apache.activemq.artemis.core.transaction.ResourceManager;
import org.apache.activemq.artemis.core.transaction.Transaction;
import org.apache.activemq.artemis.core.transaction.impl.TransactionImpl;
import org.apache.activemq.artemis.utils.collections.LinkedListIterator;
import org.jboss.logging.Logger;
public class PostOfficeJournalLoader implements JournalLoader {
@ -380,30 +381,33 @@ public class PostOfficeJournalLoader implements JournalLoader {
if (store != null && store.checkPageFileExists(pageId.intValue())) {
// on this case we need to recalculate the records
Page pg = store.createPage(pageId.intValue());
Page pg = store.newPageObject(pageId.intValue());
pg.open(true);
List<PagedMessage> pgMessages = pg.read(storageManager);
org.apache.activemq.artemis.utils.collections.LinkedList<PagedMessage> pgMessages = pg.read(storageManager);
pg.close(false, false);
Map<Long, AtomicInteger> countsPerQueueOnPage = new HashMap<>();
Map<Long, AtomicLong> sizePerQueueOnPage = new HashMap<>();
for (PagedMessage pgd : pgMessages) {
if (pgd.getTransactionID() <= 0) {
for (long q : pgd.getQueueIDs()) {
AtomicInteger countQ = countsPerQueueOnPage.get(q);
AtomicLong sizeQ = sizePerQueueOnPage.get(q);
if (countQ == null) {
countQ = new AtomicInteger(0);
countsPerQueueOnPage.put(q, countQ);
}
if (sizeQ == null) {
sizeQ = new AtomicLong(0);
sizePerQueueOnPage.put(q, sizeQ);
}
countQ.incrementAndGet();
if (pgd.getPersistentSize() > 0) {
sizeQ.addAndGet(pgd.getPersistentSize());
try (LinkedListIterator<PagedMessage> iter = pgMessages.iterator()) {
while (iter.hasNext()) {
PagedMessage pgd = iter.next();
if (pgd.getTransactionID() <= 0) {
for (long q : pgd.getQueueIDs()) {
AtomicInteger countQ = countsPerQueueOnPage.get(q);
AtomicLong sizeQ = sizePerQueueOnPage.get(q);
if (countQ == null) {
countQ = new AtomicInteger(0);
countsPerQueueOnPage.put(q, countQ);
}
if (sizeQ == null) {
sizeQ = new AtomicLong(0);
sizePerQueueOnPage.put(q, sizeQ);
}
countQ.incrementAndGet();
if (pgd.getPersistentSize() > 0) {
sizeQ.addAndGet(pgd.getPersistentSize());
}
}
}
}

View File

@ -107,6 +107,7 @@ import org.apache.activemq.artemis.utils.BooleanUtil;
import org.apache.activemq.artemis.utils.Env;
import org.apache.activemq.artemis.utils.ReferenceCounter;
import org.apache.activemq.artemis.utils.ReusableLatch;
import org.apache.activemq.artemis.utils.SizeAwareMetric;
import org.apache.activemq.artemis.utils.actors.ArtemisExecutor;
import org.apache.activemq.artemis.utils.collections.ConcurrentHashSet;
import org.apache.activemq.artemis.utils.collections.NodeStore;
@ -207,8 +208,8 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
// The quantity of pagedReferences on messageReferences priority list
private final AtomicInteger pagedReferences = new AtomicInteger(0);
// The estimate of memory being consumed by this queue. Used to calculate instances of messages to depage
final AtomicInteger queueMemorySize = new AtomicInteger(0);
final SizeAwareMetric queueMemorySize = new SizeAwareMetric();
protected final QueueMessageMetrics pendingMetrics = new QueueMessageMetrics(this, "pending");
@ -1102,6 +1103,9 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
/* Called when a message is cancelled back into the queue */
@Override
public void addHead(final MessageReference ref, boolean scheduling) {
if (logger.isDebugEnabled()) {
logger.debug("AddHead, size = " + queueMemorySize + ", intermediate size = " + intermediateMessageReferences.size() + ", references size = " + messageReferences.size() + "\nreference=" + ref);
}
try (ArtemisCloseable metric = measureCritical(CRITICAL_PATH_ADD_HEAD)) {
synchronized (this) {
if (ringSize != -1) {
@ -1124,6 +1128,9 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
/* Called when a message is cancelled back into the queue */
@Override
public void addSorted(final MessageReference ref, boolean scheduling) {
if (logger.isDebugEnabled()) {
logger.debug("addSorted, size = " + queueMemorySize + ", intermediate size = " + intermediateMessageReferences.size() + ", references size = " + messageReferences.size() + "\nreference=" + ref);
}
try (ArtemisCloseable metric = measureCritical(CRITICAL_PATH_ADD_HEAD)) {
synchronized (this) {
if (ringSize != -1) {
@ -1176,7 +1183,7 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
@Override
public synchronized void reload(final MessageReference ref) {
queueMemorySize.addAndGet(ref.getMessageMemoryEstimate());
queueMemorySize.addSize(ref.getMessageMemoryEstimate());
if (!scheduledDeliveryHandler.checkAndSchedule(ref, true)) {
internalAddTail(ref);
}
@ -1246,7 +1253,7 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
}
// We only add queueMemorySize if not being delivered directly
queueMemorySize.addAndGet(ref.getMessageMemoryEstimate());
queueMemorySize.addSize(ref.getMessageMemoryEstimate());
intermediateMessageReferences.add(ref);
@ -1311,7 +1318,7 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
private void deliverAsync(boolean noWait) {
if (scheduledRunners.get() < MAX_SCHEDULED_RUNNERS) {
scheduledRunners.incrementAndGet();
checkDepage(noWait);
checkDepage();
try {
getExecutor().execute(deliverRunner);
} catch (RejectedExecutionException ignored) {
@ -1337,7 +1344,7 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
public ArtemisExecutor getExecutor() {
if (pageSubscription != null && pageSubscription.isPaging()) {
// When in page mode, we don't want to have concurrent IO on the same PageStore
return pageSubscription.getExecutor();
return pageSubscription.getPagingStore().getExecutor();
} else {
return executor;
}
@ -2476,7 +2483,7 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
}
// If empty we need to schedule depaging to make sure we would depage expired messages as well
if ((!hasElements || expired) && pageIterator != null && pageIterator.tryNext() > 0) {
if ((!hasElements || expired) && pageIterator != null && pageIterator.tryNext() != PageIterator.NextResult.noElements) {
scheduleDepage(true);
}
}
@ -2841,7 +2848,7 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
* @param ref
*/
private void internalAddHead(final MessageReference ref) {
queueMemorySize.addAndGet(ref.getMessageMemoryEstimate());
queueMemorySize.addSize(ref.getMessageMemoryEstimate());
pendingMetrics.incrementMetrics(ref);
refAdded(ref);
@ -2860,7 +2867,7 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
* @param ref
*/
private void internalAddSorted(final MessageReference ref) {
queueMemorySize.addAndGet(ref.getMessageMemoryEstimate());
queueMemorySize.addSize(ref.getMessageMemoryEstimate());
pendingMetrics.incrementMetrics(ref);
refAdded(ref);
@ -3101,29 +3108,21 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
refRemoved(ref);
}
private void checkDepage(boolean noWait) {
if (pageIterator != null && pageSubscription.isPaging() && !depagePending && needsDepage() && (noWait ? pageIterator.tryNext() > 0 : pageIterator.hasNext())) {
private void checkDepage() {
if (pageIterator != null && pageSubscription.isPaging() && !depagePending && needsDepage() && pageIterator.tryNext() != PageIterator.NextResult.noElements) {
scheduleDepage(false);
}
}
/**
* This is a common check we do before scheduling depaging.. or while depaging.
* Before scheduling a depage runnable we verify if it fits / needs depaging.
* We also check for while needsDepage While depaging.
* This is just to avoid a copy & paste dependency
*
* This is a check on page sizing.
*
* @return
*/
private boolean needsDepage() {
return queueMemorySize.get() < pageSubscription.getPagingStore().getMaxSize() &&
/**
* In most cases, one depage round following by at most MAX_SCHEDULED_RUNNERS deliver round,
* thus we just need to read MAX_DELIVERIES_IN_LOOP * MAX_SCHEDULED_RUNNERS messages. If we read too much, the message reference
* maybe discarded by gc collector in response to memory demand and we need to read it again at
* a great cost when delivering.
*/
intermediateMessageReferences.size() + messageReferences.size() < MAX_DEPAGE_NUM;
return queueMemorySize.getSize() < pageSubscription.getPagingStore().getMaxPageReadBytes() &&
queueMemorySize.getElements() < pageSubscription.getPagingStore().getMaxPageReadMessages();
}
private SimpleString extractGroupID(MessageReference ref) {
@ -3154,7 +3153,7 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
}
protected void refRemoved(MessageReference ref) {
queueMemorySize.addAndGet(-ref.getMessageMemoryEstimate());
queueMemorySize.addSize(-ref.getMessageMemoryEstimate());
pendingMetrics.decrementMetrics(ref);
if (ref.isPaged()) {
pagedReferences.decrementAndGet();
@ -3162,7 +3161,7 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
}
protected void addRefSize(MessageReference ref) {
queueMemorySize.addAndGet(ref.getMessageMemoryEstimate());
queueMemorySize.addSize(ref.getMessageMemoryEstimate());
pendingMetrics.incrementMetrics(ref);
}
@ -3178,7 +3177,7 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
logger.trace("Scheduling depage for queue " + this.getName());
}
depagePending = true;
pageSubscription.getExecutor().execute(new DepageRunner(scheduleExpiry));
pageSubscription.getPagingStore().execute(() -> depage(scheduleExpiry));
}
}
@ -3199,24 +3198,24 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
long timeout = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(DELIVERY_TIMEOUT);
if (logger.isTraceEnabled()) {
logger.trace("QueueMemorySize before depage on queue=" + this.getName() + " is " + queueMemorySize.get());
logger.trace("QueueMemorySize before depage on queue=" + this.getName() + " is " + queueMemorySize.getSize());
}
this.directDeliver = false;
int depaged = 0;
while (timeout - System.nanoTime() > 0 && needsDepage()) {
int status = pageIterator.tryNext();
if (status == 2) {
PageIterator.NextResult status = pageIterator.tryNext();
if (status == PageIterator.NextResult.retry) {
continue;
} else if (status == 0) {
} else if (status == PageIterator.NextResult.noElements) {
break;
}
depaged++;
PagedReference reference = pageIterator.next();
if (logger.isTraceEnabled()) {
logger.trace("Depaging reference " + reference + " on queue " + this.getName());
if (logger.isDebugEnabled()) {
logger.debug("Depaging reference " + reference + " on queue " + this.getName() + " depaged::" + depaged);
}
addTail(reference, false);
pageIterator.remove();
@ -3228,12 +3227,12 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
if (logger.isDebugEnabled()) {
int maxSize = pageSubscription.getPagingStore().getPageSizeBytes();
if (depaged == 0 && queueMemorySize.get() >= maxSize) {
if (depaged == 0 && queueMemorySize.getSize() >= maxSize) {
logger.debug("Couldn't depage any message as the maxSize on the queue was achieved. " + "There are too many pending messages to be acked in reference to the page configuration");
}
if (logger.isDebugEnabled()) {
logger.debug("Queue Memory Size after depage on queue=" + this.getName() + " is " + queueMemorySize.get() + " with maxSize = " + maxSize + ". Depaged " + depaged + " messages, pendingDelivery=" + messageReferences.size() + ", intermediateMessageReferences= " + intermediateMessageReferences.size() + ", queueDelivering=" + deliveringMetrics.getMessageCount());
logger.debug("Queue Memory Size after depage on queue=" + this.getName() + " is " + queueMemorySize.getSize() + " with maxSize = " + maxSize + ". Depaged " + depaged + " messages, pendingDelivery=" + messageReferences.size() + ", intermediateMessageReferences= " + intermediateMessageReferences.size() + ", queueDelivering=" + deliveringMetrics.getMessageCount());
}
}
@ -4162,28 +4161,9 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
if (needCheckDepage) {
try (ArtemisCloseable metric = measureCritical(CRITICAL_CHECK_DEPAGE)) {
checkDepage(true);
checkDepage();
}
}
} catch (Exception e) {
ActiveMQServerLogger.LOGGER.errorDelivering(e);
}
}
}
private final class DepageRunner implements Runnable {
final boolean scheduleExpiry;
private DepageRunner(boolean scheduleExpiry) {
this.scheduleExpiry = scheduleExpiry;
}
@Override
public void run() {
try {
depage(scheduleExpiry);
} catch (Exception e) {
ActiveMQServerLogger.LOGGER.errorDelivering(e);
}

View File

@ -1345,7 +1345,8 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
final int readBytes = context.readInto(bodyBuffer);
assert readBytes == localChunkLen;
assert readBytes == localChunkLen : "readBytes = " + readBytes + ", localChunkLen=" + localChunkLen + " on large message " + largeMessage.getMessageID() + ", hash = " + System.identityHashCode(largeMessage);
final byte[] body = bodyBuffer.array();

View File

@ -62,7 +62,7 @@ public final class SharedNothingBackupActivation extends Activation implements R
//this is how we act when we start as a backup
private ReplicaPolicy replicaPolicy;
//this is the endpoint where we replicate too
//this is the endpoint where we replicate to
private ReplicationEndpoint replicationEndpoint;
private final ActiveMQServerImpl activeMQServer;

View File

@ -42,6 +42,8 @@ public class AddressSettings implements Mergeable<AddressSettings>, Serializable
public static final long DEFAULT_MAX_SIZE_MESSAGES = -1;
public static final int DEFAULT_MAX_READ_PAGE_MESSAGES = 1000;
public static final AddressFullMessagePolicy DEFAULT_ADDRESS_FULL_MESSAGE_POLICY = AddressFullMessagePolicy.PAGE;
public static final int DEFAULT_PAGE_SIZE = 10 * 1024 * 1024;
@ -141,6 +143,10 @@ public class AddressSettings implements Mergeable<AddressSettings>, Serializable
private Long maxSizeBytes = null;
private Integer maxReadPageBytes = null;
private Integer maxReadPageMessages = null;
private Long maxSizeMessages = null;
private Integer pageSizeBytes = null;
@ -281,6 +287,8 @@ public class AddressSettings implements Mergeable<AddressSettings>, Serializable
this.addressFullMessagePolicy = other.addressFullMessagePolicy;
this.maxSizeBytes = other.maxSizeBytes;
this.maxSizeMessages = other.maxSizeMessages;
this.maxReadPageMessages = other.maxReadPageMessages;
this.maxReadPageBytes = other.maxReadPageBytes;
this.pageSizeBytes = other.pageSizeBytes;
this.pageMaxCache = other.pageMaxCache;
this.dropMessagesWhenFull = other.dropMessagesWhenFull;
@ -628,6 +636,24 @@ public class AddressSettings implements Mergeable<AddressSettings>, Serializable
return this;
}
public int getMaxReadPageMessages() {
return maxReadPageMessages != null ? maxReadPageMessages : AddressSettings.DEFAULT_MAX_READ_PAGE_MESSAGES;
}
public AddressSettings setMaxReadPageMessages(final int maxReadPageMessages) {
this.maxReadPageMessages = maxReadPageMessages;
return this;
}
public int getMaxReadPageBytes() {
return maxReadPageBytes != null ? maxReadPageBytes : 2 * getPageSizeBytes();
}
public AddressSettings setMaxReadPageBytes(final int maxReadPageBytes) {
this.maxReadPageBytes = maxReadPageBytes;
return this;
}
public int getMaxDeliveryAttempts() {
return maxDeliveryAttempts != null ? maxDeliveryAttempts : AddressSettings.DEFAULT_MAX_DELIVERY_ATTEMPTS;
}
@ -1003,6 +1029,12 @@ public class AddressSettings implements Mergeable<AddressSettings>, Serializable
if (maxSizeMessages == null) {
maxSizeMessages = merged.maxSizeMessages;
}
if (maxReadPageBytes == null) {
maxReadPageBytes = merged.maxReadPageBytes;
}
if (maxReadPageMessages == null) {
maxReadPageMessages = merged.maxReadPageMessages;
}
if (pageMaxCache == null) {
pageMaxCache = merged.pageMaxCache;
}
@ -1433,6 +1465,14 @@ public class AddressSettings implements Mergeable<AddressSettings>, Serializable
if (buffer.readableBytes() > 0) {
maxSizeMessages = BufferHelper.readNullableLong(buffer);
}
if (buffer.readableBytes() > 0) {
maxReadPageBytes = BufferHelper.readNullableInteger(buffer);
}
if (buffer.readableBytes() > 0) {
maxReadPageMessages = BufferHelper.readNullableInteger(buffer);
}
}
@Override
@ -1501,7 +1541,9 @@ public class AddressSettings implements Mergeable<AddressSettings>, Serializable
BufferHelper.sizeOfNullableInteger(managementMessageAttributeSizeLimit) +
BufferHelper.sizeOfNullableInteger(slowConsumerThresholdMeasurementUnit.getValue()) +
BufferHelper.sizeOfNullableBoolean(enableIngressTimestamp) +
BufferHelper.sizeOfNullableLong(maxSizeMessages);
BufferHelper.sizeOfNullableLong(maxSizeMessages) +
BufferHelper.sizeOfNullableInteger(maxReadPageMessages) +
BufferHelper.sizeOfNullableInteger(maxReadPageBytes);
}
@Override
@ -1637,6 +1679,10 @@ public class AddressSettings implements Mergeable<AddressSettings>, Serializable
buffer.writeNullableSimpleString(configDeleteDiverts != null ? new SimpleString(configDeleteDiverts.toString()) : null);
BufferHelper.writeNullableLong(buffer, maxSizeMessages);
BufferHelper.writeNullableInteger(buffer, maxReadPageBytes);
BufferHelper.writeNullableInteger(buffer, maxReadPageMessages);
}
/* (non-Javadoc)

View File

@ -3987,6 +3987,25 @@
</xsd:annotation>
</xsd:element>
<xsd:element name="max-read-page-messages" type="xsd:int" default="10" maxOccurs="1" minOccurs="0">
<xsd:annotation>
<xsd:documentation>
How many messages are we allowed to read from page into memory each time. We should read more data from pages as messages are acknowledged until it fills up the size.
Between this and max-read-page-bytes the system will stop reading based on whatever hits the mark first
</xsd:documentation>
</xsd:annotation>
</xsd:element>
<xsd:element name="max-read-page-bytes" type="xsd:string" default="20M" maxOccurs="1" minOccurs="0">
<xsd:annotation>
<xsd:documentation>
How many bytes are we allowed to read from page into memory each time. We should read more data from pages as messages are acknowledged until it fills up the size.
Between this and max-read-page-messages the system will stop reading based on whatever hits the mark first
</xsd:documentation>
</xsd:annotation>
</xsd:element>
<xsd:element name="address-full-policy" maxOccurs="1" minOccurs="0">
<xsd:annotation>
<xsd:documentation>

View File

@ -369,6 +369,20 @@ public class FileConfigurationParserTest extends ActiveMQTestBase {
Assert.assertEquals(123, settings.getMaxSizeMessages());
}
@Test
public void testParseMaxReadAddressSettings() throws Exception {
String configStr = "<configuration><address-settings>" + "\n" + "<address-setting match=\"foo\">" + "\n" + "<max-read-page-bytes>1k</max-read-page-bytes><max-read-page-messages>33</max-read-page-messages>.\n" + "</address-setting>" + "\n" + "</address-settings></configuration>" + "\n";
FileConfigurationParser parser = new FileConfigurationParser();
ByteArrayInputStream input = new ByteArrayInputStream(configStr.getBytes(StandardCharsets.UTF_8));
Configuration configuration = parser.parseMainConfig(input);
AddressSettings settings = configuration.getAddressSettings().get("foo");
Assert.assertEquals(1024, settings.getMaxReadPageBytes());
Assert.assertEquals(33, settings.getMaxReadPageMessages());
}
// you should not use K, M notations on address settings max-size-messages
@Test
public void testExpectedErrorOverMaxMessageNotation() throws Exception {

View File

@ -70,6 +70,7 @@ import org.apache.activemq.artemis.core.server.plugin.ActiveMQServerPlugin;
import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
import org.apache.activemq.artemis.core.settings.impl.SlowConsumerPolicy;
import org.apache.activemq.artemis.core.settings.impl.SlowConsumerThresholdMeasurementUnit;
import org.apache.activemq.artemis.logs.AssertionLoggerHandler;
import org.apache.activemq.artemis.utils.RandomUtil;
import org.apache.activemq.artemis.utils.critical.CriticalAnalyzerPolicy;
import org.junit.AfterClass;
@ -933,4 +934,23 @@ public class FileConfigurationTest extends ConfigurationImplTest {
return null;
}
}
@Test
public void testValidateCache() throws Exception {
AssertionLoggerHandler.startCapture(true);
try {
FileConfiguration fc = new FileConfiguration();
FileDeploymentManager deploymentManager = new FileDeploymentManager(getConfigurationName());
deploymentManager.addDeployable(fc);
deploymentManager.readConfiguration();
Assert.assertTrue(AssertionLoggerHandler.findText("AMQ224117"));
Assert.assertEquals(1, AssertionLoggerHandler.countText("AMQ224117"));
} finally {
AssertionLoggerHandler.stopCapture();
}
}
}

View File

@ -0,0 +1,27 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.activemq.artemis.core.paging.cursor.impl;
import org.apache.activemq.artemis.core.paging.cursor.PageCursorProvider;
public class PageCursorProviderAccessor {
public static void cleanup(PageCursorProvider provider) {
((PageCursorProviderImpl)provider).cleanup();
}
}

View File

@ -1,129 +0,0 @@
/*
* Copyright The Apache Software Foundation.
*
* Licensed 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.activemq.artemis.core.paging.cursor.impl;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.core.io.SequentialFile;
import org.apache.activemq.artemis.core.io.SequentialFileFactory;
import org.apache.activemq.artemis.core.paging.PagingStore;
import org.apache.activemq.artemis.core.paging.cursor.PageCache;
import org.apache.activemq.artemis.core.paging.impl.Page;
import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.utils.actors.ArtemisExecutor;
import org.junit.Assert;
import org.junit.Test;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class PageCursorProviderImplTest {
@Test(timeout = 30_000)
public void shouldAllowConcurrentPageReads() throws Exception {
final PagingStore pagingStore = mock(PagingStore.class);
final StorageManager storageManager = mock(StorageManager.class);
when(storageManager.beforePageRead(anyLong(), any(TimeUnit.class))).thenReturn(true);
final int pages = 2;
final ArtemisExecutor artemisExecutor = mock(ArtemisExecutor.class);
final PageCursorProviderImpl pageCursorProvider = new PageCursorProviderImpl(pagingStore, storageManager, artemisExecutor, 2);
when(pagingStore.getCurrentWritingPage()).thenReturn(pages);
when(pagingStore.checkPageFileExists(anyInt())).thenReturn(true);
SequentialFileFactory sequentialFileFactory = mock(SequentialFileFactory.class);
SequentialFile firstPageFile = mock(SequentialFile.class);
when(firstPageFile.size()).thenReturn(0L);
Page firstPage = new Page(new SimpleString("StorageManager"), storageManager, sequentialFileFactory, firstPageFile, 1);
when(pagingStore.createPage(1)).thenReturn(firstPage);
SequentialFile secondPageFile = mock(SequentialFile.class);
when(secondPageFile.size()).thenReturn(0L);
when(secondPageFile.isOpen()).thenReturn(true);
Page secondPage = new Page(new SimpleString("StorageManager"), storageManager, sequentialFileFactory, secondPageFile, 2);
when(pagingStore.createPage(2)).thenReturn(secondPage);
final CountDownLatch finishFirstPageRead = new CountDownLatch(1);
final Thread concurrentRead = new Thread(() -> {
try {
final PageCache cache = pageCursorProvider.getPageCache(2);
Assert.assertNotNull(cache);
} finally {
finishFirstPageRead.countDown();
}
});
try {
when(firstPageFile.isOpen()).then(invocationOnMock -> {
boolean pageReading = false;
for (StackTraceElement element : Thread.currentThread().getStackTrace()) {
if (element.getClassName().compareTo(Page.class.getName()) == 0 && element.getMethodName().compareTo("read") == 0) {
pageReading = true;
break;
}
}
if (pageReading) {
concurrentRead.start();
finishFirstPageRead.await();
}
return true;
});
Assert.assertNotNull(pageCursorProvider.getPageCache(1));
} finally {
pageCursorProvider.stop();
concurrentRead.interrupt();
concurrentRead.join();
}
}
@Test(timeout = 30_000)
public void returnPageCacheImplIfEvicted() throws Exception {
returnCacheIfEvicted(true);
}
@Test(timeout = 30_000)
public void returnPageReaderIfEvicted() throws Exception {
returnCacheIfEvicted(false);
}
private void returnCacheIfEvicted(boolean readWholePage) throws Exception {
final PagingStore pagingStore = mock(PagingStore.class);
final StorageManager storageManager = mock(StorageManager.class);
when(storageManager.beforePageRead(anyLong(), any(TimeUnit.class))).thenReturn(true);
final int pages = 2;
final ArtemisExecutor artemisExecutor = mock(ArtemisExecutor.class);
final PageCursorProviderImpl pageCursorProvider = new PageCursorProviderImpl(pagingStore, storageManager, artemisExecutor, 1, readWholePage);
when(pagingStore.getCurrentWritingPage()).thenReturn(pages);
when(pagingStore.checkPageFileExists(anyInt())).thenReturn(true);
final Page firstPage = mock(Page.class);
when(firstPage.getPageId()).thenReturn(1);
when(pagingStore.createPage(1)).thenReturn(firstPage);
final Page secondPage = mock(Page.class);
when(secondPage.getPageId()).thenReturn(2);
when(pagingStore.createPage(2)).thenReturn(secondPage);
Assert.assertTrue(pageCursorProvider.getPageCache(1) instanceof PageCacheImpl);
Assert.assertTrue(pageCursorProvider.getPageCache(2) instanceof PageCacheImpl);
if (readWholePage) {
Assert.assertTrue(pageCursorProvider.getPageCache(1) instanceof PageCacheImpl);
} else {
Assert.assertTrue(pageCursorProvider.getPageCache(1) instanceof PageReader);
}
Assert.assertEquals(pageCursorProvider.getCacheSize(), 1);
Assert.assertTrue(pageCursorProvider.getPageCache(2) instanceof PageCacheImpl);
pageCursorProvider.stop();
}
}

View File

@ -1,214 +0,0 @@
/**
* 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.activemq.artemis.core.paging.cursor.impl;
import java.util.Arrays;
import org.apache.activemq.artemis.api.core.ICoreMessage;
import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.core.io.SequentialFile;
import org.apache.activemq.artemis.core.io.SequentialFileFactory;
import org.apache.activemq.artemis.core.io.nio.NIOSequentialFileFactory;
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
import org.apache.activemq.artemis.core.paging.PagedMessage;
import org.apache.activemq.artemis.core.paging.cursor.NonExistentPage;
import org.apache.activemq.artemis.core.paging.cursor.PagePosition;
import org.apache.activemq.artemis.core.paging.impl.Page;
import org.apache.activemq.artemis.core.paging.impl.PagedMessageImpl;
import org.apache.activemq.artemis.core.persistence.impl.nullpm.NullStorageManager;
import org.apache.activemq.artemis.logs.AssertionLoggerHandler;
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
import org.junit.Assert;
import org.junit.Test;
import static org.apache.activemq.artemis.core.paging.cursor.impl.PageSubscriptionImpl.PagePositionAndFileOffset;
import static org.apache.activemq.artemis.utils.RandomUtil.randomBoolean;
public class PageReaderTest extends ActiveMQTestBase {
@Test
public void testPageReadMessage() throws Exception {
recreateDirectory(getTestDir());
int num = 50;
int[] offsets = createPage(num);
PageReader pageReader = getPageReader();
PagedMessage[] pagedMessages = pageReader.readMessages();
assertEquals(pagedMessages.length, num);
PagedMessage pagedMessage = null;
for (int i = 0; i < num; i++) {
if (randomBoolean()) {
PagePosition pagePosition = new PagePositionImpl(10, i);
pagedMessage = pageReader.getMessage(pagePosition);
} else {
int nextFileOffset = pagedMessage == null ? -1 : offsets[i - 1] + pagedMessage.getStoredSize() + Page.SIZE_RECORD;
PagePositionAndFileOffset startPosition = new PagePositionAndFileOffset(nextFileOffset, new PagePositionImpl(10, i - 1));
PagePosition pagePosition = startPosition.nextPagePostion();
assertEquals("Message " + i + " has wrong offset", offsets[i], pagePosition.getFileOffset());
pagedMessage = pageReader.getMessage(pagePosition);
}
assertNotNull(pagedMessage);
assertEquals(pagedMessage.getMessage().getMessageID(), i);
assertEquals(pagedMessages[i].getMessage().getMessageID(), i);
}
pageReader.close();
}
@Test
public void testShortPageReadMessage() throws Exception {
recreateDirectory(getTestDir());
int num = 2;
int[] offsets = createPage(num);
PageReader pageReader = getPageReader();
PagedMessage[] pagedMessages = pageReader.readMessages();
assertEquals(pagedMessages.length, num);
PagePosition pagePosition = new PagePositionImpl(10, 0);
PagedMessage firstPagedMessage = pageReader.getMessage(pagePosition);
assertEquals("Message 0 has a wrong encodeSize", pagedMessages[0].getEncodeSize(), firstPagedMessage.getEncodeSize());
int nextFileOffset = offsets[0] + firstPagedMessage.getStoredSize() + Page.SIZE_RECORD;
PagePositionAndFileOffset startPosition = new PagePositionAndFileOffset(nextFileOffset, new PagePositionImpl(10, 0));
PagePosition nextPagePosition = startPosition.nextPagePostion();
assertEquals("Message 1 has a wrong offset", offsets[1], nextPagePosition.getFileOffset());
PagedMessage pagedMessage = pageReader.getMessage(nextPagePosition);
assertNotNull(pagedMessage);
assertEquals(pagedMessage.getMessage().getMessageID(), 1);
assertEquals(pagedMessages[1].getMessage().getMessageID(), 1);
pageReader.close();
}
@Test
public void testForceInvalidPosition() throws Exception {
AssertionLoggerHandler.startCapture();
try {
recreateDirectory(getTestDir());
int num = 2;
int[] offsets = createPage(num);
PageReader pageReader = getPageReader();
PagedMessage[] pagedMessages = pageReader.readMessages();
assertEquals(pagedMessages.length, num);
PagePosition pagePosition = new PagePositionImpl(10, 0, 50);
PagedMessage firstPagedMessage = pageReader.getMessage(pagePosition);
assertEquals("Message 0 has a wrong encodeSize", pagedMessages[0].getEncodeSize(), firstPagedMessage.getEncodeSize());
PagePosition nextPagePosition = new PagePositionImpl(10, 1, 5000);
PagedMessage pagedMessage = pageReader.getMessage(nextPagePosition);
assertNotNull(pagedMessage);
assertEquals(pagedMessage.getMessage().getMessageID(), 1);
assertEquals(pagedMessages[1].getMessage().getMessageID(), 1);
pageReader.close();
Assert.assertTrue("Logging did not throw warn expected", AssertionLoggerHandler.findText("AMQ222288"));
} finally {
AssertionLoggerHandler.stopCapture();
AssertionLoggerHandler.clear();
}
}
@Test
public void testPageReadMessageBeyondPage() throws Exception {
recreateDirectory(getTestDir());
int num = 10;
createPage(num);
PageReader pageReader = getPageReader();
assertNull(pageReader.getMessage(new PagePositionImpl(10, num)));
try {
pageReader.getMessage(new PagePositionImpl(10, num), true, true);
assertFalse("Expect exception since message number is beyond page ", true);
} catch (NonExistentPage e) {
}
pageReader.close();
}
@Test
public void testPageReadMessageKeepOpen() throws Exception {
recreateDirectory(getTestDir());
int num = 10;
createPage(num);
PageReader pageReader = getPageReader();
pageReader.getMessage(new PagePositionImpl(10, 1), true, true);
assertFalse("Page file should keep open", pageReader.openPage());
pageReader.getMessage(new PagePositionImpl(10, 1), true, false);
assertFalse("Page file should preserve previous state", pageReader.openPage());
pageReader.close();
pageReader.getMessage(new PagePositionImpl(10, 1), true, false);
assertTrue("Page file should preserve previous state", pageReader.openPage());
pageReader.close();
}
private int[] createPage(int num) throws Exception {
SequentialFileFactory factory = new NIOSequentialFileFactory(getTestDirfile(), 1);
SequentialFile file = factory.createSequentialFile("00010.page");
Page page = new Page(new SimpleString("something"), new NullStorageManager(), factory, file, 10);
page.open(true);
SimpleString simpleDestination = new SimpleString("Test");
final int msgSize = 100;
final byte[] content = new byte[msgSize];
Arrays.fill(content, (byte) 'b');
int[] offsets = new int[num];
for (int i = 0; i < num; i++) {
Message msg = createMessage(simpleDestination, i, content);
offsets[i] = (int)page.getFile().position();
PagedMessageImpl pgdMessage = new PagedMessageImpl(msg, new long[0]);
long expectedPosition = pgdMessage.getEncodeSize() + Page.SIZE_RECORD + page.getFile().position();
page.write(pgdMessage);
Assert.assertEquals(page.getFile().position(), expectedPosition);
Assert.assertEquals(i + 1, page.getNumberOfMessages());
}
page.close(false, false);
return offsets;
}
protected Message createMessage(SimpleString address, int msgId, byte[] content) {
ICoreMessage msg = new CoreMessage().setMessageID(msgId).initBuffer(1024);
for (byte b : content) {
msg.getBodyBuffer().writeByte(b);
}
msg.setAddress(address);
return msg;
}
private PageReader getPageReader() throws Exception {
SequentialFileFactory factory = new NIOSequentialFileFactory(getTestDirfile(), 1);
SequentialFile file = factory.createSequentialFile("00010.page");
file.open();
Page page = new Page(new SimpleString("something"), new NullStorageManager(), factory, file, 10);
page.open(true);
page.read(new NullStorageManager());
PageReader pageReader = new PageReader(page, page.getNumberOfMessages());
return pageReader;
}
}

View File

@ -30,4 +30,8 @@ public class PagingManagerTestAccessor {
public static SizeAwareMetric globalSizeAwareMetric(PagingManager pagingManager) {
return ((PagingManagerImpl)pagingManager).getSizeAwareMetric();
}
public static String debugMessages(Page page) throws Exception {
return page.debugMessages();
}
}

View File

@ -17,6 +17,8 @@
package org.apache.activemq.artemis.core.paging.impl;
import java.util.function.Consumer;
import org.apache.activemq.artemis.core.io.SequentialFileFactory;
import org.apache.activemq.artemis.core.paging.PagingStore;
@ -24,4 +26,15 @@ public class PagingStoreTestAccessor {
public static SequentialFileFactory getFileFactory(PagingStore store) throws Exception {
return ((PagingStoreImpl) store).getFileFactory();
}
public static int getUsedPagesSize(PagingStore store) {
return ((PagingStoreImpl)store).getUsedPagesSize();
}
public static void forEachUsedPage(PagingStore store, Consumer<Page> consumer) {
PagingStoreImpl impl = (PagingStoreImpl) store;
impl.forEachUsedPage(consumer);
}
}

View File

@ -27,7 +27,6 @@ import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.core.io.IOCallback;
import org.apache.activemq.artemis.core.paging.cursor.PageIterator;
import org.apache.activemq.artemis.core.paging.cursor.PagePosition;
import org.apache.activemq.artemis.core.paging.cursor.PageSubscription;
import org.apache.activemq.artemis.core.paging.cursor.PagedReferenceImpl;
import org.apache.activemq.artemis.core.paging.impl.PagedMessageImpl;
@ -63,7 +62,7 @@ public class QueueImplTest {
when(pageIterator).hasNext();
Mockito.doAnswer(invocationOnMock -> {
pageIteratorIndex.incrementAndGet();
return new PagedReferenceImpl(Mockito.mock(PagePosition.class), new PagedMessageImpl(
return new PagedReferenceImpl(new PagedMessageImpl(
Mockito.mock(Message.class), new long[]{0}), pageSubscription);
}).when(pageIterator).next();
Mockito.doReturn(pageIterator).when(pageSubscription).iterator();

View File

@ -21,8 +21,8 @@ import org.apache.activemq.artemis.core.server.Queue;
public class QueueImplTestAccessor {
public static int getQueueMemorySize(Queue queue) {
return ((QueueImpl)queue).queueMemorySize.get();
public static long getQueueMemorySize(Queue queue) {
return ((QueueImpl)queue).queueMemorySize.getSize();
}
}

View File

@ -269,12 +269,12 @@ public class TransactionImplTest extends ActiveMQTestBase {
}
@Override
public void pageClosed(SimpleString storeName, int pageNumber) {
public void pageClosed(SimpleString storeName, long pageNumber) {
}
@Override
public void pageDeleted(SimpleString storeName, int pageNumber) {
public void pageDeleted(SimpleString storeName, long pageNumber) {
}
@ -289,7 +289,7 @@ public class TransactionImplTest extends ActiveMQTestBase {
}
@Override
public void pageWrite(PagedMessage message, int pageNumber) {
public void pageWrite(PagedMessage message, long pageNumber) {
}

View File

@ -1511,6 +1511,27 @@ public abstract class ActiveMQTestBase extends Assert {
return server;
}
protected ActiveMQServer createServer(final boolean realFiles,
final Configuration configuration,
final int pageSize,
final long maxAddressSize,
final int maxReadPageMessages,
final Map<String, AddressSettings> settings) {
ActiveMQServer server = addServer(ActiveMQServers.newActiveMQServer(configuration, realFiles));
if (settings != null) {
for (Map.Entry<String, AddressSettings> setting : settings.entrySet()) {
server.getAddressSettingsRepository().addMatch(setting.getKey(), setting.getValue());
}
}
AddressSettings defaultSetting = new AddressSettings().setPageSizeBytes(pageSize).setMaxSizeBytes(maxAddressSize).setAddressFullMessagePolicy(AddressFullMessagePolicy.PAGE).setMaxReadPageMessages(1);
server.getAddressSettingsRepository().addMatch("#", defaultSetting);
return server;
}
protected final ActiveMQServer createServer(final boolean realFiles,
final Configuration configuration,
final int pageSize,

View File

@ -25,7 +25,6 @@ that would be found in the `broker.xml` file.
<max-size-messages>1000</max-size-messages>
<max-size-bytes-reject-threshold>-1</max-size-bytes-reject-threshold>
<page-size-bytes>20000</page-size-bytes>
<page-max-cache-size></page-max-cache-size>
<address-full-policy>PAGE</address-full-policy>
<message-counter-history-day-limit></message-counter-history-day-limit>
<last-value-queue>true</last-value-queue> <!-- deprecated! see default-last-value-queue -->
@ -139,7 +138,7 @@ calculate an adjustment to the `redelivery-delay` (up or down). Default is
`0.0`. Valid values are between 0.0 and 1.0. Read more about [undelivered
messages](undelivered-messages.md#configuring-delayed-redelivery).
`max-size-bytes`, `max-size-messages`, `page-size-bytes`, & `page-max-cache-size` are used to
`max-size-bytes`, `max-size-messages`, `page-size-bytes`, `max-read-page-messages` & `max-read-page-bytes` are used to
configure paging on an address. This is explained
[here](paging.md#configuration).

View File

@ -136,7 +136,6 @@ attribute `match` | N/A | X | Changing this value is same as deleting the whole
`<max-delivery-attempts>` | X (no more than one can be present) | The configured max-delivery-attempts will be removed from running broker after reloading. | The max-delivery-attempts will be updated after reloading.
`<max-size-bytes>` | X (no more than one can be present) | The configured max-size-bytes will be removed from running broker after reloading. | The max-size-bytes will be updated after reloading.
`<page-size-bytes>` | X (no more than one can be present) | The configured page-size-bytes will be removed from running broker after reloading. | The page-size-bytes will be updated after reloading.
`<page-max-cache-size>` | X (no more than one can be present) | The configured page-max-cache-size will be removed from running broker after reloading. | The page-max-cache-size will be updated after reloading.
`<address-full-policy>` | X (no more than one can be present) | The configured address-full-policy will be removed from running broker after reloading. | The address-full-policy will be updated after reloading.
`<message-counter-history-day-limit>` | X (no more than one can be present) | The configured message-counter-history-day-limit will be removed from running broker after reloading. | The message-counter-history-day-limit will be updated after reloading.
`<last-value-queue>` | X (no more than one can be present) | The configured last-value-queue will be removed from running broker after reloading (no longer a last value queue). | The last-value-queue will be updated after reloading.

View File

@ -220,7 +220,6 @@ Name | Description | Default
[max-size-bytes](paging.md)| Max size a queue can be before invoking `address-full-policy` | -1
[max-size-bytes-reject-threshold]() | Used with `BLOCK`, the max size an address can reach before messages are rejected; works in combination with `max-size-bytes` **for AMQP clients only**. | -1
[page-size-bytes](paging.md) | Size of each file on page | 10485760
[page-max-cache-size](paging.md) | Maximum number of files cached from paging | 5
[address-full-policy](address-model.md)| What to do when a queue reaches `max-size-bytes` | `PAGE`
[message-counter-history-day-limit](address-model.md) | Days to keep message counter data | 0
[last-value-queue](last-value-queues.md) | **deprecated** Queue is a last value queue; see `default-last-value-queue` instead | `false`

View File

@ -87,7 +87,7 @@ Configuration is done at the address settings in `broker.xml`.
> is not considered while evaluating if [global-max-size](#global-max-size)
> is hit and can't cause other non-management addresses to trigger a
> configured `address-full-policy`.
This is the list of available parameters on the address settings.
Property Name|Description|Default
@ -96,12 +96,17 @@ Property Name|Description|Default
`max-size-messages`|The max number of messages the address could have before entering on page mode.| -1 (disabled)
`page-size-bytes`|The size of each page file used on the paging system|10MB
`address-full-policy`|This must be set to `PAGE` for paging to enable. If the value is `PAGE` then further messages will be paged to disk. If the value is `DROP` then further messages will be silently dropped. If the value is `FAIL` then the messages will be dropped and the client message producers will receive an exception. If the value is `BLOCK` then client message producers will block when they try and send further messages.|`PAGE`
`page-max-cache-size`|The system will keep up to `page-max-cache-size` page files in memory to optimize IO during paging navigation.|5
`max-read-page-messages` | how many message can be read from paging into the Queue whenever more messages are needed. The system wtill stop reading if `max-read-page-bytes hits the limit first. | 1000
`max-read-page-bytes` | how much memory the messages read from paging can take on the Queue whenever more messages are needed. The system will stop reading if `max-read-page-messages` hits the limit first. | 2 * page-size-bytes
### max-size-bytes and max-size-messages simultaneous usage
It is possible to define max-size-messages (as the maximum number of messages) and max-messages-size (as the max number of estimated memory used by the address) concurrently. The configured policy will start based on the first value to reach its mark.
#### Maximum read from page
`max-read-page-messages` and `max-read-page-bytes` are used to control messaging reading from paged file into the Queue. The broker will add messages on the Queue until either `max-read-page-meessages` or `max-read-page-bytes` reaches the limit.
## Global Max Size
Beyond the `max-size-bytes` on the address you can also set the global-max-size
@ -192,6 +197,12 @@ The pages are synced periodically and the sync period is configured through
the same value of `journal-buffer-timeout`. When using ASYNCIO, the default
should be `3333333`.
## Memory usage from Paged Messages.
The system should keep at least one paged file in memory caching ahead reading messages.
Also every active subscription could keep one paged file in memory.
So, if your system has too many queues it is recommended to minimize the page-size.
## Example
See the [Paging Example](examples.md#paging) which shows how to use paging with

View File

@ -186,7 +186,7 @@
<activemq.version.majorVersion>1</activemq.version.majorVersion>
<activemq.version.minorVersion>0</activemq.version.minorVersion>
<activemq.version.microVersion>0</activemq.version.microVersion>
<activemq.version.incrementingVersion>132,131,130,129,128,127,126,125,124,123,122</activemq.version.incrementingVersion>
<activemq.version.incrementingVersion>133,132,131,130,129,128,127,126,125,124,123,122</activemq.version.incrementingVersion>
<activemq.version.versionTag>${project.version}</activemq.version.versionTag>
<ActiveMQ-Version>${project.version}(${activemq.version.incrementingVersion})</ActiveMQ-Version>

View File

@ -452,6 +452,32 @@
<variableName>ARTEMIS-2_10_0</variableName>
</configuration>
</execution>
<execution>
<phase>compile</phase>
<goals>
<goal>dependency-scan</goal>
</goals>
<id>2_22_0-check</id>
<configuration>
<optional>true</optional>
<libListWithDeps>
<arg>org.apache.activemq:artemis-jms-server:2.22.0</arg>
<arg>org.apache.activemq:artemis-jms-client:2.22.0</arg>
<arg>org.apache.activemq:artemis-cli:2.22.0</arg>
<arg>org.apache.activemq:artemis-hornetq-protocol:2.22.0</arg>
<arg>org.apache.activemq:artemis-amqp-protocol:2.22.0</arg>
<arg>org.apache.activemq:artemis-hornetq-protocol:2.22.0</arg>
<arg>org.apache.groovy:groovy-all:pom:${groovy.version}</arg>
<arg>org.jboss.marshalling:jboss-marshalling-river:2.0.9.Final</arg>
</libListWithDeps>
<libList>
<arg>org.apache.activemq.tests:compatibility-tests:${project.version}</arg>
</libList>
<!-- for future maintainers, notice that if you add new variables you also need to add the system property
otherwise this is not captured, search for the word @@@@ on this pom where I left anothr comment -->
<variableName>ARTEMIS-2_22_0</variableName>
</configuration>
</execution>
<execution>
<phase>compile</phase>
<goals>
@ -675,6 +701,10 @@
<name>ARTEMIS-2_10_0</name> <!-- 2.10.0 -->
<value>${ARTEMIS-2_10_0}</value>
</property>
<property>
<name>ARTEMIS-2_22_0</name>
<value>${ARTEMIS-2_22_0}</value>
</property>
<property>
<name>ARTEMIS-240</name>
<value>${ARTEMIS-240}</value>

View File

@ -39,6 +39,7 @@ public class GroovyRun {
public static final String TWO_SIX_THREE = "ARTEMIS-263";
public static final String TWO_SEVEN_ZERO = "ARTEMIS-270";
public static final String TWO_TEN_ZERO = "ARTEMIS-2_10_0";
public static final String TWO_TWENTYTWO_ZERO = "ARTEMIS-2_22_0";
public static final String HORNETQ_235 = "HORNETQ-235";
public static final String HORNETQ_247 = "HORNETQ-247";
public static final String AMQ_5_11 = "AMQ_5_11";

View File

@ -0,0 +1,57 @@
package multiVersionReplica
/*
* 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.
*/
import org.apache.activemq.artemis.api.core.QueueConfiguration
import org.apache.activemq.artemis.api.core.RoutingType
import org.apache.activemq.artemis.core.config.ClusterConnectionConfiguration
import org.apache.activemq.artemis.core.config.CoreAddressConfiguration
import org.apache.activemq.artemis.core.config.ha.ReplicaPolicyConfiguration
import org.apache.activemq.artemis.core.config.impl.ConfigurationImpl
import org.apache.activemq.artemis.core.server.JournalType
import org.apache.activemq.artemis.core.server.embedded.EmbeddedActiveMQ
import org.apache.activemq.artemis.core.settings.impl.AddressFullMessagePolicy
import org.apache.activemq.artemis.core.settings.impl.AddressSettings
String folder = arg[0];
String id = arg[1];
String port = arg[2];
String backupPort = arg[3]
configuration = new ConfigurationImpl();
configuration.setJournalType(JournalType.NIO);
configuration.setBrokerInstance(new File(folder + "/" + id));
configuration.addAcceptorConfiguration("artemis", "tcp://localhost:" + port);
configuration.addConnectorConfiguration("local", "tcp://localhost:" + port);
configuration.setSecurityEnabled(false);
configuration.setPersistenceEnabled(true);
configuration.setGlobalMaxMessages(100);
configuration.setHAPolicyConfiguration(new ReplicaPolicyConfiguration().setClusterName("main"))
configuration.addAddressesSetting("#", new AddressSettings().setAddressFullMessagePolicy(AddressFullMessagePolicy.PAGE));
ClusterConnectionConfiguration backToMain = new ClusterConnectionConfiguration(new URI("static://(tcp://localhost:" + backupPort + ")")).setName("main").setConnectorName("local")
configuration.addClusterConfiguration(backToMain)
configuration.addAddressConfiguration(new CoreAddressConfiguration().setName("MultiVersionReplicaTestQueue"));
configuration.addQueueConfiguration(new QueueConfiguration("MultiVersionReplicaTestQueue").setAddress("MultiVersionReplicaTestQueue").setRoutingType(RoutingType.ANYCAST));
theBackupServer = new EmbeddedActiveMQ();
theBackupServer.setConfiguration(configuration);
theBackupServer.start();

View File

@ -0,0 +1,28 @@
package multiVersionReplica
/*
* 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.
*/
import org.apache.activemq.artemis.tests.compatibility.GroovyRun
for (int i = 0; i < 100 && !theBackupServer.getActiveMQServer().isActive(); i++) {
Thread.sleep(100);
}
System.out.println("*******************************************************************************************************************************");
System.out.println("backup active = " + theBackupServer.getActiveMQServer().isActive())
System.out.println("*******************************************************************************************************************************");
GroovyRun.assertTrue(theBackupServer.getActiveMQServer().isActive());

View File

@ -14,11 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.activemq.artemis.core.paging.cursor;
import org.apache.activemq.artemis.core.paging.PagedMessage;
public interface BulkPageCache extends PageCache {
PagedMessage[] getMessages();
}
if (theBackupServer != null) {
theBackupServer.stop()
}

View File

@ -0,0 +1,57 @@
package multiVersionReplica
/*
* 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.
*/
import org.apache.activemq.artemis.api.core.QueueConfiguration
import org.apache.activemq.artemis.api.core.RoutingType
import org.apache.activemq.artemis.core.config.ClusterConnectionConfiguration
import org.apache.activemq.artemis.core.config.CoreAddressConfiguration
import org.apache.activemq.artemis.core.config.ha.ReplicatedPolicyConfiguration
import org.apache.activemq.artemis.core.config.impl.ConfigurationImpl
// starts an artemis server
import org.apache.activemq.artemis.core.server.JournalType
import org.apache.activemq.artemis.core.server.embedded.EmbeddedActiveMQ
import org.apache.activemq.artemis.core.settings.impl.AddressFullMessagePolicy
import org.apache.activemq.artemis.core.settings.impl.AddressSettings
String folder = arg[0];
String id = arg[1];
String port = arg[2];
String backupPort = arg[3]
configuration = new ConfigurationImpl();
configuration.setJournalType(JournalType.NIO);
configuration.setBrokerInstance(new File(folder + "/" + id));
configuration.addAcceptorConfiguration("artemis", "tcp://localhost:" + port);
configuration.addConnectorConfiguration("local", "tcp://localhost:" + port);
configuration.setSecurityEnabled(false);
configuration.setPersistenceEnabled(true);
configuration.setGlobalMaxMessages(10);
configuration.setHAPolicyConfiguration(new ReplicatedPolicyConfiguration().setClusterName("main"))
configuration.addAddressesSetting("#", new AddressSettings().setAddressFullMessagePolicy(AddressFullMessagePolicy.PAGE));
ClusterConnectionConfiguration backupConfiguration = new ClusterConnectionConfiguration(new URI("static://(tcp://localhost:" + backupPort + ")")).setName("main").setConnectorName("local")
configuration.addClusterConfiguration(backupConfiguration)
configuration.addAddressConfiguration(new CoreAddressConfiguration().setName("MultiVersionReplicaTestQueue"));
configuration.addQueueConfiguration(new QueueConfiguration("MultiVersionReplicaTestQueue").setAddress("MultiVersionReplicaTestQueue").setRoutingType(RoutingType.ANYCAST));
theMainServer = new EmbeddedActiveMQ();
theMainServer.setConfiguration(configuration);
theMainServer.start();

View File

@ -0,0 +1,29 @@
package multiVersionReplica
/*
* 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.
*/
import org.apache.activemq.artemis.tests.compatibility.GroovyRun
for (int i = 0; i < 100 && !theMainServer.getActiveMQServer().isReplicaSync(); i++) {
Thread.sleep(100);
}
System.out.println("*******************************************************************************************************************************");
System.out.println("replica sync sync = " + theMainServer.getActiveMQServer().isReplicaSync())
System.out.println("*******************************************************************************************************************************");
GroovyRun.assertTrue(theMainServer.getActiveMQServer().isReplicaSync());

View File

@ -14,11 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.activemq.artemis.core.paging.cursor;
import org.apache.activemq.artemis.core.paging.PagedMessage;
public interface LivePageCache extends BulkPageCache {
void addLiveMessage(PagedMessage message);
}
if (theMainServer != null) {
theMainServer.stop();
}

View File

@ -0,0 +1,154 @@
/*
* 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.activemq.artemis.tests.compatibility;
import javax.jms.Connection;
import javax.jms.ConnectionFactory;
import javax.jms.MessageConsumer;
import javax.jms.MessageProducer;
import javax.jms.Queue;
import javax.jms.Session;
import javax.jms.TextMessage;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import org.apache.activemq.artemis.jms.client.ActiveMQConnectionFactory;
import org.apache.activemq.artemis.tests.compatibility.base.ClasspathBase;
import org.apache.qpid.jms.JmsConnectionFactory;
import org.junit.After;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import static org.apache.activemq.artemis.tests.compatibility.GroovyRun.SNAPSHOT;
import static org.apache.activemq.artemis.tests.compatibility.GroovyRun.TWO_TWENTYTWO_ZERO;
@RunWith(Parameterized.class)
public class MultiVersionReplicaTest extends ClasspathBase {
private static final String QUEUE_NAME = "MultiVersionReplicaTestQueue";
private final String main;
private final ClassLoader mainClassloader;
private final String backup;
private final ClassLoader backupClassLoader;
@Parameterized.Parameters(name = "main={0}, backup={1}")
public static Collection getParameters() {
List<Object[]> combinations = new ArrayList<>();
combinations.add(new Object[]{TWO_TWENTYTWO_ZERO, SNAPSHOT});
combinations.add(new Object[]{SNAPSHOT, TWO_TWENTYTWO_ZERO});
// The SNAPSHOT/SNAPSHOT is here as a teest validation only, like in other cases where SNAPSHOT/SNAPSHOT is used.
combinations.add(new Object[]{SNAPSHOT, SNAPSHOT});
return combinations;
}
public MultiVersionReplicaTest(String main, String backup) throws Exception {
this.main = main;
this.mainClassloader = getClasspath(main);
this.backup = backup;
this.backupClassLoader = getClasspath(backup);
}
@After
public void cleanupServers() {
try {
evaluate(mainClassloader, "multiVersionReplica/mainServerStop.groovy");
} catch (Exception ignored) {
}
try {
evaluate(backupClassLoader, "multiVersionReplica/backupServerStop.groovy");
} catch (Exception ignored) {
}
}
@Test
public void testReplica() throws Throwable {
System.out.println("Starting live");
evaluate(mainClassloader, "multiVersionReplica/mainServer.groovy", serverFolder.getRoot().getAbsolutePath(), "1", "61000", "61001");
System.out.println("Starting backup");
evaluate(backupClassLoader, "multiVersionReplica/backupServer.groovy", serverFolder.getRoot().getAbsolutePath(), "2", "61001", "61000");
evaluate(mainClassloader, "multiVersionReplica/mainServerIsReplicated.groovy");
send(new ActiveMQConnectionFactory("tcp://localhost:61000"), 2000);
send(new JmsConnectionFactory("amqp://localhost:61000"), 2000);
evaluate(mainClassloader, "multiVersionReplica/mainServerStop.groovy");
evaluate(backupClassLoader, "multiVersionReplica/backupServerIsActive.groovy");
receive(new ActiveMQConnectionFactory("tcp://localhost:61001"), 2000);
receive(new JmsConnectionFactory("amqp://localhost:61001"), 2000);
evaluate(backupClassLoader, "multiVersionReplica/backupServerStop.groovy");
}
private void send(ConnectionFactory factory, int numberOfMessages) throws Throwable {
try (Connection connection = factory.createConnection()) {
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
Queue queue = session.createQueue(QUEUE_NAME);
MessageProducer producer = session.createProducer(queue);
boolean pending = false;
for (int i = 0; i < numberOfMessages; i++) {
producer.send(session.createTextMessage("Hello world!!!!!"));
pending = true;
if (i > 0 && i % 100 == 0) {
session.commit();
pending = false;
}
}
if (pending) {
session.commit();
}
}
}
private void receive(ConnectionFactory factory, int numberOfMessages) throws Throwable {
try (Connection connection = factory.createConnection()) {
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
Queue queue = session.createQueue(QUEUE_NAME);
connection.start();
MessageConsumer consumer = session.createConsumer(queue);
boolean pending = false;
for (int i = 0; i < numberOfMessages; i++) {
TextMessage message = (TextMessage) consumer.receive(1000);
Assert.assertNotNull(message);
pending = true;
if (i > 0 && i % 100 == 0) {
session.commit();
pending = false;
}
}
if (pending) {
session.commit();
}
}
}
}

View File

@ -44,6 +44,7 @@ public class AMQPBridgeDisconnectTest extends AmqpClientTestSupport {
public static void main(String[] arg) {
try {
AMQPBridgeDisconnectTest reconnect = new AMQPBridgeDisconnectTest();
reconnect.setTestDir(arg[1]);
if (arg[0].equals("client")) {
reconnect.runExternal(true);
} else {
@ -122,7 +123,7 @@ public class AMQPBridgeDisconnectTest extends AmqpClientTestSupport {
this.server.getConfiguration().addAMQPConnection(connectConfiguration);
}
this.server.start();
Process process = SpawnedVMSupport.spawnVM(AMQPBridgeDisconnectTest.class.getName(), true, startClient ? "server" : "client");
Process process = SpawnedVMSupport.spawnVM(AMQPBridgeDisconnectTest.class.getName(), true, startClient ? "server" : "client", getTestDir());
try {
ActiveMQServer var10000 = this.server;

View File

@ -1,83 +0,0 @@
/*
* 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.activemq.artemis.tests.integration.amqp.paging;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.ActiveMQBuffers;
import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.core.paging.cursor.impl.PageReaderTest;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPStandardMessage;
import org.apache.activemq.artemis.spi.core.protocol.MessagePersister;
import org.apache.activemq.artemis.tests.integration.amqp.AmqpTestSupport;
import org.apache.activemq.transport.amqp.client.AmqpMessage;
import org.apache.qpid.proton.message.impl.MessageImpl;
import org.junit.Assert;
import org.junit.Test;
public class AmqpPageReaderTest extends PageReaderTest {
public MessageImpl createProtonMessage(String address) {
AmqpMessage message = new AmqpMessage();
final StringBuilder builder = new StringBuilder();
for (int i = 0; i < 1000; i++) {
builder.append('0');
}
final String data = builder.toString();
message.setText(data);
message.setAddress(address);
message.setDurable(true);
MessageImpl protonMessage = (MessageImpl) message.getWrappedMessage();
return protonMessage;
}
@Override
protected Message createMessage(SimpleString address, int msgId, byte[] content) {
MessageImpl protonMessage = createProtonMessage(address.toString());
AMQPStandardMessage amqpStandardMessage = AmqpTestSupport.encodeAndDecodeMessage(0, protonMessage, 2 * 1024);
amqpStandardMessage.setMessageID(msgId);
return amqpStandardMessage;
}
@Test
public void testEncodeSize() throws Exception {
Message message = createMessage(SimpleString.toSimpleString("Test"), 1, new byte[10]);
MessagePersister persister = (MessagePersister)message.getPersister();
ActiveMQBuffer buffer = ActiveMQBuffers.dynamicBuffer(1024);
persister.encode(buffer, message);
Assert.assertEquals(persister.getEncodeSize(message), buffer.writerIndex());
// the very first byte is the persisterID, we skip that since we are calling the Persister directly
buffer.readerIndex(1);
Message messageRead = persister.decode(buffer, null, null);
// The current persister does not guarantee the same encode size after loading
/// if this ever changes we can uncomment the next line.
// Assert.assertEquals(persister.getEncodeSize(message), persister.getEncodeSize(messageRead));
}
}

View File

@ -104,7 +104,7 @@ public class AmqpPagingTest extends AmqpClientTestSupport {
Wait.assertEquals(MSG_COUNT, queueView::getMessageCount);
PagingStore pagingStore = server.getPagingManager().getPageStore(SimpleString.toSimpleString(getQueueName()));
Assert.assertTrue(pagingStore.isPaging());
final int pageCacheMaxSize = server.getConfiguration().getAddressSettings().get("#").getPageCacheMaxSize();
final long pageCacheMaxSize = server.getConfiguration().getAddressSettings().get("#").getPageCacheMaxSize();
Assert.assertThat("the size of the messages or the number of messages isn't enough",
pagingStore.getNumberOfPages(), Matchers.greaterThan(pageCacheMaxSize));
receiver.flow(MSG_COUNT);

View File

@ -196,7 +196,7 @@ public class JMSPagingFileDeleteTest extends JMSTestBase {
timeout = System.currentTimeMillis() + 10000;
Wait.assertEquals(0, pagingStore::getNumberOfPages); //I expected number of the page is 1, but It was not.
Wait.assertEquals(1, pagingStore::getNumberOfPages);
} finally {
if (connection != null) {
connection.close();

View File

@ -2714,10 +2714,6 @@ public class LargeMessageTest extends LargeMessageTestBase {
session.commit();
if (isPage) {
server.getPagingManager().getPageStore(ADDRESS).getCursorProvider().clearCache();
}
if (isPage) {
Assert.assertEquals(0, server.getPagingManager().getPageStore(ADDRESS).getAddressSize());
Assert.assertEquals(0, server.getPagingManager().getGlobalSize());

View File

@ -355,17 +355,17 @@ public class SendAckFailTest extends SpawnedTestBase {
}
@Override
public void pageClosed(SimpleString storeName, int pageNumber) {
public void pageClosed(SimpleString storeName, long pageNumber) {
manager.pageClosed(storeName, pageNumber);
}
@Override
public void pageDeleted(SimpleString storeName, int pageNumber) {
public void pageDeleted(SimpleString storeName, long pageNumber) {
manager.pageDeleted(storeName, pageNumber);
}
@Override
public void pageWrite(PagedMessage message, int pageNumber) {
public void pageWrite(PagedMessage message, long pageNumber) {
manager.pageWrite(message, pageNumber);
}

View File

@ -82,7 +82,7 @@ public class AddressControlUsingCoreTest extends AddressControlTest {
}
@Override
public int getNumberOfPages() {
public long getNumberOfPages() {
return (int) proxy.retrieveAttributeValue("numberOfPages", Integer.class);
}

View File

@ -85,6 +85,7 @@ import org.apache.activemq.artemis.core.paging.PagingStoreFactory;
import org.apache.activemq.artemis.core.paging.cursor.PageCursorProvider;
import org.apache.activemq.artemis.core.paging.cursor.PageIterator;
import org.apache.activemq.artemis.core.paging.cursor.PageSubscription;
import org.apache.activemq.artemis.core.paging.cursor.impl.PageCursorProviderAccessor;
import org.apache.activemq.artemis.core.paging.cursor.impl.PageCursorProviderImpl;
import org.apache.activemq.artemis.core.paging.cursor.impl.PagePositionImpl;
import org.apache.activemq.artemis.core.paging.impl.Page;
@ -209,15 +210,6 @@ public class PagingTest extends ActiveMQTestBase {
@Test
public void testPageOnLargeMessageMultipleQueues() throws Exception {
internaltestOnLargetMessageMultipleQueues(MESSAGE_SIZE, true);
}
@Test
public void testPageOnLargeMessageMultipleQueuesNoPersistence() throws Exception {
internaltestOnLargetMessageMultipleQueues(LARGE_MESSAGE_SIZE, false);
}
private void internaltestOnLargetMessageMultipleQueues(final int messageSize, final boolean enablePersistence) throws Exception, ActiveMQException {
clearDataRecreateServerDirs();
Configuration config = createDefaultInVMConfig();
@ -226,7 +218,7 @@ public class PagingTest extends ActiveMQTestBase {
final int PAGE_SIZE = 10 * 1024;
ActiveMQServer server = createServer(enablePersistence, config, PAGE_SIZE, PAGE_MAX);
ActiveMQServer server = createServer(true, config, PAGE_SIZE, PAGE_MAX);
server.start();
locator.setBlockOnNonDurableSend(true).setBlockOnDurableSend(true).setBlockOnAcknowledge(true);
@ -247,11 +239,7 @@ public class PagingTest extends ActiveMQTestBase {
message.getBodyBuffer().writerIndex(0);
message.getBodyBuffer().writeBytes(new byte[messageSize]);
for (int j = 1; j <= messageSize; j++) {
message.getBodyBuffer().writeInt(j);
}
message.getBodyBuffer().writeBytes(new byte[LARGE_MESSAGE_SIZE]);
producer.send(message);
}
@ -268,14 +256,12 @@ public class PagingTest extends ActiveMQTestBase {
for (int i = 0; i < 201; i++) {
ClientMessage message2 = consumer.receive(10000);
Assert.assertNotNull(message2);
Assert.assertNotNull("message was null, ad= " + ad, message2);
message2.acknowledge();
Assert.assertNotNull(message2);
}
if (ad > -1) {
if (ad >= 1) {
session.commit();
} else {
session.rollback();
@ -388,6 +374,139 @@ public class PagingTest extends ActiveMQTestBase {
}
}
@Test
public void testSimpleCursorIterator() throws Exception {
Configuration config = createDefaultInVMConfig();
final int PAGE_MAX = 20 * 1024;
final int PAGE_SIZE = 10 * 1024;
ActiveMQServer server = createServer(true, config, PAGE_SIZE, PAGE_MAX);
server.start();
final int numberOfBytes = 124;
final int NUMBER_OF_MESSAGES = 201;
locator.setBlockOnNonDurableSend(false).setBlockOnDurableSend(false).setBlockOnAcknowledge(false);
ClientSessionFactory sf = addSessionFactory(createSessionFactory(locator));
ClientSession session = sf.createSession(null, null, false, false, false, false, 0);
session.createQueue(new QueueConfiguration(ADDRESS.concat("-0")).setAddress(ADDRESS));
server.getPagingManager().getPageStore(ADDRESS).startPaging();
server.getPagingManager().getPageStore(ADDRESS).disableCleanup();
session.start();
ClientProducer producer = session.createProducer(ADDRESS);
ClientMessage message = null;
for (int i = 0; i < NUMBER_OF_MESSAGES; i++) {
message = session.createMessage(true);
message.getBodyBuffer().writerIndex(0);
message.getBodyBuffer().writeBytes(new byte[numberOfBytes]);
for (int j = 1; j <= numberOfBytes; j++) {
message.getBodyBuffer().writeInt(j);
}
message.putIntProperty("i", i);
producer.send(message);
}
session.commit();
Queue queue = server.locateQueue(ADDRESS.concat("-0"));
PagingStore store = server.getPagingManager().getPageStore(ADDRESS);
PageCursorProvider provider = store.getCursorProvider();
PageSubscription cursorSubscription = provider.getSubscription(queue.getID());
PageIterator iterator = cursorSubscription.iterator(true);
for (int i = 0; i < NUMBER_OF_MESSAGES; i++) {
Assert.assertTrue(iterator.hasNext());
PagedMessage messageReceived = iterator.next().getPagedMessage();
System.out.println("Page " + messageReceived.getPageNumber() + " , message = " + messageReceived.getMessageNumber());
Assert.assertNotNull(messageReceived);
Assert.assertEquals(i, (int)messageReceived.getMessage().getIntProperty("i"));
}
Assert.assertFalse(iterator.hasNext());
}
@Test
public void testSimpleCursorIteratorLargeMessage() throws Exception {
Configuration config = createDefaultInVMConfig();
final int PAGE_MAX = 20 * 1024;
final int PAGE_SIZE = 10 * 1024;
ActiveMQServer server = createServer(true, config, PAGE_SIZE, PAGE_MAX);
server.start();
final int numberOfBytes = 200 * 1024;
final int NUMBER_OF_MESSAGES = 50;
locator.setBlockOnNonDurableSend(false).setBlockOnDurableSend(false).setBlockOnAcknowledge(false);
ClientSessionFactory sf = addSessionFactory(createSessionFactory(locator));
ClientSession session = sf.createSession(null, null, false, false, false, false, 0);
session.createQueue(new QueueConfiguration(ADDRESS.concat("-0")).setAddress(ADDRESS));
server.getPagingManager().getPageStore(ADDRESS).startPaging();
server.getPagingManager().getPageStore(ADDRESS).disableCleanup();
session.start();
ClientProducer producer = session.createProducer(ADDRESS);
ClientMessage message = null;
for (int i = 0; i < NUMBER_OF_MESSAGES; i++) {
message = session.createMessage(true);
message.getBodyBuffer().writerIndex(0);
message.putIntProperty("i", i);
producer.send(message);
}
session.commit();
Queue queue = server.locateQueue(ADDRESS.concat("-0"));
queue.pause();
PagingStore store = server.getPagingManager().getPageStore(ADDRESS);
PageCursorProvider provider = store.getCursorProvider();
PageSubscription cursorSubscription = provider.getSubscription(queue.getID());
PageIterator iterator = cursorSubscription.iterator(true);
for (int i = 0; i < NUMBER_OF_MESSAGES; i++) {
Assert.assertTrue(iterator.hasNext());
PagedMessage messageReceived = iterator.next().getPagedMessage();
System.out.println("Page " + messageReceived.getPageNumber() + " , message = " + messageReceived.getMessageNumber());
Assert.assertNotNull(messageReceived);
Assert.assertEquals(i, (int)messageReceived.getMessage().getIntProperty("i"));
}
Assert.assertFalse(iterator.hasNext());
}
@Test
public void testPageCleanup() throws Exception {
clearDataRecreateServerDirs();
@ -693,9 +812,11 @@ public class PagingTest extends ActiveMQTestBase {
queue.getPagingStore().forceAnotherPage(); // forcing an empty file, just to make it more challenging
int page = 1;
for (int i = 0; i < numberOfMessages; i++) {
if (i % 10 == 0 && i > 0) {
queue.getPagingStore().forceAnotherPage();
page++;
}
message = session.createMessage(true);
@ -704,6 +825,7 @@ public class PagingTest extends ActiveMQTestBase {
bodyLocal.writeBytes(body);
message.putIntProperty("i", i);
message.putIntProperty("page", page);
producer.send(message);
}
@ -748,9 +870,9 @@ public class PagingTest extends ActiveMQTestBase {
server.getStorageManager().getMessageJournal().scheduleCompactAndBlock(5000);
Page page4 = queue.getPagingStore().createPage(4);
Page page4 = queue.getPagingStore().newPageObject(4);
page4.open(true);
List<PagedMessage> messagesRead = page4.read(server.getStorageManager());
org.apache.activemq.artemis.utils.collections.LinkedList<PagedMessage> messagesRead = page4.read(server.getStorageManager());
Assert.assertEquals(10, messagesRead.size());
page4.close(false);
page4.delete(null);
@ -772,6 +894,9 @@ public class PagingTest extends ActiveMQTestBase {
sf = createSessionFactory(locator);
session = sf.createSession(false, true, true);
logger.info("*******************************************************************************************************************************");
logger.info("Creating consumer");
consumer = session.createConsumer(ADDRESS);
session.start();
@ -787,7 +912,83 @@ public class PagingTest extends ActiveMQTestBase {
Assert.assertNull(msgClient);
session.commit();
Wait.assertFalse(queue.getPagingStore()::isPaging);
Wait.assertFalse(queue.getPagingStore()::isPaging, 5000, 100);
}
@Test
public void testPageReadOneMessage() throws Exception {
clearDataRecreateServerDirs();
Configuration config = createDefaultInVMConfig().setJournalSyncNonTransactional(false);
server = createServer(true, config, PAGE_SIZE, PAGE_MAX, 1, new HashMap<>());
server.start();
final int numberOfMessages = 100;
locator = createInVMNonHALocator().setBlockOnNonDurableSend(true).setBlockOnDurableSend(true).setBlockOnAcknowledge(true);
sf = createSessionFactory(locator);
ClientSession session = sf.createSession(false, false, false);
session.createQueue(new QueueConfiguration(PagingTest.ADDRESS));
ClientProducer producer = session.createProducer(PagingTest.ADDRESS);
ClientMessage message = null;
byte[] body = new byte[10];
ByteBuffer bb = ByteBuffer.wrap(body);
for (int j = 1; j <= 10; j++) {
bb.put(getSamplebyte(j));
}
Queue queue = server.locateQueue(ADDRESS);
for (int repeat = 0; repeat < 2; repeat++) {
queue.getPagingStore().startPaging();
int page = 1;
for (int i = 0; i < numberOfMessages; i++) {
if (i % 10 == 0 && i > 0) {
queue.getPagingStore().forceAnotherPage();
page++;
}
message = session.createMessage(true);
ActiveMQBuffer bodyLocal = message.getBodyBuffer();
bodyLocal.writeBytes(body);
message.putIntProperty("i", i);
message.putIntProperty("page", page);
producer.send(message);
}
session.commit();
session.start();
ClientConsumer consumer = session.createConsumer(ADDRESS);
for (int i = 0; i < numberOfMessages; i++) {
ClientMessage messReceived = consumer.receive(5000);
Assert.assertNotNull(messReceived);
System.out.println("Receiving " + messReceived);
messReceived.acknowledge();
}
consumer.close();
session.commit();
Wait.assertFalse(queue.getPagingStore()::isPaging, 5000, 100);
Wait.assertEquals(1, () -> PagingStoreTestAccessor.getUsedPagesSize(queue.getPagingStore()), 1000, 100);
}
Wait.assertFalse(queue.getPagingStore()::isPaging, 5000, 100);
}
@ -872,7 +1073,7 @@ public class PagingTest extends ActiveMQTestBase {
Assert.assertEquals(10, queue.getPagingStore().getNumberOfPages());
queue.getPagingStore().getCursorProvider().cleanup();
PageCursorProviderAccessor.cleanup(queue.getPagingStore().getCursorProvider());
Assert.assertEquals(9, queue.getPagingStore().getNumberOfPages());
@ -1015,7 +1216,7 @@ public class PagingTest extends ActiveMQTestBase {
session.commit();
consumer.close();
queue.getPagingStore().getCursorProvider().cleanup();
PageCursorProviderAccessor.cleanup(queue.getPagingStore().getCursorProvider());
Wait.assertFalse(queue.getPagingStore()::isPaging, 5000, 100);
}
@ -1263,7 +1464,7 @@ public class PagingTest extends ActiveMQTestBase {
Wait.assertEquals(0, purgeQueue::getMessageCount);
Wait.assertEquals(0, purgeQueue.getPageSubscription().getPagingStore()::getAddressSize);
Wait.assertFalse(purgeQueue.getPageSubscription()::isPaging);
Wait.assertFalse(purgeQueue.getPageSubscription()::isPaging, 5000, 100);
StorageManager sm = server.getStorageManager();
@ -1565,6 +1766,111 @@ public class PagingTest extends ActiveMQTestBase {
}
@Test
public void testSimplePreparedAck() throws Exception {
clearDataRecreateServerDirs();
Configuration config = createDefaultInVMConfig().setJournalSyncNonTransactional(false);
server = createServer(true, config, PagingTest.PAGE_SIZE, PagingTest.PAGE_MAX);
server.start();
final int numberOfMessages = 50;
// namespace for the first client (before the server restart)
{
locator = createInVMNonHALocator().setBlockOnNonDurableSend(true).setBlockOnDurableSend(true).setBlockOnAcknowledge(true).setAckBatchSize(0);
sf = createSessionFactory(locator);
ClientSession session = sf.createSession(false, true, true);
session.createQueue(new QueueConfiguration(PagingTest.ADDRESS));
Queue queue = server.locateQueue(PagingTest.ADDRESS);
ClientProducer producer = session.createProducer(PagingTest.ADDRESS);
byte[] body = new byte[MESSAGE_SIZE];
ByteBuffer bb = ByteBuffer.wrap(body);
for (int j = 1; j <= MESSAGE_SIZE; j++) {
bb.put(getSamplebyte(j));
}
queue.getPageSubscription().getPagingStore().startPaging();
// Send many messages, 5 on each page
for (int i = 0; i < numberOfMessages; i++) {
ClientMessage message = session.createMessage(true);
message.putIntProperty("count", i);
ActiveMQBuffer bodyLocal = message.getBodyBuffer();
bodyLocal.writeBytes(body);
producer.send(message);
if ((i + 1) % 5 == 0) {
session.commit();
queue.getPageSubscription().getPagingStore().forceAnotherPage();
}
}
session.close();
session = sf.createSession(true, false, false);
Xid xidConsumeNoCommit = newXID();
session.start(xidConsumeNoCommit, XAResource.TMNOFLAGS);
ClientConsumer cons = session.createConsumer(ADDRESS);
session.start();
// First message is consumed, prepared, will be rolled back later
ClientMessage firstMessageConsumed = cons.receive(5000);
assertNotNull(firstMessageConsumed);
firstMessageConsumed.acknowledge();
session.end(xidConsumeNoCommit, XAResource.TMSUCCESS);
session.prepare(xidConsumeNoCommit);
}
server.stop();
server.start();
// Namespace for the second client, after the restart
{
locator = createInVMNonHALocator().setBlockOnNonDurableSend(true).setBlockOnDurableSend(true).setBlockOnAcknowledge(true).setAckBatchSize(0);
sf = createSessionFactory(locator);
ClientSession session = sf.createSession(false, true, true);
ClientConsumer cons = session.createConsumer(ADDRESS);
session.start();
for (int i = 1; i < numberOfMessages; i++) {
ClientMessage message = cons.receive(5000);
Assert.assertNotNull(message);
Assert.assertEquals(i, message.getIntProperty("count").intValue());
message.acknowledge();
}
session.commit();
}
}
@Test
public void testPreparedACKRemoveAndRestart() throws Exception {
Assume.assumeTrue(storeType == StoreConfiguration.StoreType.FILE);
@ -1798,7 +2104,8 @@ public class PagingTest extends ActiveMQTestBase {
server.start();
final int numberOfMessages = 5000;
final int NUM_MESSAGES = 100;
final int COMMIT_INTERVAL = 10;
locator = createInVMNonHALocator().setConsumerWindowSize(10 * 1024 * 1024).setBlockOnNonDurableSend(true).setBlockOnDurableSend(true).setBlockOnAcknowledge(true);
@ -1828,7 +2135,7 @@ public class PagingTest extends ActiveMQTestBase {
bb.put(getSamplebyte(j));
}
for (int i = 0; i < numberOfMessages; i++) {
for (int i = 0; i < NUM_MESSAGES; i++) {
message = session.createMessage(true);
ActiveMQBuffer bodyLocal = message.getBodyBuffer();
@ -1836,7 +2143,7 @@ public class PagingTest extends ActiveMQTestBase {
bodyLocal.writeBytes(body);
producer.send(message);
if (i % 1000 == 0) {
if (i % COMMIT_INTERVAL == 0) {
session.commit();
}
}
@ -1846,13 +2153,8 @@ public class PagingTest extends ActiveMQTestBase {
long timeout = System.currentTimeMillis() + 30000;
// I want the buffer full to make sure there are pending messages on the server's side
while (System.currentTimeMillis() < timeout && (cons.getBufferSize() < 1000 || cons2.getBufferSize() < 1000)) {
Thread.sleep(100);
}
assertTrue(cons.getBufferSize() >= 1000);
assertTrue(cons2.getBufferSize() >= 1000);
waitBuffer(cons, NUM_MESSAGES / 5);
waitBuffer(cons2, NUM_MESSAGES / 5);
session.close();
@ -1910,11 +2212,11 @@ public class PagingTest extends ActiveMQTestBase {
cons = (ClientConsumerInternal) session.createConsumer(ADDRESS);
session.start();
for (int i = 0; i < numberOfMessages; i++) {
for (int i = 0; i < NUM_MESSAGES; i++) {
message = cons.receive(1000);
assertNotNull(message);
message.acknowledge();
if (i % 1000 == 0) {
if (i % COMMIT_INTERVAL == 0) {
session.commit();
}
}
@ -1926,15 +2228,17 @@ public class PagingTest extends ActiveMQTestBase {
Wait.assertEquals(0, queue::getMessageCount);
timeout = System.currentTimeMillis() + 10000;
while (timeout > System.currentTimeMillis() && queue.getPageSubscription().getPagingStore().isPaging()) {
Thread.sleep(100);
}
Wait.assertFalse(queue.getPageSubscription().getPagingStore()::isPaging, 5000, 100);
assertFalse(queue.getPageSubscription().getPagingStore().isPaging());
server.stop();
}
private void waitBuffer(ClientConsumerInternal clientBuffer, int bufferSize) {
Wait.assertTrue(() -> "expected " + bufferSize + " but got " + clientBuffer.getBufferSize(), () -> clientBuffer.getBufferSize() > bufferSize, 5000, 100);
}
@Test
public void testPreparePersistent() throws Exception {
clearDataRecreateServerDirs();
@ -2111,6 +2415,8 @@ public class PagingTest extends ActiveMQTestBase {
Wait.assertEquals(0, queue::getMessageCount);
PageCursorProviderAccessor.cleanup(queue.getPagingStore().getCursorProvider());
waitForNotPaging(queue);
}
@ -2554,7 +2860,7 @@ public class PagingTest extends ActiveMQTestBase {
}
@Override
public void cleanup() {
protected void cleanup() {
if (mainCleanup.get()) {
super.cleanup();
} else {
@ -2662,7 +2968,7 @@ public class PagingTest extends ActiveMQTestBase {
queue = server.locateQueue(ADDRESS);
queue.getPageSubscription().cleanupEntries(false);
queue.getPageSubscription().getPagingStore().getCursorProvider().cleanup();
PageCursorProviderAccessor.cleanup(queue.getPageSubscription().getPagingStore().getCursorProvider());
consumer = session.createConsumer(ADDRESS);
session.start();
@ -2783,7 +3089,7 @@ public class PagingTest extends ActiveMQTestBase {
q2.getPageSubscription().cleanupEntries(false);
PageCursorProvider provider = q1.getPageSubscription().getPagingStore().getCursorProvider();
provider.cleanup();
PageCursorProviderAccessor.cleanup(provider);
waitForNotPaging(q1);
@ -2984,7 +3290,7 @@ public class PagingTest extends ActiveMQTestBase {
session.commit();
PagingStore store = server.getPagingManager().getPageStore(ADDRESS);
store.getCursorProvider().cleanup();
PageCursorProviderAccessor.cleanup(store.getCursorProvider());
Wait.assertFalse(server.getPagingManager().getPageStore(ADDRESS)::isPaging, 5000, 100);
}
@ -4196,7 +4502,7 @@ public class PagingTest extends ActiveMQTestBase {
queue.getPageSubscription().getPagingStore().disableCleanup();
queue.getPageSubscription().getPagingStore().getCursorProvider().cleanup();
PageCursorProviderAccessor.cleanup(queue.getPagingStore().getCursorProvider());
consumerSession.start();
ClientConsumer consumer = consumerSession.createConsumer(ADDRESS, SimpleString.toSimpleString("id > 0"));
@ -4209,7 +4515,7 @@ public class PagingTest extends ActiveMQTestBase {
// The only reason I'm calling cleanup directly is that it would be easy to debug in case of bugs
// if you see an issue with cleanup here, enjoy debugging this method
queue.getPageSubscription().getPagingStore().getCursorProvider().cleanup();
PageCursorProviderAccessor.cleanup(queue.getPagingStore().getCursorProvider());
}
queue.getPageSubscription().getPagingStore().enableCleanup();
@ -4357,7 +4663,7 @@ public class PagingTest extends ActiveMQTestBase {
queue = server.locateQueue(ADDRESS);
queue.getPageSubscription().cleanupEntries(false);
queue.getPageSubscription().getPagingStore().getCursorProvider().cleanup();
PageCursorProviderAccessor.cleanup(queue.getPageSubscription().getPagingStore().getCursorProvider());
ClientConsumer consumer = session.createConsumer(ADDRESS);
session.start();
@ -4974,6 +5280,83 @@ public class PagingTest extends ActiveMQTestBase {
session.close();
}
@Test
public void testSimplePaging() throws Exception {
SimpleString PAGED_ADDRESS = new SimpleString("paged");
Configuration configuration = createDefaultInVMConfig();
Map<String, AddressSettings> addresses = new HashMap<>();
addresses.put("#", new AddressSettings());
AddressSettings pagedDestination = new AddressSettings().setPageSizeBytes(1024).setMaxSizeBytes(0);
addresses.put(PAGED_ADDRESS.toString(), pagedDestination);
server = createServer(true, configuration, -1, -1, addresses);
server.start();
sf = createSessionFactory(locator);
ClientSession session = sf.createSession(false, true, false);
session.createQueue(new QueueConfiguration(PAGED_ADDRESS));
Wait.assertTrue(() -> null != server.locateQueue(PAGED_ADDRESS));
Queue serverQueue = server.locateQueue(PAGED_ADDRESS);
//serverQueue.getPagingStore().startPaging();
//Wait.assertTrue(serverQueue.getPagingStore()::isPaging);
ClientProducer producerPaged = session.createProducer(PAGED_ADDRESS);
int NUMBER_OF_MESSAGES = 100;
for (int i = 0; i < NUMBER_OF_MESSAGES; i++) {
ClientMessage msg = session.createMessage(true);
msg.putIntProperty("i", i);
msg.getBodyBuffer().writeBytes(new byte[512]);
producerPaged.send(msg);
}
session.close();
//System.exit(-1);
Assert.assertTrue(server.getPagingManager().getPageStore(PAGED_ADDRESS).isPaging());
session = sf.createSession(false, true, false);
session.start();
ClientConsumer consumerPaged = session.createConsumer(PAGED_ADDRESS);
session.commit();
for (int i = 0; i < NUMBER_OF_MESSAGES; i++) {
ClientMessage msg = consumerPaged.receive(5000);
Assert.assertNotNull("expected message at " + i, msg);
int recI = msg.getIntProperty("i");
Assert.assertEquals(i, recI);
if (recI != i) {
System.err.println("Expected " + i + " and received " + recI);
}
System.out.println("*******************************************************************************************************************************");
System.out.println("msg " + msg);
System.out.println("*******************************************************************************************************************************");
msg.acknowledge();
session.commit();
}
Assert.assertNull(consumerPaged.receiveImmediate());
session.close();
}
@Test
public void testPagingDifferentSizes() throws Exception {
SimpleString PAGED_ADDRESS_A = new SimpleString("paged-a");
@ -5178,9 +5561,9 @@ public class PagingTest extends ActiveMQTestBase {
}
// It's async, so need to wait a bit for it happening
assertFalse(server.getPagingManager().getPageStore(ADDRESS).isPaging());
Wait.assertFalse(server.getPagingManager().getPageStore(ADDRESS)::isPaging);
Wait.assertEquals(0, ()->server.getPagingManager().getPageStore(ADDRESS).getNumberOfPages());
Wait.assertEquals(1, ()->server.getPagingManager().getPageStore(ADDRESS).getNumberOfPages());
}
@Test
@ -5254,7 +5637,7 @@ public class PagingTest extends ActiveMQTestBase {
}
PagingStore store = server.getPagingManager().getPageStore(ADDRESS);
store.getCursorProvider().cleanup();
PageCursorProviderAccessor.cleanup(store.getCursorProvider());
long timeout = System.currentTimeMillis() + 5000;
while (store.isPaging() && timeout > System.currentTimeMillis()) {
@ -5334,14 +5717,14 @@ public class PagingTest extends ActiveMQTestBase {
}
PagingStore store = server.getPagingManager().getPageStore(ADDRESS);
store.getCursorProvider().cleanup();
PageCursorProviderAccessor.cleanup(store.getCursorProvider());
long timeout = System.currentTimeMillis() + 5000;
while (store.isPaging() && timeout > System.currentTimeMillis()) {
Thread.sleep(100);
}
store.getCursorProvider().cleanup();
PageCursorProviderAccessor.cleanup(store.getCursorProvider());
waitForNotPaging(server.locateQueue(PagingTest.ADDRESS.concat("=1")));
@ -5416,7 +5799,7 @@ public class PagingTest extends ActiveMQTestBase {
session.commit();
session.close();
store.getCursorProvider().cleanup();
PageCursorProviderAccessor.cleanup(store.getCursorProvider());
waitForNotPaging(server.locateQueue(PagingTest.ADDRESS.concat("=1")));
@ -5724,7 +6107,7 @@ public class PagingTest extends ActiveMQTestBase {
pgStoreAddress.getCursorProvider().getSubscription(serverQueue.getID()).cleanupEntries(false);
pgStoreAddress.getCursorProvider().cleanup();
PageCursorProviderAccessor.cleanup(pgStoreAddress.getCursorProvider());
while (timeout > System.currentTimeMillis() && pgStoreAddress.isPaging()) {
Thread.sleep(50);
@ -5854,15 +6237,9 @@ public class PagingTest extends ActiveMQTestBase {
cons.close();
long timeout = System.currentTimeMillis() + 5000;
pgStoreAddress = server.getPagingManager().getPageStore(ADDRESS);
while (timeout > System.currentTimeMillis() && pgStoreAddress.isPaging()) {
Thread.sleep(50);
}
assertFalse(pgStoreAddress.isPaging());
Wait.assertFalse(pgStoreAddress::isPaging, 5000, 100);
session.close();
} finally {
@ -6539,22 +6916,19 @@ public class PagingTest extends ActiveMQTestBase {
consumer.close();
store.getCursorProvider().cleanup();
PageCursorProviderAccessor.cleanup(store.getCursorProvider());
Wait.assertEquals(0, queue::getMessageCount);
long timeout = System.currentTimeMillis() + 5000;
while (store.isPaging() && timeout > System.currentTimeMillis()) {
Thread.sleep(100);
}
PageCursorProviderAccessor.cleanup(store.getCursorProvider());
store.getCursorProvider().cleanup();
Wait.assertFalse(store::isPaging, 5000, 100);
sf.close();
locator.close();
Wait.assertEquals(0, store::getNumberOfPages);
Wait.assertEquals(1L, store::getNumberOfPages, 5000, 100);
} finally {
try {
@ -7211,7 +7585,7 @@ public class PagingTest extends ActiveMQTestBase {
consumer.close();
session.close();
store.getCursorProvider().cleanup();
PageCursorProviderAccessor.cleanup(store.getCursorProvider());
waitForNotPaging(server.locateQueue(PagingTest.ADDRESS.concat("=1")));
sf.close();
locator.close();
@ -7335,7 +7709,6 @@ public class PagingTest extends ActiveMQTestBase {
ClientConsumer cons = session.createConsumer(ADDRESS);
ClientMessage msgReceivedCons = null;
// simulate the live page cache evicted
store.getCursorProvider().clearCache();
for (int i = 0; i < num; i++) {
msgReceivedCons = cons.receive(1000);
assertNotNull(msgReceivedCons);

View File

@ -42,6 +42,7 @@ import org.apache.activemq.artemis.core.settings.impl.AddressFullMessagePolicy;
import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
import org.apache.activemq.artemis.core.transaction.Transaction;
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
import org.apache.activemq.artemis.utils.actors.ArtemisExecutor;
import org.junit.Assert;
import org.junit.Test;
@ -245,6 +246,16 @@ public class PersistMultiThreadTest extends ActiveMQTestBase {
class FakePagingStore implements PagingStore {
@Override
public void execute(Runnable runnable) {
runnable.run();
}
@Override
public ArtemisExecutor getExecutor() {
return null;
}
@Override
public void durableDown(Message message, int durableCount) {
@ -276,12 +287,12 @@ public class PersistMultiThreadTest extends ActiveMQTestBase {
}
@Override
public int getNumberOfPages() {
public long getNumberOfPages() {
return 0;
}
@Override
public int getCurrentWritingPage() {
public long getCurrentWritingPage() {
return 0;
}
@ -345,11 +356,36 @@ public class PersistMultiThreadTest extends ActiveMQTestBase {
}
@Override
public Page usePage(long page, boolean create) {
return null;
}
@Override
public Page usePage(long page) {
return null;
}
@Override
public Page newPageObject(long page) throws Exception {
return null;
}
@Override
public void ioSync() throws Exception {
}
@Override
public int getMaxPageReadBytes() {
return 0;
}
@Override
public int getMaxPageReadMessages() {
return 0;
}
@Override
public boolean page(Message message,
Transaction tx,
@ -358,12 +394,7 @@ public class PersistMultiThreadTest extends ActiveMQTestBase {
}
@Override
public Page createPage(int page) throws Exception {
return null;
}
@Override
public boolean checkPageFileExists(int page) throws Exception {
public boolean checkPageFileExists(long page) throws Exception {
return false;
}

View File

@ -33,11 +33,9 @@ import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.core.config.Configuration;
import org.apache.activemq.artemis.core.filter.Filter;
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
import org.apache.activemq.artemis.core.paging.cursor.PageCache;
import org.apache.activemq.artemis.core.paging.cursor.PageCursorProvider;
import org.apache.activemq.artemis.core.paging.cursor.PageSubscription;
import org.apache.activemq.artemis.core.paging.cursor.PagedReference;
import org.apache.activemq.artemis.core.paging.cursor.impl.PageCursorProviderImpl;
import org.apache.activemq.artemis.core.paging.impl.PagingStoreImpl;
import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.core.persistence.impl.journal.OperationContextImpl;
@ -75,30 +73,6 @@ public class PageCursorStressTest extends ActiveMQTestBase {
private static final int PAGE_SIZE = 10 * 1024 * 1024;
// Read more cache than what would fit on the memory, and validate if the memory would be cleared through soft-caches
@Test
public void testReadCache() throws Exception {
final int NUM_MESSAGES = 100;
int numberOfPages = addMessages(NUM_MESSAGES, 1024 * 1024);
System.out.println("NumberOfPages = " + numberOfPages);
PageCursorProviderImpl cursorProvider = new PageCursorProviderImpl(lookupPageStore(ADDRESS), server.getStorageManager(), server.getExecutorFactory().getExecutor(), 5);
for (int i = 0; i < numberOfPages; i++) {
PageCache cache = cursorProvider.getPageCache(i + 1);
System.out.println("Page " + i + " had " + cache.getNumberOfMessages() + " messages");
}
forceGC();
assertTrue(cursorProvider.getCacheSize() < numberOfPages);
System.out.println("Cache size = " + cursorProvider.getCacheSize());
}
@Test
public void testSimpleCursor() throws Exception {
@ -109,7 +83,7 @@ public class PageCursorStressTest extends ActiveMQTestBase {
Iterator<PagedReference> iterEmpty = cursor.iterator();
int numberOfPages = addMessages(NUM_MESSAGES, 1024 * 1024);
long numberOfPages = addMessages(NUM_MESSAGES, 1024 * 1024);
System.out.println("NumberOfPages = " + numberOfPages);
@ -119,7 +93,7 @@ public class PageCursorStressTest extends ActiveMQTestBase {
int key = 0;
while ((msg = iterator.next()) != null) {
assertEquals(key++, msg.getMessage().getIntProperty("key").intValue());
cursor.confirmPosition(msg.getPosition());
cursor.confirmPosition(msg.getPagedMessage().newPositionObject());
}
assertEquals(NUM_MESSAGES, key);
@ -203,7 +177,7 @@ public class PageCursorStressTest extends ActiveMQTestBase {
});
int numberOfPages = addMessages(NUM_MESSAGES, 1024 * 1024);
long numberOfPages = addMessages(NUM_MESSAGES, 1024 * 1024);
System.out.println("NumberOfPages = " + numberOfPages);
@ -221,7 +195,7 @@ public class PageCursorStressTest extends ActiveMQTestBase {
assertEquals(key, msg.getMessage().getIntProperty("key").intValue());
assertTrue(msg.getMessage().getBooleanProperty("even").booleanValue());
key += 2;
cursorEven.confirmPosition(msg.getPosition());
cursorEven.confirmPosition(msg.getPagedMessage().newPositionObject());
}
assertEquals(NUM_MESSAGES, key);
@ -230,7 +204,7 @@ public class PageCursorStressTest extends ActiveMQTestBase {
assertEquals(key, msg.getMessage().getIntProperty("key").intValue());
assertFalse(msg.getMessage().getBooleanProperty("even").booleanValue());
key += 2;
cursorOdd.confirmPosition(msg.getPosition());
cursorOdd.confirmPosition(msg.getPagedMessage().newPositionObject());
}
assertEquals(NUM_MESSAGES + 1, key);
@ -245,28 +219,12 @@ public class PageCursorStressTest extends ActiveMQTestBase {
}
@Test
public void testReadNextPage() throws Exception {
final int NUM_MESSAGES = 1;
int numberOfPages = addMessages(NUM_MESSAGES, 1024);
System.out.println("NumberOfPages = " + numberOfPages);
PageCursorProvider cursorProvider = lookupCursorProvider();
PageCache cache = cursorProvider.getPageCache(2);
assertNull(cache);
}
@Test
public void testRestartWithHoleOnAck() throws Exception {
final int NUM_MESSAGES = 1000;
int numberOfPages = addMessages(NUM_MESSAGES, 10 * 1024);
long numberOfPages = addMessages(NUM_MESSAGES, 10 * 1024);
System.out.println("Number of pages = " + numberOfPages);
@ -319,7 +277,7 @@ public class PageCursorStressTest extends ActiveMQTestBase {
public void testRestartWithHoleOnAckAndTransaction() throws Exception {
final int NUM_MESSAGES = 1000;
int numberOfPages = addMessages(NUM_MESSAGES, 10 * 1024);
long numberOfPages = addMessages(NUM_MESSAGES, 10 * 1024);
System.out.println("Number of pages = " + numberOfPages);
@ -696,7 +654,7 @@ public class PageCursorStressTest extends ActiveMQTestBase {
final int NUM_MESSAGES = 10;
int numberOfPages = addMessages(NUM_MESSAGES, 1024 * 1024);
long numberOfPages = addMessages(NUM_MESSAGES, 1024 * 1024);
System.out.println("NumberOfPages = " + numberOfPages);
@ -752,11 +710,11 @@ public class PageCursorStressTest extends ActiveMQTestBase {
}
private int addMessages(final int numMessages, final int messageSize) throws Exception {
private long addMessages(final int numMessages, final int messageSize) throws Exception {
return addMessages(0, numMessages, messageSize);
}
private int addMessages(final int start, final int numMessages, final int messageSize) throws Exception {
private long addMessages(final int start, final int numMessages, final int messageSize) throws Exception {
PagingStoreImpl pageStore = lookupPageStore(ADDRESS);
pageStore.startPaging();

View File

@ -18,7 +18,6 @@ package org.apache.activemq.artemis.tests.unit.core.paging.impl;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
@ -42,6 +41,8 @@ import org.apache.activemq.artemis.tests.unit.core.journal.impl.fakes.FakeSequen
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
import org.apache.activemq.artemis.utils.ActiveMQThreadFactory;
import org.apache.activemq.artemis.utils.actors.OrderedExecutorFactory;
import org.apache.activemq.artemis.utils.collections.LinkedList;
import org.apache.activemq.artemis.utils.collections.LinkedListIterator;
import org.apache.activemq.artemis.utils.critical.EmptyCriticalAnalyzer;
import org.junit.Assert;
import org.junit.Before;
@ -142,7 +143,7 @@ public class PageTest extends ActiveMQTestBase {
file.open();
page = new Page(new SimpleString("something"), storageManager, factory, file, 10);
List<PagedMessage> msgs = page.read(storageManager, largeMessages);
LinkedList<PagedMessage> msgs = page.read(storageManager, largeMessages);
Assert.assertEquals(numberOfElements, msgs.size());
@ -162,12 +163,14 @@ public class PageTest extends ActiveMQTestBase {
Assert.assertEquals(numberOfElements, tmpPage.getNumberOfMessages());
}
Assert.assertTrue(page.delete(msgs.toArray(new PagedMessage[msgs.size()])));
Assert.assertTrue(page.delete(msgs));
for (PagedMessage pagedMessage : msgs) {
Assert.assertEquals(0, pagedMessage.getMessage().getUsage());
try (LinkedListIterator<PagedMessage> iter = msgs.iterator()) {
while (iter.hasNext()) {
PagedMessage pagedMessage = iter.next();
Assert.assertEquals(0, pagedMessage.getMessage().getUsage());
}
}
Assert.assertEquals(0, factory.listFiles(".page").size());
}
@ -219,7 +222,7 @@ public class PageTest extends ActiveMQTestBase {
file.open();
Page page1 = new Page(new SimpleString("something"), new NullStorageManager(), factory, file, 10);
List<PagedMessage> msgs = page1.read(new NullStorageManager());
LinkedList<PagedMessage> msgs = page1.read(new NullStorageManager());
Assert.assertEquals(numberOfElements, msgs.size());
@ -245,6 +248,81 @@ public class PageTest extends ActiveMQTestBase {
addPageElements(new NullStorageManager(), simpleDestination, page, numberOfElements, false, startMessageID);
}
@Test
public void testAddMessages() throws Exception {
recreateDirectory(getTestDir());
testAddMessages(new NullStorageManager(), new NIOSequentialFileFactory(getTestDirfile(), 1), 1000, false);
}
protected void testAddMessages(final StorageManager storageManager,
final SequentialFileFactory factory,
final int numberOfElements,
final boolean largeMessages) throws Exception {
SequentialFile file = factory.createSequentialFile("00010.page");
Page page = new Page(new SimpleString("something"), storageManager, factory, file, 10);
Assert.assertEquals(10, page.getPageId());
page.open(true);
Assert.assertEquals(1, factory.listFiles("page").size());
SimpleString simpleDestination = new SimpleString("Test");
final long startMessageID = 1;
addPageElements(storageManager, simpleDestination, page, numberOfElements, largeMessages, startMessageID);
LinkedList<PagedMessage> msgsBefore = page.getMessages();
page.sync();
page.close(false, false);
file = factory.createSequentialFile("00010.page");
page = new Page(new SimpleString("something"), storageManager, factory, file, 10);
LinkedList<PagedMessage> msgs = page.getMessages();
Assert.assertEquals(numberOfElements, msgs.size());
Assert.assertEquals(numberOfElements, page.getNumberOfMessages());
for (int i = 0; i < msgs.size(); i++) {
final PagedMessage pagedMessage = msgs.get(i);
Assert.assertEquals(simpleDestination, pagedMessage.getMessage().getAddressSimpleString());
Assert.assertEquals(largeMessages, pagedMessage.getMessage().isLargeMessage());
Assert.assertEquals(startMessageID + i, pagedMessage.getMessage().getMessageID());
Assert.assertEquals(largeMessages ? 1 : 0, pagedMessage.getMessage().getUsage());
}
for (int i = 0; i < msgs.size(); i++) {
final PagedMessage pagedMessage = msgsBefore.get(i);
Assert.assertEquals(simpleDestination, pagedMessage.getMessage().getAddressSimpleString());
Assert.assertEquals(largeMessages, pagedMessage.getMessage().isLargeMessage());
Assert.assertEquals(startMessageID + i, pagedMessage.getMessage().getMessageID());
Assert.assertEquals(largeMessages ? 1 : 0, pagedMessage.getMessage().getUsage());
}
Assert.assertTrue(page.delete(msgs));
try (LinkedListIterator<PagedMessage> iter = msgs.iterator()) {
while (iter.hasNext()) {
PagedMessage pagedMessage = iter.next();
Assert.assertEquals(0, pagedMessage.getMessage().getUsage());
}
}
Assert.assertEquals(0, factory.listFiles(".page").size());
}
/**
* @param simpleDestination
* @param page

View File

@ -18,7 +18,6 @@ package org.apache.activemq.artemis.tests.unit.core.paging.impl;
import java.io.File;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
@ -39,6 +38,7 @@ import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
import org.apache.activemq.artemis.core.settings.impl.HierarchicalObjectRepository;
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
import org.apache.activemq.artemis.utils.RandomUtil;
import org.apache.activemq.artemis.utils.collections.LinkedList;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@ -76,7 +76,7 @@ public class PagingManagerImplTest extends ActiveMQTestBase {
page.open(true);
List<PagedMessage> msgs = page.read(new NullStorageManager());
LinkedList<PagedMessage> msgs = page.read(new NullStorageManager());
page.close(false, false);

View File

@ -25,6 +25,7 @@ import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReentrantReadWriteLock;
@ -48,10 +49,14 @@ import org.apache.activemq.artemis.core.paging.cursor.PageCursorProvider;
import org.apache.activemq.artemis.core.paging.cursor.PageIterator;
import org.apache.activemq.artemis.core.paging.cursor.PageSubscription;
import org.apache.activemq.artemis.core.paging.cursor.PagedReference;
import org.apache.activemq.artemis.core.paging.cursor.PagedReferenceImpl;
import org.apache.activemq.artemis.core.paging.cursor.impl.PageCursorProviderAccessor;
import org.apache.activemq.artemis.core.paging.cursor.impl.PageCursorProviderImpl;
import org.apache.activemq.artemis.core.paging.impl.Page;
import org.apache.activemq.artemis.core.paging.impl.PageReadWriter;
import org.apache.activemq.artemis.core.paging.impl.PageTransactionInfoImpl;
import org.apache.activemq.artemis.core.paging.impl.PagingStoreImpl;
import org.apache.activemq.artemis.core.paging.impl.PagingStoreTestAccessor;
import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.core.persistence.impl.nullpm.NullStorageManager;
import org.apache.activemq.artemis.core.server.files.FileStoreMonitor;
@ -69,8 +74,11 @@ import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
import org.apache.activemq.artemis.utils.ActiveMQThreadFactory;
import org.apache.activemq.artemis.utils.ExecutorFactory;
import org.apache.activemq.artemis.utils.RandomUtil;
import org.apache.activemq.artemis.utils.ReusableLatch;
import org.apache.activemq.artemis.utils.Wait;
import org.apache.activemq.artemis.utils.actors.ArtemisExecutor;
import org.apache.activemq.artemis.utils.collections.LinkedList;
import org.apache.activemq.artemis.utils.collections.LinkedListIterator;
import org.jboss.logging.Logger;
import org.junit.After;
import org.junit.Assert;
@ -231,7 +239,7 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
page.open(true);
List<PagedMessage> msg = page.read(new NullStorageManager());
LinkedList<PagedMessage> msg = page.read(new NullStorageManager());
Assert.assertEquals(numMessages, msg.size());
Assert.assertEquals(1, storeImpl.getNumberOfPages());
@ -267,12 +275,15 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
PageSubscription subscription = storeImpl.getCursorProvider().createSubscription(1, null, true);
FakeQueue fakeQueue = new FakeQueue(destination, 1).setDurable(true).setPageSubscription(subscription);
storeImpl.getCursorProvider().disableCleanup();
storeImpl.start();
for (int repeat = 0; repeat < 5; repeat++) {
log.debug("###############################################################################################################################");
log.debug("#repeat " + repeat);
storeImpl.startPaging();
Assert.assertEquals(1, storeImpl.getNumberOfPages());
storeImpl.getCursorProvider().disableCleanup();
int numMessages = 100;
{
@ -290,6 +301,7 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
if (i > 0 && i % 10 == 0) {
storeImpl.forceAnotherPage();
page++;
Assert.assertEquals(page, storeImpl.getNumberOfPages());
}
}
}
@ -309,7 +321,11 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
}
iterator.close();
storeImpl.getCursorProvider().cleanup();
if (log.isDebugEnabled()) {
debugPage(storeImpl, subscription, storeImpl.getFirstPage(), storeImpl.getCurrentWritingPage());
}
PageCursorProviderAccessor.cleanup(storeImpl.getCursorProvider());
Assert.assertTrue(storeImpl.isPaging());
@ -320,9 +336,13 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
if (reference == null) {
break;
}
Assert.assertTrue(subscription.contains(reference));
log.debug("#received message " + messagesRead + ", " + reference);
messagesRead++;
int pageOnMsg = reference.getMessage().getIntProperty("page");
Assert.assertTrue(pageOnMsg <= 2 || pageOnMsg >= 10);
Assert.assertTrue("received " + reference, pageOnMsg <= 2 || pageOnMsg >= 10);
subscription.ack(reference);
}
@ -330,14 +350,36 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
Assert.assertEquals(30, messagesRead);
storeImpl.getCursorProvider().cleanup();
Assert.assertEquals(3, storeImpl.getNumberOfPages());
PageCursorProviderAccessor.cleanup(storeImpl.getCursorProvider());
Assert.assertFalse(storeImpl.isPaging());
Assert.assertEquals(1, PagingStoreTestAccessor.getUsedPagesSize(storeImpl));
Assert.assertEquals(1, storeImpl.getNumberOfPages());
}
}
private void debugPage(PagingStoreImpl storeImpl, PageSubscription subscription, long startPage, long endPage) throws Exception {
for (long pgID = startPage; pgID <= endPage; pgID++) {
Page page = storeImpl.newPageObject(pgID);
page.open(false);
log.debug("# Page " + pgID);
page.getMessages().forEach(p -> {
String acked;
try {
acked = subscription.contains(new PagedReferenceImpl(p, subscription)) + "...";
} catch (Exception e) {
e.printStackTrace();
acked = "";
}
log.debug(acked + p);
});
page.close(false);
}
}
@Test
public void testRemoveCurrentPage() throws Exception {
@ -356,7 +398,7 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
for (int repeat = 0; repeat < 5; repeat++) {
System.out.println("#repeat " + repeat);
log.debug("#repeat " + repeat);
storeImpl.startPaging();
@ -401,7 +443,7 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
Assert.assertEquals(7, messagesRead);
storeImpl.getCursorProvider().cleanup();
PageCursorProviderAccessor.cleanup(storeImpl.getCursorProvider());
Assert.assertEquals(10, factory.listFiles("page").size());
@ -411,7 +453,7 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
Assert.assertEquals(11, factory.listFiles("page").size());
storeImpl.getCursorProvider().cleanup();
PageCursorProviderAccessor.cleanup(storeImpl.getCursorProvider());
Assert.assertEquals(10, factory.listFiles("page").size());
@ -436,12 +478,61 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
Assert.assertEquals(90, messagesRead);
storeImpl.getCursorProvider().cleanup();
PageCursorProviderAccessor.cleanup(storeImpl.getCursorProvider());
Assert.assertFalse(storeImpl.isPaging());
}
}
@Test
public void testReadNumberOfMessages() throws Exception {
SequentialFileFactory factory = new NIOSequentialFileFactory(getTestDirfile(), 1);
SimpleString destination = new SimpleString("test");
PagingStoreFactory storeFactory = new FakeStoreFactory(factory);
StorageManager storageManager = createStorageManagerMock();
PagingStoreImpl storeImpl = new PagingStoreImpl(PagingStoreImplTest.destinationTestName, null, 100, createMockManager(), storageManager, factory, storeFactory, PagingStoreImplTest.destinationTestName, new AddressSettings().setAddressFullMessagePolicy(AddressFullMessagePolicy.PAGE), getExecutorFactory().getExecutor(), true);
PageSubscription subscription = storeImpl.getCursorProvider().createSubscription(1, null, true);
FakeQueue fakeQueue = new FakeQueue(destination, 1).setDurable(true).setPageSubscription(subscription);
storeImpl.getCursorProvider().disableCleanup();
storeImpl.start();
storeImpl.startPaging();
for (int i = 1; i <= 10; i++) {
ActiveMQBuffer buffer = createRandomBuffer(i + 1L, 10);
Message msg = createMessage(i, storeImpl, destination, buffer);
msg.putIntProperty("i", i);
msg.putIntProperty("page", 1);
final RoutingContextImpl ctx = new RoutingContextImpl(null);
ctx.addQueue(fakeQueue.getName(), fakeQueue);
Assert.assertTrue(storeImpl.page(msg, ctx.getTransaction(), ctx.getContextListing(storeImpl.getStoreName())));
}
Assert.assertEquals(1, storeImpl.getNumberOfPages());
Assert.assertEquals(1, factory.listFiles("page").size());
String fileName = storeImpl.createFileName(1);
ArrayList<PagedMessage> messages = new ArrayList<>();
SequentialFile file = factory.createSequentialFile(storeImpl.createFileName(1));
file.open();
int size = PageReadWriter.readFromSequentialFile(storageManager, storeImpl.getStoreName(), factory, file, 1, messages::add, PageReadWriter.SKIP_ALL, null, null);
file.close();
Assert.assertEquals(0, messages.size());
Assert.assertEquals(10, size);
}
@Test
public void testDepageMultiplePages() throws Exception {
SequentialFileFactory factory = new NIOSequentialFileFactory(new File(getPageDir()), 1).setDatasync(false);
@ -490,7 +581,7 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
page.open(true);
List<PagedMessage> msg = page.read(new NullStorageManager());
LinkedList<PagedMessage> msg = page.read(new NullStorageManager());
page.close(false, false);
@ -541,9 +632,11 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
Page page = store.depage();
Assert.assertNotNull(page);
page.open(true);
List<PagedMessage> msgs = page.read(new NullStorageManager());
LinkedList<PagedMessage> msgs = page.read(new NullStorageManager());
Assert.assertEquals(1, msgs.size());
@ -690,18 +783,22 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
for (Page page : readPages) {
page.open(true);
List<PagedMessage> msgs = page.read(new NullStorageManager());
LinkedList<PagedMessage> msgs = page.read(new NullStorageManager());
page.close(false, false);
for (PagedMessage msg : msgs) {
long id = msg.getMessage().toCore().getBodyBuffer().readLong();
msg.getMessage().toCore().getBodyBuffer().resetReaderIndex();
Message msgWritten = buffers.remove(id);
buffers2.put(id, msg.getMessage());
Assert.assertNotNull(msgWritten);
Assert.assertEquals(msg.getMessage().getAddress(), msgWritten.getAddress());
ActiveMQTestBase.assertEqualsBuffers(10, msgWritten.toCore().getBodyBuffer(), msg.getMessage().toCore().getBodyBuffer());
try (LinkedListIterator<PagedMessage> iter = msgs.iterator()) {
while (iter.hasNext()) {
PagedMessage msg = iter.next();
long id = msg.getMessage().toCore().getBodyBuffer().readLong();
msg.getMessage().toCore().getBodyBuffer().resetReaderIndex();
Message msgWritten = buffers.remove(id);
buffers2.put(id, msg.getMessage());
Assert.assertNotNull(msgWritten);
Assert.assertEquals(msg.getMessage().getAddress(), msgWritten.getAddress());
ActiveMQTestBase.assertEqualsBuffers(10, msgWritten.toCore().getBodyBuffer(), msg.getMessage().toCore().getBodyBuffer());
}
}
}
@ -721,7 +818,7 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
PagingStore storeImpl2 = new PagingStoreImpl(PagingStoreImplTest.destinationTestName, null, 100, createMockManager(), createStorageManagerMock(), factory, storeFactory, new SimpleString("test"), settings, getExecutorFactory().getExecutor(), true);
storeImpl2.start();
int numberOfPages = storeImpl2.getNumberOfPages();
long numberOfPages = storeImpl2.getNumberOfPages();
Assert.assertTrue(numberOfPages != 0);
storeImpl2.startPaging();
@ -750,22 +847,21 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
page.open(true);
List<PagedMessage> msgs = page.read(new NullStorageManager());
LinkedList<PagedMessage> msgs = page.read(new NullStorageManager());
page.close(false, false);
for (PagedMessage msg : msgs) {
msgs.forEach((msg) -> {
long id = msg.getMessage().toCore().getBodyBuffer().readLong();
Message msgWritten = buffers2.remove(id);
Assert.assertNotNull(msgWritten);
Assert.assertEquals(msg.getMessage().getAddress(), msgWritten.getAddress());
ActiveMQTestBase.assertEqualsByteArrays(msgWritten.toCore().getBodyBuffer().writerIndex(), msgWritten.toCore().getBodyBuffer().toByteBuffer().array(), msg.getMessage().toCore().getBodyBuffer().toByteBuffer().array());
}
});
}
lastPage.open(true);
List<PagedMessage> lastMessages = lastPage.read(new NullStorageManager());
LinkedList<PagedMessage> lastMessages = lastPage.read(new NullStorageManager());
lastPage.close(false, false);
Assert.assertEquals(1, lastMessages.size());
@ -812,43 +908,38 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
@Test
public void testOrderOnPaging() throws Throwable {
clearDataRecreateServerDirs();
SequentialFileFactory factory = new NIOSequentialFileFactory(new File(getPageDir()), 1).setDatasync(false);
ExecutorService executorService = Executors.newFixedThreadPool(2);
try {
clearDataRecreateServerDirs();
SequentialFileFactory factory = new FakeSequentialFileFactory();
PagingStoreFactory storeFactory = new FakeStoreFactory(factory);
PagingStoreFactory storeFactory = new FakeStoreFactory(factory);
final int MAX_SIZE = 1024 * 10;
final int MAX_SIZE = 1024 * 10;
AddressSettings settings = new AddressSettings().setPageSizeBytes(MAX_SIZE).setAddressFullMessagePolicy(AddressFullMessagePolicy.PAGE);
AddressSettings settings = new AddressSettings().setPageSizeBytes(MAX_SIZE).setAddressFullMessagePolicy(AddressFullMessagePolicy.PAGE);
final PagingStore store = new PagingStoreImpl(PagingStoreImplTest.destinationTestName, null, 100, createMockManager(), createStorageManagerMock(), factory, storeFactory, new SimpleString("test"), settings, getExecutorFactory().getExecutor(), false);
final PagingStore store = new PagingStoreImpl(PagingStoreImplTest.destinationTestName, null, 100, createMockManager(), createStorageManagerMock(), factory, storeFactory, new SimpleString("test"), settings, getExecutorFactory().getExecutor(), false);
store.start();
store.start();
Assert.assertEquals(0, store.getNumberOfPages());
Assert.assertEquals(0, store.getNumberOfPages());
// Marked the store to be paged
store.startPaging();
// Marked the store to be paged
store.startPaging();
final CountDownLatch producedLatch = new CountDownLatch(1);
Assert.assertEquals(1, store.getNumberOfPages());
Assert.assertEquals(1, store.getNumberOfPages());
final SimpleString destination = new SimpleString("test");
final SimpleString destination = new SimpleString("test");
final long NUMBER_OF_MESSAGES = 10000;
final long NUMBER_OF_MESSAGES = 100000;
final List<Throwable> errors = new ArrayList<>();
final List<Throwable> errors = new ArrayList<>();
class WriterThread extends Thread {
WriterThread() {
super("PageWriter");
}
@Override
public void run() {
ReusableLatch done = new ReusableLatch(0);
done.countUp();
executorService.execute(() -> {
try {
for (long i = 0; i < NUMBER_OF_MESSAGES; i++) {
// Each thread will Keep paging until all the messages are depaged.
@ -858,77 +949,62 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
msg.putLongProperty("count", i);
final RoutingContextImpl ctx2 = new RoutingContextImpl(null);
while (!store.page(msg, ctx2.getTransaction(), ctx2.getContextListing(store.getStoreName()))) {
store.startPaging();
}
if (i == 0) {
producedLatch.countDown();
}
store.page(msg, ctx2.getTransaction(), ctx2.getContextListing(store.getStoreName()));
}
} catch (Throwable e) {
e.printStackTrace();
errors.add(e);
} finally {
done.countDown();
}
}
}
});
class ReaderThread extends Thread {
Assert.assertTrue(done.await(10, TimeUnit.SECONDS));
done.countUp();
ReaderThread() {
super("PageReader");
}
@Override
public void run() {
executorService.execute(() -> {
try {
AtomicInteger msgsRead = new AtomicInteger(0);
long msgsRead = 0;
while (msgsRead < NUMBER_OF_MESSAGES) {
while (msgsRead.get() < NUMBER_OF_MESSAGES) {
Page page = store.depage();
AtomicInteger countOnPage = new AtomicInteger(0);
if (page != null) {
page.open(true);
List<PagedMessage> messages = page.read(new NullStorageManager());
for (PagedMessage pgmsg : messages) {
LinkedList<PagedMessage> messages = page.read(new NullStorageManager());
messages.forEach(pgmsg -> {
Assert.assertEquals(countOnPage.getAndIncrement(), pgmsg.getMessageNumber());
Message msg = pgmsg.getMessage();
Assert.assertEquals(msgsRead++, msg.getMessageID());
Assert.assertEquals(msgsRead.getAndIncrement(), msg.getMessageID());
Assert.assertEquals(msg.getMessageID(), msg.getLongProperty("count").longValue());
}
});
page.close(false, false);
page.delete(null);
} else {
log.debug("Depaged!!!! numerOfMessages = " + msgsRead + " of " + NUMBER_OF_MESSAGES);
Thread.sleep(500);
}
}
} catch (Throwable e) {
e.printStackTrace();
errors.add(e);
} finally {
done.countDown();
}
});
Assert.assertTrue(done.await(10, TimeUnit.SECONDS));
store.stop();
for (Throwable e : errors) {
throw e;
}
}
WriterThread producerThread = new WriterThread();
producerThread.start();
ReaderThread consumer = new ReaderThread();
consumer.start();
producerThread.join();
consumer.join();
store.stop();
for (Throwable e : errors) {
throw e;
} finally {
executorService.shutdownNow();
}
}
@Test
public void testWriteIncompletePage() throws Exception {
clearDataRecreateServerDirs();
@ -978,20 +1054,20 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
storeImpl.stop();
storeImpl.start();
long msgsRead = 0;
AtomicLong msgsRead = new AtomicLong(0);
while (msgsRead < num1 + num2) {
while (msgsRead.get() < num1 + num2) {
page = storeImpl.depage();
assertNotNull("no page after read " + msgsRead + " msg", page);
page.open(true);
List<PagedMessage> messages = page.read(new NullStorageManager());
LinkedList<PagedMessage> messages = page.read(new NullStorageManager());
for (PagedMessage pgmsg : messages) {
messages.forEach(pgmsg -> {
Message msg = pgmsg.getMessage();
Assert.assertEquals(msgsRead, msg.getMessageID());
Assert.assertEquals(msgsRead.longValue(), msg.getMessageID());
Assert.assertEquals(msg.getMessageID(), msg.getLongProperty("count").longValue());
msgsRead++;
}
msgsRead.incrementAndGet();
});
page.close(false);
}

View File

@ -0,0 +1,50 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.activemq.artemis.tests.unit.util;
import java.util.Iterator;
import java.util.NoSuchElementException;
import org.apache.activemq.artemis.utils.collections.EmptyList;
import org.apache.activemq.artemis.utils.collections.LinkedList;
import org.junit.Assert;
import org.junit.Test;
public class EmptyListTest {
@Test
public void testEmpty() {
LinkedList<String> stringEmpty = EmptyList.getEmptyList();
Assert.assertEquals(0, stringEmpty.size());
Iterator<String> stringIterator = stringEmpty.iterator();
Assert.assertFalse(stringIterator.hasNext());
try {
stringIterator.next();
Assert.fail("Exception expected");
} catch (NoSuchElementException e) {
}
try {
stringEmpty.get(0);
Assert.fail("Exception expected");
} catch (IndexOutOfBoundsException e) {
}
}
}

Some files were not shown because too many files have changed in this diff Show More