ARTEMIS-3243 Implementing dual mirror with Broker Connections

There are some major tests added as part of this PR.

This PR has been done through an extensive collaboration with Robbie Gemmel on https://github.com/apache/activemq-artemis/pull/3633
This commit is contained in:
Clebert Suconic 2021-08-03 16:50:43 -04:00
parent 043d7eabd7
commit 813ed88ecb
94 changed files with 7704 additions and 729 deletions

View File

@ -16,8 +16,6 @@
*/
package org.apache.activemq.artemis.utils.collections;
import java.util.function.ToLongFunction;
public interface LinkedList<E> {
void addHead(E e);
@ -34,10 +32,9 @@ public interface LinkedList<E> {
void clearID();
/** The ID Supplier function needs to return positive IDs (greater or equal to 0)
* If you spply a negative ID, it will be considered a null value, and
* the value will just be ignored. */
void setIDSupplier(ToLongFunction<E> supplier);
/** this makes possibl to use {@link #removeWithID(String, long)} */
void setNodeStore(NodeStore<E> store);
E removeWithID(long id);
/** you need to call {@link #setNodeStore(NodeStore)} before you are able to call this method. */
E removeWithID(String listID, long id);
}

View File

@ -20,9 +20,6 @@ import java.lang.reflect.Array;
import java.util.Comparator;
import java.util.NoSuchElementException;
import java.util.Objects;
import java.util.function.ToLongFunction;
import io.netty.util.collection.LongObjectHashMap;
/**
* A linked list implementation which allows multiple iterators to exist at the same time on the queue, and which see any
@ -36,14 +33,13 @@ public class LinkedListImpl<E> implements LinkedList<E> {
private final Node<E> head = new NodeHolder<>(null);
private final Comparator<E> comparator;
LongObjectHashMap<Node<E>> nodeMap;
private Node<E> tail = null;
private int size;
// We store in an array rather than a Map for the best performance
private volatile Iterator[] iters;
private int numIters;
private int nextIndex;
private ToLongFunction<E> idSupplier;
private NodeStore<E> nodeStore;
public LinkedListImpl() {
this(null, null);
@ -53,30 +49,23 @@ public class LinkedListImpl<E> implements LinkedList<E> {
this(comparator, null);
}
public LinkedListImpl(Comparator<E> comparator, ToLongFunction<E> supplier) {
public LinkedListImpl(Comparator<E> comparator, NodeStore<E> supplier) {
iters = createIteratorArray(INITIAL_ITERATOR_ARRAY_SIZE);
this.comparator = comparator;
this.idSupplier = supplier;
if (idSupplier != null) {
this.nodeMap = newLongHashMap();
} else {
this.nodeMap = null;
}
this.nodeStore = supplier;
}
@Override
public void clearID() {
idSupplier = null;
if (nodeMap != null) {
nodeMap.clear();
nodeMap = null;
if (nodeStore != null) {
nodeStore.clear();
}
nodeStore = null;
}
@Override
public void setIDSupplier(ToLongFunction<E> supplier) {
this.idSupplier = supplier;
nodeMap = newLongHashMap();
public void setNodeStore(NodeStore<E> supplier) {
this.nodeStore = supplier;
try (Iterator iterator = (Iterator) iterator()) {
while (iterator.hasNext()) {
@ -87,15 +76,8 @@ public class LinkedListImpl<E> implements LinkedList<E> {
}
}
private LongObjectHashMap<Node<E>> newLongHashMap() {
return new LongObjectHashMap<>(Math.max(8, this.size));
}
private void putID(E value, Node<E> position) {
long id = idSupplier.applyAsLong(value);
if (id >= 0) {
nodeMap.put(id, position);
}
private void putID(E element, Node<E> node) {
nodeStore.storeNode(element, node);
}
@Override
@ -121,12 +103,11 @@ public class LinkedListImpl<E> implements LinkedList<E> {
}
@Override
public E removeWithID(long id) {
if (nodeMap == null) {
return null;
}
public E removeWithID(String listID, long id) {
assert nodeStore != null; // it is assumed the code will call setNodeStore before callin removeWithID
Node<E> node = nodeStore.getNode(listID, id);
Node<E> node = nodeMap.get(id);
if (node == null) {
return null;
}
@ -137,17 +118,14 @@ public class LinkedListImpl<E> implements LinkedList<E> {
}
private void itemAdded(Node<E> node, E item) {
if (nodeMap != null) {
if (nodeStore != null) {
putID(item, node);
}
}
private void itemRemoved(Node<E> node) {
if (nodeMap != null) {
long id = idSupplier.applyAsLong(node.val());
if (id >= 0) {
nodeMap.remove(id);
}
if (nodeStore != null) {
nodeStore.removeNode(node.val(), node);
}
}
@ -254,13 +232,6 @@ public class LinkedListImpl<E> implements LinkedList<E> {
return size;
}
/**
* Return the number of elements we have on suppliedIDs
*/
public int getSizeOfSuppliedIDs() {
return nodeMap == null ? 0 : nodeMap.size();
}
@Override
public LinkedListIterator<E> iterator() {
return new Iterator();

View File

@ -0,0 +1,40 @@
/*
* 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;
/**
* This interface is meant to encapsulate the usage of {@code HashMap<ListID, LongObjectHashMap<ElementType>>}
* The implementation should store the node in such way that you can access it through {@link #getNode(String, long)}
*/
public interface NodeStore<E> {
/** When you store the node, make sure you find what is the ID and ListID for the element you are storing
* as later one you will need to provide the node based on list and id as specified on {@link #getNode(String, long)} */
void storeNode(E element, LinkedListImpl.Node<E> node);
LinkedListImpl.Node<E> getNode(String listID, long id);
void removeNode(E element, LinkedListImpl.Node<E> node);
/** this is meant to be a quick help to Garbage Collection.
* Whenever the IDSupplier list is being cleared, you should first call the clear method and
* empty every list before you let the instance go. */
void clear();
/** ths should return the sum of all the sizes. for test assertions. */
int size();
}

View File

@ -16,8 +16,6 @@
*/
package org.apache.activemq.artemis.utils.collections;
import java.util.function.ToLongFunction;
/**
* A type of linked list which maintains items according to a priority
* and allows adding and removing of elements at both ends, and peeking.<br>
@ -36,12 +34,12 @@ public interface PriorityLinkedList<E> {
void clear();
/**
* @see LinkedList#setIDSupplier(ToLongFunction)
* @see LinkedList#setNodeStore(NodeStore)
* @param supplier
*/
void setIDSupplier(ToLongFunction<E> supplier);
void setNodeStore(NodeStore<E> supplier);
E removeWithID(long id);
E removeWithID(String listID, long id);
/**
* Returns the size of this list.<br>

View File

@ -20,7 +20,6 @@ import java.lang.reflect.Array;
import java.util.Comparator;
import java.util.NoSuchElementException;
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
import java.util.function.ToLongFunction;
/**
* A priority linked list implementation
@ -97,18 +96,18 @@ public class PriorityLinkedListImpl<E> implements PriorityLinkedList<E> {
}
@Override
public void setIDSupplier(ToLongFunction<E> supplier) {
public void setNodeStore(NodeStore<E> supplier) {
for (LinkedList<E> list : levels) {
list.setIDSupplier(supplier);
list.setNodeStore(supplier);
}
}
@Override
public E removeWithID(long id) {
public E removeWithID(String listID, long id) {
// we start at 4 just as an optimization, since most times we only use level 4 as the level on messages
if (levels.length > 4) {
for (int l = 4; l < levels.length; l++) {
E removed = levels[l].removeWithID(id);
E removed = levels[l].removeWithID(listID, id);
if (removed != null) {
exclusiveIncrementSize(-1);
return removed;
@ -117,7 +116,7 @@ public class PriorityLinkedListImpl<E> implements PriorityLinkedList<E> {
}
for (int l = Math.min(3, levels.length); l >= 0; l--) {
E removed = levels[l].removeWithID(id);
E removed = levels[l].removeWithID(listID, id);
if (removed != null) {
exclusiveIncrementSize(-1);
return removed;

View File

@ -351,32 +351,48 @@ public class ByteUtilTest {
}
}
@Test
public void testIntToByte() {
for (int i = 0; i < 1000; i++) {
int randomInt = RandomUtil.randomInt();
byte[] expected = ByteBuffer.allocate(4).putInt(randomInt).array();
public void testIntToBytes() {
internalIntToBytesTest(RandomUtil.randomInt(), null);
internalIntToBytesTest(0, new byte[]{0, 0, 0, 0});
internalIntToBytesTest(-1, new byte[] {(byte)0xFF, (byte)0xFF, (byte)0xFF, (byte)0xFF});
internalIntToBytesTest(Integer.MIN_VALUE, new byte[] {(byte)0x80, (byte)0x00, (byte)0x00, (byte)0x00});
internalIntToBytesTest(Integer.MAX_VALUE, new byte[] {(byte)0x7F, (byte)0xFF, (byte)0xFF, (byte)0xFF});
}
byte[] actual = ByteUtil.intToBytes(randomInt);
assertArrayEquals(expected, actual);
assertEquals(randomInt, ByteUtil.bytesToInt(expected));
assertEquals(randomInt, ByteUtil.bytesToInt(actual));
private void internalIntToBytesTest(int intValue, byte[] manualExpect) {
byte[] expected = ByteBuffer.allocate(4).putInt(intValue).array();
byte[] actual = ByteUtil.intToBytes(intValue);
if (manualExpect != null) {
Assert.assertEquals(4, manualExpect.length);
assertArrayEquals(manualExpect, actual);
}
assertArrayEquals(expected, actual);
assertEquals(intValue, ByteUtil.bytesToInt(expected));
assertEquals(intValue, ByteUtil.bytesToInt(actual));
}
@Test
public void testLongToBytes() {
ByteBuffer buffer = ByteBuffer.allocate(8);
long randomLong = RandomUtil.randomLong();
buffer.putLong(randomLong);
byte[] longArrayAssert = buffer.array();
internalLongToBytesTest(RandomUtil.randomLong(), null);
internalLongToBytesTest(0, new byte[] {0, 0, 0, 0, 0, 0, 0, 0});
internalLongToBytesTest(-1, new byte[] {(byte)0xFF, (byte)0xFF, (byte)0xFF, (byte)0xFF, (byte)0xFF, (byte)0xFF, (byte)0xFF, (byte)0xFF});
internalLongToBytesTest(Long.MIN_VALUE, new byte[] {(byte)0x80, (byte)0x00, (byte)0x00, (byte)0x00, (byte)0x00, (byte)0x00, (byte)0x00, (byte)0x00});
internalLongToBytesTest(Long.MAX_VALUE, new byte[] {(byte)0x7F, (byte)0xFF, (byte)0xFF, (byte)0xFF, (byte)0xFF, (byte)0xFF, (byte)0xFF, (byte)0xFF});
}
byte[] convertedArray = ByteUtil.longToBytes(randomLong);
private void internalLongToBytesTest(long longValue, byte[] manualExpected) {
byte[] expected = ByteBuffer.allocate(8).putLong(longValue).array();
byte[] actual = ByteUtil.longToBytes(longValue);
if (manualExpected != null) {
Assert.assertEquals(8, manualExpected.length);
assertArrayEquals(manualExpected, actual);
}
assertArrayEquals(expected, actual);
assertArrayEquals(longArrayAssert, convertedArray);
assertEquals(longValue, ByteUtil.bytesToLong(expected));
assertEquals(longValue, ByteUtil.bytesToLong(actual));
}
@Test

View File

@ -110,7 +110,7 @@ public class Wait {
assertTrue(DEFAULT_FAILURE_MESSAGE, condition);
}
public static void assertFalse(Condition condition) throws Exception {
public static void assertFalse(Condition condition) {
assertTrue(() -> !condition.isSatisfied());
}
@ -159,7 +159,7 @@ public class Wait {
public static boolean waitFor(final Condition condition,
final long durationMillis,
final long sleepMillis) {
return waitFor(condition, durationMillis, sleepMillis, true);
return waitFor(condition, durationMillis, sleepMillis, false);
}
public static boolean waitFor(final Condition condition,

View File

@ -0,0 +1,52 @@
/*
* 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.uri;
import java.net.URI;
import java.util.List;
import org.apache.activemq.artemis.api.core.TransportConfiguration;
import org.apache.activemq.artemis.uri.ConnectorTransportConfigurationParser;
import org.jboss.logging.Logger;
import org.junit.Assert;
import org.junit.Test;
public class ConnectorTransportConfigurationParserURITest {
private static final Logger logger = Logger.getLogger(ConnectorTransportConfigurationParserURITest.class);
@Test
public void testParse() throws Exception {
ConnectorTransportConfigurationParser parser = new ConnectorTransportConfigurationParser(false);
URI transportURI = parser.expandURI("tcp://live:1#tcp://backupA:2,tcp://backupB:3");
System.out.println(transportURI);
List<TransportConfiguration> objects = parser.newObject(transportURI, "test");
if (logger.isInfoEnabled()) {
objects.forEach((t) -> logger.info("transportConfig:" + t.toString()));
}
Assert.assertEquals(3, objects.size());
Assert.assertEquals("live", objects.get(0).getParams().get("host"));
Assert.assertEquals("1", objects.get(0).getParams().get("port"));
Assert.assertEquals("backupA", objects.get(1).getParams().get("host"));
Assert.assertEquals("2", objects.get(1).getParams().get("port"));
Assert.assertEquals("backupB", objects.get(2).getParams().get("host"));
Assert.assertEquals("3", objects.get(2).getParams().get("port"));
}
}

View File

@ -29,7 +29,6 @@ export TEST_TARGET="./target"
. ./installHome.sh
cd $ARTEMIS_HOME/examples/features/standard/
cd auto-closeable; mvn verify; cd ..
cd broker-plugin; mvn verify; cd ..
cd browser; mvn verify; cd ..
@ -57,10 +56,7 @@ cd interceptor-mqtt; mvn verify; cd ..
cd jms-bridge; mvn verify; cd ..
cd jmx; mvn verify; cd ..
cd jmx-ssl; mvn verify; cd ..
# too big for most CI machines
#cd large-message; mvn verify; cd ..
#cd large-message; mvn verify; cd .. # too big for most CI machines, keeping it out
cd last-value-queue; mvn verify; cd ..
cd management; mvn verify; cd ..
cd management-notifications; mvn verify; cd ..
@ -100,10 +96,7 @@ cd xa-heuristic; mvn verify; cd ..
cd xa-receive; mvn verify; cd ..
cd xa-send; mvn verify; cd ..
cd $ARTEMIS_HOME/examples/features/clustered/
cd client-side-load-balancing; mvn verify; cd ..
cd clustered-durable-subscription; mvn verify; cd ..
cd clustered-grouping; mvn verify; cd ..
@ -119,10 +112,7 @@ cd symmetric-cluster; mvn verify; cd ..
cd shared-storage-static-cluster; mvn verify; cd ..
# TODO: these will hung eventually when ran in series
cd $ARTEMIS_HOME/examples/features/ha/
cd application-layer-failover; mvn verify; cd ..
cd client-side-failoverlistener; mvn verify; cd ..
cd colocated-failover; mvn verify; cd ..
@ -133,39 +123,26 @@ cd multiple-failover-failback; mvn verify; cd ..
cd non-transaction-failover; mvn verify; cd ..
cd replicated-failback; mvn verify; cd ..
cd replicated-failback-static; mvn verify; cd ..
cd replicated-multiple-failover; mvn verify; cd ..
cd replicated-transaction-failover; mvn verify; cd ..
cd scale-down; mvn verify; cd ..
cd transaction-failover; mvn verify; cd ..
cd $ARTEMIS_HOME/examples/protocols/amqp/
cd queue; mvn verify; cd ..
cd $ARTEMIS_HOME/examples/protocols/mqtt/
cd clustered-queue-mqtt; mvn verify; cd ..
cd publish-subscribe; mvn verify; cd ..
cd $ARTEMIS_HOME/examples/protocols/openwire/
cd queue; mvn verify; cd ..
cd message-listener; mvn verify; cd ..
cd message-recovery; mvn verify; cd ..
cd virtual-topic-mapping; mvn verify; cd ..
cd $ARTEMIS_HOME/examples/protocols/stomp/
cd stomp; mvn verify; cd ..
cd stomp1.1; mvn verify; cd ..
cd stomp1.2; mvn verify; cd ..
@ -173,6 +150,11 @@ cd stomp-dual-authentication; mvn verify; cd ..
cd stomp-embedded-interceptor; mvn verify; cd ..
cd stomp-jms; mvn verify; cd ..
cd $ARTEMIS_HOME/examples/features/broker-connection
cd amqp-receiving-messages; mvn verify; cd ..
cd amqp-sending-messages; mvn verify; cd ..
cd amqp-sending-overssl; mvn verify; cd ..
cd disaster-recovery; mvn verify; cd ..
cd $CURRENT_DIR
rm -rf target

View File

@ -33,6 +33,7 @@ import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.management.Notification;
import org.apache.activemq.artemis.core.server.management.NotificationListener;
import org.apache.activemq.artemis.protocol.amqp.client.ProtonClientProtocolManager;
import org.apache.activemq.artemis.protocol.amqp.connect.mirror.ReferenceNodeStore;
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPConnectionContext;
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPConstants;
import org.apache.activemq.artemis.protocol.amqp.proton.AmqpSupport;
@ -63,8 +64,16 @@ public class ProtonProtocolManager extends AbstractProtocolManager<AMQPMessage,
private final List<AmqpInterceptor> incomingInterceptors = new ArrayList<>();
private final List<AmqpInterceptor> outgoingInterceptors = new ArrayList<>();
public static String getMirrorAddress(String connectionName) {
return MIRROR_ADDRESS + "_" + connectionName;
}
private final ActiveMQServer server;
// We must use one referenceIDSupplier per server.
// protocol manager is the perfect aggregation for that.
private ReferenceNodeStore referenceIDSupplier;
private final ProtonProtocolManagerFactory factory;
private final Map<SimpleString, RoutingType> prefixes = new HashMap<>();
@ -111,6 +120,15 @@ public class ProtonProtocolManager extends AbstractProtocolManager<AMQPMessage,
this.updateInterceptors(incomingInterceptors, outgoingInterceptors);
}
public synchronized ReferenceNodeStore getReferenceIDSupplier() {
if (referenceIDSupplier == null) {
// we lazy start the instance.
// only create it when needed
referenceIDSupplier = new ReferenceNodeStore(server);
}
return referenceIDSupplier;
}
public ActiveMQServer getServer() {
return server;
}

View File

@ -20,6 +20,7 @@ package org.apache.activemq.artemis.protocol.amqp.connect;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@ -27,6 +28,7 @@ import java.util.concurrent.Executor;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Stream;
import org.apache.activemq.artemis.api.core.ActiveMQException;
@ -59,6 +61,7 @@ import org.apache.activemq.artemis.protocol.amqp.broker.ProtonProtocolManager;
import org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerAggregation;
import org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource;
import org.apache.activemq.artemis.protocol.amqp.logger.ActiveMQAMQPProtocolLogger;
import org.apache.activemq.artemis.protocol.amqp.logger.ActiveMQAMQPProtocolMessageBundle;
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPSessionContext;
import org.apache.activemq.artemis.protocol.amqp.proton.ProtonServerSenderContext;
import org.apache.activemq.artemis.protocol.amqp.proton.SenderController;
@ -77,6 +80,8 @@ import org.apache.qpid.proton.amqp.messaging.Source;
import org.apache.qpid.proton.amqp.messaging.Target;
import org.apache.qpid.proton.amqp.transport.ReceiverSettleMode;
import org.apache.qpid.proton.amqp.transport.SenderSettleMode;
import org.apache.qpid.proton.engine.EndpointState;
import org.apache.qpid.proton.engine.Link;
import org.apache.qpid.proton.engine.Receiver;
import org.apache.qpid.proton.engine.Sender;
import org.apache.qpid.proton.engine.Session;
@ -96,7 +101,9 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
private ActiveMQProtonRemotingConnection protonRemotingConnection;
private volatile boolean started = false;
private final AMQPBrokerConnectionManager bridgeManager;
private AMQPMirrorControllerSource mirrorControllerSource;
private int retryCounter = 0;
private int lastRetryCounter;
private boolean connecting = false;
private volatile ScheduledFuture reconnectFuture;
private final Set<Queue> senders = new HashSet<>();
@ -141,6 +148,10 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
return started;
}
public boolean isConnecting() {
return connecting;
}
@Override
public void stop() {
if (!started) return;
@ -167,7 +178,7 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
if (brokerConnectConfiguration != null && brokerConnectConfiguration.getConnectionElements() != null) {
for (AMQPBrokerConnectionElement connectionElement : brokerConnectConfiguration.getConnectionElements()) {
if (connectionElement.getType() == AMQPBrokerConnectionAddressType.MIRROR) {
installMirrorController(this, (AMQPMirrorBrokerConnectionElement) connectionElement, server);
installMirrorController((AMQPMirrorBrokerConnectionElement) connectionElement, server);
}
}
}
@ -205,11 +216,12 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
public void createLink(Queue queue, AMQPBrokerConnectionElement connectionElement) {
if (connectionElement.getType() == AMQPBrokerConnectionAddressType.PEER) {
connectSender(queue, queue.getAddress().toString(), null, Symbol.valueOf("qd.waypoint"));
connectReceiver(protonRemotingConnection, session, sessionContext, queue, Symbol.valueOf("qd.waypoint"));
Symbol[] dispatchCapability = new Symbol[]{AMQPMirrorControllerSource.QPID_DISPATCH_WAYPOINT_CAPABILITY};
connectSender(queue, queue.getAddress().toString(), null, null, null, null, dispatchCapability);
connectReceiver(protonRemotingConnection, session, sessionContext, queue, dispatchCapability);
} else {
if (connectionElement.getType() == AMQPBrokerConnectionAddressType.SENDER) {
connectSender(queue, queue.getAddress().toString(), null);
connectSender(queue, queue.getAddress().toString(), null, null, null, null, null);
}
if (connectionElement.getType() == AMQPBrokerConnectionAddressType.RECEIVER) {
connectReceiver(protonRemotingConnection, session, sessionContext, queue);
@ -217,13 +229,28 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
}
}
SimpleString getMirrorSNF(AMQPMirrorBrokerConnectionElement mirrorElement) {
SimpleString snf = mirrorElement.getMirrorSNF();
if (snf == null) {
snf = SimpleString.toSimpleString(ProtonProtocolManager.getMirrorAddress(this.brokerConnectConfiguration.getName()));
mirrorElement.setMirrorSNF(snf);
}
return snf;
}
private void linkClosed(Link link) {
if (link.getLocalState() == EndpointState.ACTIVE) {
error(ActiveMQAMQPProtocolMessageBundle.BUNDLE.brokerConnectionRemoteLinkClosed(), lastRetryCounter);
}
}
private void doConnect() {
try {
connecting = true;
List<TransportConfiguration> configurationList = brokerConnectConfiguration.getTransportConfigurations();
TransportConfiguration tpConfig = configurationList.get(0);
TransportConfiguration tpConfig = configurationList.get(retryCounter % configurationList.size());
String hostOnParameter = ConfigurationHelper.getStringProperty(TransportConstants.HOST_PROP_NAME, TransportConstants.DEFAULT_HOST, tpConfig.getParams());
int portOnParameter = ConfigurationHelper.getIntProperty(TransportConstants.PORT_PROP_NAME, TransportConstants.DEFAULT_PORT, tpConfig.getParams());
@ -236,10 +263,12 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
return;
}
int currentRetryCounter = retryCounter;
reconnectFuture = null;
lastRetryCounter = retryCounter;
retryCounter = 0;
reconnectFuture = null;
// before we retry the connection we need to remove any previous links
// as they will need to be recreated
senders.clear();
@ -250,6 +279,7 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
ConnectionEntry entry = protonProtocolManager.createOutgoingConnectionEntry(connection, saslFactory);
server.getRemotingService().addConnectionEntry(connection, entry);
protonRemotingConnection = (ActiveMQProtonRemotingConnection) entry.connection;
protonRemotingConnection.getAmqpConnection().setLinkCloseListener(this::linkClosed);
connection.getChannel().pipeline().addLast(new AMQPBrokerConnectionChannelHandler(bridgesConnector.getChannelGroup(), protonRemotingConnection.getAmqpConnection().getHandler(), this, server.getExecutorFactory().getExecutor()));
@ -277,9 +307,11 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
for (AMQPBrokerConnectionElement connectionElement : brokerConnectConfiguration.getConnectionElements()) {
if (connectionElement.getType() == AMQPBrokerConnectionAddressType.MIRROR) {
AMQPMirrorBrokerConnectionElement replica = (AMQPMirrorBrokerConnectionElement)connectionElement;
Queue queue = server.locateQueue(replica.getSourceMirrorAddress());
connectSender(queue, ProtonProtocolManager.MIRROR_ADDRESS, (r) -> AMQPMirrorControllerSource.validateProtocolData(r, replica.getSourceMirrorAddress()));
Queue queue = server.locateQueue(getMirrorSNF(replica));
connectSender(queue, queue.getName().toString(), mirrorControllerSource::setLink, (r) -> AMQPMirrorControllerSource.validateProtocolData(protonProtocolManager.getReferenceIDSupplier(), r, getMirrorSNF(replica)), server.getNodeID().toString(),
new Symbol[]{AMQPMirrorControllerSource.MIRROR_CAPABILITY}, null);
}
}
}
@ -288,7 +320,7 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
bridgeManager.connected(connection, this);
ActiveMQAMQPProtocolLogger.LOGGER.successReconnect(brokerConnectConfiguration.getName(), host + ":" + port, currentRetryCounter);
ActiveMQAMQPProtocolLogger.LOGGER.successReconnect(brokerConnectConfiguration.getName(), host + ":" + port, lastRetryCounter);
connecting = false;
} catch (Throwable e) {
@ -297,6 +329,7 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
}
public void retryConnection() {
lastRetryCounter = retryCounter;
if (bridgeManager.isStarted() && started) {
if (brokerConnectConfiguration.getReconnectAttempts() < 0 || retryCounter < brokerConnectConfiguration.getReconnectAttempts()) {
retryCounter++;
@ -306,8 +339,10 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
}
reconnectFuture = scheduledExecutorService.schedule(() -> connectExecutor.execute(() -> doConnect()), brokerConnectConfiguration.getRetryInterval(), TimeUnit.MILLISECONDS);
} else {
retryCounter = 0;
started = false;
connecting = false;
ActiveMQAMQPProtocolLogger.LOGGER.retryConnectionFailed(brokerConnectConfiguration.getName(), host + ":" + port, retryCounter);
ActiveMQAMQPProtocolLogger.LOGGER.retryConnectionFailed(brokerConnectConfiguration.getName(), host + ":" + port, lastRetryCounter);
if (logger.isDebugEnabled()) {
logger.debug("no more reconnections as the retry counter reached " + retryCounter + " out of " + brokerConnectConfiguration.getReconnectAttempts());
}
@ -319,18 +354,13 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
// TODO implement this as part of https://issues.apache.org/jira/browse/ARTEMIS-2965
}
/** The reason this method is static is the following:
*
* It is returning the snfQueue to the replica, and I needed isolation from the actual instance.
* During development I had a mistake where I used a property from the Object,
* so, I needed this isolation for my organization and making sure nothing would be shared. */
private static Queue installMirrorController(AMQPBrokerConnection brokerConnection, AMQPMirrorBrokerConnectionElement replicaConfig, ActiveMQServer server) throws Exception {
private Queue installMirrorController(AMQPMirrorBrokerConnectionElement replicaConfig, ActiveMQServer server) throws Exception {
MirrorController currentMirrorController = server.getMirrorController();
// This following block is to avoid a duplicate on mirror controller
if (currentMirrorController != null && currentMirrorController instanceof AMQPMirrorControllerSource) {
Queue queue = checkCurrentMirror(brokerConnection, (AMQPMirrorControllerSource)currentMirrorController);
Queue queue = checkCurrentMirror(this, (AMQPMirrorControllerSource) currentMirrorController);
// on this case we already had a mirror installed before, we won't duplicate it
if (queue != null) {
return queue;
@ -339,7 +369,7 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
AMQPMirrorControllerAggregation aggregation = (AMQPMirrorControllerAggregation) currentMirrorController;
for (AMQPMirrorControllerSource source : aggregation.getPartitions()) {
Queue queue = checkCurrentMirror(brokerConnection, source);
Queue queue = checkCurrentMirror(this, source);
// on this case we already had a mirror installed before, we won't duplicate it
if (queue != null) {
return queue;
@ -347,9 +377,9 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
}
}
AddressInfo addressInfo = server.getAddressInfo(replicaConfig.getSourceMirrorAddress());
AddressInfo addressInfo = server.getAddressInfo(getMirrorSNF(replicaConfig));
if (addressInfo == null) {
addressInfo = new AddressInfo(replicaConfig.getSourceMirrorAddress()).addRoutingType(RoutingType.ANYCAST).setAutoCreated(false).setTemporary(!replicaConfig.isDurable());
addressInfo = new AddressInfo(getMirrorSNF(replicaConfig)).addRoutingType(RoutingType.ANYCAST).setAutoCreated(false).setTemporary(!replicaConfig.isDurable()).setInternal(true);
server.addAddressInfo(addressInfo);
}
@ -357,15 +387,15 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
throw new IllegalArgumentException("sourceMirrorAddress is not ANYCAST");
}
Queue mirrorControlQueue = server.locateQueue(replicaConfig.getSourceMirrorAddress());
Queue mirrorControlQueue = server.locateQueue(getMirrorSNF(replicaConfig));
if (mirrorControlQueue == null) {
mirrorControlQueue = server.createQueue(new QueueConfiguration(replicaConfig.getSourceMirrorAddress()).setAddress(replicaConfig.getSourceMirrorAddress()).setRoutingType(RoutingType.ANYCAST).setDurable(replicaConfig.isDurable()), true);
mirrorControlQueue = server.createQueue(new QueueConfiguration(getMirrorSNF(replicaConfig)).setAddress(getMirrorSNF(replicaConfig)).setRoutingType(RoutingType.ANYCAST).setDurable(replicaConfig.isDurable()).setInternal(true), true);
}
mirrorControlQueue.setMirrorController(true);
QueueBinding snfReplicaQueueBinding = (QueueBinding)server.getPostOffice().getBinding(replicaConfig.getSourceMirrorAddress());
QueueBinding snfReplicaQueueBinding = (QueueBinding)server.getPostOffice().getBinding(getMirrorSNF(replicaConfig));
if (snfReplicaQueueBinding == null) {
logger.warn("Queue does not exist even after creation! " + replicaConfig);
throw new IllegalAccessException("Cannot start replica");
@ -373,12 +403,14 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
Queue snfQueue = snfReplicaQueueBinding.getQueue();
if (!snfQueue.getAddress().equals(replicaConfig.getSourceMirrorAddress())) {
if (!snfQueue.getAddress().equals(getMirrorSNF(replicaConfig))) {
logger.warn("Queue " + snfQueue + " belong to a different address (" + snfQueue.getAddress() + "), while we expected it to be " + addressInfo.getName());
throw new IllegalAccessException("Cannot start replica");
}
AMQPMirrorControllerSource newPartition = new AMQPMirrorControllerSource(snfQueue, server, replicaConfig.isMessageAcknowledgements(), replicaConfig.isQueueCreation(), replicaConfig.isQueueRemoval(), brokerConnection);
AMQPMirrorControllerSource newPartition = new AMQPMirrorControllerSource(protonProtocolManager, snfQueue, server, replicaConfig, this);
this.mirrorControllerSource = newPartition;
server.scanAddresses(newPartition);
@ -458,8 +490,11 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
private void connectSender(Queue queue,
String targetName,
java.util.function.Consumer<Sender> senderConsumer,
java.util.function.Consumer<? super MessageReference> beforeDeliver,
Symbol... capabilities) {
String brokerID,
Symbol[] desiredCapabilities,
Symbol[] targetCapabilities) {
if (logger.isDebugEnabled()) {
logger.debug("Connecting outbound for " + queue);
}
@ -482,20 +517,80 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
sender.setReceiverSettleMode(ReceiverSettleMode.FIRST);
Target target = new Target();
target.setAddress(targetName);
if (capabilities != null) {
target.setCapabilities(capabilities);
if (targetCapabilities != null) {
target.setCapabilities(targetCapabilities);
}
sender.setTarget(target);
Source source = new Source();
source.setAddress(queue.getAddress().toString());
sender.setSource(source);
if (brokerID != null) {
HashMap<Symbol, Object> mapProperties = new HashMap<>(1, 1); // this map is expected to have a single element, so load factor = 1
mapProperties.put(AMQPMirrorControllerSource.BROKER_ID, brokerID);
sender.setProperties(mapProperties);
}
if (desiredCapabilities != null) {
sender.setDesiredCapabilities(desiredCapabilities);
}
AMQPOutgoingController outgoingInitializer = new AMQPOutgoingController(queue, sender, sessionContext.getSessionSPI());
ProtonServerSenderContext senderContext = new ProtonServerSenderContext(protonRemotingConnection.getAmqpConnection(), sender, sessionContext, sessionContext.getSessionSPI(), outgoingInitializer).setBeforeDelivery(beforeDeliver);
sender.open();
sessionContext.addSender(sender, senderContext);
final ScheduledFuture futureTimeout;
AtomicBoolean cancelled = new AtomicBoolean(false);
if (bridgesConnector.getConnectTimeoutMillis() > 0) {
futureTimeout = server.getScheduledPool().schedule(() -> {
cancelled.set(true);
error(ActiveMQAMQPProtocolMessageBundle.BUNDLE.brokerConnectionTimeout(), lastRetryCounter);
}, bridgesConnector.getConnectTimeoutMillis(), TimeUnit.MILLISECONDS);
} else {
futureTimeout = null;
}
// Using attachments to set up a Runnable that will be executed inside AMQPBrokerConnection::remoteLinkOpened
sender.attachments().set(Runnable.class, Runnable.class, () -> {
ProtonServerSenderContext senderContext = new ProtonServerSenderContext(protonRemotingConnection.getAmqpConnection(), sender, sessionContext, sessionContext.getSessionSPI(), outgoingInitializer).setBeforeDelivery(beforeDeliver);
try {
if (!cancelled.get()) {
if (futureTimeout != null) {
futureTimeout.cancel(false);
}
if (sender.getRemoteTarget() == null) {
error(ActiveMQAMQPProtocolMessageBundle.BUNDLE.senderLinkRefused(sender.getTarget().getAddress()), lastRetryCounter);
return;
}
if (desiredCapabilities != null) {
if (!verifyOfferedCapabilities(sender, desiredCapabilities)) {
error(ActiveMQAMQPProtocolMessageBundle.BUNDLE.missingOfferedCapability(Arrays.toString(desiredCapabilities)), lastRetryCounter);
return;
}
}
if (brokerID != null) {
if (sender.getRemoteProperties() == null || !sender.getRemoteProperties().containsKey(AMQPMirrorControllerSource.BROKER_ID)) {
error(ActiveMQAMQPProtocolMessageBundle.BUNDLE.missingBrokerID(), lastRetryCounter);
return;
}
Object remoteBrokerID = sender.getRemoteProperties().get(AMQPMirrorControllerSource.BROKER_ID);
if (remoteBrokerID.equals(brokerID)) {
error(ActiveMQAMQPProtocolMessageBundle.BUNDLE.brokerConnectionMirrorItself(), lastRetryCounter);
return;
}
}
sessionContext.addSender(sender, senderContext);
if (senderConsumer != null) {
senderConsumer.accept(sender);
}
}
} catch (Exception e) {
error(e);
}
});
} catch (Exception e) {
error(e);
}
@ -503,7 +598,39 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
});
}
protected boolean verifyOfferedCapabilities(Sender sender, Symbol[] capabilities) {
if (sender.getRemoteOfferedCapabilities() == null) {
return false;
}
for (Symbol s : capabilities) {
boolean foundS = false;
for (Symbol b : sender.getRemoteOfferedCapabilities()) {
if (b.equals(s)) {
foundS = true;
break;
}
}
if (!foundS) {
return false;
}
}
return true;
}
protected void error(Throwable e) {
error(e, 0);
}
// the retryCounter is passed here
// in case the error happened after the actual connection
// say the connection is invalid due to an invalid attribute or wrong password
// but the max retry should not be affected by such cases
// otherwise we would always retry from 0 and never reach a max
protected void error(Throwable e, int retryCounter) {
this.retryCounter = retryCounter;
connecting = false;
logger.warn(e.getMessage(), e);
redoConnection();
@ -553,6 +680,11 @@ public class AMQPBrokerConnection implements ClientConnectionLifeCycleListener,
private void redoConnection() {
// avoiding retro-feeding an error call from the close after anyting else that happened.
if (protonRemotingConnection != null) {
protonRemotingConnection.getAmqpConnection().setLinkCloseListener(null);
}
// we need to use the connectExecutor to initiate a redoConnection
// otherwise we would need to add synchronized blocks along this class
// to control when connecting becomes true and when it becomes false

View File

@ -54,6 +54,15 @@ public class AMQPMirrorControllerAggregation implements MirrorController, Active
}
@Override
public String getRemoteMirrorId() {
if (partitions != null && partitions.size() > 0) {
return partitions.get(0).getRemoteMirrorId();
} else {
throw new IllegalStateException();
}
}
@Override
public boolean isStarted() {
return false;
@ -106,17 +115,4 @@ public class AMQPMirrorControllerAggregation implements MirrorController, Active
}
}
@Override
public void startAddressScan() throws Exception {
for (MirrorController partition : partitions) {
partition.startAddressScan();
}
}
@Override
public void endAddressScan() throws Exception {
for (MirrorController partition : partitions) {
partition.endAddressScan();
}
}
}

View File

@ -23,6 +23,7 @@ import java.util.Map;
import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.api.core.QueueConfiguration;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPMirrorBrokerConnectionElement;
import org.apache.activemq.artemis.core.postoffice.impl.PostOfficeImpl;
import org.apache.activemq.artemis.core.server.ActiveMQComponent;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
@ -36,42 +37,55 @@ import org.apache.activemq.artemis.core.server.mirror.MirrorController;
import org.apache.activemq.artemis.core.transaction.Transaction;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessageBrokerAccessor;
import org.apache.activemq.artemis.protocol.amqp.broker.ProtonProtocolManager;
import org.apache.activemq.artemis.protocol.amqp.connect.AMQPBrokerConnection;
import org.apache.qpid.proton.amqp.Symbol;
import org.apache.qpid.proton.amqp.messaging.DeliveryAnnotations;
import org.apache.qpid.proton.amqp.messaging.Properties;
import org.apache.qpid.proton.engine.Sender;
import org.jboss.logging.Logger;
public class AMQPMirrorControllerSource implements MirrorController, ActiveMQComponent {
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerTarget.getControllerInUse;
public class AMQPMirrorControllerSource extends BasicMirrorController<Sender> implements MirrorController, ActiveMQComponent {
private static final Logger logger = Logger.getLogger(AMQPMirrorControllerSource.class);
public static final Symbol EVENT_TYPE = Symbol.getSymbol("x-opt-amq-mr-ev-type");
public static final Symbol ADDRESS = Symbol.getSymbol("x-opt-amq-mr-adr");
public static final Symbol QUEUE = Symbol.getSymbol("x-opt-amq-mr-qu");
public static final Symbol BROKER_ID = Symbol.getSymbol("x-opt-amq-bkr-id");
// Events:
public static final Symbol ADD_ADDRESS = Symbol.getSymbol("addAddress");
public static final Symbol DELETE_ADDRESS = Symbol.getSymbol("deleteAddress");
public static final Symbol CREATE_QUEUE = Symbol.getSymbol("createQueue");
public static final Symbol DELETE_QUEUE = Symbol.getSymbol("deleteQueue");
public static final Symbol ADDRESS_SCAN_START = Symbol.getSymbol("addressCanStart");
public static final Symbol ADDRESS_SCAN_END = Symbol.getSymbol("addressScanEnd");
public static final Symbol POST_ACK = Symbol.getSymbol("postAck");
// Delivery annotation property used on mirror control routing and Ack
public static final Symbol INTERNAL_ID = Symbol.getSymbol("x-opt-amq-mr-id");
public static final Symbol INTERNAL_DESTINATION = Symbol.getSymbol("x-opt-amq-mr-dst");
// Capabilities
public static final Symbol MIRROR_CAPABILITY = Symbol.getSymbol("amq.mirror");
public static final Symbol QPID_DISPATCH_WAYPOINT_CAPABILITY = Symbol.valueOf("qd.waypoint");
public static final SimpleString INTERNAL_ID_EXTRA_PROPERTY = SimpleString.toSimpleString(INTERNAL_ID.toString());
public static final SimpleString INTERNAL_BROKER_ID_EXTRA_PROPERTY = SimpleString.toSimpleString(BROKER_ID.toString());
private static final ThreadLocal<MirrorControlRouting> mirrorControlRouting = ThreadLocal.withInitial(() -> new MirrorControlRouting(null));
final Queue snfQueue;
final ActiveMQServer server;
final ReferenceNodeStore idSupplier;
final boolean acks;
final boolean addQueues;
final boolean deleteQueues;
private final AMQPBrokerConnection brokerConnection;
final AMQPMirrorBrokerConnectionElement replicaConfig;
boolean started;
@Override
@ -87,12 +101,16 @@ public class AMQPMirrorControllerSource implements MirrorController, ActiveMQCom
return started;
}
public AMQPMirrorControllerSource(Queue snfQueue, ActiveMQServer server, boolean acks, boolean addQueues, boolean deleteQueues, AMQPBrokerConnection brokerConnection) {
public AMQPMirrorControllerSource(ProtonProtocolManager protonProtocolManager, Queue snfQueue, ActiveMQServer server, AMQPMirrorBrokerConnectionElement replicaConfig,
AMQPBrokerConnection brokerConnection) {
super(server);
this.replicaConfig = replicaConfig;
this.snfQueue = snfQueue;
this.server = server;
this.acks = acks;
this.addQueues = addQueues;
this.deleteQueues = deleteQueues;
this.idSupplier = protonProtocolManager.getReferenceIDSupplier();
this.addQueues = replicaConfig.isQueueCreation();
this.deleteQueues = replicaConfig.isQueueRemoval();
this.acks = replicaConfig.isMessageAcknowledgements();
this.brokerConnection = brokerConnection;
}
@ -104,62 +122,104 @@ public class AMQPMirrorControllerSource implements MirrorController, ActiveMQCom
return brokerConnection;
}
@Override
public void startAddressScan() throws Exception {
Message message = createMessage(null, null, ADDRESS_SCAN_START, null);
route(server, message);
}
@Override
public void endAddressScan() throws Exception {
Message message = createMessage(null, null, ADDRESS_SCAN_END, null);
route(server, message);
}
@Override
public void addAddress(AddressInfo addressInfo) throws Exception {
if (logger.isTraceEnabled()) {
logger.trace(server + " addAddress " + addressInfo);
}
if (getControllerInUse() != null && !addressInfo.isInternal()) {
return;
}
if (addQueues) {
Message message = createMessage(addressInfo.getName(), null, ADD_ADDRESS, addressInfo.toJSON());
Message message = createMessage(addressInfo.getName(), null, ADD_ADDRESS, null, addressInfo.toJSON());
route(server, message);
}
}
@Override
public void deleteAddress(AddressInfo addressInfo) throws Exception {
if (logger.isTraceEnabled()) {
logger.trace(server + " deleteAddress " + addressInfo);
}
if (invalidTarget(getControllerInUse()) || addressInfo.isInternal()) {
return;
}
if (deleteQueues) {
Message message = createMessage(addressInfo.getName(), null, DELETE_ADDRESS, addressInfo.toJSON());
Message message = createMessage(addressInfo.getName(), null, DELETE_ADDRESS, null, addressInfo.toJSON());
route(server, message);
}
}
@Override
public void createQueue(QueueConfiguration queueConfiguration) throws Exception {
if (logger.isTraceEnabled()) {
logger.trace(server + " createQueue " + queueConfiguration);
}
if (invalidTarget(getControllerInUse()) || queueConfiguration.isInternal()) {
if (logger.isTraceEnabled()) {
logger.trace("Rejecting ping pong on create " + queueConfiguration + " as isInternal=" + queueConfiguration.isInternal() + " and mirror target = " + getControllerInUse());
}
return;
}
if (addQueues) {
Message message = createMessage(queueConfiguration.getAddress(), queueConfiguration.getName(), CREATE_QUEUE, queueConfiguration.toJSON());
Message message = createMessage(queueConfiguration.getAddress(), queueConfiguration.getName(), CREATE_QUEUE, null, queueConfiguration.toJSON());
route(server, message);
}
}
@Override
public void deleteQueue(SimpleString address, SimpleString queue) throws Exception {
if (logger.isTraceEnabled()) {
logger.trace(server + " deleteQueue " + address + "/" + queue);
}
if (invalidTarget(getControllerInUse())) {
return;
}
if (deleteQueues) {
Message message = createMessage(address, queue, DELETE_QUEUE, queue.toString());
Message message = createMessage(address, queue, DELETE_QUEUE, null, queue.toString());
route(server, message);
}
}
private boolean invalidTarget(MirrorController controller) {
return controller != null && sameNode(getRemoteMirrorId(), controller.getRemoteMirrorId());
}
private boolean sameNode(String remoteID, String sourceID) {
return (remoteID != null && sourceID != null && remoteID.equals(sourceID));
}
@Override
public void sendMessage(Message message, RoutingContext context, List<MessageReference> refs) {
if (invalidTarget(context.getMirrorSource())) {
if (logger.isTraceEnabled()) {
logger.trace("server " + server + " is discarding send to avoid infinite loop (reflection with the mirror)");
}
return;
}
if (logger.isTraceEnabled()) {
logger.trace(server + " send message " + message);
}
try {
context.setReusable(false);
MessageReference ref = MessageReference.Factory.createReference(message, snfQueue);
String nodeID = setProtocolData(idSupplier, ref);
if (nodeID != null && nodeID.equals(getRemoteMirrorId())) {
if (logger.isTraceEnabled()) {
logger.trace("Message " + message + "already belonged to the node, " + getRemoteMirrorId() + ", it won't circle send");
}
return;
}
snfQueue.refUp(ref);
refs.add(ref);
message.usageUp();
setProtocolData(ref);
if (message.isDurable() && snfQueue.isDurable()) {
PostOfficeImpl.storeDurableReference(server.getStorageManager(), message, context.getTransaction(), snfQueue, true);
@ -170,16 +230,30 @@ public class AMQPMirrorControllerSource implements MirrorController, ActiveMQCom
}
}
public static void validateProtocolData(MessageReference ref, SimpleString snfAddress) {
public static void validateProtocolData(ReferenceNodeStore referenceIDSupplier, MessageReference ref, SimpleString snfAddress) {
if (ref.getProtocolData() == null && !ref.getMessage().getAddressSimpleString().equals(snfAddress)) {
setProtocolData(ref);
setProtocolData(referenceIDSupplier, ref);
}
}
private static void setProtocolData(MessageReference ref) {
/** This method will return the brokerID used by the message */
private static String setProtocolData(ReferenceNodeStore referenceIDSupplier, MessageReference ref) {
Map<Symbol, Object> daMap = new HashMap<>();
DeliveryAnnotations deliveryAnnotations = new DeliveryAnnotations(daMap);
daMap.put(INTERNAL_ID, ref.getMessage().getMessageID());
String brokerID = referenceIDSupplier.getServerID(ref);
// getListID will return null when the message was generated on this broker.
// on this case we do not send the brokerID, and the ControllerTarget will get the information from the link.
// this is just to safe a few bytes and some processing on the wire.
if (brokerID != null) {
// not sending the brokerID, will make the other side to get the brokerID from the remote link's property
daMap.put(BROKER_ID, brokerID);
}
long id = referenceIDSupplier.getID(ref);
daMap.put(INTERNAL_ID, id);
String address = ref.getMessage().getAddress();
if (address != null) { // this is the message that was set through routing
Properties amqpProperties = getProperties(ref.getMessage());
@ -190,6 +264,8 @@ public class AMQPMirrorControllerSource implements MirrorController, ActiveMQCom
}
}
ref.setProtocolData(deliveryAnnotations);
return brokerID;
}
private static Properties getProperties(Message message) {
@ -202,16 +278,42 @@ public class AMQPMirrorControllerSource implements MirrorController, ActiveMQCom
@Override
public void postAcknowledge(MessageReference ref, AckReason reason) throws Exception {
if (acks && !ref.getQueue().isMirrorController()) { // we don't call postACK on snfqueues, otherwise we would get infinite loop because of this feedback
Message message = createMessage(ref.getQueue().getAddress(), ref.getQueue().getName(), POST_ACK, ref.getMessage().getMessageID());
route(server, message);
ref.getMessage().usageDown();
MirrorController controllerInUse = getControllerInUse();
if (!acks || ref.getQueue().isMirrorController()) { // we don't call postACK on snfqueues, otherwise we would get infinite loop because of this feedback/
return;
}
if (invalidTarget(controllerInUse)) {
return;
}
if ((ref.getQueue() != null && (ref.getQueue().isInternalQueue() || ref.getQueue().isMirrorController()))) {
if (logger.isDebugEnabled()) {
logger.debug(server + " rejecting postAcknowledge queue=" + ref.getQueue().getName() + ", ref=" + ref + " to avoid infinite loop with the mirror (reflection)");
}
return;
}
if (logger.isTraceEnabled()) {
logger.trace(server + " postAcknowledge " + ref);
}
String nodeID = idSupplier.getServerID(ref); // notice the brokerID will be null for any message generated on this broker.
long internalID = idSupplier.getID(ref);
if (logger.isTraceEnabled()) {
logger.trace(server + " sending ack message from server " + nodeID + " with messageID=" + internalID);
}
Message message = createMessage(ref.getQueue().getAddress(), ref.getQueue().getName(), POST_ACK, nodeID, internalID);
route(server, message);
ref.getMessage().usageDown();
}
private Message createMessage(SimpleString address, SimpleString queue, Object event, Object body) {
return AMQPMirrorMessageFactory.createMessage(snfQueue.getAddress().toString(), address, queue, event, body);
private Message createMessage(SimpleString address, SimpleString queue, Object event, String brokerID, Object body) {
return AMQPMirrorMessageFactory.createMessage(snfQueue.getAddress().toString(), address, queue, event, brokerID, body);
}
public static void route(ActiveMQServer server, Message message) throws Exception {
message.setMessageID(server.getStorageManager().generateID());
MirrorControlRouting ctx = mirrorControlRouting.get();

View File

@ -16,19 +16,16 @@
*/
package org.apache.activemq.artemis.protocol.amqp.connect.mirror;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.ToLongFunction;
import java.util.stream.Stream;
import org.apache.activemq.artemis.api.core.ActiveMQAddressDoesNotExistException;
import org.apache.activemq.artemis.api.core.ActiveMQNonExistentQueueException;
import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.api.core.QueueConfiguration;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.core.postoffice.Binding;
import org.apache.activemq.artemis.core.postoffice.impl.LocalQueueBinding;
import org.apache.activemq.artemis.core.io.IOCallback;
import org.apache.activemq.artemis.core.persistence.impl.journal.OperationContextImpl;
import org.apache.activemq.artemis.core.postoffice.DuplicateIDCache;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.MessageReference;
import org.apache.activemq.artemis.core.server.Queue;
@ -38,12 +35,18 @@ import org.apache.activemq.artemis.core.server.impl.AddressInfo;
import org.apache.activemq.artemis.core.server.impl.RoutingContextImpl;
import org.apache.activemq.artemis.core.server.mirror.MirrorController;
import org.apache.activemq.artemis.core.transaction.Transaction;
import org.apache.activemq.artemis.core.transaction.TransactionOperationAbstract;
import org.apache.activemq.artemis.core.transaction.impl.TransactionImpl;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessageBrokerAccessor;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPSessionCallback;
import org.apache.activemq.artemis.protocol.amqp.broker.ProtonProtocolManager;
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPConnectionContext;
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPSessionContext;
import org.apache.activemq.artemis.protocol.amqp.proton.ProtonAbstractReceiver;
import org.apache.activemq.artemis.utils.ByteUtil;
import org.apache.activemq.artemis.utils.collections.NodeStore;
import org.apache.activemq.artemis.utils.pools.MpscPool;
import org.apache.qpid.proton.amqp.messaging.Accepted;
import org.apache.qpid.proton.amqp.messaging.AmqpValue;
import org.apache.qpid.proton.amqp.transport.ReceiverSettleMode;
@ -51,30 +54,99 @@ import org.apache.qpid.proton.engine.Delivery;
import org.apache.qpid.proton.engine.Receiver;
import org.jboss.logging.Logger;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.EVENT_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.ADDRESS;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.ADD_ADDRESS;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.BROKER_ID;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.CREATE_QUEUE;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.DELETE_ADDRESS;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.DELETE_QUEUE;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.EVENT_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.INTERNAL_BROKER_ID_EXTRA_PROPERTY;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.INTERNAL_DESTINATION;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.INTERNAL_ID;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.POST_ACK;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.QUEUE;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.ADD_ADDRESS;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.DELETE_ADDRESS;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.CREATE_QUEUE;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.DELETE_QUEUE;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.INTERNAL_ID;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.ADDRESS_SCAN_START;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.ADDRESS_SCAN_END;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.INTERNAL_ID_EXTRA_PROPERTY;
public class AMQPMirrorControllerTarget extends ProtonAbstractReceiver implements MirrorController {
public static final SimpleString INTERNAL_ID_EXTRA_PROPERTY = SimpleString.toSimpleString(INTERNAL_ID.toString());
private static final Logger logger = Logger.getLogger(AMQPMirrorControllerTarget.class);
final ActiveMQServer server;
private static ThreadLocal<MirrorController> controllerThreadLocal = new ThreadLocal<>();
public static void setControllerInUse(MirrorController controller) {
controllerThreadLocal.set(controller);
}
public static MirrorController getControllerInUse() {
return controllerThreadLocal.get();
}
/** Objects of this class can be used by either transaction or by OperationContext.
* It is important that when you're using the transactions you clear any references to
* the operation context. Don't use transaction and OperationContext at the same time
* as that would generate duplicates on the objects cache.
*/
class ACKMessageOperation implements IOCallback, Runnable {
Delivery delivery;
/** notice that when you use the Transaction, you need to make sure you don't use the IO*/
public TransactionOperationAbstract tx = new TransactionOperationAbstract() {
@Override
public void afterCommit(Transaction tx) {
completeOperation();
}
};
void reset() {
this.delivery = null;
}
ACKMessageOperation setDelivery(Delivery delivery) {
this.delivery = delivery;
return this;
}
@Override
public void run() {
if (logger.isTraceEnabled()) {
logger.trace("Delivery settling for " + delivery + ", context=" + delivery.getContext());
}
delivery.disposition(Accepted.getInstance());
settle(delivery);
connection.flush();
AMQPMirrorControllerTarget.this.ackMessageMpscPool.release(ACKMessageOperation.this);
}
@Override
public void done() {
completeOperation();
}
private void completeOperation() {
connection.runNow(ACKMessageOperation.this);
}
@Override
public void onError(int errorCode, String errorMessage) {
logger.warn(errorMessage + "-" + errorMessage);
}
}
// in a regular case we should not have more than amqpCredits on the pool, that's the max we would need
private final MpscPool<ACKMessageOperation> ackMessageMpscPool = new MpscPool<>(amqpCredits, ACKMessageOperation::reset, () -> new ACKMessageOperation());
final RoutingContextImpl routingContext = new RoutingContextImpl(null);
Map<SimpleString, Map<SimpleString, QueueConfiguration>> scanAddresses;
final BasicMirrorController<Receiver> basicController;
final ActiveMQServer server;
DuplicateIDCache lruduplicateIDCache;
String lruDuplicateIDKey;
private final NodeStore<MessageReference> referenceNodeStore;
public AMQPMirrorControllerTarget(AMQPSessionCallback sessionSPI,
AMQPConnectionContext connection,
@ -82,7 +154,15 @@ public class AMQPMirrorControllerTarget extends ProtonAbstractReceiver implement
Receiver receiver,
ActiveMQServer server) {
super(sessionSPI, connection, protonSession, receiver);
this.basicController = new BasicMirrorController(server);
this.basicController.setLink(receiver);
this.server = server;
this.referenceNodeStore = sessionSPI.getProtocolManager().getReferenceIDSupplier();
}
@Override
public String getRemoteMirrorId() {
return basicController.getRemoteMirrorId();
}
@Override
@ -92,12 +172,20 @@ public class AMQPMirrorControllerTarget extends ProtonAbstractReceiver implement
@Override
protected void actualDelivery(AMQPMessage message, Delivery delivery, Receiver receiver, Transaction tx) {
recoverContext();
incrementSettle();
if (logger.isDebugEnabled()) {
logger.debug(server.getIdentity() + "::Received " + message);
if (logger.isTraceEnabled()) {
logger.trace(server + "::actualdelivery call for " + message);
}
setControllerInUse(this);
delivery.setContext(message);
ACKMessageOperation messageAckOperation = this.ackMessageMpscPool.borrow().setDelivery(delivery);
try {
/** We use message annotations, because on the same link we will receive control messages
* coming from mirror events,
@ -106,28 +194,22 @@ public class AMQPMirrorControllerTarget extends ProtonAbstractReceiver implement
* The body of the message may still be used on control messages, on cases where a JSON string is sent. */
Object eventType = AMQPMessageBrokerAccessor.getMessageAnnotationProperty(message, EVENT_TYPE);
if (eventType != null) {
if (eventType.equals(ADDRESS_SCAN_START)) {
logger.debug("Starting scan for removed queues");
startAddressScan();
} else if (eventType.equals(ADDRESS_SCAN_END)) {
logger.debug("Ending scan for removed queues");
endAddressScan();
} else if (eventType.equals(ADD_ADDRESS)) {
if (eventType.equals(ADD_ADDRESS)) {
AddressInfo addressInfo = parseAddress(message);
if (logger.isDebugEnabled()) {
logger.debug("Adding Address " + addressInfo);
logger.debug(server + " Adding Address " + addressInfo);
}
addAddress(addressInfo);
} else if (eventType.equals(DELETE_ADDRESS)) {
AddressInfo addressInfo = parseAddress(message);
if (logger.isDebugEnabled()) {
logger.debug("Removing Address " + addressInfo);
logger.debug(server + " Removing Address " + addressInfo);
}
deleteAddress(addressInfo);
} else if (eventType.equals(CREATE_QUEUE)) {
QueueConfiguration queueConfiguration = parseQueue(message);
if (logger.isDebugEnabled()) {
logger.debug("Creating queue " + queueConfiguration);
logger.debug(server + " Creating queue " + queueConfiguration);
}
createQueue(queueConfiguration);
} else if (eventType.equals(DELETE_QUEUE)) {
@ -135,31 +217,42 @@ public class AMQPMirrorControllerTarget extends ProtonAbstractReceiver implement
String address = (String) AMQPMessageBrokerAccessor.getMessageAnnotationProperty(message, ADDRESS);
String queueName = (String) AMQPMessageBrokerAccessor.getMessageAnnotationProperty(message, QUEUE);
if (logger.isDebugEnabled()) {
logger.debug("Deleting queue " + queueName + " on address " + address);
logger.debug(server + " Deleting queue " + queueName + " on address " + address);
}
deleteQueue(SimpleString.toSimpleString(address), SimpleString.toSimpleString(queueName));
} else if (eventType.equals(POST_ACK)) {
String address = (String) AMQPMessageBrokerAccessor.getMessageAnnotationProperty(message, ADDRESS);
String nodeID = (String) AMQPMessageBrokerAccessor.getMessageAnnotationProperty(message, BROKER_ID);
if (nodeID == null) {
nodeID = getRemoteMirrorId(); // not sending the nodeID means it's data generated on that broker
}
String queueName = (String) AMQPMessageBrokerAccessor.getMessageAnnotationProperty(message, QUEUE);
AmqpValue value = (AmqpValue) message.getBody();
Long messageID = (Long) value.getValue();
if (logger.isDebugEnabled()) {
logger.debug("Post ack address=" + address + " queueName = " + queueName + " messageID=" + messageID);
logger.debug(server + " Post ack address=" + address + " queueName = " + queueName + " messageID=" + messageID + ", nodeID=" + nodeID);
}
if (postAcknowledge(address, queueName, nodeID, messageID, messageAckOperation)) {
messageAckOperation = null;
}
postAcknowledge(address, queueName, messageID);
}
} else {
if (logger.isDebugEnabled()) {
logger.debug("Sending message " + message);
logger.debug(server + " Sending message " + message);
}
if (sendMessage(message, messageAckOperation)) {
// since the send was successful, we give up the reference here,
// so there won't be any call on afterCompleteOperations
messageAckOperation = null;
}
sendMessage(message);
}
} catch (Throwable e) {
logger.warn(e.getMessage(), e);
} finally {
delivery.disposition(Accepted.getInstance());
settle(delivery);
connection.flush();
setControllerInUse(null);
if (messageAckOperation != null) {
server.getStorageManager().afterCompleteOperations(messageAckOperation);
}
}
}
@ -190,61 +283,10 @@ public class AMQPMirrorControllerTarget extends ProtonAbstractReceiver implement
return addressInfo;
}
@Override
public void startAddressScan() throws Exception {
scanAddresses = new HashMap<>();
}
@Override
public void endAddressScan() throws Exception {
Map<SimpleString, Map<SimpleString, QueueConfiguration>> scannedAddresses = scanAddresses;
this.scanAddresses = null;
Stream<Binding> bindings = server.getPostOffice().getAllBindings();
bindings.forEach((binding) -> {
if (binding instanceof LocalQueueBinding) {
LocalQueueBinding localQueueBinding = (LocalQueueBinding) binding;
Map<SimpleString, QueueConfiguration> scannedQueues = scannedAddresses.get(localQueueBinding.getQueue().getAddress());
if (scannedQueues == null) {
if (logger.isDebugEnabled()) {
logger.debug("There's no address " + localQueueBinding.getQueue().getAddress() + " so, removing queue");
}
try {
deleteQueue(localQueueBinding.getQueue().getAddress(), localQueueBinding.getQueue().getName());
} catch (Exception e) {
logger.warn(e.getMessage(), e);
}
} else {
QueueConfiguration queueConfg = scannedQueues.get(localQueueBinding.getQueue().getName());
if (queueConfg == null) {
if (logger.isDebugEnabled()) {
logger.debug("There no queue for " + localQueueBinding.getQueue().getName() + " so, removing queue");
}
try {
deleteQueue(localQueueBinding.getQueue().getAddress(), localQueueBinding.getQueue().getName());
} catch (Exception e) {
logger.warn(e.getMessage(), e);
}
}
}
}
});
}
private Map<SimpleString, QueueConfiguration> getQueueScanMap(SimpleString address) {
Map<SimpleString, QueueConfiguration> queueMap = scanAddresses.get(address);
if (queueMap == null) {
queueMap = new HashMap<>();
scanAddresses.put(address, queueMap);
}
return queueMap;
}
@Override
public void addAddress(AddressInfo addressInfo) throws Exception {
if (logger.isDebugEnabled()) {
logger.debug("Adding address " + addressInfo);
logger.debug(server + " Adding address " + addressInfo);
}
server.addAddressInfo(addressInfo);
}
@ -252,7 +294,7 @@ public class AMQPMirrorControllerTarget extends ProtonAbstractReceiver implement
@Override
public void deleteAddress(AddressInfo addressInfo) throws Exception {
if (logger.isDebugEnabled()) {
logger.debug("delete address " + addressInfo);
logger.debug(server + " delete address " + addressInfo);
}
try {
server.removeAddressInfo(addressInfo.getName(), null, true);
@ -267,88 +309,162 @@ public class AMQPMirrorControllerTarget extends ProtonAbstractReceiver implement
@Override
public void createQueue(QueueConfiguration queueConfiguration) throws Exception {
if (logger.isDebugEnabled()) {
logger.debug("Adding queue " + queueConfiguration);
logger.debug(server + " Adding queue " + queueConfiguration);
}
server.createQueue(queueConfiguration, true);
if (scanAddresses != null) {
getQueueScanMap(queueConfiguration.getAddress()).put(queueConfiguration.getName(), queueConfiguration);
try {
server.createQueue(queueConfiguration, true);
} catch (Exception ignored) {
logger.debug("Queue could not be created, already existed " + queueConfiguration, ignored);
}
}
@Override
public void deleteQueue(SimpleString addressName, SimpleString queueName) throws Exception {
if (logger.isDebugEnabled()) {
logger.debug("destroy queue " + queueName + " on address = " + addressName);
logger.debug(server + " destroy queue " + queueName + " on address = " + addressName + " server " + server.getIdentity());
}
try {
server.destroyQueue(queueName);
server.destroyQueue(queueName,null, false, true, false, false);
} catch (ActiveMQNonExistentQueueException expected) {
logger.debug("queue " + queueName + " was previously removed", expected);
logger.debug(server + " queue " + queueName + " was previously removed", expected);
}
}
private static ToLongFunction<MessageReference> referenceIDSupplier = (source) -> {
Long id = (Long) source.getMessage().getBrokerProperty(INTERNAL_ID_EXTRA_PROPERTY);
if (id == null) {
return -1;
} else {
return id;
}
};
public boolean postAcknowledge(String address, String queue, String nodeID, long messageID, ACKMessageOperation ackMessage) throws Exception {
final Queue targetQueue = server.locateQueue(queue);
if (targetQueue == null) {
logger.warn("Queue " + queue + " not found on mirror target, ignoring ack for queue=" + queue + ", messageID=" + messageID + ", nodeID=" + nodeID);
return false;
}
public void postAcknowledge(String address, String queue, long messageID) {
if (logger.isDebugEnabled()) {
logger.debug("post acking " + address + ", queue = " + queue + ", messageID = " + messageID);
// we only do the following check if debug
if (targetQueue.getConsumerCount() > 0) {
logger.debug("server " + server.getIdentity() + ", queue " + targetQueue.getName() + " has consumers while delivering ack for " + messageID);
}
}
Queue targetQueue = server.locateQueue(queue);
if (targetQueue != null) {
MessageReference reference = targetQueue.removeWithSuppliedID(messageID, referenceIDSupplier);
if (reference != null) {
if (logger.isDebugEnabled()) {
logger.debug("Acking reference " + reference);
}
try {
targetQueue.acknowledge(reference);
} catch (Exception e) {
// TODO anything else I can do here?
// such as close the connection with error?
logger.warn(e.getMessage(), e);
}
} else {
if (logger.isTraceEnabled()) {
logger.trace("There is no reference to ack on " + messageID);
}
if (logger.isTraceEnabled()) {
logger.trace("Server " + server.getIdentity() + " with queue = " + queue + " being acked for " + messageID + " coming from " + messageID + " targetQueue = " + targetQueue);
}
performAck(nodeID, messageID, targetQueue, ackMessage, true);
return true;
}
private void performAck(String nodeID, long messageID, Queue targetQueue, ACKMessageOperation ackMessageOperation, boolean retry) {
if (logger.isTraceEnabled()) {
logger.trace("performAck (nodeID=" + nodeID + ", messageID=" + messageID + ")" + ", targetQueue=" + targetQueue.getName());
}
MessageReference reference = targetQueue.removeWithSuppliedID(nodeID, messageID, referenceNodeStore);
if (reference == null && retry) {
if (logger.isDebugEnabled()) {
logger.debug("Retrying Reference not found on messageID=" + messageID + " nodeID=" + nodeID);
}
targetQueue.flushOnIntermediate(() -> {
recoverContext();
performAck(nodeID, messageID, targetQueue, ackMessageOperation, false);
});
return;
}
if (reference != null) {
if (logger.isTraceEnabled()) {
logger.trace("Post ack Server " + server + " worked well for messageID=" + messageID + " nodeID=" + nodeID);
}
try {
targetQueue.acknowledge(reference);
OperationContextImpl.getContext().executeOnCompletion(ackMessageOperation);
} catch (Exception e) {
logger.warn(e.getMessage(), e);
}
} else {
if (logger.isDebugEnabled()) {
logger.debug("Post ack Server " + server + " could not find messageID = " + messageID +
" representing nodeID=" + nodeID);
}
}
}
private void sendMessage(AMQPMessage message) throws Exception {
/**
* this method returning true means the sendMessage was successful, and the IOContext should no longer be used.
* as the sendMessage was successful the OperationContext of the transaction will take care of the completion.
* The caller of this method should give up any reference to messageCompletionAck when this method returns true.
* */
private boolean sendMessage(AMQPMessage message, ACKMessageOperation messageCompletionAck) throws Exception {
if (message.getMessageID() <= 0) {
message.setMessageID(server.getStorageManager().generateID());
}
Long internalID = (Long) AMQPMessageBrokerAccessor.getDeliveryAnnotationProperty(message, INTERNAL_ID);
String internalMirrorID = (String)AMQPMessageBrokerAccessor.getDeliveryAnnotationProperty(message, BROKER_ID);
if (internalMirrorID == null) {
internalMirrorID = getRemoteMirrorId(); // not pasisng the ID means the data was generated on the remote broker
}
Long internalIDLong = (Long) AMQPMessageBrokerAccessor.getDeliveryAnnotationProperty(message, INTERNAL_ID);
String internalAddress = (String) AMQPMessageBrokerAccessor.getDeliveryAnnotationProperty(message, INTERNAL_DESTINATION);
if (internalID != null) {
message.setBrokerProperty(INTERNAL_ID_EXTRA_PROPERTY, internalID);
long internalID = 0;
if (internalIDLong != null) {
internalID = internalIDLong;
}
if (logger.isTraceEnabled()) {
logger.trace("sendMessage on server " + server + " for message " + message +
" with internalID = " + internalIDLong + " mirror id " + internalMirrorID);
}
routingContext.setDuplicateDetection(false); // we do our own duplicate detection here
DuplicateIDCache duplicateIDCache;
if (lruDuplicateIDKey != null && lruDuplicateIDKey.equals(internalMirrorID)) {
duplicateIDCache = lruduplicateIDCache;
} else {
// we use the number of credits for the duplicate detection, as that means the maximum number of elements you can have pending
if (logger.isDebugEnabled()) {
logger.trace("Setting up duplicate detection cache on " + ProtonProtocolManager.MIRROR_ADDRESS + ", ServerID=" + internalMirrorID + " with " + connection.getAmqpCredits() + " elements, being the number of credits");
}
lruDuplicateIDKey = internalMirrorID;
lruduplicateIDCache = server.getPostOffice().getDuplicateIDCache(SimpleString.toSimpleString(ProtonProtocolManager.MIRROR_ADDRESS + "_" + internalMirrorID), connection.getAmqpCredits());
duplicateIDCache = lruduplicateIDCache;
}
byte[] duplicateIDBytes = ByteUtil.longToBytes(internalIDLong);
if (duplicateIDCache.contains(duplicateIDBytes)) {
flow();
return false;
}
message.setBrokerProperty(INTERNAL_ID_EXTRA_PROPERTY, internalID);
message.setBrokerProperty(INTERNAL_BROKER_ID_EXTRA_PROPERTY, internalMirrorID);
if (internalAddress != null) {
message.setAddress(internalAddress);
}
routingContext.clear();
final TransactionImpl transaction = new MirrorTransaction(server.getStorageManager());
transaction.addOperation(messageCompletionAck.tx);
routingContext.setTransaction(transaction);
duplicateIDCache.addToCache(duplicateIDBytes, transaction);
routingContext.clear().setMirrorSource(this);
server.getPostOffice().route(message, routingContext, false);
// We use this as part of a transaction because of the duplicate detection cache that needs to be done atomically
transaction.commit();
flow();
// return true here will instruct the caller to ignore any references to messageCompletionAck
return true;
}
/**
* not implemented on the target, treated at {@link #postAcknowledge(String, String, long)}
*
* @param ref
* @param reason
*/
@ -356,13 +472,6 @@ public class AMQPMirrorControllerTarget extends ProtonAbstractReceiver implement
public void postAcknowledge(MessageReference ref, AckReason reason) {
}
/**
* not implemented on the target, treated at {@link #sendMessage(AMQPMessage)}
*
* @param message
* @param context
* @param refs
*/
@Override
public void sendMessage(Message message, RoutingContext context, List<MessageReference> refs) {
}

View File

@ -36,6 +36,7 @@ import org.apache.qpid.proton.codec.EncoderImpl;
import org.apache.qpid.proton.codec.WritableBuffer;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.ADDRESS;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.BROKER_ID;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.EVENT_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.QUEUE;
@ -48,7 +49,7 @@ public class AMQPMirrorMessageFactory {
* This method is open to make it testable,
* do not use on your applications.
*/
public static Message createMessage(String to, SimpleString address, SimpleString queue, Object event, Object body) {
public static Message createMessage(String to, SimpleString address, SimpleString queue, Object event, String brokerID, Object body) {
Header header = new Header();
header.setDurable(true);
@ -57,6 +58,9 @@ public class AMQPMirrorMessageFactory {
if (address != null) {
annotations.put(ADDRESS, address.toString());
}
if (brokerID != null) {
annotations.put(BROKER_ID, brokerID);
}
if (queue != null) {
annotations.put(QUEUE, queue.toString());
}

View File

@ -0,0 +1,65 @@
/*
* 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.protocol.amqp.connect.mirror;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.qpid.proton.engine.Link;
public class BasicMirrorController<T extends Link> {
protected final ActiveMQServer server;
protected T link;
protected String remoteMirrorId;
protected final String localMirrorId;
public String getLocalMirrorId() {
return this.localMirrorId;
}
public String getRemoteMirrorId() {
if (remoteMirrorId == null) {
remoteMirrorId = getRemoteMirrorID(link);
}
return remoteMirrorId;
}
public static String getRemoteMirrorID(Link link) {
if ( link != null && link.getRemoteProperties() != null && link.getRemoteProperties().containsKey(AMQPMirrorControllerSource.BROKER_ID)) {
return (String)link.getRemoteProperties().get(AMQPMirrorControllerSource.BROKER_ID);
} else {
return null;
}
}
public BasicMirrorController(ActiveMQServer server) {
this.server = server;
this.localMirrorId = server.getNodeID().toString();
}
public T getLink() {
return link;
}
public BasicMirrorController<T> setLink(T link) {
this.link = link;
return this;
}
}

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
*
* 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.protocol.amqp.connect.mirror;
import java.util.List;
import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.core.server.mirror.MirrorController;
import org.apache.activemq.artemis.core.transaction.TransactionOperation;
import org.apache.activemq.artemis.core.transaction.impl.TransactionImpl;
import org.jboss.logging.Logger;
public class MirrorTransaction extends TransactionImpl {
private static final Logger logger = Logger.getLogger(MirrorTransaction.class);
MirrorController controlInUse;
public MirrorTransaction(StorageManager storageManager) {
super(storageManager);
this.controlInUse = AMQPMirrorControllerTarget.getControllerInUse();
logger.debug("controlTarget = " + controlInUse + " transactionID = " + getID());
}
@Override
protected synchronized void afterCommit(List<TransactionOperation> operationsToComplete) {
MirrorController beforeController = AMQPMirrorControllerTarget.getControllerInUse();
AMQPMirrorControllerTarget.setControllerInUse(controlInUse);
try {
super.afterCommit(operationsToComplete);
} finally {
AMQPMirrorControllerTarget.setControllerInUse(beforeController);
}
}
}

View File

@ -0,0 +1,148 @@
/**
* 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.protocol.amqp.connect.mirror;
import java.util.HashMap;
import io.netty.util.collection.LongObjectHashMap;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.MessageReference;
import org.apache.activemq.artemis.utils.collections.NodeStore;
import org.apache.activemq.artemis.utils.collections.LinkedListImpl;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.INTERNAL_BROKER_ID_EXTRA_PROPERTY;
import static org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource.INTERNAL_ID_EXTRA_PROPERTY;
public class ReferenceNodeStore implements NodeStore<MessageReference> {
private final String serverID;
public ReferenceNodeStore(ActiveMQServer server) {
this.serverID = server.getNodeID().toString();
}
// This is where the messages are stored by server id...
HashMap<String, LongObjectHashMap<LinkedListImpl.Node<MessageReference>>> lists;
String lruListID;
LongObjectHashMap<LinkedListImpl.Node<MessageReference>> lruMap;
@Override
public void storeNode(MessageReference element, LinkedListImpl.Node<MessageReference> node) {
String list = getServerID(element);
long id = getID(element);
storeNode(list, id, node);
}
private void storeNode(String serverID, long id, LinkedListImpl.Node<MessageReference> node) {
LongObjectHashMap<LinkedListImpl.Node<MessageReference>> nodesMap = getMap(serverID);
if (nodesMap != null) {
synchronized (nodesMap) {
nodesMap.put(id, node);
}
}
}
@Override
public void removeNode(MessageReference element, LinkedListImpl.Node<MessageReference> node) {
long id = getID(element);
String serverID = getServerID(element);
LongObjectHashMap<LinkedListImpl.Node<MessageReference>> nodeMap = getMap(serverID);
if (nodeMap != null) {
synchronized (nodeMap) {
nodeMap.remove(id);
}
}
}
@Override
public LinkedListImpl.Node<MessageReference> getNode(String serverID, long id) {
LongObjectHashMap<LinkedListImpl.Node<MessageReference>> nodeMap = getMap(serverID);
assert nodeMap != null;
synchronized (nodeMap) {
return nodeMap.get(id);
}
}
/** notice getMap should always return an instance. It should never return null. */
private synchronized LongObjectHashMap<LinkedListImpl.Node<MessageReference>> getMap(String serverID) {
if (serverID == null) {
serverID = this.serverID; // returning for the localList in case it's null
}
if (lruListID != null && lruListID.equals(serverID)) {
return lruMap;
}
if (lists == null) {
lists = new HashMap<>();
}
LongObjectHashMap<LinkedListImpl.Node<MessageReference>> theList = lists.get(serverID);
if (theList == null) {
theList = new LongObjectHashMap<>();
lists.put(serverID, theList);
}
lruMap = theList; // cached result
lruListID = serverID;
return theList;
}
public String getServerID(MessageReference element) {
Object nodeID = element.getMessage().getBrokerProperty(INTERNAL_BROKER_ID_EXTRA_PROPERTY);
if (nodeID != null) {
return nodeID.toString();
} else {
// it is important to return null here, as the MirrorSource is expecting it to be null
// in the case the nodeID being from the originating server.
// don't be tempted to return this.serverID here.
return null;
}
}
public long getID(MessageReference element) {
Long id = (Long) element.getMessage().getBrokerProperty(INTERNAL_ID_EXTRA_PROPERTY);
if (id == null) {
return element.getMessageID();
} else {
return id;
}
}
@Override
public synchronized void clear() {
lists.forEach((k, v) -> v.clear());
lists.clear();
lruListID = null;
lruMap = null;
}
@Override
public int size() {
int size = 0;
for (LongObjectHashMap mapValue : lists.values()) {
size += mapValue.size();
}
return size;
}
}

View File

@ -63,7 +63,7 @@ public interface ActiveMQAMQPProtocolLogger extends BasicLogger {
@LogMessage(level = Logger.Level.INFO)
@Message(id = 111003, value = "\n*******************************************************************************************************************************" +
"\nSuccess on Server AMQP Connection {0} on {1} after {2} retries" +
"\nConnected on Server AMQP Connection {0} on {1} after {2} retries" +
"\n*******************************************************************************************************************************\n", format = Message.Format.MESSAGE_FORMAT)
void successReconnect(String name, String hostAndPort, int currentRetry);

View File

@ -87,4 +87,26 @@ public interface ActiveMQAMQPProtocolMessageBundle {
@Message(id = 119017, value = "not authorized to create producer, {0}", format = Message.Format.MESSAGE_FORMAT)
ActiveMQAMQPSecurityException securityErrorCreatingProducer(String message);
@Message(id = 119018, value = "link is missing an offered capability declaration {0}", format = Message.Format.MESSAGE_FORMAT)
ActiveMQAMQPIllegalStateException missingOfferedCapability(String capability);
@Message(id = 119019, value = "There is no brokerID defined on the target connection. Connection will be closed.", format = Message.Format.MESSAGE_FORMAT)
ActiveMQAMQPIllegalStateException missingBrokerID();
@Message(id = 119020, value = "The Broker Connection Open Callback Has Timed Out.", format = Message.Format.MESSAGE_FORMAT)
ActiveMQAMQPIllegalStateException brokerConnectionTimeout();
@Message(id = 119021, value = "The broker connection had a remote link closed unexpectedly", format = Message.Format.MESSAGE_FORMAT)
ActiveMQAMQPIllegalStateException brokerConnectionRemoteLinkClosed();
@Message(id = 119022, value = "The broker connection is trying to connect to itself. Check your configuration.", format = Message.Format.MESSAGE_FORMAT)
ActiveMQAMQPIllegalStateException brokerConnectionMirrorItself();
@Message(id = 119023, value = "Sender link refused for address {0}", format = Message.Format.MESSAGE_FORMAT)
ActiveMQAMQPIllegalStateException senderLinkRefused(String address);
@Message(id = 119024, value = "link is missing a desired capability declaration {0}", format = Message.Format.MESSAGE_FORMAT)
ActiveMQAMQPIllegalStateException missingDesiredCapability(String capability);
}

View File

@ -41,6 +41,7 @@ import org.apache.activemq.artemis.core.security.SecurityAuth;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPConnectionCallback;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPSessionCallback;
import org.apache.activemq.artemis.protocol.amqp.broker.ProtonProtocolManager;
import org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource;
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPException;
import org.apache.activemq.artemis.protocol.amqp.logger.ActiveMQAMQPProtocolLogger;
import org.apache.activemq.artemis.protocol.amqp.logger.ActiveMQAMQPProtocolMessageBundle;
@ -94,6 +95,8 @@ public class AMQPConnectionContext extends ProtonInitializable implements EventH
private final Map<Symbol, Object> connectionProperties = new HashMap<>();
private final ScheduledExecutorService scheduledPool;
private LinkCloseListener linkCloseListener;
private final Map<Session, AMQPSessionContext> sessions = new ConcurrentHashMap<>();
private final ProtonProtocolManager protocolManager;
@ -118,8 +121,8 @@ public class AMQPConnectionContext extends ProtonInitializable implements EventH
ClientSASLFactory saslClientFactory,
Map<Symbol, Object> connectionProperties) {
this(protocolManager, connectionSP, containerId, idleTimeout, maxFrameSize, channelMax, useCoreSubscriptionNaming, scheduledPool, isIncomingConnection, saslClientFactory, connectionProperties, false);
}
public AMQPConnectionContext(ProtonProtocolManager protocolManager,
AMQPConnectionCallback connectionSP,
String containerId,
@ -132,7 +135,6 @@ public class AMQPConnectionContext extends ProtonInitializable implements EventH
ClientSASLFactory saslClientFactory,
Map<Symbol, Object> connectionProperties,
boolean bridgeConnection) {
this.protocolManager = protocolManager;
this.bridgeConnection = bridgeConnection;
this.connectionCallback = connectionSP;
@ -172,6 +174,15 @@ public class AMQPConnectionContext extends ProtonInitializable implements EventH
}
}
public LinkCloseListener getLinkCloseListener() {
return linkCloseListener;
}
public AMQPConnectionContext setLinkCloseListener(LinkCloseListener linkCloseListener) {
this.linkCloseListener = linkCloseListener;
return this;
}
public boolean isBridgeConnection() {
return bridgeConnection;
}
@ -310,6 +321,13 @@ public class AMQPConnectionContext extends ProtonInitializable implements EventH
AMQPSessionContext protonSession = getSessionExtension(link.getSession());
Runnable runnable = link.attachments().get(Runnable.class, Runnable.class);
if (runnable != null) {
link.attachments().set(Runnable.class, Runnable.class, null);
runnable.run();
return;
}
if (link.getLocalState() == EndpointState.ACTIVE) { // if already active it's probably from the AMQP bridge and hence we just ignore it
return;
}
@ -324,10 +342,26 @@ public class AMQPConnectionContext extends ProtonInitializable implements EventH
} else {
if (isReplicaTarget(receiver)) {
try {
protonSession.getSessionSPI().check(SimpleString.toSimpleString(ProtonProtocolManager.MIRROR_ADDRESS), CheckType.SEND, getSecurityAuth());
} catch (ActiveMQSecurityException e) {
throw ActiveMQAMQPProtocolMessageBundle.BUNDLE.securityErrorCreatingProducer(e.getMessage());
try {
protonSession.getSessionSPI().check(SimpleString.toSimpleString(link.getTarget().getAddress()), CheckType.SEND, getSecurityAuth());
} catch (ActiveMQSecurityException e) {
throw ActiveMQAMQPProtocolMessageBundle.BUNDLE.securityErrorCreatingProducer(e.getMessage());
}
if (!verifyDesiredCapabilities(receiver, AMQPMirrorControllerSource.MIRROR_CAPABILITY)) {
throw ActiveMQAMQPProtocolMessageBundle.BUNDLE.missingDesiredCapability(AMQPMirrorControllerSource.MIRROR_CAPABILITY.toString());
}
} catch (ActiveMQAMQPException e) {
log.warn(e.getMessage(), e);
link.setTarget(null);
link.setCondition(new ErrorCondition(e.getAmqpError(), e.getMessage()));
link.close();
return;
}
receiver.setOfferedCapabilities(new Symbol[]{AMQPMirrorControllerSource.MIRROR_CAPABILITY});
protonSession.addReplicaTarget(receiver);
} else {
protonSession.addReceiver(receiver);
@ -339,8 +373,30 @@ public class AMQPConnectionContext extends ProtonInitializable implements EventH
}
}
protected boolean verifyDesiredCapabilities(Receiver reciever, Symbol s) {
if (reciever.getRemoteDesiredCapabilities() == null) {
return false;
}
boolean foundS = false;
for (Symbol b : reciever.getRemoteDesiredCapabilities()) {
if (b.equals(s)) {
foundS = true;
break;
}
}
if (!foundS) {
return false;
}
return true;
}
private boolean isReplicaTarget(Link link) {
return link != null && link.getTarget() != null && link.getTarget().getAddress() != null && link.getTarget().getAddress().equals(ProtonProtocolManager.MIRROR_ADDRESS);
return link != null && link.getTarget() != null && link.getTarget().getAddress() != null && link.getTarget().getAddress().startsWith(ProtonProtocolManager.MIRROR_ADDRESS);
}
public Symbol[] getConnectionCapabilitiesOffered() {
@ -610,27 +666,26 @@ public class AMQPConnectionContext extends ProtonInitializable implements EventH
public void onRemoteClose(Link link) throws Exception {
handler.requireHandler();
// We scheduled it for later, as that will work through anything that's pending on the current deliveries.
runNow(() -> {
if (linkCloseListener != null) {
linkCloseListener.onClose(link);
}
ProtonDeliveryHandler linkContext = (ProtonDeliveryHandler) link.getContext();
if (linkContext != null) {
try {
linkContext.close(true);
} catch (Exception e) {
log.error(e.getMessage(), e);
}
ProtonDeliveryHandler linkContext = (ProtonDeliveryHandler) link.getContext();
if (linkContext != null) {
try {
linkContext.close(true);
} catch (Exception e) {
log.error(e.getMessage(), e);
}
}
/// we have to perform the link.close after the linkContext.close is finished.
// linkeContext.close will perform a few executions on the netty loop,
// this has to come next
runLater(() -> {
link.close();
link.free();
flush();
});
/// we have to perform the link.close after the linkContext.close is finished.
// linkeContext.close will perform a few executions on the netty loop,
// this has to come next
runLater(() -> {
link.close();
link.free();
flush();
});
}

View File

@ -16,6 +16,7 @@
*/
package org.apache.activemq.artemis.protocol.amqp.proton;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
@ -25,6 +26,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQSecurityException;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.ServerProducer;
import org.apache.activemq.artemis.core.server.impl.ServerProducerImpl;
import org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource;
import org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerTarget;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPSessionCallback;
import org.apache.activemq.artemis.protocol.amqp.client.ProtonClientSenderContext;
@ -34,6 +36,7 @@ import org.apache.activemq.artemis.protocol.amqp.proton.transaction.ProtonTransa
import org.apache.qpid.proton.amqp.Symbol;
import org.apache.qpid.proton.amqp.transaction.Coordinator;
import org.apache.qpid.proton.amqp.transport.ErrorCondition;
import org.apache.qpid.proton.engine.EndpointState;
import org.apache.qpid.proton.engine.Receiver;
import org.apache.qpid.proton.engine.Sender;
import org.apache.qpid.proton.engine.Session;
@ -189,10 +192,13 @@ public class AMQPSessionContext extends ProtonInitializable {
senders.put(sender, protonSender);
serverSenders.put(protonSender.getBrokerConsumer(), protonSender);
sender.setContext(protonSender);
connection.runNow(() -> {
sender.open();
connection.flush();
});
if (sender.getLocalState() != EndpointState.ACTIVE) {
connection.runNow(() -> {
sender.open();
connection.flush();
});
}
protonSender.start();
} catch (ActiveMQAMQPException e) {
@ -224,6 +230,9 @@ public class AMQPSessionContext extends ProtonInitializable {
ServerProducer serverProducer = new ServerProducerImpl(receiver.getName(), "AMQP", receiver.getTarget().getAddress());
sessionSPI.addProducer(serverProducer);
receiver.setContext(protonReceiver);
HashMap<Symbol, Object> brokerIDProperties = new HashMap<>();
brokerIDProperties.put(AMQPMirrorControllerSource.BROKER_ID, server.getNodeID().toString());
receiver.setProperties(brokerIDProperties);
connection.runNow(() -> {
receiver.open();
connection.flush();

View File

@ -0,0 +1,24 @@
/**
* 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.protocol.amqp.proton;
import org.apache.qpid.proton.engine.Link;
public interface LinkCloseListener {
void onClose(Link objectClosed);
}

View File

@ -93,6 +93,10 @@ public abstract class ProtonAbstractReceiver extends ProtonInitializable impleme
}
}
protected void recoverContext() {
sessionSPI.recoverContext();
}
protected void clearLargeMessage() {
connection.runNow(() -> {

View File

@ -234,16 +234,13 @@ public class ProtonServerSenderContext extends ProtonInitializable implements Pr
*/
public void start() throws ActiveMQAMQPException {
sessionSPI.start();
// protonSession.getServerSession().start();
// todo add flow control
try {
// to do whatever you need to make the broker start sending messages to the consumer
// this could be null if a link reattach has happened
if (brokerConsumer != null) {
sessionSPI.startSender(brokerConsumer);
}
// protonSession.getServerSession().receiveConsumerCredits(consumerID, -1);
} catch (Exception e) {
throw ActiveMQAMQPProtocolMessageBundle.BUNDLE.errorStartingConsumer(e.getMessage());
}

View File

@ -17,12 +17,9 @@
package org.apache.activemq.artemis.core.config.amqpBrokerConnectivity;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.utils.RandomUtil;
public class AMQPMirrorBrokerConnectionElement extends AMQPBrokerConnectionElement {
SimpleString sourceMirrorAddress;
boolean durable;
boolean queueCreation = true;
@ -31,6 +28,17 @@ public class AMQPMirrorBrokerConnectionElement extends AMQPBrokerConnectionEleme
boolean messageAcknowledgements = true;
SimpleString mirrorSNF;
public SimpleString getMirrorSNF() {
return mirrorSNF;
}
public AMQPMirrorBrokerConnectionElement setMirrorSNF(SimpleString mirrorSNF) {
this.mirrorSNF = mirrorSNF;
return this;
}
public AMQPMirrorBrokerConnectionElement() {
this.setType(AMQPBrokerConnectionAddressType.MIRROR);
}
@ -41,22 +49,11 @@ public class AMQPMirrorBrokerConnectionElement extends AMQPBrokerConnectionEleme
return durable;
}
public AMQPMirrorBrokerConnectionElement setSourceMirrorAddress(String mirrorAddress) {
return this.setSourceMirrorAddress(SimpleString.toSimpleString(mirrorAddress));
}
public AMQPMirrorBrokerConnectionElement setSourceMirrorAddress(SimpleString souceMirrorAddress) {
this.sourceMirrorAddress = souceMirrorAddress;
this.durable = sourceMirrorAddress != null;
public AMQPMirrorBrokerConnectionElement setDurable(boolean durable) {
this.durable = durable;
return this;
}
public SimpleString getSourceMirrorAddress() {
if (sourceMirrorAddress == null) {
sourceMirrorAddress = SimpleString.toSimpleString(parent.getName() + RandomUtil.randomString());
}
return sourceMirrorAddress;
}
public boolean isQueueCreation() {
return queueCreation;
}

View File

@ -1982,11 +1982,10 @@ public final class FileConfigurationParser extends XMLConfigurationUtil {
if (nodeType == AMQPBrokerConnectionAddressType.MIRROR) {
boolean messageAcks = getBooleanAttribute(e2, "message-acknowledgements", true);
boolean queueCreation = getBooleanAttribute(e2,"queue-creation", true);
boolean durable = getBooleanAttribute(e2, "durable", true);
boolean queueRemoval = getBooleanAttribute(e2, "queue-removal", true);
String sourceMirrorAddress = getAttributeValue(e2, "source-mirror-address");
AMQPMirrorBrokerConnectionElement amqpMirrorConnectionElement = new AMQPMirrorBrokerConnectionElement();
amqpMirrorConnectionElement.setMessageAcknowledgements(messageAcks).setQueueCreation(queueCreation).setQueueRemoval(queueRemoval).
setSourceMirrorAddress(sourceMirrorAddress);
amqpMirrorConnectionElement.setMessageAcknowledgements(messageAcks).setQueueCreation(queueCreation).setQueueRemoval(queueRemoval).setDurable(durable);
connectionElement = amqpMirrorConnectionElement;
connectionElement.setType(AMQPBrokerConnectionAddressType.MIRROR);
} else {

View File

@ -197,6 +197,8 @@ public interface PostOffice extends ActiveMQComponent {
DuplicateIDCache getDuplicateIDCache(SimpleString address);
DuplicateIDCache getDuplicateIDCache(SimpleString address, int idCacheSize);
void sendQueueInfoToQueue(SimpleString queueName, SimpleString address) throws Exception;
Object getNotificationLock();

View File

@ -491,7 +491,7 @@ public class PostOfficeImpl implements PostOffice, NotificationListener, Binding
server.callBrokerAddressPlugins(plugin -> plugin.beforeAddAddress(addressInfo, reload));
}
if (mirrorControllerSource != null) {
if (!reload && mirrorControllerSource != null) {
mirrorControllerSource.addAddress(addressInfo);
}
@ -1327,13 +1327,18 @@ public class PostOfficeImpl implements PostOffice, NotificationListener, Binding
@Override
public DuplicateIDCache getDuplicateIDCache(final SimpleString address) {
return getDuplicateIDCache(address, idCacheSize);
}
@Override
public DuplicateIDCache getDuplicateIDCache(final SimpleString address, int cacheSizeToUse) {
DuplicateIDCache cache = duplicateIDCaches.get(address);
if (cache == null) {
if (persistIDCache) {
cache = DuplicateIDCaches.persistent(address, idCacheSize, storageManager);
cache = DuplicateIDCaches.persistent(address, cacheSizeToUse, storageManager);
} else {
cache = DuplicateIDCaches.inMemory(address, idCacheSize);
cache = DuplicateIDCaches.inMemory(address, cacheSizeToUse);
}
DuplicateIDCache oldCache = duplicateIDCaches.putIfAbsent(address, cache);

View File

@ -376,19 +376,21 @@ public class SimpleAddressManager implements AddressManager {
@Override
public void scanAddresses(MirrorController mirrorController) throws Exception {
mirrorController.startAddressScan();
for (AddressInfo info : addressInfoMap.values()) {
mirrorController.addAddress(info);
if (!info.isInternal()) {
mirrorController.addAddress(info);
}
Bindings bindings = mappings.get(info.getName());
if (bindings != null) {
for (Binding binding : bindings.getBindings()) {
if (binding instanceof LocalQueueBinding) {
LocalQueueBinding localQueueBinding = (LocalQueueBinding)binding;
mirrorController.createQueue(localQueueBinding.getQueue().getQueueConfiguration());
if (!localQueueBinding.getQueue().isMirrorController() && !localQueueBinding.getQueue().isInternalQueue()) {
mirrorController.createQueue(localQueueBinding.getQueue().getQueueConfiguration());
}
}
}
}
}
mirrorController.endAddressScan();
}
}

View File

@ -38,6 +38,11 @@ public interface MessageReference {
return new MessageReferenceImpl(encode, queue);
}
}
default boolean skipDelivery() {
return false;
}
boolean isPaged();
Message getMessage();

View File

@ -20,7 +20,6 @@ import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Executor;
import java.util.function.ToLongFunction;
import org.apache.activemq.artemis.api.core.ActiveMQException;
import org.apache.activemq.artemis.api.core.Message;
@ -36,6 +35,7 @@ import org.apache.activemq.artemis.core.postoffice.Binding;
import org.apache.activemq.artemis.core.server.impl.AckReason;
import org.apache.activemq.artemis.core.transaction.Transaction;
import org.apache.activemq.artemis.utils.ReferenceCounter;
import org.apache.activemq.artemis.utils.collections.NodeStore;
import org.apache.activemq.artemis.utils.collections.LinkedListIterator;
import org.apache.activemq.artemis.utils.critical.CriticalComponent;
@ -76,8 +76,8 @@ public interface Queue extends Bindable,CriticalComponent {
/** Remove item with a supplied non-negative {@literal (>= 0) } ID.
* If the idSupplier returns {@literal < 0} the ID is considered a non value (null) and it will be ignored.
*
* @see org.apache.activemq.artemis.utils.collections.LinkedList#setIDSupplier(ToLongFunction) */
MessageReference removeWithSuppliedID(long id, ToLongFunction<MessageReference> idSupplier);
* @see org.apache.activemq.artemis.utils.collections.LinkedList#setNodeStore(NodeStore) */
MessageReference removeWithSuppliedID(String serverID, long id, NodeStore<MessageReference> nodeStore);
/**
* The queue definition could be durable, but the messages could eventually be considered non durable.
@ -193,6 +193,9 @@ public interface Queue extends Bindable,CriticalComponent {
void reload(MessageReference ref);
default void flushOnIntermediate(Runnable runnable) {
}
void addTail(MessageReference ref);
void addTail(MessageReference ref, boolean direct);
@ -409,6 +412,16 @@ public interface Queue extends Bindable,CriticalComponent {
*/
LinkedListIterator<MessageReference> iterator();
default void forEach(java.util.function.Consumer<MessageReference> consumer) {
synchronized (this) {
try (LinkedListIterator<MessageReference> iterator = iterator()) {
while (iterator.hasNext()) {
consumer.accept(iterator.next());
}
}
}
}
LinkedListIterator<MessageReference> browserIterator();
SimpleString getExpiryAddress();

View File

@ -22,6 +22,7 @@ import java.util.Map;
import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.api.core.RoutingType;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.core.server.mirror.MirrorController;
import org.apache.activemq.artemis.core.transaction.Transaction;
public interface RoutingContext {
@ -40,6 +41,10 @@ public interface RoutingContext {
* to avoid*/
boolean isMirrorController();
MirrorController getMirrorSource();
RoutingContext setMirrorSource(MirrorController mirrorController);
int getPreviousBindingsVersion();
SimpleString getPreviousAddress();

View File

@ -41,7 +41,6 @@ import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicLongFieldUpdater;
import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.ToLongFunction;
import org.apache.activemq.artemis.api.config.ActiveMQDefaultConfiguration;
import org.apache.activemq.artemis.api.core.ActiveMQException;
@ -110,6 +109,7 @@ import org.apache.activemq.artemis.utils.ReferenceCounter;
import org.apache.activemq.artemis.utils.ReusableLatch;
import org.apache.activemq.artemis.utils.actors.ArtemisExecutor;
import org.apache.activemq.artemis.utils.collections.ConcurrentHashSet;
import org.apache.activemq.artemis.utils.collections.NodeStore;
import org.apache.activemq.artemis.utils.collections.LinkedListIterator;
import org.apache.activemq.artemis.utils.collections.PriorityLinkedList;
import org.apache.activemq.artemis.utils.collections.PriorityLinkedListImpl;
@ -196,12 +196,12 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
// This is where messages are stored
private final PriorityLinkedList<MessageReference> messageReferences = new PriorityLinkedListImpl<>(QueueImpl.NUM_PRIORITIES, MessageReferenceImpl.getIDComparator());
private ToLongFunction<MessageReference> idSupplier;
private NodeStore<MessageReference> nodeStore;
private void checkIDSupplier(ToLongFunction<MessageReference> idSupplier) {
if (this.idSupplier != idSupplier) {
this.idSupplier = idSupplier;
messageReferences.setIDSupplier(idSupplier);
private void checkIDSupplier(NodeStore<MessageReference> nodeStore) {
if (this.nodeStore != nodeStore) {
this.nodeStore = nodeStore;
messageReferences.setNodeStore(nodeStore);
}
}
@ -1193,6 +1193,18 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
addTail(ref, false);
}
@Override
public void flushOnIntermediate(Runnable runnable) {
intermediateMessageReferences.add(new MessageReferenceImpl() {
@Override
public boolean skipDelivery() {
runnable.run();
return true;
}
});
deliverAsync();
}
@Override
public void addTail(final MessageReference ref, final boolean direct) {
try (ArtemisCloseable metric = measureCritical(CRITICAL_PATH_ADD_TAIL)) {
@ -1842,6 +1854,10 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
refsOperation = getRefsOperation(tx, reason);
}
if (logger.isTraceEnabled()) {
logger.trace(this + " acknowledge tx=" + transactional + " ref=" + ref + ", reason=" + reason + ", consumer=" + consumer);
}
if (nonDestructive && reason == AckReason.NORMAL) {
if (transactional) {
refsOperation.addOnlyRefAck(ref);
@ -2926,6 +2942,9 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
MessageReference ref;
while ((ref = intermediateMessageReferences.poll()) != null) {
if (ref.skipDelivery()) {
continue;
}
internalAddTail(ref);
if (!ref.isPaged()) {
@ -2946,7 +2965,7 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
*/
private boolean deliver() {
if (logger.isDebugEnabled()) {
logger.debug(this + " doing deliver. messageReferences=" + messageReferences.size());
logger.debug("Queue " + this.getName() + " doing deliver. messageReferences=" + messageReferences.size() + " with consumers=" + getConsumerCount());
}
scheduledRunners.decrementAndGet();
@ -3281,9 +3300,9 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
}
@Override
public MessageReference removeWithSuppliedID(long id, ToLongFunction<MessageReference> idSupplier) {
checkIDSupplier(idSupplier);
return messageReferences.removeWithID(id);
public synchronized MessageReference removeWithSuppliedID(String serverID, long id, NodeStore<MessageReference> nodeStore) {
checkIDSupplier(nodeStore);
return messageReferences.removeWithID(serverID, id);
}
private void internalAddRedistributor(final ArtemisExecutor executor) {

View File

@ -31,6 +31,7 @@ import org.apache.activemq.artemis.core.server.Queue;
import org.apache.activemq.artemis.core.server.RouteContextList;
import org.apache.activemq.artemis.core.server.RoutingContext;
import org.apache.activemq.artemis.api.core.RoutingType;
import org.apache.activemq.artemis.core.server.mirror.MirrorController;
import org.apache.activemq.artemis.core.transaction.Transaction;
import org.jboss.logging.Logger;
@ -51,6 +52,9 @@ public class RoutingContextImpl implements RoutingContext {
private RoutingType previousRoutingType;
/* To be set by the Mirror target on the server, to avoid ping pongs or reflections of messages between mirrors */
private MirrorController mirrorControllerSource;
private RoutingType routingType;
Boolean reusable = null;
@ -137,6 +141,17 @@ public class RoutingContextImpl implements RoutingContext {
return this;
}
@Override
public MirrorController getMirrorSource() {
return mirrorControllerSource;
}
@Override
public RoutingContext setMirrorSource(MirrorController mirrorController) {
this.mirrorControllerSource = mirrorController;
return this;
}
@Override
public void addQueue(final SimpleString address, final Queue queue) {

View File

@ -31,9 +31,6 @@ import org.apache.activemq.artemis.core.server.impl.AddressInfo;
* This represents the contract we will use to send messages to replicas.
* */
public interface MirrorController {
void startAddressScan() throws Exception;
void endAddressScan() throws Exception;
void addAddress(AddressInfo addressInfo) throws Exception;
void deleteAddress(AddressInfo addressInfo) throws Exception;
void createQueue(QueueConfiguration queueConfiguration) throws Exception;
@ -41,4 +38,6 @@ public interface MirrorController {
void sendMessage(Message message, RoutingContext context, List<MessageReference> refs);
void postAcknowledge(MessageReference ref, AckReason reason) throws Exception;
String getRemoteMirrorId();
}

View File

@ -17,9 +17,7 @@
package org.apache.activemq.artemis.core.transaction.impl;
import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.core.server.Queue;
import org.apache.activemq.artemis.core.server.impl.AckReason;
import org.apache.activemq.artemis.core.server.impl.RefsOperation;
public class BindingsTransactionImpl extends TransactionImpl {
@ -38,15 +36,11 @@ public class BindingsTransactionImpl extends TransactionImpl {
}
}
@Override
protected void doRollback() throws Exception {
if (isContainsPersistent()) {
storageManager.rollbackBindings(getID());
setState(State.ROLLEDBACK);
}
}
@Override
public RefsOperation createRefsOperation(Queue queue, AckReason reason) {
return null;
}
}

View File

@ -546,7 +546,7 @@ public class TransactionImpl implements Transaction {
// Private
// -------------------------------------------------------------------
private void doRollback() throws Exception {
protected void doRollback() throws Exception {
if (containsPersistent || xid != null && state == State.PREPARED) {
storageManager.rollback(id);
}
@ -558,13 +558,13 @@ public class TransactionImpl implements Transaction {
}
}
private synchronized void afterCommit(List<TransactionOperation> oeprationsToComplete) {
if (oeprationsToComplete != null) {
for (TransactionOperation operation : oeprationsToComplete) {
protected synchronized void afterCommit(List<TransactionOperation> operationsToComplete) {
if (operationsToComplete != null) {
for (TransactionOperation operation : operationsToComplete) {
operation.afterCommit(this);
}
// Help out GC here
oeprationsToComplete.clear();
operationsToComplete.clear();
}
}

View File

@ -1712,7 +1712,7 @@
<xsd:element name="concurrency" type="xsd:int" default="1" maxOccurs="1" minOccurs="0">
<xsd:annotation>
<xsd:documentation>
Number of concurrent workers, more workers can help increase throughput on high latency networks.
Number of concurrent workers, more workers can help increase throughput on high latency networks.
Defaults to 1
</xsd:documentation>
</xsd:annotation>
@ -2185,11 +2185,11 @@
</xsd:documentation>
</xsd:annotation>
</xsd:attribute>
<xsd:attribute name="source-mirror-address" type="xsd:string" use="optional" default="">
<xsd:attribute name="durable" type="xsd:boolean" use="optional" default="true">
<xsd:annotation>
<xsd:documentation>
By default the replica will use a temporary store and forward queue to store events towards the mirror / replica.
However if this is set, we will use a defined durable queue.
This property will determine if the mirror will use a durable queue or not as a Store and Forward Queue.
This is true by default.
</xsd:documentation>
</xsd:annotation>
</xsd:attribute>

View File

@ -20,6 +20,9 @@ import java.util.HashSet;
import java.util.Iterator;
import java.util.Objects;
import io.netty.util.collection.LongObjectHashMap;
import org.apache.activemq.artemis.utils.collections.NodeStore;
import org.apache.activemq.artemis.utils.collections.LinkedListImpl;
import org.apache.activemq.artemis.utils.collections.LinkedListIterator;
import org.apache.activemq.artemis.utils.collections.PriorityLinkedListImpl;
import org.junit.Assert;
@ -878,15 +881,44 @@ public final class PriorityLinkedListTest extends Assert {
@Test
public void testRemoveWithID() {
for (int i = 0; i < 3000; i++) {
for (int i = 1; i <= 3000; i++) {
list.addHead(new Wibble("" + i, i), i % 10);
}
list.setIDSupplier(source -> source.id);
class WibbleNodeStore implements NodeStore<Wibble> {
LongObjectHashMap<LinkedListImpl.Node<Wibble>> list = new LongObjectHashMap<>();
@Override
public void storeNode(Wibble element, LinkedListImpl.Node<Wibble> node) {
list.put(element.id, node);
}
@Override
public LinkedListImpl.Node<Wibble> getNode(String listID, long id) {
return list.get(id);
}
@Override
public void removeNode(Wibble element, LinkedListImpl.Node<Wibble> node) {
list.remove(element.id);
}
@Override
public void clear() {
list.clear();
}
@Override
public int size() {
return list.size();
}
}
list.setNodeStore(new WibbleNodeStore());
// remove every 3rd
for (int i = 0; i < 3000; i += 3) {
Assert.assertEquals(new Wibble("" + i, i), list.removeWithID(i));
for (int i = 3; i <= 3000; i += 3) {
Assert.assertEquals(new Wibble("" + i, i), list.removeWithID("", i));
}
Assert.assertEquals(2000, list.size());
@ -901,7 +933,7 @@ public final class PriorityLinkedListTest extends Assert {
Assert.assertEquals(2000, values.size());
for (int i = 0; i < 3000; i += 3) {
for (int i = 1; i <= 3000; i += 3) {
if (i % 3 == 0) {
Assert.assertFalse(values.contains("" + i));
} else {

View File

@ -29,7 +29,6 @@ import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.ToLongFunction;
import io.netty.buffer.ByteBuf;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
@ -57,6 +56,7 @@ import org.apache.activemq.artemis.utils.ActiveMQThreadFactory;
import org.apache.activemq.artemis.utils.RandomUtil;
import org.apache.activemq.artemis.utils.ReferenceCounter;
import org.apache.activemq.artemis.utils.UUID;
import org.apache.activemq.artemis.utils.collections.NodeStore;
import org.apache.activemq.artemis.utils.collections.LinkedListIterator;
import org.apache.activemq.artemis.utils.critical.CriticalComponentImpl;
import org.apache.activemq.artemis.utils.critical.EmptyCriticalAnalyzer;
@ -859,7 +859,7 @@ public class ScheduledDeliveryHandlerTest extends Assert {
}
@Override
public MessageReference removeWithSuppliedID(long id, ToLongFunction<MessageReference> idSupplier) {
public MessageReference removeWithSuppliedID(String serverID, long id, NodeStore<MessageReference> nodeStore) {
return null;
}
@ -1670,6 +1670,5 @@ public class ScheduledDeliveryHandlerTest extends Assert {
public void setExclusive(boolean exclusive) {
}
}
}

View File

@ -388,10 +388,10 @@
<receiver address-match="TEST-RECEIVER" />
<peer address-match="TEST-PEER"/>
<receiver queue-name="TEST-WITH-QUEUE-NAME"/>
<mirror message-acknowledgements="false" queue-creation="false" source-mirror-address="TEST-REPLICA" queue-removal="false"/>
<mirror message-acknowledgements="false" queue-creation="false" durable="false" queue-removal="false"/>
</amqp-connection>
<amqp-connection uri="tcp://test2:222" name="test2">
<mirror/>
<mirror durable="false"/>
</amqp-connection>
<amqp-connection uri="tcp://false" name="auto-start-false" auto-start="false">
<mirror/>

View File

@ -266,6 +266,21 @@
<discovery-group-ref discovery-group-name="dg1"/>
</cluster-connection>
</cluster-connections>
<broker-connections>
<amqp-connection uri="tcp://test1:111" name="test1" retry-interval="333" reconnect-attempts="33" user="testuser" password="testpassword">
<sender address-match="TEST-SENDER" />
<receiver address-match="TEST-RECEIVER" />
<peer address-match="TEST-PEER"/>
<receiver queue-name="TEST-WITH-QUEUE-NAME"/>
<mirror message-acknowledgements="false" queue-creation="false" durable="false" queue-removal="false"/>
</amqp-connection>
<amqp-connection uri="tcp://test2:222" name="test2">
<mirror durable="false"/>
</amqp-connection>
<amqp-connection uri="tcp://false" name="auto-start-false" auto-start="false">
<mirror/>
</amqp-connection>
</broker-connections>
<grouping-handler name="gh1">
<type>LOCAL</type>
<address>jms</address>

View File

@ -4,11 +4,11 @@ Instead of waiting for clients to connect, a broker can initiate a connection to
Currently, this feature supports only the AMQP protocol. However, in the future, it might be expanded to other protocols.
You configure broker connections using a `<broker-connections>` XML element in the `broker.xml` configuration file.
Broker connections are configured by the `<broker-connections>` XML element in the `broker.xml` configuration file.
```xml
<broker-connections>
...
...
</broker-connections>
```
@ -20,9 +20,9 @@ To define an AMQP broker connection, add an `<amqp-connection>` element within t
```xml
<broker-connections>
<amqp-connection uri="tcp://MY_HOST:MY_PORT" name="my-broker" retry-interval="100" reconnect-attempts="-1" user="john" password="doe">
...
</amqp-connection>
<amqp-connection uri="tcp://HOST:PORT" name="other-server" retry-interval="100" reconnect-attempts="-1" user="john" password="doe">
...
</amqp-connection>
</broker-connections>
```
@ -32,23 +32,147 @@ To define an AMQP broker connection, add an `<amqp-connection>` element within t
- `password`: Password with which to connect to the endpoint (this is an optional argument)
- `retry-interval`: Time, in milliseconds to wait before retrying a connection after an error. The default value is `5000`.
- `reconnect-attempts`: default is -1 meaning infinite
- `auto-start` : Should the broker connection start automatically with the broker. Default is `true`. If false you need to call a management operation to start it.
- `auto-start` : Should the broker connection start automatically with the broker. Default is `true`. If false it is necessary to call a management operation to start it.
*Notice:* If you disable auto-start on the broker connection, the start of the broker connection will only happen after the management method `startBrokerConnection(connectionName)` is called on the ServerController.
*Notice:* If auto-start is disabled on the broker connection, the start of the broker connection will only happen after the management method `startBrokerConnection(connectionName)` is called on the ServerController.
*Important*: The target endpoint needs permission for all operations that you configure. Therefore, If you are using a security manager, ensure that you perform the configured operations as a user with sufficient permissions.
*Important*: The target endpoint needs permission for all operations that configured. Therefore, If a security manager is being used, ensure to perform the configured operations with a user with sufficient permissions.
<div style="page-break-after: always"></div>
# AMQP Server Connection Operations
The following types of operations are supported on a AMQP server connection:
The following types of operations are supported on an AMQP server connection:
* Senders
* Messages received on specific queues are transferred to another endpoint
* Receivers
* The broker pulls messages from another endpoint
* Peers
* The broker creates both senders and receivers on another endpoint that knows how to handle them. Currently, this is implemented by Apache Qpid Dispatch.
* Mirrors
* The broker uses an AMQP connection to another broker and duplicate messages and sends acknowledgements over the wire.
* The broker uses an AMQP connection to another broker and duplicates messages and sends acknowledgements over the wire.
* Senders
* Messages received on specific queues are transferred to another endpoint.
* Receivers
* The broker pulls messages from another endpoint.
* Peers
* The broker creates both senders and receivers on another endpoint that knows how to handle them. This is currently implemented by Apache Qpid Dispatch.
<div style="page-break-after: always"></div>
# Reconnecting and Failover
It is possible to determine how reconnection will happen on a broker connection.
These are the attributes are available on amqp-connection XML element:
- reconnect-attempts: default as -1 (infinite). How many attempts will be done after a failed connection
- retry-interval: default as 5000, in milliseconds, the wait between each retry in connections.
It is also possible to specify alternate hosts on a broker connection by appending a comma separated list after a # at the end of the URI.
The broker connection would keep trying on the alternate list until one of the targets is available to connect. Example:
```xml
<broker-connections>
<amqp-connection uri="tcp://ServerA:5672#BackupA:5672,BackupB:5672" name="ServerA" reconnect-attempts="-1" retry-interval="5000">
...
</amqp-connection>
</broker-connections>
```
![Broker Connection Failover](images/broker-connection-failover.jpg)
Figure 1. Broker Connection - Reconnecting and Failover.
The previous example portrays a case of connection failure towards ServerA. The system would try to connect to serverA, backupA, and backupB until it successfully connects to one of these nodes.
<div style="page-break-after: always"></div>
## Mirroring
The idea of mirroring is to send events that happen on a broker towards another broker, without blocking any operations from producers and consumers, allowing them to keep operating as fast as possible.
It can be used for Disaster Recovery, and depending on the requirements even for failing over the data.
The following events are sent through mirroring:
* Message sending
* Messages sent to one broker will be "replicated" to the target broker.
* Message acknowledgement
* Acknowledgements removing messages at one broker will be sent to the target broker.
* Note that if the message is pending for a consumer on the target mirror, the ack will not succeed and the message might be delivered by both brokers.
* Queue and address creation.
* Queue and address deletion.
### Mirror configuration
Add a `<mirror>` element within the `<amqp-connection>` element to configure mirroring to the target broker.
The following optional arguments can be utilized:
* `queue-removal`: Specifies whether a queue- or address-removal event is sent. The default value is `true`.
* `message-acknowledgements`: Specifies whether message acknowledgements are sent. The default value is `true`.
* `queue-creation`: Specifies whether a queue- or address-creation event is sent. The default value is `true`.
An example of a mirror configuration is shown below:
```xml
<broker-connections>
<amqp-connection uri="tcp://HOST:PORT" name="mirror">
<mirror/>
</amqp-connection>
</broker-connections>
```
### Store and Forward Queue
Mirror events are always stored on a local queue prefixed as "$ACTIVEMQ_ARTEMIS_MIRROR_" and then concatenated with the broker connection's configured name.
So, in the following configuration mirror events will be stored on a queue named "$ACTIVEMQ_ARTEMIS_MIRROR_brokerB".
```xml
<broker-connection>
<amqp-connection uri="tcp://brokerB:5672" name="brokerB">
<mirror/>
</amqp-connection>
</broker-connection>
```
These messages are then transferred to brokerB:5672. A producer to the address $ACTIVEMQ_ARTEMIS_MIRROR_brokerB will be created towards brokerB. If there is a security manager configured, security roles must be provided to the user on the broker connection.
Notice the queue $ACTIVEMQ_ARTEMIS_MIRROR_brokerB will not actually exist on brokerB and so it wont be visible on the administration console. The target broker will treat these messages accordingly as mirror events and perform the appropriate operations at the target broker.
### Pre Existing Messages
The broker will only mirror messages arriving from the point in time the mirror was configured. Previously existing messages will not be forwarded to other brokers.
## Dual Mirror (Disaster Recovery)
ActiveMQ Artemis supports automatic fallback mirroring. Every sent message and every acknowledgement is asynchronously replicated to the mirrored broker.
On the following diagram, there will be two servers called DataCenter1, and DataCenter2. In order to have a dual mirror configuration, it is necessary is to add the mirror broker connection on each broker.xml:
![Broker Connection DR](images/broker-connection-DR.jpg)
Figure 2. Broker Connection - Disaster Recovery.
on DataCenter1, the following code should be added the broker.xml:
```xml
<broker-connections>
<amqp-connection uri="tcp://DataCenter2:5672" name="DC2">
<mirror/>
</amqp-connection>
</broker-connections>
```
The following xml should be added on DataCenter2's broker.xml:
```xml
<broker-connections>
<amqp-connection uri="tcp://DataCenter1:5672" name="DC1">
<mirror/>
</amqp-connection>
</broker-connections>
```
The broker connections will replicate sends and acknowledgements to the other broker, no matter where they originated. If messages are sent on DC1 (DataCenter1) these will be automatically transferred to DC2 (DataCenter2). Messages acknowledgements received on DC2 will be automatically related back to DC1.
The only exception to that rule would be if there were already consumers with pending messages on any server, where a mirrored acknowledgement will not prevent the message being consumed by both consumers. It is recommended to not have active consumers on both servers.
## Example
There is an example as part of the distribution showing dual broker configuration (or disaster recovery) under ./examples/features/broker-connection/disaster-recovery.
On the example two brokers are configured to mirror each other and whatever happens in one broker is immediately copied over to the other broker.
<div style="page-break-after: always"></div>
## Senders and Receivers
It is possible to connect an ActiveMQ Artemis broker to another AMQP endpoint simply by creating a sender or receiver broker connection element.
@ -57,9 +181,9 @@ For a `sender`, the broker creates a message consumer on a queue that sends mess
For a `receiver`, the broker creates a message producer on an address that receives messages from another AMQP endpoint.
Both elements work like a message bridge. However, there is no additional overhead required to process messages. Senders and receivers behave just like any other consumer or producer in ActiveMQ Artemis.
Both elements function as a message bridge. However, there is no additional overhead required to process messages. Senders and receivers behave just like any other consumer or producer in ActiveMQ Artemis.
You can configure senders or receivers for specific queues. You can also match senders and receivers to specific addresses or _sets_ of addresses, using wildcard expressions. When configuring a sender or receiver, you can set the following properties:
Specific queues can be configured by senders or receivers. Wildcard expressions can be used to match senders and receivers to specific addresses or _sets_ of addresses. When configuring a sender or receiver, the following properties can be set:
- `address-match`: Match the sender or receiver to a specific address or __set__ of addresses, using a wildcard expression
- `queue-name`: Configure the sender or receiver for a specific queue
@ -70,83 +194,85 @@ Some examples are shown below.
Using address expressions:
```xml
<broker-connections>
<amqp-connection uri="tcp://MY_HOST:MY_PORT" name="my-broker">
<sender address-match="queues.#"/>
<!-- notice the local queues for remotequeues.# need to be created on this broker -->
<receiver address-match="remotequeues.#"/>
</amqp-connection>
<amqp-connection uri="tcp://HOST:PORT" name="other-server">
<sender address-match="queues.#"/>
<!-- notice the local queues for remotequeues.# need to be created on this broker -->
<receiver address-match="remotequeues.#"/>
</amqp-connection>
</broker-connections>
<addresses>
<address name="remotequeues.A">
<anycast>
<queue name="remoteQueueA"/>
</anycast>
</address>
<address name="queues.B">
<anycast>
<queue name="localQueueB"/>
</anycast>
</address>
<address name="remotequeues.A">
<anycast>
<queue name="remoteQueueA"/>
</anycast>
</address>
<address name="queues.B">
<anycast>
<queue name="localQueueB"/>
</anycast>
</address>
</addresses>
```
Using queue names:
```xml
<broker-connections>
<amqp-connection uri="tcp://MY_HOST:MY_PORT" name="my-broker">
<receiver queue-name="remoteQueueA"/>
<sender queue-name="localQueueB"/>
</amqp-connection>
<amqp-connection uri="tcp://HOST:PORT" name="other-server">
<receiver queue-name="remoteQueueA"/>
<sender queue-name="localQueueB"/>
</amqp-connection>
</broker-connections>
<addresses>
<address name="remotequeues.A">
<anycast>
<queue name="remoteQueueA"/>
</anycast>
</address>
<address name="queues.B">
<anycast>
<queue name="localQueueB"/>
</anycast>
</address>
<address name="remotequeues.A">
<anycast>
<queue name="remoteQueueA"/>
</anycast>
</address>
<address name="queues.B">
<anycast>
<queue name="localQueueB"/>
</anycast>
</address>
</addresses>
```
*Important:* You can match a receiver only to a local queue that already exists. Therefore, if you are using receivers, make sure that you pre-create the queue locally. Otherwise, the broker cannot match the remote queues and addresses.
*Important:* Receivers can only be matched to a local queue that already exists. Therefore, if receivers are being used, ensure that queues are pre-created locally. Otherwise, the broker cannot match the remote queues and addresses.
*Important:* Do not create a sender and a receiver to the same destination. This creates an infinite loop of sends and receives.
<div style="page-break-after: always"></div>
## Peers
The broker can be configured as a peer which connects to the [Apache Qpid Dispatch Router](https://qpid.apache.org/components/dispatch-router/) and instructs it the broker it will act as a store-and-forward queue for a given AMQP waypoint address configured on the router. In this scenario, clients connect to a router to send and receive messages using a waypointed address, and the router routes these messages to or from the queue on the broker.
The broker can be configured as a peer which connects to the [Apache Qpid Dispatch Router](https://qpid.apache.org/components/dispatch-router/) and instructs it that the broker will act as a store-and-forward queue for a given AMQP waypoint address configured on the router. In this scenario, clients connect to a router to send and receive messages using a waypointed address, and the router routes these messages to or from the queue on the broker.
The peer configuration causes ActiveMQ Artemis to create a sender and receiver pair for each destination matched in the broker-connection configuration, with these carrying special configuration to let Qpid Dispatch know to collaborate with the broker. This replaces the traditional need of a router-initiated connection and auto-links.
You can experiment with advanced networking scenarios with Qpid Dispatch Router and get a lot of benefit from the AMQP protocol and its ecosystem.
Qpid Dispatch Router offers a lot of advanced networking options that be used together with ActiveMQ Artemis.
With a peer configuration, the same properties are present as when there are senders and receivers. For example, a configuration where queues with names beginning "queue." act as storage for the matching router waypoint address would be:
With a peer configuration, you have the same properties that you have on a sender and receiver. For example, a configuration where queues with names beginning "queue." act as storage for the matching router waypoint address would be:
```xml
<broker-connections>
<amqp-connection uri="tcp://MY_HOST:MY_PORT" name="my-router">
<peer address-match="queues.#"/>
</amqp-connection>
<amqp-connection uri="tcp://HOST:PORT" name="router">
<peer address-match="queues.#"/>
</amqp-connection>
</broker-connections>
<addresses>
<address name="queues.A">
<anycast>
<queue name="queues.A"/>
</anycast>
</address>
<address name="queues.B">
<address name="queues.A">
<anycast>
<queue name="queues.B"/>
<queue name="queues.A"/>
</anycast>
</address>
</address>
<address name="queues.B">
<anycast>
<queue name="queues.B"/>
</anycast>
</address>
</addresses>
```
There must be a matching address waypoint configuration on the router, instructing it that the particular router addresses the broker attaches to should be treated as waypoints. For example, a similar prefix- based router address configuration would be:
There must be a matching address waypoint configuration on the router instructing it that the particular router addresses the broker attaches to should be treated as waypoints. For example, a similar prefix- based router address configuration would be:
```
address {
@ -159,119 +285,26 @@ For more information refer to the "brokered messaging" documentation for [Apache
*Important:* Do not use this feature to connect to another broker, otherwise any message sent will be immediately ready to consume creating an infinite echo of sends and receives.
*Important:* You do not need to configure the router with a connector or auto-links to communicate with the broker. The brokers peer configuration replaces these aspects of the router waypoint usage.
*Important:* It is not necessary to configure the router with a connector or auto-links to communicate with the broker. The brokers peer configuration replaces these aspects of the router waypoint usage.
## Address Consideration
It is highly recommended that you keep `address name` and `queue name` the same, as when you use a queue with its distinct name (as in the following example), senders and receivers will always use the `address name` when creating the remote endpoint.
It is highly recommended that `address name` and `queue name` are the same. When a queue with its distinct name (as in the following example) is used, senders and receivers will always use the `address name` when creating the remote endpoint.
```xml
<broker-connections>
<amqp-connection uri="tcp://MY_HOST:MY_PORT" name="my-broker">
<sender address-match="queues.#"/>
</amqp-connection>
<amqp-connection uri="tcp://HOST:PORT" name="other-server">
<sender address-match="queues.#"/>
</amqp-connection>
</broker-connections>
<addresses>
<address name="queues.A">
<anycast>
<queue name="distinctNameQueue.A"/>
</anycast>
</address>
<address name="queues.A">
<anycast>
<queue name="distinctNameQueue.A"/>
</anycast>
</address>
</addresses>
```
In the above example the `broker connection` would create an AMQP sender towards "queues.A".
*Important:* To avoid confusion it is recommended that you keep the `address name` and `queue name` the same.
## Mirror
The mirror option on the broker connection can capture events from the broker and pass them over the wire to another broker. This enables you to capture multiple asynchronous replicas. The following types of events are captured:
* Message routing
* Message acknowledgement
* Queue and address creation
* queue and address deletion
When you configure a mirror, these events are captured from the broker, stored on a local queue, and later forwarded to a target destination on another ActiveMQ Artemis broker.
To configure a mirror, you add a `<mirror>` element within the `<amqp-connection>` element.
The local queue is called `source-mirror-address`
You can specify the following optional arguments.
* `queue-removal`: Specifies whether a queue- or address-removal event is sent. The default value is `true`.
* `message-acknowledgements`: Specifies whether message acknowledgements are sent. The default value is `true`.
* `queue-creation`: Specifies whether a queue- or address-creation event is sent. The default value is `true`.
* `source-mirror-address`: By default, the mirror creates a non-durable temporary queue to store messages before they are sent to the other broker. If you define a name value for this property, an ANYCAST durable queue and address is created with the specified name.
An example of a mirror configuration is shown below:
```xml
<broker-connections>
<amqp-connection uri="tcp://MY_HOST:MY_PORT" name="my-mirror">
<mirror queue-removal="true" queue-creation="true" message-acknowledgements="true" source-mirror-address="myLocalSNFMirrorQueue"/>
</amqp-connection>
</broker-connections>
```
*Important*: A broker can mirror to multiple replicas (1 to many). However a replica broker can only have a single mirror source. Make sure you do not mirror multiple source brokers to a single replica broker.
### Pre Existing Messages
The broker will not send pre existing messages through the mirror. So, If you add mirror to your configuration and the journal had pre existing messages these messages will not be sent.
## Broker Connection Stop and Disconnect
Once you start the broker connection with a mirror the mirror events will always be sent to the intermediate queue configured at the `source-mirror-address`.
It is possible to stop the broker connection with the operation stopBrokerConnection(connectionName) on the ServerControl, but it is only effective to disconnect the brokers, while the mirror events are always captured.
## Disaster & Recovery Considerations
As you use the mirror option to replicate data across datacenters, you have to take a few considerations:
* Currently we don't support quorums for activating the replica, so you have to manually control when your clients connect to the replica site.
* Make sure the replica site is passive. Having producers and consumers connected into both sites would be messy and could lead you to data integrity issues.
* You can disable auto-start on the acceptor your clients use to connect, and only enable it after a disaster has occurred.
* Only the queues and addresses are mirrored. Consumer states will have to be reapplied on the replica when the clients reconnects (that applies to message groups, exclusive consumers or anything related to clients)
* Make sure your configuration options are copied over, including Diverts, security, last value queues, address settings and other configuration options.
* Have a way back route after a disaster.
* You can have a disabled broker connection to be enabled after the disaster.
## Mirror example with Failback
On this example lets play with two brokers:
- sourceBroker
- replicaBroker
Add this configuration on sourceBroker:
```xml
<broker-connections>
<amqp-connection uri="tcp://replicaBroker:6700" name="DRSite">
<mirror message-acknowledgements="true"/>
</amqp-connection>
</broker-connections>
```
On the replicaBroker, add a disabled broker connection for failing back after a disaster, and also set the acceptors with autoStart=false
```xml
<acceptors>
<!-- this one is for clients -->
<acceptor name="artemis">tcp://0.0.0.0:61616?autoStart=false;tcpSendBufferSize=1048576;tcpReceiveBufferSize=1048576;amqpMinLargeMessageSize=102400;protocols=CORE,AMQP,STOMP,HORNETQ,MQTT,OPENWIRE;useEpoll=true;amqpCredits=1000;amqpLowCredits=300;amqpDuplicateDetection=true;autoStart=false</acceptor>
<!-- this one is for DR communication -->
<acceptor name="amqp">tcp://0.0.0.0:6700?autoStart=true;tcpSendBufferSize=1048576;tcpReceiveBufferSize=1048576;protocols=AMQP;useEpoll=true;amqpCredits=1000;amqpLowCredits=300;amqpMinLargeMessageSize=102400;amqpDuplicateDetection=true;autoStart=false</acceptor>
</acceptors>
<broker-connections>
<amqp-connection uri="tcp://sourceBroker:6700" name="failbackBroker" auto-start="false">
<mirror message-acknowledgements="true"/>
</amqp-connection>
</broker-connections>
```
After a failure has occurred, you can use a management operation start on the acceptor:
- AcceptorControl.start();
And you can call startBrokerConnection to enable the failback towards the live site:
- ActiveMQServerControl.startBrokerConnection("failbackBroker")
In the above example the `broker connection` would create an AMQP sender towards "queues.A".
*Important:* To avoid confusion it is recommended that `address name` and `queue name` are kept the same.

Binary file not shown.

After

Width:  |  Height:  |  Size: 10 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 12 KiB

View File

@ -0,0 +1,165 @@
<?xml version='1.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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.activemq.examples.broker-connection</groupId>
<artifactId>broker-connections</artifactId>
<version>2.18.0-SNAPSHOT</version>
</parent>
<artifactId>disaster-recovery</artifactId>
<packaging>jar</packaging>
<name>disaster-recovery</name>
<properties>
<activemq.basedir>${project.basedir}/../../../..</activemq.basedir>
</properties>
<dependencies>
<dependency>
<groupId>org.apache.qpid</groupId>
<artifactId>qpid-jms-client</artifactId>
<version>${qpid.jms.version}</version>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.activemq</groupId>
<artifactId>artemis-maven-plugin</artifactId>
<executions>
<execution>
<id>create0</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<ignore>${noServer}</ignore>
<instance>${basedir}/target/server0</instance>
<allowAnonymous>true</allowAnonymous>
<configuration>${basedir}/target/classes/activemq/server0</configuration>
<!-- this makes it easier in certain envs -->
<javaOptions>-Djava.net.preferIPv4Stack=true</javaOptions>
</configuration>
</execution>
<execution>
<id>create1</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<ignore>${noServer}</ignore>
<instance>${basedir}/target/server1</instance>
<allowAnonymous>true</allowAnonymous>
<configuration>${basedir}/target/classes/activemq/server1</configuration>
<!-- this makes it easier in certain envs -->
<javaOptions>-Djava.net.preferIPv4Stack=true</javaOptions>
</configuration>
</execution>
<execution>
<id>start1</id>
<goals>
<goal>cli</goal>
</goals>
<configuration>
<ignore>${noServer}</ignore>
<spawn>true</spawn>
<location>${basedir}/target/server1</location>
<testURI>tcp://localhost:5661</testURI>
<args>
<param>run</param>
</args>
<name>server1</name>
</configuration>
</execution>
<execution>
<id>start0</id>
<goals>
<goal>cli</goal>
</goals>
<configuration>
<spawn>true</spawn>
<ignore>${noServer}</ignore>
<location>${basedir}/target/server0</location>
<testURI>tcp://localhost:5660</testURI>
<args>
<param>run</param>
</args>
<name>server0</name>
</configuration>
</execution>
<execution>
<id>runClient</id>
<goals>
<goal>runClient</goal>
</goals>
<configuration>
<!-- you may have to set export MAVEN_OPTS="-Djava.net.preferIPv4Stack=true"
if you are on MacOS for instance -->
<clientClass>org.apache.activemq.artemis.jms.example.DisasterAndRecovery</clientClass>
</configuration>
</execution>
<execution>
<id>stop0</id>
<goals>
<goal>cli</goal>
</goals>
<configuration>
<ignore>${noServer}</ignore>
<location>${basedir}/target/server0</location>
<args>
<param>stop</param>
</args>
</configuration>
</execution>
<execution>
<id>stop1</id>
<goals>
<goal>cli</goal>
</goals>
<configuration>
<ignore>${noServer}</ignore>
<location>${basedir}/target/server1</location>
<args>
<param>stop</param>
</args>
</configuration>
</execution>
</executions>
<dependencies>
<dependency>
<groupId>org.apache.activemq.examples.broker-connection</groupId>
<artifactId>disaster-recovery</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-clean-plugin</artifactId>
</plugin>
</plugins>
</build>
</project>

View File

@ -0,0 +1,25 @@
# AMQP Broker Disaster Recovery
To run the example, simply type **mvn verify** from this directory, or **mvn -PnoServer verify** if you want to create and start the broker manually.
On this broker you will have two brokers connected to each other.
This broker configure two servers:
- server0
- server1
Each broker has a broker connection towards the other broker, with a mirror tag configured:
```xml
<broker-connections>
<amqp-connection uri="tcp://localhost:5661" name="otherBroker" retry-interval="1000">
<mirror/>
</amqp-connection>
</broker-connections>
```
Watever happens in server0 is replicated to server1 and vice versa.
In case you want to play this in real life, all you have to do is to move your consumers and producers from one broker towards the other broker.

View File

@ -0,0 +1,87 @@
/*
* 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.jms.example;
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 org.apache.qpid.jms.JmsConnectionFactory;
/**
* On this example, two brokers are mirrored.
* Everything that is happening on the first broker will be mirrored on the second, and Vice Versa.
*/
public class DisasterAndRecovery {
public static void main(final String[] args) throws Exception {
ConnectionFactory cfServer0 = new JmsConnectionFactory("amqp://localhost:5660");
ConnectionFactory cfServer1 = new JmsConnectionFactory("amqp://localhost:5661");
try (Connection connection = cfServer0.createConnection()) {
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
Queue queue = session.createQueue("exampleQueue");
MessageProducer producer = session.createProducer(queue);
for (int i = 0; i < 100; i++) {
producer.send(session.createTextMessage("Message " + i));
}
}
// Every message send on server0, will be mirrored into server1
try (Connection connection = cfServer1.createConnection()) {
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
Queue queue = session.createQueue("exampleQueue");
connection.start();
MessageConsumer consumer = session.createConsumer(queue);
// we will consume only half of the messages on this server
for (int i = 0; i < 50; i++) {
TextMessage message = (TextMessage) consumer.receive(5000);
System.out.println("Received Message on server1: " + message.getText());
if (!message.getText().equals("Message " + i)) {
// This is really not supposed to happen. We will throw an exception and in case it happens it needs to be investigated
throw new IllegalStateException("Mirror Example is not working as expected");
}
}
}
// mirroring of acknowledgemnts are asynchronous They are fast but still asynchronous. So lets wait some time to let the ack be up to date between the servers
// a few milliseconds would do, but I'm waiting a second just in case
Thread.sleep(1000);
// Every message send on server0, will be mirrored into server1
try (Connection connection = cfServer0.createConnection()) {
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
Queue queue = session.createQueue("exampleQueue");
connection.start();
MessageConsumer consumer = session.createConsumer(queue);
// we will consume only half of the messages on this server
for (int i = 50; i < 100; i++) {
TextMessage message = (TextMessage) consumer.receive(5000);
System.out.println("Received Message on the original server0: " + message.getText());
if (!message.getText().equals("Message " + i)) {
// This is really not supposed to happen. We will throw an exception and in case it happens it needs to be investigated
throw new IllegalStateException("Mirror Example is not working as expected");
}
}
}
}
}

View File

@ -0,0 +1,76 @@
<?xml version="1.0" encoding="UTF-8" standalone="no"?>
<!--
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.
-->
<configuration xmlns="urn:activemq" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="urn:activemq /schema/artemis-configuration.xsd">
<core xmlns="urn:activemq:core">
<bindings-directory>./data/messaging/bindings</bindings-directory>
<journal-directory>./data/messaging/journal</journal-directory>
<large-messages-directory>./data/messaging/largemessages</large-messages-directory>
<paging-directory>./data/messaging/paging</paging-directory>
<journal-datasync>true</journal-datasync>
<journal-min-files>2</journal-min-files>
<journal-pool-files>10</journal-pool-files>
<journal-device-block-size>4096</journal-device-block-size>
<journal-file-size>10M</journal-file-size>
<!-- Acceptors -->
<acceptors>
<acceptor name="artemis">tcp://0.0.0.0:5660?tcpSendBufferSize=1048576;tcpReceiveBufferSize=1048576;amqpMinLargeMessageSize=102400;protocols=CORE,AMQP,STOMP,HORNETQ,MQTT,OPENWIRE;useEpoll=true;amqpCredits=1000;amqpLowCredits=300;amqpDuplicateDetection=true</acceptor>
</acceptors>
<broker-connections>
<amqp-connection uri="tcp://localhost:5661" name="otherBroker" retry-interval="1000">
<mirror/>
</amqp-connection>
</broker-connections>
<security-settings>
<security-setting match="#">
<permission type="createNonDurableQueue" roles="guest"/>
<permission type="deleteNonDurableQueue" roles="guest"/>
<permission type="createDurableQueue" roles="guest"/>
<permission type="deleteDurableQueue" roles="guest"/>
<permission type="createAddress" roles="guest"/>
<permission type="deleteAddress" roles="guest"/>
<permission type="consume" roles="guest"/>
<permission type="browse" roles="guest"/>
<permission type="send" roles="guest"/>
<!-- we need this otherwise ./artemis data imp wouldn't work -->
<permission type="manage" roles="guest"/>
</security-setting>
</security-settings>
<addresses>
<address name="exampleQueue">
<anycast>
<queue name="exampleQueue"/>
</anycast>
</address>
</addresses>
</core>
</configuration>

View File

@ -0,0 +1,76 @@
<?xml version="1.0" encoding="UTF-8" standalone="no"?>
<!--
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.
-->
<configuration xmlns="urn:activemq" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="urn:activemq /schema/artemis-configuration.xsd">
<core xmlns="urn:activemq:core">
<bindings-directory>./data/messaging/bindings</bindings-directory>
<journal-directory>./data/messaging/journal</journal-directory>
<large-messages-directory>./data/messaging/largemessages</large-messages-directory>
<paging-directory>./data/messaging/paging</paging-directory>
<journal-datasync>true</journal-datasync>
<journal-min-files>2</journal-min-files>
<journal-pool-files>10</journal-pool-files>
<journal-device-block-size>4096</journal-device-block-size>
<journal-file-size>10M</journal-file-size>
<acceptors>
<acceptor name="artemis">tcp://0.0.0.0:5661?tcpSendBufferSize=1048576;tcpReceiveBufferSize=1048576;amqpMinLargeMessageSize=102400;protocols=CORE,AMQP,STOMP,HORNETQ,MQTT,OPENWIRE;useEpoll=true;amqpCredits=1000;amqpLowCredits=300;amqpDuplicateDetection=true</acceptor>
</acceptors>
<broker-connections>
<amqp-connection uri="tcp://localhost:5660" name="otherBroker" retry-interval="1000">
<mirror/>
</amqp-connection>
</broker-connections>
<security-settings>
<security-setting match="#">
<permission type="createNonDurableQueue" roles="guest"/>
<permission type="deleteNonDurableQueue" roles="guest"/>
<permission type="createDurableQueue" roles="guest"/>
<permission type="deleteDurableQueue" roles="guest"/>
<permission type="createAddress" roles="guest"/>
<permission type="deleteAddress" roles="guest"/>
<permission type="consume" roles="guest"/>
<permission type="browse" roles="guest"/>
<permission type="send" roles="guest"/>
<!-- we need this otherwise ./artemis data imp wouldn't work -->
<permission type="manage" roles="guest"/>
</security-setting>
</security-settings>
<addresses>
<address name="exampleQueue">
<anycast>
<queue name="exampleQueue"/>
</anycast>
</address>
</addresses>
</core>
</configuration>

View File

@ -50,6 +50,7 @@ under the License.
<module>amqp-sending-messages</module>
<module>amqp-receiving-messages</module>
<module>amqp-sending-overssl</module>
<module>disaster-recovery</module>
</modules>
</profile>
<profile>
@ -58,6 +59,7 @@ under the License.
<module>amqp-sending-messages</module>
<module>amqp-receiving-messages</module>
<module>amqp-sending-overssl</module>
<module>disaster-recovery</module>
</modules>
</profile>
</profiles>

View File

@ -211,7 +211,6 @@ public class AmqpClientTestSupport extends AmqpTestSupport {
createAddressAndQueues(server);
}
return server;
}
@ -343,13 +342,6 @@ public class AmqpClientTestSupport extends AmqpTestSupport {
return getName() + "-" + index;
}
public AmqpClientTestSupport() {
}
public AmqpClientTestSupport(String connectorScheme, boolean useSSL) {
this.useSSL = useSSL;
}
protected void sendMessages(String destinationName, int count) throws Exception {
sendMessages(destinationName, count, null);
}
@ -449,7 +441,10 @@ public class AmqpClientTestSupport extends AmqpTestSupport {
sendMessagesOpenWire(destinationName, count, durable, null);
}
protected void sendMessagesOpenWire(String destinationName, int count, boolean durable, byte[] payload) throws Exception {
protected void sendMessagesOpenWire(String destinationName,
int count,
boolean durable,
byte[] payload) throws Exception {
ConnectionFactory cf = new ActiveMQConnectionFactory("tcp://127.0.0.1:5672");
Connection connection = cf.createConnection();
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);

View File

@ -0,0 +1,197 @@
/**
* 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.integration.amqp.connect;
import javax.jms.Connection;
import javax.jms.ConnectionFactory;
import javax.jms.MessageConsumer;
import javax.jms.MessageProducer;
import javax.jms.Session;
import javax.jms.TextMessage;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPBrokerConnectConfiguration;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPMirrorBrokerConnectionElement;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.Queue;
import org.apache.activemq.artemis.tests.integration.amqp.AmqpClientTestSupport;
import org.apache.activemq.artemis.tests.util.CFUtil;
import org.apache.activemq.artemis.utils.Wait;
import org.junit.Assert;
import org.junit.Test;
public class AMQPChainedReplicaTest extends AmqpClientTestSupport {
protected static final int AMQP_PORT_2 = 5673;
protected static final int AMQP_PORT_3 = 5674;
ActiveMQServer server_2;
ActiveMQServer server_3;
@Override
protected ActiveMQServer createServer() throws Exception {
return createServer(AMQP_PORT, false);
}
@Test
public void testChained() throws Exception {
server_2 = createServer(AMQP_PORT_2, false);
server_3 = createServer(AMQP_PORT_3, false);
{
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("test", "tcp://localhost:" + AMQP_PORT_2).setRetryInterval(100).setReconnectAttempts(-1);
amqpConnection.addElement(new AMQPMirrorBrokerConnectionElement());
server.getConfiguration().addAMQPConnection(amqpConnection);
}
{
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("test", "tcp://localhost:" + AMQP_PORT_3).setRetryInterval(100).setReconnectAttempts(-1);
amqpConnection.addElement(new AMQPMirrorBrokerConnectionElement());
server_2.getConfiguration().addAMQPConnection(amqpConnection);
}
{
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("test", "tcp://localhost:" + AMQP_PORT).setRetryInterval(100).setReconnectAttempts(-1);
amqpConnection.addElement(new AMQPMirrorBrokerConnectionElement());
server_3.getConfiguration().addAMQPConnection(amqpConnection);
}
server.start();
server_2.start();
server_3.start();
createAddressAndQueues(server);
Wait.assertTrue(() -> server.locateQueue(getQueueName()) != null);
Wait.assertTrue(() -> server_2.locateQueue(getQueueName()) != null);
Wait.assertTrue(() -> server_3.locateQueue(getQueueName()) != null);
Queue q1 = server.locateQueue(getQueueName());
Assert.assertNotNull(q1);
Queue q2 = server.locateQueue(getQueueName());
Assert.assertNotNull(q2);
Queue q3 = server.locateQueue(getQueueName());
Assert.assertNotNull(q3);
ConnectionFactory factory = CFUtil.createConnectionFactory("AMQP", "tcp://localhost:" + AMQP_PORT);
ConnectionFactory factory2 = CFUtil.createConnectionFactory("AMQP", "tcp://localhost:" + AMQP_PORT_2);
ConnectionFactory factory3 = CFUtil.createConnectionFactory("AMQP", "tcp://localhost:" + AMQP_PORT_3);
try (Connection conn = factory.createConnection()) {
Session session = conn.createSession();
MessageProducer producer = session.createProducer(session.createQueue(getQueueName()));
for (int i = 0; i < 40; i++) {
producer.send(session.createTextMessage("message " + i));
}
}
Thread.sleep(100); // some time to allow eventual loops
Wait.assertEquals(40L, q1::getMessageCount, 1000, 100);
Wait.assertEquals(40L, q2::getMessageCount, 1000, 100);
Wait.assertEquals(40L, q3::getMessageCount, 1000, 100);
try (Connection conn = factory.createConnection()) {
Session session = conn.createSession();
conn.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(getQueueName()));
for (int i = 0; i < 10; i++) {
TextMessage message = (TextMessage) consumer.receive(1000);
Assert.assertNotNull(message);
Assert.assertEquals("message " + i, message.getText());
}
consumer.close();
}
Wait.assertEquals(30L, q1::getMessageCount, 1000, 100);
Wait.assertEquals(30L, q2::getMessageCount, 1000, 100);
Wait.assertEquals(30L, q3::getMessageCount, 1000, 100);
try (Connection conn = factory2.createConnection()) {
Session session = conn.createSession();
conn.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(getQueueName()));
for (int i = 10; i < 20; i++) {
TextMessage message = (TextMessage) consumer.receive(1000);
Assert.assertNotNull(message);
Assert.assertEquals("message " + i, message.getText());
}
consumer.close();
}
Wait.assertEquals(20L, q1::getMessageCount, 1000, 100);
Wait.assertEquals(20L, q2::getMessageCount, 1000, 100);
Wait.assertEquals(20L, q3::getMessageCount, 1000, 100);
try (Connection conn = factory3.createConnection()) {
Session session = conn.createSession();
conn.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(getQueueName()));
for (int i = 20; i < 30; i++) {
TextMessage message = (TextMessage) consumer.receive(1000);
Assert.assertNotNull(message);
Assert.assertEquals("message " + i, message.getText());
}
consumer.close();
}
Wait.assertEquals(10L, q1::getMessageCount, 1000, 100);
Wait.assertEquals(10L, q2::getMessageCount, 1000, 100);
Wait.assertEquals(10L, q3::getMessageCount, 1000, 100);
try (Connection conn = factory.createConnection()) {
Session session = conn.createSession();
conn.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(getQueueName()));
for (int i = 30; i < 40; i++) {
TextMessage message = (TextMessage) consumer.receive(1000);
Assert.assertNotNull(message);
Assert.assertEquals("message " + i, message.getText());
}
consumer.close();
}
Wait.assertEquals(0L, q1::getMessageCount, 1000, 100);
Wait.assertEquals(0L, q2::getMessageCount, 1000, 100);
Wait.assertEquals(0L, q3::getMessageCount, 1000, 100);
try (Connection conn = factory.createConnection()) {
Session session = conn.createSession();
conn.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(getQueueName()));
Assert.assertNull(consumer.receiveNoWait());
consumer.close();
}
try (Connection conn = factory2.createConnection()) {
Session session = conn.createSession();
conn.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(getQueueName()));
Assert.assertNull(consumer.receiveNoWait());
consumer.close();
}
try (Connection conn = factory3.createConnection()) {
Session session = conn.createSession();
conn.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(getQueueName()));
Assert.assertNull(consumer.receiveNoWait());
consumer.close();
}
}
}

View File

@ -203,7 +203,6 @@ public class AMQPReplicaTest extends AmqpClientTestSupport {
server_2.start();
// if this does not succeed the catch up did not arrive at the other server
Wait.assertTrue(() -> server.locateQueue("sometest") != null);
Wait.assertTrue(() -> server.locateQueue("ToBeGone") == null);
server_2.stop();
server.stop();
}
@ -214,8 +213,6 @@ public class AMQPReplicaTest extends AmqpClientTestSupport {
server.start();
server.setIdentity("Server1");
server.addAddressInfo(new AddressInfo("sometest").setAutoCreated(false).addRoutingType(RoutingType.MULTICAST));
// This queue will disappear from the source, so it should go
server.createQueue(new QueueConfiguration("ToBeGone").setDurable(true).setRoutingType(RoutingType.MULTICAST));
server.stop();
server_2 = createServer(AMQP_PORT_2, false);
@ -243,12 +240,10 @@ public class AMQPReplicaTest extends AmqpClientTestSupport {
server.start();
Assert.assertTrue(server.locateQueue("sometest") == null);
Assert.assertTrue(server.locateQueue("ToBeGone") != null);
Wait.assertTrue(server::isActive);
server_2.start();
// if this does not succeed the catch up did not arrive at the other server
Wait.assertTrue(() -> server.locateQueue("sometest") != null);
Wait.assertTrue(() -> server.locateQueue("ToBeGone") == null);
server_2.stop();
server.stop();
}
@ -466,7 +461,7 @@ public class AMQPReplicaTest extends AmqpClientTestSupport {
server_2.getConfiguration().setName("thisone");
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("OtherSide", "tcp://localhost:" + AMQP_PORT).setReconnectAttempts(-1).setRetryInterval(100);
AMQPMirrorBrokerConnectionElement replica = new AMQPMirrorBrokerConnectionElement().setSourceMirrorAddress("TheSource");
AMQPMirrorBrokerConnectionElement replica = new AMQPMirrorBrokerConnectionElement().setDurable(true);
amqpConnection.addElement(replica);
server_2.getConfiguration().addAMQPConnection(amqpConnection);
@ -595,13 +590,13 @@ public class AMQPReplicaTest extends AmqpClientTestSupport {
} else {
queueOnServer1 = locateQueue(server, getQueueName());
}
Queue snfreplica = server_2.locateQueue(replica.getSourceMirrorAddress());
Queue snfreplica = server_2.locateQueue(replica.getMirrorSNF());
Assert.assertNotNull(snfreplica);
Wait.assertEquals(0, snfreplica::getMessageCount);
Wait.assertEquals(NUMBER_OF_MESSAGES, queueOnServer1::getMessageCount);
Wait.assertEquals(NUMBER_OF_MESSAGES, queueOnServer1::getMessageCount, 2000);
Queue queueOnServer2 = locateQueue(server_2, getQueueName());
Wait.assertEquals(NUMBER_OF_MESSAGES, queueOnServer1::getMessageCount);
Wait.assertEquals(NUMBER_OF_MESSAGES, queueOnServer2::getMessageCount);
@ -627,7 +622,6 @@ public class AMQPReplicaTest extends AmqpClientTestSupport {
if (largeMessage) {
validateNoFilesOnLargeDir(server.getConfiguration().getLargeMessagesDirectory(), 0);
//validateNoFilesOnLargeDir(server_2.getConfiguration().getLargeMessagesDirectory(), 50); // we kept half of the messages
}
} else {
@ -765,8 +759,8 @@ public class AMQPReplicaTest extends AmqpClientTestSupport {
Wait.assertEquals(NUMBER_OF_MESSAGES, queue_server_3::getMessageCount);
Wait.assertEquals(NUMBER_OF_MESSAGES, queue_server_1::getMessageCount);
Queue replica1Queue = server_2.locateQueue(replica1.getSourceMirrorAddress());
Queue replica2Queue = server_2.locateQueue(replica2.getSourceMirrorAddress());
Queue replica1Queue = server_2.locateQueue(replica1.getMirrorSNF());
Queue replica2Queue = server_2.locateQueue(replica2.getMirrorSNF());
Wait.assertEquals(0L, replica2Queue.getPagingStore()::getAddressSize, 1000, 100);
Wait.assertEquals(0L, replica1Queue.getPagingStore()::getAddressSize, 1000, 100);

View File

@ -0,0 +1,425 @@
/**
* 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.integration.amqp.connect;
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.io.PrintStream;
import java.net.URI;
import java.util.concurrent.TimeUnit;
import org.apache.activemq.artemis.api.core.QueueConfiguration;
import org.apache.activemq.artemis.api.core.RoutingType;
import org.apache.activemq.artemis.cli.commands.tools.PrintData;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPBrokerConnectConfiguration;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPMirrorBrokerConnectionElement;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.impl.AddressInfo;
import org.apache.activemq.artemis.logs.AssertionLoggerHandler;
import org.apache.activemq.artemis.tests.integration.amqp.AmqpClientTestSupport;
import org.apache.activemq.artemis.tests.util.CFUtil;
import org.apache.activemq.artemis.utils.StringPrintStream;
import org.apache.activemq.artemis.utils.Wait;
import org.apache.activemq.transport.amqp.client.AmqpClient;
import org.apache.activemq.transport.amqp.client.AmqpConnection;
import org.apache.activemq.transport.amqp.client.AmqpMessage;
import org.apache.activemq.transport.amqp.client.AmqpReceiver;
import org.apache.activemq.transport.amqp.client.AmqpSender;
import org.apache.activemq.transport.amqp.client.AmqpSession;
import org.jboss.logging.Logger;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class BrokerInSyncTest extends AmqpClientTestSupport {
public static final int TIME_BEFORE_RESTART = 1000;
protected static final int AMQP_PORT_2 = 5673;
protected static final int AMQP_PORT_3 = 5674;
private static final Logger logger = Logger.getLogger(BrokerInSyncTest.class);
ActiveMQServer server_2;
@Before
public void startLogging() {
AssertionLoggerHandler.startCapture();
}
@After
public void stopLogging() {
try {
Assert.assertFalse(AssertionLoggerHandler.findText("AMQ222214"));
} finally {
AssertionLoggerHandler.stopCapture();
}
}
@Override
protected ActiveMQServer createServer() throws Exception {
return createServer(AMQP_PORT, false);
}
@Test
public void testSyncOnCreateQueues() throws Exception {
server.setIdentity("Server1");
{
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("connectTowardsServer2", "tcp://localhost:" + AMQP_PORT_2).setReconnectAttempts(3).setRetryInterval(100);
amqpConnection.addElement(new AMQPMirrorBrokerConnectionElement().setDurable(true));
server.getConfiguration().addAMQPConnection(amqpConnection);
}
server.start();
server_2 = createServer(AMQP_PORT_2, false);
server_2.setIdentity("Server2");
{
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("connectTowardsServer1", "tcp://localhost:" + AMQP_PORT).setReconnectAttempts(-1).setRetryInterval(100);
amqpConnection.addElement(new AMQPMirrorBrokerConnectionElement().setDurable(true));
server_2.getConfiguration().addAMQPConnection(amqpConnection);
}
server_2.start();
server_2.addAddressInfo(new AddressInfo("sometest").setAutoCreated(false));
server_2.createQueue(new QueueConfiguration("sometest").setDurable(true));
Wait.assertTrue(() -> server_2.locateQueue("sometest") != null);
Wait.assertTrue(() -> server.locateQueue("sometest") != null);
server.addAddressInfo(new AddressInfo("OnServer1").setAutoCreated(false));
server.createQueue(new QueueConfiguration("OnServer1").setDurable(true));
Wait.assertTrue(() -> server.locateQueue("OnServer1") != null);
Wait.assertTrue("Sync is not working on the way back", () -> server_2.locateQueue("OnServer1") != null, 2000);
Wait.assertTrue(() -> server_2.locateQueue("sometest") != null);
Wait.assertTrue(() -> server.locateQueue("sometest") != null);
for (int i = 0; i < 10; i++) {
final int queueID = i;
server_2.createQueue(new QueueConfiguration("test2_" + i).setDurable(true));
server.createQueue(new QueueConfiguration("test1_" + i).setDurable(true));
Wait.assertTrue(() -> server.locateQueue("test2_" + queueID) != null);
Wait.assertTrue(() -> server.locateQueue("test1_" + queueID) != null);
Wait.assertTrue(() -> server_2.locateQueue("test2_" + queueID) != null);
Wait.assertTrue(() -> server_2.locateQueue("test1_" + queueID) != null);
}
server_2.stop();
server.stop();
}
@Test
public void testSingleMessage() throws Exception {
server.setIdentity("Server1");
{
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("connectTowardsServer2", "tcp://localhost:" + AMQP_PORT_2).setReconnectAttempts(3).setRetryInterval(100);
amqpConnection.addElement(new AMQPMirrorBrokerConnectionElement().setDurable(true));
server.getConfiguration().addAMQPConnection(amqpConnection);
}
server.start();
server_2 = createServer(AMQP_PORT_2, false);
server_2.setIdentity("Server2");
{
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("connectTowardsServer1", "tcp://localhost:" + AMQP_PORT).setReconnectAttempts(-1).setRetryInterval(100);
amqpConnection.addElement(new AMQPMirrorBrokerConnectionElement().setDurable(true));
server_2.getConfiguration().addAMQPConnection(amqpConnection);
}
server_2.start();
server_2.addAddressInfo(new AddressInfo(getQueueName()).setAutoCreated(false).addRoutingType(RoutingType.ANYCAST));
server_2.createQueue(new QueueConfiguration(getQueueName()).setDurable(true).setRoutingType(RoutingType.ANYCAST));
Wait.assertTrue(() -> server_2.locateQueue(getQueueName()) != null);
Wait.assertTrue(() -> server.locateQueue(getQueueName()) != null);
ConnectionFactory cf1 = CFUtil.createConnectionFactory("AMQP", "tcp://localhost:" + AMQP_PORT);
Connection connection1 = cf1.createConnection();
Session session1 = connection1.createSession(true, Session.SESSION_TRANSACTED);
connection1.start();
ConnectionFactory cf2 = CFUtil.createConnectionFactory("AMQP", "tcp://localhost:" + AMQP_PORT_2);
Connection connection2 = cf2.createConnection();
Session session2 = connection2.createSession(true, Session.SESSION_TRANSACTED);
connection2.start();
Queue queue = session1.createQueue(getQueueName());
MessageProducer producerServer1 = session1.createProducer(queue);
MessageProducer producerServer2 = session2.createProducer(queue);
TextMessage message = session1.createTextMessage("test");
message.setIntProperty("i", 0);
message.setStringProperty("server", server.getIdentity());
producerServer1.send(message);
session1.commit();
org.apache.activemq.artemis.core.server.Queue queueOnServer1 = server.locateQueue(getQueueName());
org.apache.activemq.artemis.core.server.Queue queueOnServer2 = server_2.locateQueue(getQueueName());
Assert.assertNotNull(queueOnServer1);
Assert.assertNotNull(queueOnServer2);
Wait.assertEquals(1, queueOnServer1::getMessageCount);
Wait.assertEquals(1, queueOnServer2::getMessageCount);
message = session1.createTextMessage("test");
message.setIntProperty("i", 1);
message.setStringProperty("server", server_2.getIdentity());
producerServer2.send(message);
session2.commit();
if (logger.isDebugEnabled() && !Wait.waitFor(() -> queueOnServer1.getMessageCount() == 2)) {
debugData();
}
Wait.assertEquals(2, queueOnServer1::getMessageCount);
Wait.assertEquals(2, queueOnServer2::getMessageCount);
connection1.close();
connection2.close();
server_2.stop();
server.stop();
}
@Test
public void testSyncData() throws Exception {
int NUMBER_OF_MESSAGES = 100;
server.setIdentity("Server1");
{
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("connectTowardsServer2", "tcp://localhost:" + AMQP_PORT_2).setReconnectAttempts(3).setRetryInterval(100);
amqpConnection.addElement(new AMQPMirrorBrokerConnectionElement().setDurable(true));
server.getConfiguration().addAMQPConnection(amqpConnection);
}
server.start();
server_2 = createServer(AMQP_PORT_2, false);
server_2.setIdentity("Server2");
{
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("connectTowardsServer1", "tcp://localhost:" + AMQP_PORT).setReconnectAttempts(-1).setRetryInterval(100);
amqpConnection.addElement(new AMQPMirrorBrokerConnectionElement().setDurable(true));
server_2.getConfiguration().addAMQPConnection(amqpConnection);
}
server_2.start();
server_2.addAddressInfo(new AddressInfo(getQueueName()).setAutoCreated(false).addRoutingType(RoutingType.ANYCAST));
server_2.createQueue(new QueueConfiguration(getQueueName()).setDurable(true).setRoutingType(RoutingType.ANYCAST));
Wait.assertTrue(() -> server_2.locateQueue(getQueueName()) != null);
Wait.assertTrue(() -> server.locateQueue(getQueueName()) != null);
ConnectionFactory cf1 = CFUtil.createConnectionFactory("AMQP", "tcp://localhost:" + AMQP_PORT);
Connection connection1 = cf1.createConnection();
Session session1 = connection1.createSession(true, Session.SESSION_TRANSACTED);
connection1.start();
ConnectionFactory cf2 = CFUtil.createConnectionFactory("AMQP", "tcp://localhost:" + AMQP_PORT_2);
Connection connection2 = cf2.createConnection();
Session session2 = connection2.createSession(true, Session.SESSION_TRANSACTED);
connection2.start();
Queue queue = session1.createQueue(getQueueName());
MessageProducer producerServer1 = session1.createProducer(queue);
MessageProducer producerServer2 = session2.createProducer(queue);
for (int i = 0; i < NUMBER_OF_MESSAGES; i++) {
TextMessage message = session1.createTextMessage("test " + i);
message.setIntProperty("i", i);
message.setStringProperty("server", server.getIdentity());
producerServer1.send(message);
}
session1.commit();
org.apache.activemq.artemis.core.server.Queue queueOnServer1 = server.locateQueue(getQueueName());
org.apache.activemq.artemis.core.server.Queue queueOnServer2 = server_2.locateQueue(getQueueName());
Assert.assertNotNull(queueOnServer1);
Assert.assertNotNull(queueOnServer2);
Wait.assertEquals(NUMBER_OF_MESSAGES, queueOnServer1::getMessageCount);
Wait.assertEquals(NUMBER_OF_MESSAGES, queueOnServer2::getMessageCount);
for (int i = NUMBER_OF_MESSAGES; i < NUMBER_OF_MESSAGES * 2; i++) {
TextMessage message = session1.createTextMessage("test " + i);
message.setIntProperty("i", i);
message.setStringProperty("server", server_2.getIdentity());
producerServer2.send(message);
}
session2.commit();
if (logger.isDebugEnabled() && !Wait.waitFor(() -> queueOnServer1.getMessageCount() == NUMBER_OF_MESSAGES * 2)) {
debugData();
}
Wait.assertEquals(NUMBER_OF_MESSAGES * 2, queueOnServer1::getMessageCount);
Wait.assertEquals(NUMBER_OF_MESSAGES * 2, queueOnServer2::getMessageCount);
MessageConsumer consumerOn1 = session1.createConsumer(queue);
for (int i = 0; i < NUMBER_OF_MESSAGES * 2; i++) {
TextMessage message = (TextMessage) consumerOn1.receive(5000);
logger.debug("### Client acking message(" + i + ") on server 1, a message that was original sent on " + message.getStringProperty("server") + " text = " + message.getText());
Assert.assertNotNull(message);
Assert.assertEquals(i, message.getIntProperty("i"));
Assert.assertEquals("test " + i, message.getText());
session1.commit();
}
boolean bothConsumed = Wait.waitFor(() -> {
long q1 = queueOnServer1.getMessageCount();
long q2 = queueOnServer2.getMessageCount();
logger.debug("Queue on Server 1 = " + q1);
logger.debug("Queue on Server 2 = " + q2);
return q1 == 0 && q2 == 0;
}, 5_000, 1000);
if (logger.isDebugEnabled() && !bothConsumed) {
debugData();
Assert.fail("q1 = " + queueOnServer1.getMessageCount() + ", q2 = " + queueOnServer2.getMessageCount());
}
Assert.assertEquals(0, queueOnServer1.getMessageCount());
Assert.assertEquals(0, queueOnServer2.getConsumerCount());
System.out.println("Queue on Server 1 = " + queueOnServer1.getMessageCount());
System.out.println("Queue on Server 2 = " + queueOnServer2.getMessageCount());
server_2.stop();
server.stop();
}
private void debugData() throws Exception {
StringPrintStream stringPrintStream = new StringPrintStream();
PrintStream out = stringPrintStream.newStream();
org.apache.activemq.artemis.core.server.Queue queueToDebugOn1 = server.locateQueue(getQueueName());
org.apache.activemq.artemis.core.server.Queue queueToDebugOn2 = server_2.locateQueue(getQueueName());
out.println("*******************************************************************************************************************************");
out.println("Queue on Server 1 with count = " + queueToDebugOn1.getMessageCount());
queueToDebugOn1.forEach((r) -> out.println("Server1 has reference " + r.getMessage()));
out.println("*******************************************************************************************************************************");
out.println("Queue on Server 2 with count = " + queueToDebugOn2.getMessageCount());
queueToDebugOn2.forEach((r) -> out.println("Server2 has reference " + r.getMessage()));
out.println("*******************************************************************************************************************************");
out.println("PrintData Server 1");
PrintData.printMessages(server.getConfiguration().getJournalLocation(), out, false, false, true, false);
out.println("*******************************************************************************************************************************");
out.println("PrintData Server 2");
PrintData.printMessages(server_2.getConfiguration().getJournalLocation(), out, false, false, true, false);
logger.debug("Data Available on Servers:\n" + stringPrintStream.toString());
}
@Test
public void testSyncDataNoSuppliedID() throws Exception {
int NUMBER_OF_MESSAGES = 100;
server.setIdentity("Server1");
{
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("connectTowardsServer2", "tcp://localhost:" + AMQP_PORT_2).setReconnectAttempts(3).setRetryInterval(100);
amqpConnection.addElement(new AMQPMirrorBrokerConnectionElement().setDurable(true));
server.getConfiguration().addAMQPConnection(amqpConnection);
}
server.start();
server_2 = createServer(AMQP_PORT_2, false);
server_2.setIdentity("Server2");
{
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("connectTowardsServer1", "tcp://localhost:" + AMQP_PORT).setReconnectAttempts(-1).setRetryInterval(100);
amqpConnection.addElement(new AMQPMirrorBrokerConnectionElement().setDurable(true));
server_2.getConfiguration().addAMQPConnection(amqpConnection);
}
server_2.start();
server_2.addAddressInfo(new AddressInfo(getQueueName()).setAutoCreated(false).addRoutingType(RoutingType.ANYCAST));
server_2.createQueue(new QueueConfiguration(getQueueName()).setDurable(true).setRoutingType(RoutingType.ANYCAST));
Wait.assertTrue(() -> server_2.locateQueue(getQueueName()) != null);
Wait.assertTrue(() -> server.locateQueue(getQueueName()) != null);
AmqpClient cf1 = new AmqpClient(new URI("tcp://localhost:" + AMQP_PORT), null, null);
AmqpConnection connection1 = cf1.createConnection();
connection1.connect();
AmqpSession session1 = connection1.createSession();
AmqpClient cf2 = new AmqpClient(new URI("tcp://localhost:" + AMQP_PORT_2), null, null);
AmqpConnection connection2 = cf2.createConnection();
connection2.connect();
AmqpSession session2 = connection2.createSession();
AmqpSender producerServer1 = session1.createSender(getQueueName());
AmqpSender producerServer2 = session2.createSender(getQueueName());
for (int i = 0; i < NUMBER_OF_MESSAGES; i++) {
AmqpMessage message = new AmqpMessage();
message.setDurable(true);
message.setApplicationProperty("i", i);
producerServer1.send(message);
}
org.apache.activemq.artemis.core.server.Queue queueOnServer1 = server.locateQueue(getQueueName());
org.apache.activemq.artemis.core.server.Queue queueOnServer2 = server_2.locateQueue(getQueueName());
Assert.assertNotNull(queueOnServer1);
Assert.assertNotNull(queueOnServer2);
Wait.assertEquals(NUMBER_OF_MESSAGES, queueOnServer1::getMessageCount);
Wait.assertEquals(NUMBER_OF_MESSAGES, queueOnServer2::getMessageCount);
for (int i = NUMBER_OF_MESSAGES; i < NUMBER_OF_MESSAGES * 2; i++) {
AmqpMessage message = new AmqpMessage();
message.setDurable(true);
message.setApplicationProperty("i", i);
producerServer2.send(message);
}
Wait.assertEquals(NUMBER_OF_MESSAGES * 2, queueOnServer1::getMessageCount);
Wait.assertEquals(NUMBER_OF_MESSAGES * 2, queueOnServer2::getMessageCount);
AmqpReceiver consumerOn1 = session1.createReceiver(getQueueName());
consumerOn1.flow(NUMBER_OF_MESSAGES * 2 + 1);
for (int i = 0; i < NUMBER_OF_MESSAGES * 2; i++) {
AmqpMessage message = consumerOn1.receive(5, TimeUnit.SECONDS);
Assert.assertNotNull(message);
message.accept();
Assert.assertEquals(i, (int) message.getApplicationProperty("i"));
}
Wait.assertEquals(0, queueOnServer1::getMessageCount);
Wait.assertEquals(0, () -> {
System.out.println(queueOnServer2.getMessageCount());
return queueOnServer2.getMessageCount();
});
connection1.close();
connection2.close();
server_2.stop();
server.stop();
}
}

View File

@ -92,7 +92,7 @@ public class MirrorControllerBasicTest extends ActiveMQTestBase {
server.addAddressInfo(new AddressInfo("test").addRoutingType(RoutingType.ANYCAST));
server.createQueue(new QueueConfiguration("test").setAddress("test").setRoutingType(RoutingType.ANYCAST));
Message message = AMQPMirrorMessageFactory.createMessage("test", SimpleString.toSimpleString("ad1"), SimpleString.toSimpleString("qu1"), "test", "body-test");
Message message = AMQPMirrorMessageFactory.createMessage("test", SimpleString.toSimpleString("ad1"), SimpleString.toSimpleString("qu1"), "test", "someUID", "body-test");
AMQPMirrorControllerSource.route(server, message);
AmqpClient client = new AmqpClient(new URI("tcp://localhost:61616"), null, null);
@ -106,6 +106,7 @@ public class MirrorControllerBasicTest extends ActiveMQTestBase {
Assert.assertEquals("body-test", (String)value.getValue());
Assert.assertEquals("ad1",amqpMessage.getMessageAnnotation(AMQPMirrorControllerSource.ADDRESS.toString()));
Assert.assertEquals("qu1", amqpMessage.getMessageAnnotation(AMQPMirrorControllerSource.QUEUE.toString()));
Assert.assertEquals("someUID", amqpMessage.getMessageAnnotation(AMQPMirrorControllerSource.BROKER_ID.toString()));
Assert.assertEquals("test", amqpMessage.getMessageAnnotation(AMQPMirrorControllerSource.EVENT_TYPE.toString()));
connection.close();

View File

@ -0,0 +1,646 @@
/**
* 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.integration.amqp.connect;
import javax.jms.Connection;
import javax.jms.ConnectionFactory;
import javax.jms.MessageConsumer;
import javax.jms.MessageProducer;
import javax.jms.Session;
import javax.jms.TextMessage;
import java.net.URI;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import io.vertx.core.Vertx;
import io.vertx.proton.ProtonConnection;
import io.vertx.proton.ProtonServerOptions;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPBrokerConnectConfiguration;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPBrokerConnectionAddressType;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPBrokerConnectionElement;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPMirrorBrokerConnectionElement;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.logs.AssertionLoggerHandler;
import org.apache.activemq.artemis.protocol.amqp.broker.ActiveMQProtonRemotingConnection;
import org.apache.activemq.artemis.protocol.amqp.broker.ProtonProtocolManager;
import org.apache.activemq.artemis.protocol.amqp.connect.AMQPBrokerConnection;
import org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource;
import org.apache.activemq.artemis.tests.integration.amqp.AmqpClientTestSupport;
import org.apache.activemq.artemis.tests.util.CFUtil;
import org.apache.activemq.artemis.tests.util.Wait;
import org.apache.activemq.artemis.utils.collections.ConcurrentHashSet;
import org.apache.activemq.transport.amqp.client.AmqpClient;
import org.apache.activemq.transport.amqp.client.AmqpConnection;
import org.apache.activemq.transport.amqp.client.AmqpSession;
import org.apache.activemq.transport.amqp.client.AmqpValidator;
import org.apache.qpid.proton.amqp.Symbol;
import org.apache.qpid.proton.amqp.messaging.Accepted;
import org.apache.qpid.proton.amqp.transport.AmqpError;
import org.apache.qpid.proton.amqp.transport.ErrorCondition;
import org.apache.qpid.proton.amqp.transport.Target;
import org.apache.qpid.proton.engine.Link;
import org.apache.qpid.proton.engine.Receiver;
import org.apache.qpid.proton.engine.Sender;
import org.apache.qpid.proton.engine.impl.ConnectionImpl;
import org.jboss.logging.Logger;
import org.junit.After;
import org.junit.Assert;
import org.junit.Test;
import static java.util.EnumSet.of;
import static org.apache.qpid.proton.engine.EndpointState.ACTIVE;
/**
* This test will make sure the Broker connection will react accordingly to a few misconfigs and possible errors on the network of brokers and eventually qipd-dispatch.
*/
public class ValidateAMQPErrorsTest extends AmqpClientTestSupport {
protected static final int AMQP_PORT_2 = 5673;
private static final Logger logger = Logger.getLogger(ValidateAMQPErrorsTest.class);
protected Vertx vertx;
protected MockServer mockServer;
public void startVerx() {
vertx = Vertx.vertx();
}
@After
public void stop() throws Exception {
if (mockServer != null) {
mockServer.close();
mockServer = null;
}
if (vertx != null) {
try {
CountDownLatch latch = new CountDownLatch(1);
vertx.close((x) -> latch.countDown());
Assert.assertTrue(latch.await(10, TimeUnit.SECONDS));
} finally {
vertx = null;
}
}
AssertionLoggerHandler.stopCapture(); // Just in case startCapture was called in any of the tests here
}
@Override
protected ActiveMQServer createServer() throws Exception {
return createServer(AMQP_PORT, false);
}
/**
* Connecting to itself should issue an error.
* and the max retry should still be counted, not just keep connecting forever.
*/
@Test
public void testConnectItself() throws Exception {
try {
AssertionLoggerHandler.startCapture();
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("test", "tcp://localhost:" + AMQP_PORT).setReconnectAttempts(10).setRetryInterval(1);
amqpConnection.addElement(new AMQPMirrorBrokerConnectionElement());
server.getConfiguration().addAMQPConnection(amqpConnection);
server.start();
Assert.assertEquals(1, server.getBrokerConnections().size());
server.getBrokerConnections().forEach((t) -> Wait.assertFalse(t::isStarted));
Wait.assertTrue(() -> AssertionLoggerHandler.findText("AMQ111001")); // max retry
AssertionLoggerHandler.clear();
Thread.sleep(100);
Assert.assertFalse(AssertionLoggerHandler.findText("AMQ111002")); // there shouldn't be a retry after the last failure
Assert.assertFalse(AssertionLoggerHandler.findText("AMQ111003")); // there shouldn't be a retry after the last failure
} finally {
AssertionLoggerHandler.stopCapture();
}
}
@Test
public void testCloseLinkOnMirror() throws Exception {
try {
AssertionLoggerHandler.startCapture();
ActiveMQServer server2 = createServer(AMQP_PORT_2, false);
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("test", "tcp://localhost:" + AMQP_PORT_2).setReconnectAttempts(-1).setRetryInterval(10);
amqpConnection.addElement(new AMQPMirrorBrokerConnectionElement());
server.getConfiguration().addAMQPConnection(amqpConnection);
server.start();
Assert.assertEquals(1, server.getBrokerConnections().size());
Wait.assertTrue(() -> AssertionLoggerHandler.findText("AMQ111002"));
server.getBrokerConnections().forEach((t) -> Wait.assertTrue(() -> ((AMQPBrokerConnection) t).isConnecting()));
server2.start();
server.getBrokerConnections().forEach((t) -> Wait.assertFalse(() -> ((AMQPBrokerConnection) t).isConnecting()));
createAddressAndQueues(server);
Wait.assertTrue(() -> server2.locateQueue(getQueueName()) != null);
Wait.assertEquals(1, server2.getRemotingService()::getConnectionCount);
server2.getRemotingService().getConnections().forEach((t) -> {
try {
ActiveMQProtonRemotingConnection connection = (ActiveMQProtonRemotingConnection) t;
ConnectionImpl protonConnection = (ConnectionImpl) connection.getAmqpConnection().getHandler().getConnection();
Wait.waitFor(() -> protonConnection.linkHead(of(ACTIVE), of(ACTIVE)) != null);
connection.getAmqpConnection().runNow(() -> {
Receiver receiver = (Receiver) protonConnection.linkHead(of(ACTIVE), of(ACTIVE));
receiver.close();
connection.flush();
});
} catch (Exception e) {
e.printStackTrace();
}
});
ConnectionFactory cf1 = CFUtil.createConnectionFactory("AMQP", "tcp://localhost:" + AMQP_PORT);
ConnectionFactory cf2 = CFUtil.createConnectionFactory("AMQP", "tcp://localhost:" + AMQP_PORT_2);
try (Connection connection = cf1.createConnection()) {
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
MessageProducer producer = session.createProducer(session.createQueue(getQueueName()));
for (int i = 0; i < 10; i++) {
producer.send(session.createTextMessage("message " + i));
}
}
// messages should still flow after a disconnect on the link
// the server should reconnect as if it was a failure
try (Connection connection = cf2.createConnection()) {
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
MessageConsumer consumer = session.createConsumer(session.createQueue(getQueueName()));
connection.start();
for (int i = 0; i < 10; i++) {
Assert.assertEquals("message " + i, ((TextMessage) consumer.receive(5000)).getText());
}
}
} finally {
AssertionLoggerHandler.stopCapture();
}
}
@Test
public void testCloseLinkOnSender() throws Exception {
testCloseLink(true);
}
@Test
public void testCloseLinkOnReceiver() throws Exception {
testCloseLink(false);
}
public void testCloseLink(boolean isSender) throws Exception {
AtomicInteger errors = new AtomicInteger(0);
AssertionLoggerHandler.startCapture(true);
ActiveMQServer server2 = createServer(AMQP_PORT_2, false);
if (isSender) {
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("test", "tcp://localhost:" + AMQP_PORT_2).setReconnectAttempts(-1).setRetryInterval(10);
amqpConnection.addElement(new AMQPBrokerConnectionElement().setMatchAddress(getQueueName()).setType(AMQPBrokerConnectionAddressType.SENDER));
server.getConfiguration().addAMQPConnection(amqpConnection);
} else {
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("test", "tcp://localhost:" + AMQP_PORT).setReconnectAttempts(-1).setRetryInterval(10);
amqpConnection.addElement(new AMQPBrokerConnectionElement().setMatchAddress(getQueueName()).setType(AMQPBrokerConnectionAddressType.RECEIVER));
server2.getConfiguration().addAMQPConnection(amqpConnection);
}
if (isSender) {
server.start();
Assert.assertEquals(1, server.getBrokerConnections().size());
} else {
server2.start();
Assert.assertEquals(1, server2.getBrokerConnections().size());
}
Wait.assertTrue(() -> AssertionLoggerHandler.findText("AMQ111002"));
server.getBrokerConnections().forEach((t) -> Wait.assertTrue(() -> ((AMQPBrokerConnection) t).isConnecting()));
if (isSender) {
server2.start();
} else {
server.start();
}
server.getBrokerConnections().forEach((t) -> Wait.assertFalse(() -> ((AMQPBrokerConnection) t).isConnecting()));
createAddressAndQueues(server);
createAddressAndQueues(server2);
Wait.assertTrue(() -> server.locateQueue(getQueueName()) != null);
Wait.assertTrue(() -> server2.locateQueue(getQueueName()) != null);
ActiveMQServer serverReceivingConnections = isSender ? server2 : server;
Wait.assertEquals(1, serverReceivingConnections.getRemotingService()::getConnectionCount);
serverReceivingConnections.getRemotingService().getConnections().forEach((t) -> {
try {
ActiveMQProtonRemotingConnection connection = (ActiveMQProtonRemotingConnection) t;
ConnectionImpl protonConnection = (ConnectionImpl) connection.getAmqpConnection().getHandler().getConnection();
Wait.waitFor(() -> protonConnection.linkHead(of(ACTIVE), of(ACTIVE)) != null);
connection.getAmqpConnection().runNow(() -> {
Link theLink = protonConnection.linkHead(of(ACTIVE), of(ACTIVE));
theLink.close();
connection.flush();
});
} catch (Exception e) {
errors.incrementAndGet();
e.printStackTrace();
}
});
Wait.assertEquals(1, () -> AssertionLoggerHandler.countText("AMQ119021"));
ConnectionFactory cf1 = CFUtil.createConnectionFactory("AMQP", "tcp://localhost:" + AMQP_PORT);
ConnectionFactory cf2 = CFUtil.createConnectionFactory("AMQP", "tcp://localhost:" + AMQP_PORT_2);
try (Connection connection = cf1.createConnection()) {
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
MessageProducer producer = session.createProducer(session.createQueue(getQueueName()));
for (int i = 0; i < 10; i++) {
producer.send(session.createTextMessage("message " + i));
}
}
// messages should still flow after a disconnect on the link
// the server should reconnect as if it was a failure
try (Connection connection = cf2.createConnection()) {
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
MessageConsumer consumer = session.createConsumer(session.createQueue(getQueueName()));
connection.start();
for (int i = 0; i < 10; i++) {
Assert.assertEquals("message " + i, ((TextMessage) consumer.receive(5000)).getText());
}
}
Assert.assertEquals(0, errors.get());
}
@Test
public void testTimeoutOnSenderOpen() throws Exception {
startVerx();
ProtonServerOptions serverOptions = new ProtonServerOptions();
mockServer = new MockServer(vertx, serverOptions, null, serverConnection -> {
serverConnection.openHandler(serverSender -> {
serverConnection.closeHandler(x -> serverConnection.close());
serverConnection.open();
});
serverConnection.sessionOpenHandler((s) -> {
s.open();
});
serverConnection.senderOpenHandler((x) -> {
x.open();
});
serverConnection.receiverOpenHandler((x) -> {
//x.open(); // I'm missing the open, so it won't ever connect
});
});
try {
AssertionLoggerHandler.startCapture(true);
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("test", "tcp://localhost:" + mockServer.actualPort() + "?connect-timeout-millis=20").setReconnectAttempts(5).setRetryInterval(10);
amqpConnection.addElement(new AMQPBrokerConnectionElement().setMatchAddress(getQueueName()).setType(AMQPBrokerConnectionAddressType.SENDER));
amqpConnection.addElement(new AMQPMirrorBrokerConnectionElement());
server.getConfiguration().addAMQPConnection(amqpConnection);
server.start();
Wait.assertTrue(() -> AssertionLoggerHandler.findText("AMQ111001"));
Wait.assertEquals(6, () -> AssertionLoggerHandler.countText("AMQ119020")); // 0..5 == 6
} finally {
mockServer.close();
}
}
@Test
public void testReconnectAfterSenderOpenTimeout() throws Exception {
startVerx();
AssertionLoggerHandler.startCapture(true);
ProtonServerOptions serverOptions = new ProtonServerOptions();
AtomicInteger countOpen = new AtomicInteger(0);
CyclicBarrier startFlag = new CyclicBarrier(2);
CountDownLatch blockBeforeOpen = new CountDownLatch(1);
AtomicInteger disconnects = new AtomicInteger(0);
AtomicInteger messagesReceived = new AtomicInteger(0);
AtomicInteger errors = new AtomicInteger(0);
ConcurrentHashSet<ProtonConnection> connections = new ConcurrentHashSet<>();
mockServer = new MockServer(vertx, serverOptions, null, serverConnection -> {
serverConnection.disconnectHandler(c -> {
disconnects.incrementAndGet(); // number of retries
connections.remove(c);
});
serverConnection.openHandler(serverSender -> {
serverConnection.closeHandler(x -> {
serverConnection.close();
connections.remove(serverConnection);
});
serverConnection.open();
connections.add(serverConnection);
});
serverConnection.sessionOpenHandler((s) -> {
s.open();
});
serverConnection.senderOpenHandler((x) -> {
x.open();
});
serverConnection.receiverOpenHandler((x) -> {
if (countOpen.incrementAndGet() > 2) {
if (countOpen.get() == 3) {
try {
startFlag.await(10, TimeUnit.SECONDS);
blockBeforeOpen.await(10, TimeUnit.SECONDS);
return;
} catch (Throwable ignored) {
}
}
HashMap<Symbol, Object> brokerIDProperties = new HashMap<>();
brokerIDProperties.put(AMQPMirrorControllerSource.BROKER_ID, "fake-id");
x.setProperties(brokerIDProperties);
x.setOfferedCapabilities(new Symbol[]{AMQPMirrorControllerSource.MIRROR_CAPABILITY});
x.setTarget(x.getRemoteTarget());
x.open();
x.handler((del, msg) -> {
if (msg.getApplicationProperties() != null) {
Map map = msg.getApplicationProperties().getValue();
Object value = map.get("sender");
if (value != null) {
if (messagesReceived.get() != ((Integer) value).intValue()) {
logger.warn("Message out of order. Expected " + messagesReceived.get() + " but received " + value);
errors.incrementAndGet();
}
messagesReceived.incrementAndGet();
}
}
});
}
});
});
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("test", "tcp://localhost:" + mockServer.actualPort() + "?connect-timeout-millis=1000").setReconnectAttempts(10).setRetryInterval(10);
amqpConnection.addElement(new AMQPMirrorBrokerConnectionElement());
server.getConfiguration().addAMQPConnection(amqpConnection);
server.start();
startFlag.await(10, TimeUnit.SECONDS);
blockBeforeOpen.countDown();
Wait.assertEquals(2, disconnects::intValue);
Wait.assertEquals(1, connections::size);
Wait.assertEquals(3, () -> AssertionLoggerHandler.countText("AMQ119020"));
ConnectionFactory factory = CFUtil.createConnectionFactory("AMQP", "tcp://localhost:" + AMQP_PORT);
try (Connection connection = factory.createConnection()) {
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
MessageProducer producer = session.createProducer(session.createQueue(getQueueName()));
for (int i = 0; i < 100; i++) {
TextMessage message = session.createTextMessage("hello");
message.setIntProperty("sender", i);
producer.send(message);
}
}
Wait.assertEquals(100, messagesReceived::intValue, 5000);
Assert.assertEquals(0, errors.get(), 5000);
}
@Test
public void testNoServerOfferedMirrorCapability() throws Exception {
startVerx();
mockServer = new MockServer(vertx, serverConnection -> {
serverConnection.openHandler(serverSender -> {
serverConnection.open();
});
serverConnection.sessionOpenHandler((s) -> {
s.open();
});
serverConnection.senderOpenHandler((x) -> {
x.open();
});
serverConnection.receiverOpenHandler((x) -> {
x.setTarget(x.getRemoteTarget());
x.open();
});
});
AssertionLoggerHandler.startCapture(true);
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("test", "tcp://localhost:" + mockServer.actualPort() + "?connect-timeout-millis=100").setReconnectAttempts(5).setRetryInterval(10);
amqpConnection.addElement(new AMQPMirrorBrokerConnectionElement());
server.getConfiguration().addAMQPConnection(amqpConnection);
server.start();
Wait.assertTrue(() -> AssertionLoggerHandler.findText("AMQ111001"));
Assert.assertEquals(6, AssertionLoggerHandler.countText("AMQ119018")); // 0..5 = 6
}
/**
* Refuse the first mirror link, verify broker handles it and reconnects
*
* @throws Exception
*/
@Test
public void testReconnectAfterMirrorLinkRefusal() throws Exception {
startVerx();
AtomicInteger errors = new AtomicInteger(0);
AtomicInteger messagesReceived = new AtomicInteger(0);
List<ProtonConnection> connections = Collections.synchronizedList(new ArrayList<ProtonConnection>());
List<ProtonConnection> disconnected = Collections.synchronizedList(new ArrayList<ProtonConnection>());
AtomicInteger refusedLinkMessageCount = new AtomicInteger();
AtomicInteger linkOpens = new AtomicInteger(0);
mockServer = new MockServer(vertx, serverConnection -> {
serverConnection.disconnectHandler(c -> {
disconnected.add(serverConnection);
});
serverConnection.openHandler(c -> {
connections.add(serverConnection);
serverConnection.open();
});
serverConnection.closeHandler(c -> {
serverConnection.close();
connections.remove(serverConnection);
});
serverConnection.sessionOpenHandler(session -> {
session.open();
});
serverConnection.receiverOpenHandler(serverReceiver -> {
Target remoteTarget = serverReceiver.getRemoteTarget();
String remoteAddress = remoteTarget == null ? null : remoteTarget.getAddress();
if (remoteAddress == null || !remoteAddress.startsWith(ProtonProtocolManager.MIRROR_ADDRESS)) {
errors.incrementAndGet();
logger.warn("Receiving address as " + remoteAddress);
return;
}
if (linkOpens.incrementAndGet() != 2) {
logger.debug("Link Opens::" + linkOpens);
logger.debug("ServerReceiver = " + serverReceiver.getTarget());
serverReceiver.setTarget(null);
serverReceiver.handler((del, msg) -> {
refusedLinkMessageCount.incrementAndGet();
logger.debug("Should not have got message on refused link: " + msg);
});
serverReceiver.open();
vertx.setTimer(20, x -> {
serverReceiver.setCondition(new ErrorCondition(AmqpError.ILLEGAL_STATE, "Testing refusal of mirror link for $reasons"));
serverReceiver.close();
});
} else {
serverReceiver.setTarget(serverReceiver.getRemoteTarget());
HashMap<Symbol, Object> linkProperties = new HashMap<>();
linkProperties.put(AMQPMirrorControllerSource.BROKER_ID, "fake-id");
serverReceiver.setProperties(linkProperties);
serverReceiver.setOfferedCapabilities(new Symbol[]{AMQPMirrorControllerSource.MIRROR_CAPABILITY});
serverReceiver.handler((del, msg) -> {
logger.debug("prefetch = " + serverReceiver.getPrefetch() + ", Got message: " + msg);
if (msg.getApplicationProperties() != null) {
Map map = msg.getApplicationProperties().getValue();
Object value = map.get("sender");
if (value != null) {
if (messagesReceived.get() != ((Integer) value).intValue()) {
logger.warn("Message out of order. Expected " + messagesReceived.get() + " but received " + value);
errors.incrementAndGet();
}
messagesReceived.incrementAndGet();
}
}
del.disposition(Accepted.getInstance(), true);
if (serverReceiver.getPrefetch() == 0) {
serverReceiver.flow(1);
}
});
serverReceiver.open();
}
});
});
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration("test", "tcp://localhost:" + mockServer.actualPort()).setReconnectAttempts(3).setRetryInterval(10);
amqpConnection.addElement(new AMQPMirrorBrokerConnectionElement());
server.getConfiguration().addAMQPConnection(amqpConnection);
server.start();
Wait.assertEquals(1, disconnected::size, 6000);
Wait.assertEquals(2, connections::size, 6000);
assertSame(connections.get(0), disconnected.get(0));
assertFalse(connections.get(1).isDisconnected());
assertEquals("Should not have got any message on refused link", 0, refusedLinkMessageCount.get());
assertEquals(0, errors.get());
ConnectionFactory factory = CFUtil.createConnectionFactory("AMQP", "tcp://localhost:" + AMQP_PORT);
try (Connection connection = factory.createConnection()) {
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
MessageProducer producer = session.createProducer(session.createQueue(getQueueName()));
for (int i = 0; i < 100; i++) {
TextMessage message = session.createTextMessage("hello");
message.setIntProperty("sender", i);
producer.send(message);
}
}
Wait.assertEquals(100, messagesReceived::intValue);
assertEquals(0, errors.get()); // Meant to check again. the errors check before was because of connection issues. This one is about duplicates on receiving
}
@Test
public void testNoClientDesiredMirrorCapability() throws Exception {
AssertionLoggerHandler.startCapture();
server.start();
AmqpClient client = new AmqpClient(new URI("tcp://localhost:" + AMQP_PORT), null, null);
client.setValidator(new AmqpValidator() {
@Override
public void inspectOpenedResource(Sender sender) {
ErrorCondition condition = sender.getRemoteCondition();
if (condition != null && condition.getCondition() != null) {
if (!condition.getCondition().equals(AmqpError.ILLEGAL_STATE)) {
markAsInvalid("Should have been closed with an illegal state error, but error was: " + condition);
}
if (!condition.getDescription().contains("AMQ119024")) {
markAsInvalid("should have indicated the error code about missing a desired capability");
}
if (!condition.getDescription().contains(AMQPMirrorControllerSource.MIRROR_CAPABILITY)) {
markAsInvalid("should have indicated the error code about missing a desired capability");
}
} else {
markAsInvalid("Sender should have been detached with an error");
}
}
});
String address = ProtonProtocolManager.getMirrorAddress(getTestName());
AmqpConnection connection = client.connect();
try {
AmqpSession session = connection.createSession();
try {
session.createSender(address);
fail("Link should have been refused.");
} catch (Exception ex) {
Assert.assertTrue(ex.getMessage().contains("AMQ119024"));
instanceLog.debug("Caught expected exception");
}
connection.getStateInspector().assertValid();
} finally {
connection.close();
}
Wait.assertTrue(() -> AssertionLoggerHandler.findText("AMQ119024"));
}
}

View File

@ -16,21 +16,18 @@
*/
package org.apache.activemq.artemis.tests.integration.amqp.journal;
import java.util.ArrayList;
import java.util.concurrent.TimeUnit;
import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.core.server.MessageReference;
import org.apache.activemq.artemis.core.server.Queue;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
import org.apache.activemq.artemis.tests.integration.amqp.AmqpClientTestSupport;
import org.apache.activemq.artemis.tests.util.Wait;
import org.apache.activemq.artemis.utils.collections.LinkedListIterator;
import org.apache.activemq.transport.amqp.client.AmqpClient;
import org.apache.activemq.transport.amqp.client.AmqpConnection;
import org.apache.activemq.transport.amqp.client.AmqpMessage;
import org.apache.activemq.transport.amqp.client.AmqpReceiver;
import org.apache.activemq.transport.amqp.client.AmqpSession;
import org.hamcrest.Matchers;
import org.junit.Assert;
import org.junit.Test;
@ -45,23 +42,22 @@ public class AmqpJournalLoadingTest extends AmqpClientTestSupport {
server.stop();
server.start();
final AMQPMessage amqpMessage;
final Queue afterRestartQueueView = getProxyToQueue(getQueueName());
Wait.assertTrue("All messages should arrive", () -> afterRestartQueueView.getMessageCount() == 1);
try (LinkedListIterator<MessageReference> iterator = afterRestartQueueView.iterator()) {
Assert.assertTrue(iterator.hasNext());
final MessageReference next = iterator.next();
Assert.assertNotNull(next);
Assert.assertFalse(iterator.hasNext());
final Message message = next.getMessage();
Assert.assertThat(message, Matchers.instanceOf(AMQPMessage.class));
amqpMessage = (AMQPMessage) message;
Assert.assertEquals(AMQPMessage.MessageDataScanningStatus.RELOAD_PERSISTENCE, amqpMessage.getDataScanningStatus());
Assert.assertTrue(amqpMessage.isDurable());
}
ArrayList<AMQPMessage> messageReference = new ArrayList<>(1);
afterRestartQueueView.forEach((next) -> {
final AMQPMessage message = (AMQPMessage)next.getMessage();
Assert.assertEquals(AMQPMessage.MessageDataScanningStatus.RELOAD_PERSISTENCE, message.getDataScanningStatus());
Assert.assertTrue(message.isDurable());
// Doing the check again in case isDurable messed up the scanning status. It should not change the status by definition
Assert.assertEquals(AMQPMessage.MessageDataScanningStatus.RELOAD_PERSISTENCE, message.getDataScanningStatus());
messageReference.add(message);
});
Assert.assertEquals(1, messageReference.size());
AmqpClient client = createAmqpClient();
AmqpConnection connection = addConnection(client.connect());
@ -76,7 +72,8 @@ public class AmqpJournalLoadingTest extends AmqpClientTestSupport {
assertEquals(1, afterRestartQueueView.getMessageCount());
Assert.assertEquals(AMQPMessage.MessageDataScanningStatus.SCANNED, amqpMessage.getDataScanningStatus());
Assert.assertEquals(AMQPMessage.MessageDataScanningStatus.SCANNED, messageReference.get(0).getDataScanningStatus());
receive.accept();

View File

@ -853,6 +853,38 @@
</args>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-dualMirror-serverA</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<role>amq</role>
<user>artemis</user>
<password>artemis</password>
<allowAnonymous>false</allowAnonymous>
<noWeb>false</noWeb>
<instance>${basedir}/target/brokerConnect/serverA</instance>
<configuration>${basedir}/target/classes/servers/brokerConnect/serverA</configuration>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-dualMirror-serverB</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<role>amq</role>
<user>artemis</user>
<password>artemis</password>
<allowAnonymous>false</allowAnonymous>
<noWeb>false</noWeb>
<instance>${basedir}/target/brokerConnect/serverB</instance>
<configuration>${basedir}/target/classes/servers/brokerConnect/serverB</configuration>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-qdr</id>
@ -869,6 +901,116 @@
<configuration>${basedir}/target/classes/servers/brokerConnect/qdr</configuration>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-bc-replica-main-A</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<instance>${basedir}/target/brokerConnect/replicaMainServerA</instance>
<configuration>${basedir}/target/classes/servers/brokerConnect/replicaMainServerA</configuration>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-bc-replica-backup-A</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<instance>${basedir}/target/brokerConnect/replicaBackupServerA</instance>
<configuration>${basedir}/target/classes/servers/brokerConnect/replicaBackupServerA</configuration>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-bc-replica-main-B</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<instance>${basedir}/target/brokerConnect/replicaMainServerB</instance>
<configuration>${basedir}/target/classes/servers/brokerConnect/replicaMainServerB</configuration>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-bc-replica-backup-B</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<instance>${basedir}/target/brokerConnect/replicaBackupServerB</instance>
<configuration>${basedir}/target/classes/servers/brokerConnect/replicaBackupServerB</configuration>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-bc-dual-root</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<instance>${basedir}/target/brokerConnect/splitMirror/serverRoot</instance>
<configuration>${basedir}/target/classes/servers/brokerConnect/splitMirror/serverRoot</configuration>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-bc-dual-A</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<instance>${basedir}/target/brokerConnect/splitMirror/serverA</instance>
<configuration>${basedir}/target/classes/servers/brokerConnect/splitMirror/serverA</configuration>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-bc-dual-B</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<instance>${basedir}/target/brokerConnect/splitMirror/serverB</instance>
<configuration>${basedir}/target/classes/servers/brokerConnect/splitMirror/serverB</configuration>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-bc-chained-root</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<instance>${basedir}/target/brokerConnect/chainedMirror/serverRoot</instance>
<configuration>${basedir}/target/classes/servers/brokerConnect/chainedMirror/serverRoot</configuration>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-bc-chained-A</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<instance>${basedir}/target/brokerConnect/chainedMirror/serverA</instance>
<configuration>${basedir}/target/classes/servers/brokerConnect/chainedMirror/serverA</configuration>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-bc-chained-B</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<instance>${basedir}/target/brokerConnect/chainedMirror/serverB</instance>
<configuration>${basedir}/target/classes/servers/brokerConnect/chainedMirror/serverB</configuration>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-bridge-transfer-A</id>

View File

@ -0,0 +1,216 @@
<?xml version='1.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.
-->
<configuration xmlns="urn:activemq"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xmlns:xi="http://www.w3.org/2001/XInclude"
xsi:schemaLocation="urn:activemq /schema/artemis-configuration.xsd">
<core xmlns="urn:activemq:core" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="urn:activemq:core ">
<name>ServerA</name>
<persistence-enabled>true</persistence-enabled>
<security-enabled>false</security-enabled>
<!-- this could be ASYNCIO, MAPPED, NIO
ASYNCIO: Linux Libaio
MAPPED: mmap files
NIO: Plain Java Files
-->
<journal-type>NIO</journal-type>
<paging-directory>data/paging</paging-directory>
<bindings-directory>data/bindings</bindings-directory>
<journal-directory>data/journal</journal-directory>
<large-messages-directory>data/large-messages</large-messages-directory>
<journal-datasync>true</journal-datasync>
<journal-min-files>2</journal-min-files>
<journal-pool-files>10</journal-pool-files>
<journal-device-block-size>4096</journal-device-block-size>
<journal-file-size>10M</journal-file-size>
<!--
This value was determined through a calculation.
Your system could perform 25 writes per millisecond
on the current journal configuration.
That translates as a sync write every 40000 nanoseconds.
Note: If you specify 0 the system will perform writes directly to the disk.
We recommend this to be 0 if you are using journalType=MAPPED and journal-datasync=false.
-->
<journal-buffer-timeout>40000</journal-buffer-timeout>
<!--
When using ASYNCIO, this will determine the writing queue depth for libaio.
-->
<journal-max-io>1</journal-max-io>
<!--
You can verify the network health of a particular NIC by specifying the <network-check-NIC> element.
<network-check-NIC>theNicName</network-check-NIC>
-->
<broker-connections>
<amqp-connection uri="tcp://artemisTestB:61616" name="serverB" retry-interval="100" reconnect-attempts="-1">
<mirror/>
</amqp-connection>
</broker-connections>
<!--
Use this to use an HTTP server to validate the network
<network-check-URL-list>http://www.apache.org</network-check-URL-list> -->
<!-- <network-check-period>10000</network-check-period> -->
<!-- <network-check-timeout>1000</network-check-timeout> -->
<!-- this is a comma separated list, no spaces, just DNS or IPs
it should accept IPV6
Warning: Make sure you understand your network topology as this is meant to validate if your network is valid.
Using IPs that could eventually disappear or be partially visible may defeat the purpose.
You can use a list of multiple IPs, and if any successful ping will make the server OK to continue running -->
<!-- <network-check-list>10.0.0.1</network-check-list> -->
<!-- use this to customize the ping used for ipv4 addresses -->
<!-- <network-check-ping-command>ping -c 1 -t %d %s</network-check-ping-command> -->
<!-- use this to customize the ping used for ipv6 addresses -->
<!-- <network-check-ping6-command>ping6 -c 1 %2$s</network-check-ping6-command> -->
<!-- how often we are looking for how many bytes are being used on the disk in ms -->
<disk-scan-period>5000</disk-scan-period>
<!-- once the disk hits this limit the system will block, or close the connection in certain protocols
that won't support flow control. -->
<max-disk-usage>90</max-disk-usage>
<!-- should the broker detect dead locks and other issues -->
<critical-analyzer>true</critical-analyzer>
<critical-analyzer-timeout>120000</critical-analyzer-timeout>
<critical-analyzer-check-period>60000</critical-analyzer-check-period>
<critical-analyzer-policy>HALT</critical-analyzer-policy>
<page-sync-timeout>40000</page-sync-timeout>
<acceptors>
<acceptor name="artemis">tcp://0.0.0.0:61616?tcpSendBufferSize=1048576;tcpReceiveBufferSize=1048576;amqpMinLargeMessageSize=102400;protocols=CORE,AMQP,STOMP,HORNETQ,MQTT,OPENWIRE;useEpoll=true;amqpCredits=1000;amqpLowCredits=300;amqpDuplicateDetection=true</acceptor>
</acceptors>
<security-settings>
<security-setting match="#">
<permission type="createNonDurableQueue" roles="amq"/>
<permission type="deleteNonDurableQueue" roles="amq"/>
<permission type="createDurableQueue" roles="amq"/>
<permission type="deleteDurableQueue" roles="amq"/>
<permission type="createAddress" roles="amq"/>
<permission type="deleteAddress" roles="amq"/>
<permission type="consume" roles="amq"/>
<permission type="browse" roles="amq"/>
<permission type="send" roles="amq"/>
<!-- we need this otherwise ./artemis data imp wouldn't work -->
<permission type="manage" roles="amq"/>
</security-setting>
</security-settings>
<address-settings>
<!-- if you define auto-create on certain queues, management has to be auto-create -->
<address-setting match="activemq.management#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
<!--default for catch all-->
<address-setting match="#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
</address-settings>
<addresses>
<address name="DLQ">
<anycast>
<queue name="DLQ" />
</anycast>
</address>
<address name="ExpiryQueue">
<anycast>
<queue name="ExpiryQueue" />
</anycast>
</address>
<address name="someQueue">
<anycast>
<queue name="someQueue" />
</anycast>
</address>
</addresses>
<!-- Uncomment the following if you want to use the Standard LoggingActiveMQServerPlugin pluging to log in events
<broker-plugins>
<broker-plugin class-name="org.apache.activemq.artemis.core.server.plugin.impl.LoggingActiveMQServerPlugin">
<property key="LOG_ALL_EVENTS" value="true"/>
<property key="LOG_CONNECTION_EVENTS" value="true"/>
<property key="LOG_SESSION_EVENTS" value="true"/>
<property key="LOG_CONSUMER_EVENTS" value="true"/>
<property key="LOG_DELIVERING_EVENTS" value="true"/>
<property key="LOG_SENDING_EVENTS" value="true"/>
<property key="LOG_INTERNAL_EVENTS" value="true"/>
</broker-plugin>
</broker-plugins>
-->
</core>
</configuration>

View File

@ -0,0 +1,213 @@
<?xml version='1.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.
-->
<configuration xmlns="urn:activemq"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xmlns:xi="http://www.w3.org/2001/XInclude"
xsi:schemaLocation="urn:activemq /schema/artemis-configuration.xsd">
<core xmlns="urn:activemq:core" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="urn:activemq:core ">
<name>ServerB</name>
<security-enabled>false</security-enabled>
<persistence-enabled>true</persistence-enabled>
<!-- this could be ASYNCIO, MAPPED, NIO
ASYNCIO: Linux Libaio
MAPPED: mmap files
NIO: Plain Java Files
-->
<journal-type>NIO</journal-type>
<paging-directory>data/paging</paging-directory>
<bindings-directory>data/bindings</bindings-directory>
<journal-directory>data/journal</journal-directory>
<large-messages-directory>data/large-messages</large-messages-directory>
<journal-datasync>true</journal-datasync>
<journal-min-files>2</journal-min-files>
<journal-pool-files>10</journal-pool-files>
<journal-device-block-size>4096</journal-device-block-size>
<journal-file-size>10M</journal-file-size>
<!--
This value was determined through a calculation.
Your system could perform 25 writes per millisecond
on the current journal configuration.
That translates as a sync write every 40000 nanoseconds.
Note: If you specify 0 the system will perform writes directly to the disk.
We recommend this to be 0 if you are using journalType=MAPPED and journal-datasync=false.
-->
<journal-buffer-timeout>40000</journal-buffer-timeout>
<broker-connections>
<amqp-connection uri="tcp://artemisTestRoot:61616" name="serverRoot" retry-interval="100" reconnect-attempts="-1">
<mirror/>
</amqp-connection>
</broker-connections>
<!--
When using ASYNCIO, this will determine the writing queue depth for libaio.
-->
<journal-max-io>1</journal-max-io>
<!--
You can verify the network health of a particular NIC by specifying the <network-check-NIC> element.
<network-check-NIC>theNicName</network-check-NIC>
-->
<!--
Use this to use an HTTP server to validate the network
<network-check-URL-list>http://www.apache.org</network-check-URL-list> -->
<!-- <network-check-period>10000</network-check-period> -->
<!-- <network-check-timeout>1000</network-check-timeout> -->
<!-- this is a comma separated list, no spaces, just DNS or IPs
it should accept IPV6
Warning: Make sure you understand your network topology as this is meant to validate if your network is valid.
Using IPs that could eventually disappear or be partially visible may defeat the purpose.
You can use a list of multiple IPs, and if any successful ping will make the server OK to continue running -->
<!-- <network-check-list>10.0.0.1</network-check-list> -->
<!-- use this to customize the ping used for ipv4 addresses -->
<!-- <network-check-ping-command>ping -c 1 -t %d %s</network-check-ping-command> -->
<!-- use this to customize the ping used for ipv6 addresses -->
<!-- <network-check-ping6-command>ping6 -c 1 %2$s</network-check-ping6-command> -->
<!-- how often we are looking for how many bytes are being used on the disk in ms -->
<disk-scan-period>5000</disk-scan-period>
<!-- once the disk hits this limit the system will block, or close the connection in certain protocols
that won't support flow control. -->
<max-disk-usage>90</max-disk-usage>
<!-- should the broker detect dead locks and other issues -->
<critical-analyzer>true</critical-analyzer>
<critical-analyzer-timeout>120000</critical-analyzer-timeout>
<critical-analyzer-check-period>60000</critical-analyzer-check-period>
<critical-analyzer-policy>HALT</critical-analyzer-policy>
<page-sync-timeout>40000</page-sync-timeout>
<acceptors>
<acceptor name="artemis">tcp://0.0.0.0:61616?tcpSendBufferSize=1048576;tcpReceiveBufferSize=1048576;amqpMinLargeMessageSize=102400;protocols=CORE,AMQP,STOMP,HORNETQ,MQTT,OPENWIRE;useEpoll=true;amqpCredits=1000;amqpLowCredits=300;amqpDuplicateDetection=true</acceptor>
</acceptors>
<security-settings>
<security-setting match="#">
<permission type="createNonDurableQueue" roles="amq"/>
<permission type="deleteNonDurableQueue" roles="amq"/>
<permission type="createDurableQueue" roles="amq"/>
<permission type="deleteDurableQueue" roles="amq"/>
<permission type="createAddress" roles="amq"/>
<permission type="deleteAddress" roles="amq"/>
<permission type="consume" roles="amq"/>
<permission type="browse" roles="amq"/>
<permission type="send" roles="amq"/>
<!-- we need this otherwise ./artemis data imp wouldn't work -->
<permission type="manage" roles="amq"/>
</security-setting>
</security-settings>
<address-settings>
<!-- if you define auto-create on certain queues, management has to be auto-create -->
<address-setting match="activemq.management#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
<!--default for catch all-->
<address-setting match="#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
</address-settings>
<addresses>
<address name="DLQ">
<anycast>
<queue name="DLQ" />
</anycast>
</address>
<address name="ExpiryQueue">
<anycast>
<queue name="ExpiryQueue" />
</anycast>
</address>
<address name="someQueue">
<anycast>
<queue name="someQueue" />
</anycast>
</address>
</addresses>
<!-- Uncomment the following if you want to use the Standard LoggingActiveMQServerPlugin pluging to log in events
<broker-plugins>
<broker-plugin class-name="org.apache.activemq.artemis.core.server.plugin.impl.LoggingActiveMQServerPlugin">
<property key="LOG_ALL_EVENTS" value="true"/>
<property key="LOG_CONNECTION_EVENTS" value="true"/>
<property key="LOG_SESSION_EVENTS" value="true"/>
<property key="LOG_CONSUMER_EVENTS" value="true"/>
<property key="LOG_DELIVERING_EVENTS" value="true"/>
<property key="LOG_SENDING_EVENTS" value="true"/>
<property key="LOG_INTERNAL_EVENTS" value="true"/>
</broker-plugin>
</broker-plugins>
-->
</core>
</configuration>

View File

@ -0,0 +1,214 @@
<?xml version='1.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.
-->
<configuration xmlns="urn:activemq"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xmlns:xi="http://www.w3.org/2001/XInclude"
xsi:schemaLocation="urn:activemq /schema/artemis-configuration.xsd">
<core xmlns="urn:activemq:core" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="urn:activemq:core ">
<name>ServerRoot</name>
<security-enabled>false</security-enabled>
<persistence-enabled>true</persistence-enabled>
<!-- this could be ASYNCIO, MAPPED, NIO
ASYNCIO: Linux Libaio
MAPPED: mmap files
NIO: Plain Java Files
-->
<journal-type>NIO</journal-type>
<paging-directory>data/paging</paging-directory>
<bindings-directory>data/bindings</bindings-directory>
<journal-directory>data/journal</journal-directory>
<large-messages-directory>data/large-messages</large-messages-directory>
<journal-datasync>true</journal-datasync>
<journal-min-files>2</journal-min-files>
<journal-pool-files>10</journal-pool-files>
<journal-device-block-size>4096</journal-device-block-size>
<journal-file-size>10M</journal-file-size>
<!--
This value was determined through a calculation.
Your system could perform 25 writes per millisecond
on the current journal configuration.
That translates as a sync write every 40000 nanoseconds.
Note: If you specify 0 the system will perform writes directly to the disk.
We recommend this to be 0 if you are using journalType=MAPPED and journal-datasync=false.
-->
<journal-buffer-timeout>40000</journal-buffer-timeout>
<!--
When using ASYNCIO, this will determine the writing queue depth for libaio.
-->
<journal-max-io>1</journal-max-io>
<!--
You can verify the network health of a particular NIC by specifying the <network-check-NIC> element.
<network-check-NIC>theNicName</network-check-NIC>
-->
<!--
Use this to use an HTTP server to validate the network
<network-check-URL-list>http://www.apache.org</network-check-URL-list> -->
<!-- <network-check-period>10000</network-check-period> -->
<!-- <network-check-timeout>1000</network-check-timeout> -->
<!-- this is a comma separated list, no spaces, just DNS or IPs
it should accept IPV6
Warning: Make sure you understand your network topology as this is meant to validate if your network is valid.
Using IPs that could eventually disappear or be partially visible may defeat the purpose.
You can use a list of multiple IPs, and if any successful ping will make the server OK to continue running -->
<!-- <network-check-list>10.0.0.1</network-check-list> -->
<!-- use this to customize the ping used for ipv4 addresses -->
<!-- <network-check-ping-command>ping -c 1 -t %d %s</network-check-ping-command> -->
<!-- use this to customize the ping used for ipv6 addresses -->
<!-- <network-check-ping6-command>ping6 -c 1 %2$s</network-check-ping6-command> -->
<!-- how often we are looking for how many bytes are being used on the disk in ms -->
<disk-scan-period>5000</disk-scan-period>
<!-- once the disk hits this limit the system will block, or close the connection in certain protocols
that won't support flow control. -->
<max-disk-usage>90</max-disk-usage>
<!-- should the broker detect dead locks and other issues -->
<critical-analyzer>true</critical-analyzer>
<critical-analyzer-timeout>120000</critical-analyzer-timeout>
<critical-analyzer-check-period>60000</critical-analyzer-check-period>
<critical-analyzer-policy>HALT</critical-analyzer-policy>
<page-sync-timeout>40000</page-sync-timeout>
<acceptors>
<acceptor name="artemis">tcp://0.0.0.0:61616?tcpSendBufferSize=1048576;tcpReceiveBufferSize=1048576;amqpMinLargeMessageSize=102400;protocols=CORE,AMQP,STOMP,HORNETQ,MQTT,OPENWIRE;useEpoll=true;amqpCredits=1000;amqpLowCredits=300;amqpDuplicateDetection=true</acceptor>
</acceptors>
<broker-connections>
<amqp-connection uri="tcp://artemisTestA:61616" name="serverA" retry-interval="100" reconnect-attempts="-1">
<mirror/>
</amqp-connection>
</broker-connections>
<security-settings>
<security-setting match="#">
<permission type="createNonDurableQueue" roles="amq"/>
<permission type="deleteNonDurableQueue" roles="amq"/>
<permission type="createDurableQueue" roles="amq"/>
<permission type="deleteDurableQueue" roles="amq"/>
<permission type="createAddress" roles="amq"/>
<permission type="deleteAddress" roles="amq"/>
<permission type="consume" roles="amq"/>
<permission type="browse" roles="amq"/>
<permission type="send" roles="amq"/>
<!-- we need this otherwise ./artemis data imp wouldn't work -->
<permission type="manage" roles="amq"/>
</security-setting>
</security-settings>
<address-settings>
<!-- if you define auto-create on certain queues, management has to be auto-create -->
<address-setting match="activemq.management#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
<!--default for catch all-->
<address-setting match="#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
</address-settings>
<addresses>
<address name="DLQ">
<anycast>
<queue name="DLQ" />
</anycast>
</address>
<address name="ExpiryQueue">
<anycast>
<queue name="ExpiryQueue" />
</anycast>
</address>
<address name="someQueue">
<anycast>
<queue name="someQueue" />
</anycast>
</address>
</addresses>
<!-- Uncomment the following if you want to use the Standard LoggingActiveMQServerPlugin pluging to log in events
<broker-plugins>
<broker-plugin class-name="org.apache.activemq.artemis.core.server.plugin.impl.LoggingActiveMQServerPlugin">
<property key="LOG_ALL_EVENTS" value="true"/>
<property key="LOG_CONNECTION_EVENTS" value="true"/>
<property key="LOG_SESSION_EVENTS" value="true"/>
<property key="LOG_CONSUMER_EVENTS" value="true"/>
<property key="LOG_DELIVERING_EVENTS" value="true"/>
<property key="LOG_SENDING_EVENTS" value="true"/>
<property key="LOG_INTERNAL_EVENTS" value="true"/>
</broker-plugin>
</broker-plugins>
-->
</core>
</configuration>

View File

@ -28,8 +28,7 @@ under the License.
<name>0.0.0.0</name>
<persistence-enabled>false</persistence-enabled>
<persistence-enabled>true</persistence-enabled>
<!-- this could be ASYNCIO, MAPPED, NIO
ASYNCIO: Linux Libaio
@ -127,7 +126,7 @@ under the License.
<broker-connections>
<amqp-connection uri="tcp://localhost:61617" name="outgoing" reconnect-attempts="-1" retry-interval="100" user="B" password="B">
<mirror/>
<mirror durable="true"/>
</amqp-connection>
</broker-connections>

View File

@ -0,0 +1,86 @@
#
# 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.
#
# Additional logger names to configure (root logger is always configured)
# Root logger option
loggers=org.eclipse.jetty,org.jboss.logging,org.apache.activemq.artemis.core.server,org.apache.activemq.artemis.utils,org.apache.activemq.artemis.utils.critical,org.apache.activemq.artemis.journal,org.apache.activemq.artemis.jms.server,org.apache.activemq.artemis.integration.bootstrap,org.apache.activemq.audit.base,org.apache.activemq.audit.message,org.apache.activemq.audit.resource,org.apache.activemq.artemis.utils.pools,org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerTarget,org.apache.activemq.artemis.core.server.impl,org.apache.activemq.artemis.utils.collections,org.apache.activemq.artemis.core.postoffice.impl
# Special logger to debug mirror Security
# Root logger level
logger.level=INFO
# ActiveMQ Artemis logger levels
# These levels are candidates to eventually debug
logger.org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerTarget.level=INFO
# if you have issues with CriticalAnalyzer, setting this as TRACE would give you extra troubleshooting information.
# but do not use it regularly as it would incur in some extra CPU usage for this diagnostic.
logger.org.apache.activemq.artemis.utils.critical.level=INFO
logger.org.eclipse.jetty.level=WARN
# Root logger handlers
logger.handlers=FILE,CONSOLE
# to enable audit change the level to INFO
logger.org.apache.activemq.audit.base.level=ERROR
logger.org.apache.activemq.audit.base.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.base.useParentHandlers=false
logger.org.apache.activemq.audit.resource.level=ERROR
logger.org.apache.activemq.audit.resource.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.resource.useParentHandlers=false
logger.org.apache.activemq.audit.message.level=ERROR
logger.org.apache.activemq.audit.message.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.message.useParentHandlers=false
# Console handler configuration
handler.CONSOLE=org.jboss.logmanager.handlers.ConsoleHandler
handler.CONSOLE.properties=autoFlush
handler.CONSOLE.level=TRACE
handler.CONSOLE.autoFlush=true
handler.CONSOLE.formatter=PATTERN
# File handler configuration
handler.FILE=org.jboss.logmanager.handlers.PeriodicRotatingFileHandler
handler.FILE.level=TRACE
handler.FILE.properties=suffix,append,autoFlush,fileName
handler.FILE.suffix=.yyyy-MM-dd
handler.FILE.append=true
handler.FILE.autoFlush=true
handler.FILE.fileName=${artemis.instance}/log/artemis.log
handler.FILE.formatter=PATTERN
# Formatter pattern configuration
formatter.PATTERN=org.jboss.logmanager.formatters.PatternFormatter
formatter.PATTERN.properties=pattern
formatter.PATTERN.pattern=[%t] %d %-5p [%c] %s%E%n
#Audit logger
handler.AUDIT_FILE=org.jboss.logmanager.handlers.PeriodicRotatingFileHandler
handler.AUDIT_FILE.level=INFO
handler.AUDIT_FILE.properties=suffix,append,autoFlush,fileName
handler.AUDIT_FILE.suffix=.yyyy-MM-dd
handler.AUDIT_FILE.append=true
handler.AUDIT_FILE.autoFlush=true
handler.AUDIT_FILE.fileName=${artemis.instance}/log/audit.log
handler.AUDIT_FILE.formatter=AUDIT_PATTERN
formatter.AUDIT_PATTERN=org.jboss.logmanager.formatters.PatternFormatter
formatter.AUDIT_PATTERN.properties=pattern
formatter.AUDIT_PATTERN.pattern=%d [AUDIT](%t) %s%E%n

View File

@ -28,8 +28,7 @@ under the License.
<name>0.0.0.0</name>
<persistence-enabled>false</persistence-enabled>
<persistence-enabled>true</persistence-enabled>
<!-- this could be ASYNCIO, MAPPED, NIO
ASYNCIO: Linux Libaio
@ -122,7 +121,7 @@ under the License.
<broker-connections>
<amqp-connection uri="tcp://localhost:61616" name="outgoing" reconnect-attempts="-1" retry-interval="100" user="A" password="A">
<mirror/>
<mirror durable="true"/>
</amqp-connection>
</broker-connections>
<acceptors>

View File

@ -0,0 +1,86 @@
#
# 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.
#
# Additional logger names to configure (root logger is always configured)
# Root logger option
loggers=org.eclipse.jetty,org.jboss.logging,org.apache.activemq.artemis.core.server,org.apache.activemq.artemis.utils,org.apache.activemq.artemis.utils.critical,org.apache.activemq.artemis.journal,org.apache.activemq.artemis.jms.server,org.apache.activemq.artemis.integration.bootstrap,org.apache.activemq.audit.base,org.apache.activemq.audit.message,org.apache.activemq.audit.resource,org.apache.activemq.artemis.utils.pools,org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerTarget,org.apache.activemq.artemis.core.server.impl,org.apache.activemq.artemis.utils.collections,org.apache.activemq.artemis.core.postoffice.impl
# Special logger to debug mirror Security
# Root logger level
logger.level=INFO
# ActiveMQ Artemis logger levels
# These levels are candidates to eventually debug
logger.org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerTarget.level=INFO
# if you have issues with CriticalAnalyzer, setting this as TRACE would give you extra troubleshooting information.
# but do not use it regularly as it would incur in some extra CPU usage for this diagnostic.
logger.org.apache.activemq.artemis.utils.critical.level=INFO
logger.org.eclipse.jetty.level=WARN
# Root logger handlers
logger.handlers=FILE,CONSOLE
# to enable audit change the level to INFO
logger.org.apache.activemq.audit.base.level=ERROR
logger.org.apache.activemq.audit.base.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.base.useParentHandlers=false
logger.org.apache.activemq.audit.resource.level=ERROR
logger.org.apache.activemq.audit.resource.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.resource.useParentHandlers=false
logger.org.apache.activemq.audit.message.level=ERROR
logger.org.apache.activemq.audit.message.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.message.useParentHandlers=false
# Console handler configuration
handler.CONSOLE=org.jboss.logmanager.handlers.ConsoleHandler
handler.CONSOLE.properties=autoFlush
handler.CONSOLE.level=TRACE
handler.CONSOLE.autoFlush=true
handler.CONSOLE.formatter=PATTERN
# File handler configuration
handler.FILE=org.jboss.logmanager.handlers.PeriodicRotatingFileHandler
handler.FILE.level=TRACE
handler.FILE.properties=suffix,append,autoFlush,fileName
handler.FILE.suffix=.yyyy-MM-dd
handler.FILE.append=true
handler.FILE.autoFlush=true
handler.FILE.fileName=${artemis.instance}/log/artemis.log
handler.FILE.formatter=PATTERN
# Formatter pattern configuration
formatter.PATTERN=org.jboss.logmanager.formatters.PatternFormatter
formatter.PATTERN.properties=pattern
formatter.PATTERN.pattern=[%t] %d %-5p [%c] %s%E%n
#Audit logger
handler.AUDIT_FILE=org.jboss.logmanager.handlers.PeriodicRotatingFileHandler
handler.AUDIT_FILE.level=INFO
handler.AUDIT_FILE.properties=suffix,append,autoFlush,fileName
handler.AUDIT_FILE.suffix=.yyyy-MM-dd
handler.AUDIT_FILE.append=true
handler.AUDIT_FILE.autoFlush=true
handler.AUDIT_FILE.fileName=${artemis.instance}/log/audit.log
handler.AUDIT_FILE.formatter=AUDIT_PATTERN
formatter.AUDIT_PATTERN=org.jboss.logmanager.formatters.PatternFormatter
formatter.AUDIT_PATTERN.properties=pattern
formatter.AUDIT_PATTERN.pattern=%d [AUDIT](%t) %s%E%n

View File

@ -0,0 +1,142 @@
<?xml version="1.0" encoding="UTF-8" standalone="no"?>
<!--
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.
--><configuration xmlns="urn:activemq" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="urn:activemq /schema/artemis-server.xsd">
<core xmlns="urn:activemq:core">
<bindings-directory>./data/bindings</bindings-directory>
<journal-directory>./data/journal</journal-directory>
<large-messages-directory>./data/largemessages</large-messages-directory>
<paging-directory>./data/paging</paging-directory>
<cluster-user>exampleUser</cluster-user>
<cluster-password>secret</cluster-password>
<journal-retention-directory storage-limit="20M">./data/retention</journal-retention-directory>
<ha-policy>
<replication>
<slave>
<allow-failback>true</allow-failback>
<!-- not needed but tells the backup not to restart after failback as there will be > 0 backups saved -->
<max-saved-replicated-journals-size>0</max-saved-replicated-journals-size>
</slave>
</replication>
</ha-policy>
<!-- Connectors -->
<connectors>
<connector name="netty-live-connector">tcp://mainA:61616</connector>
<connector name="netty-connector">tcp://backupA:61616</connector>
</connectors>
<!-- Acceptors -->
<acceptors>
<acceptor name="netty-acceptor">tcp://0.0.0.0:61616</acceptor>
</acceptors>
<broker-connections>
<amqp-connection uri="tcp://mainB:61616#tcp://backupB:61616" name="serverB" reconnect-attempts="-1" retry-interval="500">
<mirror durable="true"/>
</amqp-connection>
</broker-connections>
<cluster-connections>
<cluster-connection name="my-cluster">
<connector-ref>netty-connector</connector-ref>
<static-connectors>
<connector-ref>netty-live-connector</connector-ref>
</static-connectors>
</cluster-connection>
</cluster-connections>
<!-- Other config -->
<security-settings>
<!--security for example queue-->
<security-setting match="#">
<permission type="createNonDurableQueue" roles="amq, guest"/>
<permission type="deleteNonDurableQueue" roles="amq, guest"/>
<permission type="createDurableQueue" roles="amq, guest"/>
<permission type="deleteDurableQueue" roles="amq, guest"/>
<permission type="createAddress" roles="amq, guest"/>
<permission type="deleteAddress" roles="amq, guest"/>
<permission type="consume" roles="amq, guest"/>
<permission type="browse" roles="amq, guest"/>
<permission type="send" roles="amq, guest"/>
<!-- we need this otherwise ./artemis data imp wouldn't work -->
<permission type="manage" roles="amq"/>
</security-setting>
</security-settings>
<address-settings>
<!-- if you define auto-create on certain queues, management has to be auto-create -->
<address-setting match="activemq.management#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
<!--default for catch all-->
<address-setting match="#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>10MB</max-size-bytes>
<page-size-bytes>1MB</page-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
</address-settings>
<addresses>
<address name="exampleTopic">
<multicast>
</multicast>
</address>
<address name="DLQ">
<anycast>
<queue name="DLQ"/>
</anycast>
</address>
<address name="exampleQueue">
<anycast>
<queue name="exampleQueue"/>
</anycast>
</address>
</addresses>
</core>
</configuration>

View File

@ -0,0 +1,142 @@
<?xml version="1.0" encoding="UTF-8" standalone="no"?>
<!--
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.
--><configuration xmlns="urn:activemq" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="urn:activemq /schema/artemis-server.xsd">
<core xmlns="urn:activemq:core">
<bindings-directory>./data/bindings</bindings-directory>
<journal-directory>./data/journal</journal-directory>
<large-messages-directory>./data/largemessages</large-messages-directory>
<paging-directory>./data/paging</paging-directory>
<cluster-user>exampleUser</cluster-user>
<cluster-password>secret</cluster-password>
<journal-retention-directory storage-limit="20M">./data/retention</journal-retention-directory>
<ha-policy>
<replication>
<slave>
<allow-failback>true</allow-failback>
<!-- not needed but tells the backup not to restart after failback as there will be > 0 backups saved -->
<max-saved-replicated-journals-size>0</max-saved-replicated-journals-size>
</slave>
</replication>
</ha-policy>
<!-- Connectors -->
<connectors>
<connector name="netty-live-connector">tcp://mainB:61616</connector>
<connector name="netty-connector">tcp://backupB:61616</connector>
</connectors>
<!-- Acceptors -->
<acceptors>
<acceptor name="netty-acceptor">tcp://0.0.0.0:61616</acceptor>
</acceptors>
<broker-connections>
<amqp-connection uri="tcp://mainA:61616#tcp://backupA:61616" name="serverA" reconnect-attempts="-1" retry-interval="500">
<mirror durable="true"/>
</amqp-connection>
</broker-connections>
<cluster-connections>
<cluster-connection name="my-cluster">
<connector-ref>netty-connector</connector-ref>
<static-connectors>
<connector-ref>netty-live-connector</connector-ref>
</static-connectors>
</cluster-connection>
</cluster-connections>
<!-- Other config -->
<security-settings>
<!--security for example queue-->
<security-setting match="#">
<permission type="createNonDurableQueue" roles="amq, guest"/>
<permission type="deleteNonDurableQueue" roles="amq, guest"/>
<permission type="createDurableQueue" roles="amq, guest"/>
<permission type="deleteDurableQueue" roles="amq, guest"/>
<permission type="createAddress" roles="amq, guest"/>
<permission type="deleteAddress" roles="amq, guest"/>
<permission type="consume" roles="amq, guest"/>
<permission type="browse" roles="amq, guest"/>
<permission type="send" roles="amq, guest"/>
<!-- we need this otherwise ./artemis data imp wouldn't work -->
<permission type="manage" roles="amq"/>
</security-setting>
</security-settings>
<address-settings>
<!-- if you define auto-create on certain queues, management has to be auto-create -->
<address-setting match="activemq.management#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
<!--default for catch all-->
<address-setting match="#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>10MB</max-size-bytes>
<page-size-bytes>1MB</page-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
</address-settings>
<addresses>
<address name="exampleTopic">
<multicast>
</multicast>
</address>
<address name="DLQ">
<anycast>
<queue name="DLQ"/>
</anycast>
</address>
<address name="exampleQueue">
<anycast>
<queue name="exampleQueue"/>
</anycast>
</address>
</addresses>
</core>
</configuration>

View File

@ -0,0 +1,139 @@
<?xml version="1.0" encoding="UTF-8" standalone="no"?>
<!--
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.
--><configuration xmlns="urn:activemq" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="urn:activemq /schema/artemis-server.xsd">
<core xmlns="urn:activemq:core">
<bindings-directory>./data/bindings</bindings-directory>
<journal-directory>./data/journal</journal-directory>
<large-messages-directory>./data/largemessages</large-messages-directory>
<paging-directory>./data/paging</paging-directory>
<journal-retention-directory storage-limit="20M">./data/retention</journal-retention-directory>
<cluster-user>exampleUser</cluster-user>
<cluster-password>secret</cluster-password>
<ha-policy>
<replication>
<master>
<!--we need this for auto failback-->
<check-for-live-server>false</check-for-live-server>
</master>
</replication>
</ha-policy>
<connectors>
<connector name="netty-connector">tcp://mainA:61616</connector>
<connector name="netty-backup-connector">tcp://backupA:61616</connector>
</connectors>
<!-- Acceptors -->
<acceptors>
<acceptor name="netty-acceptor">tcp://0.0.0.0:61616</acceptor>
</acceptors>
<broker-connections>
<amqp-connection uri="tcp://mainB:61616#tcp://backupB:61616" name="serverB" reconnect-attempts="-1" retry-interval="500">
<mirror durable="true"/>
</amqp-connection>
</broker-connections>
<cluster-connections>
<cluster-connection name="my-cluster">
<connector-ref>netty-connector</connector-ref>
<static-connectors>
<connector-ref>netty-backup-connector</connector-ref>
</static-connectors>
</cluster-connection>
</cluster-connections>
<security-settings>
<!--security for example queue-->
<security-setting match="#">
<permission type="createNonDurableQueue" roles="amq, guest"/>
<permission type="deleteNonDurableQueue" roles="amq, guest"/>
<permission type="createDurableQueue" roles="amq, guest"/>
<permission type="deleteDurableQueue" roles="amq, guest"/>
<permission type="createAddress" roles="amq, guest"/>
<permission type="deleteAddress" roles="amq, guest"/>
<permission type="consume" roles="amq, guest"/>
<permission type="browse" roles="amq, guest"/>
<permission type="send" roles="amq, guest"/>
<!-- we need this otherwise ./artemis data imp wouldn't work -->
<permission type="manage" roles="amq"/>
</security-setting>
</security-settings>
<address-settings>
<!-- if you define auto-create on certain queues, management has to be auto-create -->
<address-setting match="activemq.management#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
<!--default for catch all-->
<address-setting match="#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>10MB</max-size-bytes>
<page-size-bytes>1MB</page-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
</address-settings>
<addresses>
<address name="exampleTopic">
<multicast>
</multicast>
</address>
<address name="DLQ">
<anycast>
<queue name="DLQ"/>
</anycast>
</address>
<address name="exampleQueue">
<anycast>
<queue name="exampleQueue"/>
</anycast>
</address>
</addresses>
</core>
</configuration>

View File

@ -0,0 +1,139 @@
<?xml version="1.0" encoding="UTF-8" standalone="no"?>
<!--
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.
--><configuration xmlns="urn:activemq" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="urn:activemq /schema/artemis-server.xsd">
<core xmlns="urn:activemq:core">
<bindings-directory>./data/bindings</bindings-directory>
<journal-directory>./data/journal</journal-directory>
<large-messages-directory>./data/largemessages</large-messages-directory>
<paging-directory>./data/paging</paging-directory>
<journal-retention-directory storage-limit="20M">./data/retention</journal-retention-directory>
<cluster-user>exampleUser</cluster-user>
<cluster-password>secret</cluster-password>
<ha-policy>
<replication>
<master>
<!--we need this for auto failback-->
<check-for-live-server>false</check-for-live-server>
</master>
</replication>
</ha-policy>
<connectors>
<connector name="netty-connector">tcp://mainB:61616</connector>
<connector name="netty-backup-connector">tcp://backupB:61616</connector>
</connectors>
<!-- Acceptors -->
<acceptors>
<acceptor name="netty-acceptor">tcp://0.0.0.0:61616</acceptor>
</acceptors>
<broker-connections>
<amqp-connection uri="tcp://mainA:61616#tcp://backupA:61616" name="serverA" reconnect-attempts="-1" retry-interval="500">
<mirror durable="true"/>
</amqp-connection>
</broker-connections>
<cluster-connections>
<cluster-connection name="my-cluster">
<connector-ref>netty-connector</connector-ref>
<static-connectors>
<connector-ref>netty-backup-connector</connector-ref>
</static-connectors>
</cluster-connection>
</cluster-connections>
<security-settings>
<!--security for example queue-->
<security-setting match="#">
<permission type="createNonDurableQueue" roles="amq, guest"/>
<permission type="deleteNonDurableQueue" roles="amq, guest"/>
<permission type="createDurableQueue" roles="amq, guest"/>
<permission type="deleteDurableQueue" roles="amq, guest"/>
<permission type="createAddress" roles="amq, guest"/>
<permission type="deleteAddress" roles="amq, guest"/>
<permission type="consume" roles="amq, guest"/>
<permission type="browse" roles="amq, guest"/>
<permission type="send" roles="amq, guest"/>
<!-- we need this otherwise ./artemis data imp wouldn't work -->
<permission type="manage" roles="amq"/>
</security-setting>
</security-settings>
<address-settings>
<!-- if you define auto-create on certain queues, management has to be auto-create -->
<address-setting match="activemq.management#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
<!--default for catch all-->
<address-setting match="#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>10MB</max-size-bytes>
<page-size-bytes>1MB</page-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
</address-settings>
<addresses>
<address name="exampleTopic">
<multicast>
</multicast>
</address>
<address name="DLQ">
<anycast>
<queue name="DLQ"/>
</anycast>
</address>
<address name="exampleQueue">
<anycast>
<queue name="exampleQueue"/>
</anycast>
</address>
</addresses>
</core>
</configuration>

View File

@ -0,0 +1,215 @@
<?xml version='1.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.
-->
<configuration xmlns="urn:activemq"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xmlns:xi="http://www.w3.org/2001/XInclude"
xsi:schemaLocation="urn:activemq /schema/artemis-configuration.xsd">
<core xmlns="urn:activemq:core" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="urn:activemq:core ">
<name>ServerA</name>
<persistence-enabled>true</persistence-enabled>
<!-- this could be ASYNCIO, MAPPED, NIO
ASYNCIO: Linux Libaio
MAPPED: mmap files
NIO: Plain Java Files
-->
<journal-type>NIO</journal-type>
<paging-directory>data/paging</paging-directory>
<bindings-directory>data/bindings</bindings-directory>
<journal-directory>data/journal</journal-directory>
<large-messages-directory>data/large-messages</large-messages-directory>
<journal-datasync>true</journal-datasync>
<journal-min-files>2</journal-min-files>
<journal-pool-files>10</journal-pool-files>
<journal-device-block-size>4096</journal-device-block-size>
<journal-file-size>10M</journal-file-size>
<!--
This value was determined through a calculation.
Your system could perform 25 writes per millisecond
on the current journal configuration.
That translates as a sync write every 40000 nanoseconds.
Note: If you specify 0 the system will perform writes directly to the disk.
We recommend this to be 0 if you are using journalType=MAPPED and journal-datasync=false.
-->
<journal-buffer-timeout>40000</journal-buffer-timeout>
<!--
When using ASYNCIO, this will determine the writing queue depth for libaio.
-->
<journal-max-io>1</journal-max-io>
<!--
You can verify the network health of a particular NIC by specifying the <network-check-NIC> element.
<network-check-NIC>theNicName</network-check-NIC>
-->
<!--
Use this to use an HTTP server to validate the network
<network-check-URL-list>http://www.apache.org</network-check-URL-list> -->
<!-- <network-check-period>10000</network-check-period> -->
<!-- <network-check-timeout>1000</network-check-timeout> -->
<!-- this is a comma separated list, no spaces, just DNS or IPs
it should accept IPV6
Warning: Make sure you understand your network topology as this is meant to validate if your network is valid.
Using IPs that could eventually disappear or be partially visible may defeat the purpose.
You can use a list of multiple IPs, and if any successful ping will make the server OK to continue running -->
<!-- <network-check-list>10.0.0.1</network-check-list> -->
<!-- use this to customize the ping used for ipv4 addresses -->
<!-- <network-check-ping-command>ping -c 1 -t %d %s</network-check-ping-command> -->
<!-- use this to customize the ping used for ipv6 addresses -->
<!-- <network-check-ping6-command>ping6 -c 1 %2$s</network-check-ping6-command> -->
<!-- how often we are looking for how many bytes are being used on the disk in ms -->
<disk-scan-period>5000</disk-scan-period>
<!-- once the disk hits this limit the system will block, or close the connection in certain protocols
that won't support flow control. -->
<max-disk-usage>90</max-disk-usage>
<!-- should the broker detect dead locks and other issues -->
<critical-analyzer>true</critical-analyzer>
<critical-analyzer-timeout>120000</critical-analyzer-timeout>
<critical-analyzer-check-period>60000</critical-analyzer-check-period>
<critical-analyzer-policy>HALT</critical-analyzer-policy>
<page-sync-timeout>40000</page-sync-timeout>
<acceptors>
<acceptor name="artemis">tcp://0.0.0.0:61616?tcpSendBufferSize=1048576;tcpReceiveBufferSize=1048576;amqpMinLargeMessageSize=102400;protocols=CORE,AMQP,STOMP,HORNETQ,MQTT,OPENWIRE;useEpoll=true;amqpCredits=1000;amqpLowCredits=300;amqpDuplicateDetection=true</acceptor>
</acceptors>
<broker-connections>
<amqp-connection uri="tcp://serverB:61616" name="another" retry-interval="100" reconnect-attempts="-1" user="artemis" password="artemis">
<mirror durable="true"/>
</amqp-connection>
</broker-connections>
<security-settings>
<security-setting match="#">
<permission type="createNonDurableQueue" roles="amq"/>
<permission type="deleteNonDurableQueue" roles="amq"/>
<permission type="createDurableQueue" roles="amq"/>
<permission type="deleteDurableQueue" roles="amq"/>
<permission type="createAddress" roles="amq"/>
<permission type="deleteAddress" roles="amq"/>
<permission type="consume" roles="amq"/>
<permission type="browse" roles="amq"/>
<permission type="send" roles="amq"/>
<!-- we need this otherwise ./artemis data imp wouldn't work -->
<permission type="manage" roles="amq"/>
</security-setting>
</security-settings>
<address-settings>
<!-- if you define auto-create on certain queues, management has to be auto-create -->
<address-setting match="activemq.management#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
<!--default for catch all-->
<address-setting match="#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
</address-settings>
<addresses>
<address name="DLQ">
<anycast>
<queue name="DLQ" />
</anycast>
</address>
<address name="ExpiryQueue">
<anycast>
<queue name="ExpiryQueue" />
</anycast>
</address>
<address name="someQueue">
<anycast>
<queue name="someQueue" />
</anycast>
</address>
</addresses>
<!-- Uncomment the following if you want to use the Standard LoggingActiveMQServerPlugin pluging to log in events
<broker-plugins>
<broker-plugin class-name="org.apache.activemq.artemis.core.server.plugin.impl.LoggingActiveMQServerPlugin">
<property key="LOG_ALL_EVENTS" value="true"/>
<property key="LOG_CONNECTION_EVENTS" value="true"/>
<property key="LOG_SESSION_EVENTS" value="true"/>
<property key="LOG_CONSUMER_EVENTS" value="true"/>
<property key="LOG_DELIVERING_EVENTS" value="true"/>
<property key="LOG_SENDING_EVENTS" value="true"/>
<property key="LOG_INTERNAL_EVENTS" value="true"/>
</broker-plugin>
</broker-plugins>
-->
</core>
</configuration>

View File

@ -0,0 +1,86 @@
#
# 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.
#
# Additional logger names to configure (root logger is always configured)
# Root logger option
loggers=org.eclipse.jetty,org.jboss.logging,org.apache.activemq.artemis.core.server,org.apache.activemq.artemis.utils,org.apache.activemq.artemis.utils.critical,org.apache.activemq.artemis.journal,org.apache.activemq.artemis.jms.server,org.apache.activemq.artemis.integration.bootstrap,org.apache.activemq.audit.base,org.apache.activemq.audit.message,org.apache.activemq.audit.resource,org.apache.activemq.artemis.utils.pools,org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerTarget,org.apache.activemq.artemis.core.server.impl,org.apache.activemq.artemis.utils.collections,org.apache.activemq.artemis.core.postoffice.impl
# Special logger to debug mirror Security
# Root logger level
logger.level=INFO
# ActiveMQ Artemis logger levels
# These levels are candidates to eventually debug
logger.org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerTarget.level=INFO
# if you have issues with CriticalAnalyzer, setting this as TRACE would give you extra troubleshooting information.
# but do not use it regularly as it would incur in some extra CPU usage for this diagnostic.
logger.org.apache.activemq.artemis.utils.critical.level=INFO
logger.org.eclipse.jetty.level=WARN
# Root logger handlers
logger.handlers=FILE,CONSOLE
# to enable audit change the level to INFO
logger.org.apache.activemq.audit.base.level=ERROR
logger.org.apache.activemq.audit.base.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.base.useParentHandlers=false
logger.org.apache.activemq.audit.resource.level=ERROR
logger.org.apache.activemq.audit.resource.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.resource.useParentHandlers=false
logger.org.apache.activemq.audit.message.level=ERROR
logger.org.apache.activemq.audit.message.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.message.useParentHandlers=false
# Console handler configuration
handler.CONSOLE=org.jboss.logmanager.handlers.ConsoleHandler
handler.CONSOLE.properties=autoFlush
handler.CONSOLE.level=TRACE
handler.CONSOLE.autoFlush=true
handler.CONSOLE.formatter=PATTERN
# File handler configuration
handler.FILE=org.jboss.logmanager.handlers.PeriodicRotatingFileHandler
handler.FILE.level=TRACE
handler.FILE.properties=suffix,append,autoFlush,fileName
handler.FILE.suffix=.yyyy-MM-dd
handler.FILE.append=true
handler.FILE.autoFlush=true
handler.FILE.fileName=${artemis.instance}/log/artemis.log
handler.FILE.formatter=PATTERN
# Formatter pattern configuration
formatter.PATTERN=org.jboss.logmanager.formatters.PatternFormatter
formatter.PATTERN.properties=pattern
formatter.PATTERN.pattern=[%t] %d %-5p [%c] %s%E%n
#Audit logger
handler.AUDIT_FILE=org.jboss.logmanager.handlers.PeriodicRotatingFileHandler
handler.AUDIT_FILE.level=INFO
handler.AUDIT_FILE.properties=suffix,append,autoFlush,fileName
handler.AUDIT_FILE.suffix=.yyyy-MM-dd
handler.AUDIT_FILE.append=true
handler.AUDIT_FILE.autoFlush=true
handler.AUDIT_FILE.fileName=${artemis.instance}/log/audit.log
handler.AUDIT_FILE.formatter=AUDIT_PATTERN
formatter.AUDIT_PATTERN=org.jboss.logmanager.formatters.PatternFormatter
formatter.AUDIT_PATTERN.properties=pattern
formatter.AUDIT_PATTERN.pattern=%d [AUDIT](%t) %s%E%n

View File

@ -0,0 +1,215 @@
<?xml version='1.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.
-->
<configuration xmlns="urn:activemq"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xmlns:xi="http://www.w3.org/2001/XInclude"
xsi:schemaLocation="urn:activemq /schema/artemis-configuration.xsd">
<core xmlns="urn:activemq:core" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="urn:activemq:core ">
<name>ServerB</name>
<persistence-enabled>true</persistence-enabled>
<!-- this could be ASYNCIO, MAPPED, NIO
ASYNCIO: Linux Libaio
MAPPED: mmap files
NIO: Plain Java Files
-->
<journal-type>NIO</journal-type>
<paging-directory>data/paging</paging-directory>
<bindings-directory>data/bindings</bindings-directory>
<journal-directory>data/journal</journal-directory>
<large-messages-directory>data/large-messages</large-messages-directory>
<journal-datasync>true</journal-datasync>
<journal-min-files>2</journal-min-files>
<journal-pool-files>10</journal-pool-files>
<journal-device-block-size>4096</journal-device-block-size>
<journal-file-size>10M</journal-file-size>
<!--
This value was determined through a calculation.
Your system could perform 25 writes per millisecond
on the current journal configuration.
That translates as a sync write every 40000 nanoseconds.
Note: If you specify 0 the system will perform writes directly to the disk.
We recommend this to be 0 if you are using journalType=MAPPED and journal-datasync=false.
-->
<journal-buffer-timeout>40000</journal-buffer-timeout>
<!--
When using ASYNCIO, this will determine the writing queue depth for libaio.
-->
<journal-max-io>1</journal-max-io>
<!--
You can verify the network health of a particular NIC by specifying the <network-check-NIC> element.
<network-check-NIC>theNicName</network-check-NIC>
-->
<!--
Use this to use an HTTP server to validate the network
<network-check-URL-list>http://www.apache.org</network-check-URL-list> -->
<!-- <network-check-period>10000</network-check-period> -->
<!-- <network-check-timeout>1000</network-check-timeout> -->
<!-- this is a comma separated list, no spaces, just DNS or IPs
it should accept IPV6
Warning: Make sure you understand your network topology as this is meant to validate if your network is valid.
Using IPs that could eventually disappear or be partially visible may defeat the purpose.
You can use a list of multiple IPs, and if any successful ping will make the server OK to continue running -->
<!-- <network-check-list>10.0.0.1</network-check-list> -->
<!-- use this to customize the ping used for ipv4 addresses -->
<!-- <network-check-ping-command>ping -c 1 -t %d %s</network-check-ping-command> -->
<!-- use this to customize the ping used for ipv6 addresses -->
<!-- <network-check-ping6-command>ping6 -c 1 %2$s</network-check-ping6-command> -->
<!-- how often we are looking for how many bytes are being used on the disk in ms -->
<disk-scan-period>5000</disk-scan-period>
<!-- once the disk hits this limit the system will block, or close the connection in certain protocols
that won't support flow control. -->
<max-disk-usage>90</max-disk-usage>
<!-- should the broker detect dead locks and other issues -->
<critical-analyzer>true</critical-analyzer>
<critical-analyzer-timeout>120000</critical-analyzer-timeout>
<critical-analyzer-check-period>60000</critical-analyzer-check-period>
<critical-analyzer-policy>HALT</critical-analyzer-policy>
<page-sync-timeout>40000</page-sync-timeout>
<acceptors>
<acceptor name="artemis">tcp://0.0.0.0:61616?tcpSendBufferSize=1048576;tcpReceiveBufferSize=1048576;amqpMinLargeMessageSize=102400;protocols=CORE,AMQP,STOMP,HORNETQ,MQTT,OPENWIRE;useEpoll=true;amqpCredits=1000;amqpLowCredits=300;amqpDuplicateDetection=true</acceptor>
</acceptors>
<broker-connections>
<amqp-connection uri="tcp://serverA:61616" name="serverA" user="artemis" password="artemis" retry-interval="100" reconnect-attempts="-1">
<mirror durable="true"/>
</amqp-connection>
</broker-connections>
<security-settings>
<security-setting match="#">
<permission type="createNonDurableQueue" roles="amq"/>
<permission type="deleteNonDurableQueue" roles="amq"/>
<permission type="createDurableQueue" roles="amq"/>
<permission type="deleteDurableQueue" roles="amq"/>
<permission type="createAddress" roles="amq"/>
<permission type="deleteAddress" roles="amq"/>
<permission type="consume" roles="amq"/>
<permission type="browse" roles="amq"/>
<permission type="send" roles="amq"/>
<!-- we need this otherwise ./artemis data imp wouldn't work -->
<permission type="manage" roles="amq"/>
</security-setting>
</security-settings>
<address-settings>
<!-- if you define auto-create on certain queues, management has to be auto-create -->
<address-setting match="activemq.management#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
<!--default for catch all-->
<address-setting match="#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
</address-settings>
<addresses>
<address name="DLQ">
<anycast>
<queue name="DLQ" />
</anycast>
</address>
<address name="ExpiryQueue">
<anycast>
<queue name="ExpiryQueue" />
</anycast>
</address>
<address name="someQueue">
<anycast>
<queue name="someQueue" />
</anycast>
</address>
</addresses>
<!-- Uncomment the following if you want to use the Standard LoggingActiveMQServerPlugin pluging to log in events
<broker-plugins>
<broker-plugin class-name="org.apache.activemq.artemis.core.server.plugin.impl.LoggingActiveMQServerPlugin">
<property key="LOG_ALL_EVENTS" value="true"/>
<property key="LOG_CONNECTION_EVENTS" value="true"/>
<property key="LOG_SESSION_EVENTS" value="true"/>
<property key="LOG_CONSUMER_EVENTS" value="true"/>
<property key="LOG_DELIVERING_EVENTS" value="true"/>
<property key="LOG_SENDING_EVENTS" value="true"/>
<property key="LOG_INTERNAL_EVENTS" value="true"/>
</broker-plugin>
</broker-plugins>
-->
</core>
</configuration>

View File

@ -0,0 +1,86 @@
#
# 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.
#
# Additional logger names to configure (root logger is always configured)
# Root logger option
loggers=org.eclipse.jetty,org.jboss.logging,org.apache.activemq.artemis.core.server,org.apache.activemq.artemis.utils,org.apache.activemq.artemis.utils.critical,org.apache.activemq.artemis.journal,org.apache.activemq.artemis.jms.server,org.apache.activemq.artemis.integration.bootstrap,org.apache.activemq.audit.base,org.apache.activemq.audit.message,org.apache.activemq.audit.resource,org.apache.activemq.artemis.utils.pools,org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerTarget,org.apache.activemq.artemis.core.server.impl,org.apache.activemq.artemis.utils.collections,org.apache.activemq.artemis.core.postoffice.impl
# Special logger to debug mirror Security
# Root logger level
logger.level=INFO
# ActiveMQ Artemis logger levels
# These levels are candidates to eventually debug
logger.org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerTarget.level=INFO
# if you have issues with CriticalAnalyzer, setting this as TRACE would give you extra troubleshooting information.
# but do not use it regularly as it would incur in some extra CPU usage for this diagnostic.
logger.org.apache.activemq.artemis.utils.critical.level=INFO
logger.org.eclipse.jetty.level=WARN
# Root logger handlers
logger.handlers=FILE,CONSOLE
# to enable audit change the level to INFO
logger.org.apache.activemq.audit.base.level=ERROR
logger.org.apache.activemq.audit.base.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.base.useParentHandlers=false
logger.org.apache.activemq.audit.resource.level=ERROR
logger.org.apache.activemq.audit.resource.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.resource.useParentHandlers=false
logger.org.apache.activemq.audit.message.level=ERROR
logger.org.apache.activemq.audit.message.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.message.useParentHandlers=false
# Console handler configuration
handler.CONSOLE=org.jboss.logmanager.handlers.ConsoleHandler
handler.CONSOLE.properties=autoFlush
handler.CONSOLE.level=TRACE
handler.CONSOLE.autoFlush=true
handler.CONSOLE.formatter=PATTERN
# File handler configuration
handler.FILE=org.jboss.logmanager.handlers.PeriodicRotatingFileHandler
handler.FILE.level=TRACE
handler.FILE.properties=suffix,append,autoFlush,fileName
handler.FILE.suffix=.yyyy-MM-dd
handler.FILE.append=true
handler.FILE.autoFlush=true
handler.FILE.fileName=${artemis.instance}/log/artemis.log
handler.FILE.formatter=PATTERN
# Formatter pattern configuration
formatter.PATTERN=org.jboss.logmanager.formatters.PatternFormatter
formatter.PATTERN.properties=pattern
formatter.PATTERN.pattern=[%t] %d %-5p [%c] %s%E%n
#Audit logger
handler.AUDIT_FILE=org.jboss.logmanager.handlers.PeriodicRotatingFileHandler
handler.AUDIT_FILE.level=INFO
handler.AUDIT_FILE.properties=suffix,append,autoFlush,fileName
handler.AUDIT_FILE.suffix=.yyyy-MM-dd
handler.AUDIT_FILE.append=true
handler.AUDIT_FILE.autoFlush=true
handler.AUDIT_FILE.fileName=${artemis.instance}/log/audit.log
handler.AUDIT_FILE.formatter=AUDIT_PATTERN
formatter.AUDIT_PATTERN=org.jboss.logmanager.formatters.PatternFormatter
formatter.AUDIT_PATTERN.properties=pattern
formatter.AUDIT_PATTERN.pattern=%d [AUDIT](%t) %s%E%n

View File

@ -0,0 +1,210 @@
<?xml version='1.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.
-->
<configuration xmlns="urn:activemq"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xmlns:xi="http://www.w3.org/2001/XInclude"
xsi:schemaLocation="urn:activemq /schema/artemis-configuration.xsd">
<core xmlns="urn:activemq:core" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="urn:activemq:core ">
<name>ServerA</name>
<persistence-enabled>true</persistence-enabled>
<security-enabled>false</security-enabled>
<!-- this could be ASYNCIO, MAPPED, NIO
ASYNCIO: Linux Libaio
MAPPED: mmap files
NIO: Plain Java Files
-->
<journal-type>NIO</journal-type>
<paging-directory>data/paging</paging-directory>
<bindings-directory>data/bindings</bindings-directory>
<journal-directory>data/journal</journal-directory>
<large-messages-directory>data/large-messages</large-messages-directory>
<journal-datasync>true</journal-datasync>
<journal-min-files>2</journal-min-files>
<journal-pool-files>10</journal-pool-files>
<journal-device-block-size>4096</journal-device-block-size>
<journal-file-size>10M</journal-file-size>
<!--
This value was determined through a calculation.
Your system could perform 25 writes per millisecond
on the current journal configuration.
That translates as a sync write every 40000 nanoseconds.
Note: If you specify 0 the system will perform writes directly to the disk.
We recommend this to be 0 if you are using journalType=MAPPED and journal-datasync=false.
-->
<journal-buffer-timeout>40000</journal-buffer-timeout>
<!--
When using ASYNCIO, this will determine the writing queue depth for libaio.
-->
<journal-max-io>1</journal-max-io>
<!--
You can verify the network health of a particular NIC by specifying the <network-check-NIC> element.
<network-check-NIC>theNicName</network-check-NIC>
-->
<!--
Use this to use an HTTP server to validate the network
<network-check-URL-list>http://www.apache.org</network-check-URL-list> -->
<!-- <network-check-period>10000</network-check-period> -->
<!-- <network-check-timeout>1000</network-check-timeout> -->
<!-- this is a comma separated list, no spaces, just DNS or IPs
it should accept IPV6
Warning: Make sure you understand your network topology as this is meant to validate if your network is valid.
Using IPs that could eventually disappear or be partially visible may defeat the purpose.
You can use a list of multiple IPs, and if any successful ping will make the server OK to continue running -->
<!-- <network-check-list>10.0.0.1</network-check-list> -->
<!-- use this to customize the ping used for ipv4 addresses -->
<!-- <network-check-ping-command>ping -c 1 -t %d %s</network-check-ping-command> -->
<!-- use this to customize the ping used for ipv6 addresses -->
<!-- <network-check-ping6-command>ping6 -c 1 %2$s</network-check-ping6-command> -->
<!-- how often we are looking for how many bytes are being used on the disk in ms -->
<disk-scan-period>5000</disk-scan-period>
<!-- once the disk hits this limit the system will block, or close the connection in certain protocols
that won't support flow control. -->
<max-disk-usage>90</max-disk-usage>
<!-- should the broker detect dead locks and other issues -->
<critical-analyzer>true</critical-analyzer>
<critical-analyzer-timeout>120000</critical-analyzer-timeout>
<critical-analyzer-check-period>60000</critical-analyzer-check-period>
<critical-analyzer-policy>HALT</critical-analyzer-policy>
<page-sync-timeout>40000</page-sync-timeout>
<acceptors>
<acceptor name="artemis">tcp://0.0.0.0:61616?tcpSendBufferSize=1048576;tcpReceiveBufferSize=1048576;amqpMinLargeMessageSize=102400;protocols=CORE,AMQP,STOMP,HORNETQ,MQTT,OPENWIRE;useEpoll=true;amqpCredits=1000;amqpLowCredits=300;amqpDuplicateDetection=true</acceptor>
</acceptors>
<security-settings>
<security-setting match="#">
<permission type="createNonDurableQueue" roles="amq"/>
<permission type="deleteNonDurableQueue" roles="amq"/>
<permission type="createDurableQueue" roles="amq"/>
<permission type="deleteDurableQueue" roles="amq"/>
<permission type="createAddress" roles="amq"/>
<permission type="deleteAddress" roles="amq"/>
<permission type="consume" roles="amq"/>
<permission type="browse" roles="amq"/>
<permission type="send" roles="amq"/>
<!-- we need this otherwise ./artemis data imp wouldn't work -->
<permission type="manage" roles="amq"/>
</security-setting>
</security-settings>
<address-settings>
<!-- if you define auto-create on certain queues, management has to be auto-create -->
<address-setting match="activemq.management#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
<!--default for catch all-->
<address-setting match="#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
</address-settings>
<addresses>
<address name="DLQ">
<anycast>
<queue name="DLQ" />
</anycast>
</address>
<address name="ExpiryQueue">
<anycast>
<queue name="ExpiryQueue" />
</anycast>
</address>
<address name="someQueue">
<anycast>
<queue name="someQueue" />
</anycast>
</address>
</addresses>
<!-- Uncomment the following if you want to use the Standard LoggingActiveMQServerPlugin pluging to log in events
<broker-plugins>
<broker-plugin class-name="org.apache.activemq.artemis.core.server.plugin.impl.LoggingActiveMQServerPlugin">
<property key="LOG_ALL_EVENTS" value="true"/>
<property key="LOG_CONNECTION_EVENTS" value="true"/>
<property key="LOG_SESSION_EVENTS" value="true"/>
<property key="LOG_CONSUMER_EVENTS" value="true"/>
<property key="LOG_DELIVERING_EVENTS" value="true"/>
<property key="LOG_SENDING_EVENTS" value="true"/>
<property key="LOG_INTERNAL_EVENTS" value="true"/>
</broker-plugin>
</broker-plugins>
-->
</core>
</configuration>

View File

@ -0,0 +1,86 @@
#
# 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.
#
# Additional logger names to configure (root logger is always configured)
# Root logger option
loggers=org.eclipse.jetty,org.jboss.logging,org.apache.activemq.artemis.core.server,org.apache.activemq.artemis.utils,org.apache.activemq.artemis.utils.critical,org.apache.activemq.artemis.journal,org.apache.activemq.artemis.jms.server,org.apache.activemq.artemis.integration.bootstrap,org.apache.activemq.audit.base,org.apache.activemq.audit.message,org.apache.activemq.audit.resource,org.apache.activemq.artemis.utils.pools,org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerTarget,org.apache.activemq.artemis.core.server.impl,org.apache.activemq.artemis.utils.collections,org.apache.activemq.artemis.core.postoffice.impl
# Special logger to debug mirror Security
# Root logger level
logger.level=INFO
# ActiveMQ Artemis logger levels
# These levels are candidates to eventually debug
logger.org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerTarget.level=INFO
# if you have issues with CriticalAnalyzer, setting this as TRACE would give you extra troubleshooting information.
# but do not use it regularly as it would incur in some extra CPU usage for this diagnostic.
logger.org.apache.activemq.artemis.utils.critical.level=INFO
logger.org.eclipse.jetty.level=WARN
# Root logger handlers
logger.handlers=FILE,CONSOLE
# to enable audit change the level to INFO
logger.org.apache.activemq.audit.base.level=ERROR
logger.org.apache.activemq.audit.base.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.base.useParentHandlers=false
logger.org.apache.activemq.audit.resource.level=ERROR
logger.org.apache.activemq.audit.resource.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.resource.useParentHandlers=false
logger.org.apache.activemq.audit.message.level=ERROR
logger.org.apache.activemq.audit.message.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.message.useParentHandlers=false
# Console handler configuration
handler.CONSOLE=org.jboss.logmanager.handlers.ConsoleHandler
handler.CONSOLE.properties=autoFlush
handler.CONSOLE.level=TRACE
handler.CONSOLE.autoFlush=true
handler.CONSOLE.formatter=PATTERN
# File handler configuration
handler.FILE=org.jboss.logmanager.handlers.PeriodicRotatingFileHandler
handler.FILE.level=TRACE
handler.FILE.properties=suffix,append,autoFlush,fileName
handler.FILE.suffix=.yyyy-MM-dd
handler.FILE.append=true
handler.FILE.autoFlush=true
handler.FILE.fileName=${artemis.instance}/log/artemis.log
handler.FILE.formatter=PATTERN
# Formatter pattern configuration
formatter.PATTERN=org.jboss.logmanager.formatters.PatternFormatter
formatter.PATTERN.properties=pattern
formatter.PATTERN.pattern=[%t] %d %-5p [%c] %s%E%n
#Audit logger
handler.AUDIT_FILE=org.jboss.logmanager.handlers.PeriodicRotatingFileHandler
handler.AUDIT_FILE.level=INFO
handler.AUDIT_FILE.properties=suffix,append,autoFlush,fileName
handler.AUDIT_FILE.suffix=.yyyy-MM-dd
handler.AUDIT_FILE.append=true
handler.AUDIT_FILE.autoFlush=true
handler.AUDIT_FILE.fileName=${artemis.instance}/log/audit.log
handler.AUDIT_FILE.formatter=AUDIT_PATTERN
formatter.AUDIT_PATTERN=org.jboss.logmanager.formatters.PatternFormatter
formatter.AUDIT_PATTERN.properties=pattern
formatter.AUDIT_PATTERN.pattern=%d [AUDIT](%t) %s%E%n

View File

@ -0,0 +1,208 @@
<?xml version='1.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.
-->
<configuration xmlns="urn:activemq"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xmlns:xi="http://www.w3.org/2001/XInclude"
xsi:schemaLocation="urn:activemq /schema/artemis-configuration.xsd">
<core xmlns="urn:activemq:core" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="urn:activemq:core ">
<name>ServerB</name>
<security-enabled>false</security-enabled>
<persistence-enabled>true</persistence-enabled>
<!-- this could be ASYNCIO, MAPPED, NIO
ASYNCIO: Linux Libaio
MAPPED: mmap files
NIO: Plain Java Files
-->
<journal-type>NIO</journal-type>
<paging-directory>data/paging</paging-directory>
<bindings-directory>data/bindings</bindings-directory>
<journal-directory>data/journal</journal-directory>
<large-messages-directory>data/large-messages</large-messages-directory>
<journal-datasync>true</journal-datasync>
<journal-min-files>2</journal-min-files>
<journal-pool-files>10</journal-pool-files>
<journal-device-block-size>4096</journal-device-block-size>
<journal-file-size>10M</journal-file-size>
<!--
This value was determined through a calculation.
Your system could perform 25 writes per millisecond
on the current journal configuration.
That translates as a sync write every 40000 nanoseconds.
Note: If you specify 0 the system will perform writes directly to the disk.
We recommend this to be 0 if you are using journalType=MAPPED and journal-datasync=false.
-->
<journal-buffer-timeout>40000</journal-buffer-timeout>
<!--
When using ASYNCIO, this will determine the writing queue depth for libaio.
-->
<journal-max-io>1</journal-max-io>
<!--
You can verify the network health of a particular NIC by specifying the <network-check-NIC> element.
<network-check-NIC>theNicName</network-check-NIC>
-->
<!--
Use this to use an HTTP server to validate the network
<network-check-URL-list>http://www.apache.org</network-check-URL-list> -->
<!-- <network-check-period>10000</network-check-period> -->
<!-- <network-check-timeout>1000</network-check-timeout> -->
<!-- this is a comma separated list, no spaces, just DNS or IPs
it should accept IPV6
Warning: Make sure you understand your network topology as this is meant to validate if your network is valid.
Using IPs that could eventually disappear or be partially visible may defeat the purpose.
You can use a list of multiple IPs, and if any successful ping will make the server OK to continue running -->
<!-- <network-check-list>10.0.0.1</network-check-list> -->
<!-- use this to customize the ping used for ipv4 addresses -->
<!-- <network-check-ping-command>ping -c 1 -t %d %s</network-check-ping-command> -->
<!-- use this to customize the ping used for ipv6 addresses -->
<!-- <network-check-ping6-command>ping6 -c 1 %2$s</network-check-ping6-command> -->
<!-- how often we are looking for how many bytes are being used on the disk in ms -->
<disk-scan-period>5000</disk-scan-period>
<!-- once the disk hits this limit the system will block, or close the connection in certain protocols
that won't support flow control. -->
<max-disk-usage>90</max-disk-usage>
<!-- should the broker detect dead locks and other issues -->
<critical-analyzer>true</critical-analyzer>
<critical-analyzer-timeout>120000</critical-analyzer-timeout>
<critical-analyzer-check-period>60000</critical-analyzer-check-period>
<critical-analyzer-policy>HALT</critical-analyzer-policy>
<page-sync-timeout>40000</page-sync-timeout>
<acceptors>
<acceptor name="artemis">tcp://0.0.0.0:61616?tcpSendBufferSize=1048576;tcpReceiveBufferSize=1048576;amqpMinLargeMessageSize=102400;protocols=CORE,AMQP,STOMP,HORNETQ,MQTT,OPENWIRE;useEpoll=true;amqpCredits=1000;amqpLowCredits=300;amqpDuplicateDetection=true</acceptor>
</acceptors>
<security-settings>
<security-setting match="#">
<permission type="createNonDurableQueue" roles="amq"/>
<permission type="deleteNonDurableQueue" roles="amq"/>
<permission type="createDurableQueue" roles="amq"/>
<permission type="deleteDurableQueue" roles="amq"/>
<permission type="createAddress" roles="amq"/>
<permission type="deleteAddress" roles="amq"/>
<permission type="consume" roles="amq"/>
<permission type="browse" roles="amq"/>
<permission type="send" roles="amq"/>
<!-- we need this otherwise ./artemis data imp wouldn't work -->
<permission type="manage" roles="amq"/>
</security-setting>
</security-settings>
<address-settings>
<!-- if you define auto-create on certain queues, management has to be auto-create -->
<address-setting match="activemq.management#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
<!--default for catch all-->
<address-setting match="#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
</address-settings>
<addresses>
<address name="DLQ">
<anycast>
<queue name="DLQ" />
</anycast>
</address>
<address name="ExpiryQueue">
<anycast>
<queue name="ExpiryQueue" />
</anycast>
</address>
<address name="someQueue">
<anycast>
<queue name="someQueue" />
</anycast>
</address>
</addresses>
<!-- Uncomment the following if you want to use the Standard LoggingActiveMQServerPlugin pluging to log in events
<broker-plugins>
<broker-plugin class-name="org.apache.activemq.artemis.core.server.plugin.impl.LoggingActiveMQServerPlugin">
<property key="LOG_ALL_EVENTS" value="true"/>
<property key="LOG_CONNECTION_EVENTS" value="true"/>
<property key="LOG_SESSION_EVENTS" value="true"/>
<property key="LOG_CONSUMER_EVENTS" value="true"/>
<property key="LOG_DELIVERING_EVENTS" value="true"/>
<property key="LOG_SENDING_EVENTS" value="true"/>
<property key="LOG_INTERNAL_EVENTS" value="true"/>
</broker-plugin>
</broker-plugins>
-->
</core>
</configuration>

View File

@ -0,0 +1,86 @@
#
# 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.
#
# Additional logger names to configure (root logger is always configured)
# Root logger option
loggers=org.eclipse.jetty,org.jboss.logging,org.apache.activemq.artemis.core.server,org.apache.activemq.artemis.utils,org.apache.activemq.artemis.utils.critical,org.apache.activemq.artemis.journal,org.apache.activemq.artemis.jms.server,org.apache.activemq.artemis.integration.bootstrap,org.apache.activemq.audit.base,org.apache.activemq.audit.message,org.apache.activemq.audit.resource,org.apache.activemq.artemis.utils.pools,org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerTarget,org.apache.activemq.artemis.core.server.impl,org.apache.activemq.artemis.utils.collections,org.apache.activemq.artemis.core.postoffice.impl
# Special logger to debug mirror Security
# Root logger level
logger.level=INFO
# ActiveMQ Artemis logger levels
# These levels are candidates to eventually debug
logger.org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerTarget.level=INFO
# if you have issues with CriticalAnalyzer, setting this as TRACE would give you extra troubleshooting information.
# but do not use it regularly as it would incur in some extra CPU usage for this diagnostic.
logger.org.apache.activemq.artemis.utils.critical.level=INFO
logger.org.eclipse.jetty.level=WARN
# Root logger handlers
logger.handlers=FILE,CONSOLE
# to enable audit change the level to INFO
logger.org.apache.activemq.audit.base.level=ERROR
logger.org.apache.activemq.audit.base.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.base.useParentHandlers=false
logger.org.apache.activemq.audit.resource.level=ERROR
logger.org.apache.activemq.audit.resource.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.resource.useParentHandlers=false
logger.org.apache.activemq.audit.message.level=ERROR
logger.org.apache.activemq.audit.message.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.message.useParentHandlers=false
# Console handler configuration
handler.CONSOLE=org.jboss.logmanager.handlers.ConsoleHandler
handler.CONSOLE.properties=autoFlush
handler.CONSOLE.level=TRACE
handler.CONSOLE.autoFlush=true
handler.CONSOLE.formatter=PATTERN
# File handler configuration
handler.FILE=org.jboss.logmanager.handlers.PeriodicRotatingFileHandler
handler.FILE.level=TRACE
handler.FILE.properties=suffix,append,autoFlush,fileName
handler.FILE.suffix=.yyyy-MM-dd
handler.FILE.append=true
handler.FILE.autoFlush=true
handler.FILE.fileName=${artemis.instance}/log/artemis.log
handler.FILE.formatter=PATTERN
# Formatter pattern configuration
formatter.PATTERN=org.jboss.logmanager.formatters.PatternFormatter
formatter.PATTERN.properties=pattern
formatter.PATTERN.pattern=[%t] %d %-5p [%c] %s%E%n
#Audit logger
handler.AUDIT_FILE=org.jboss.logmanager.handlers.PeriodicRotatingFileHandler
handler.AUDIT_FILE.level=INFO
handler.AUDIT_FILE.properties=suffix,append,autoFlush,fileName
handler.AUDIT_FILE.suffix=.yyyy-MM-dd
handler.AUDIT_FILE.append=true
handler.AUDIT_FILE.autoFlush=true
handler.AUDIT_FILE.fileName=${artemis.instance}/log/audit.log
handler.AUDIT_FILE.formatter=AUDIT_PATTERN
formatter.AUDIT_PATTERN=org.jboss.logmanager.formatters.PatternFormatter
formatter.AUDIT_PATTERN.properties=pattern
formatter.AUDIT_PATTERN.pattern=%d [AUDIT](%t) %s%E%n

View File

@ -0,0 +1,217 @@
<?xml version='1.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.
-->
<configuration xmlns="urn:activemq"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xmlns:xi="http://www.w3.org/2001/XInclude"
xsi:schemaLocation="urn:activemq /schema/artemis-configuration.xsd">
<core xmlns="urn:activemq:core" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="urn:activemq:core ">
<name>ServerRoot</name>
<security-enabled>false</security-enabled>
<persistence-enabled>true</persistence-enabled>
<!-- this could be ASYNCIO, MAPPED, NIO
ASYNCIO: Linux Libaio
MAPPED: mmap files
NIO: Plain Java Files
-->
<journal-type>NIO</journal-type>
<paging-directory>data/paging</paging-directory>
<bindings-directory>data/bindings</bindings-directory>
<journal-directory>data/journal</journal-directory>
<large-messages-directory>data/large-messages</large-messages-directory>
<journal-datasync>true</journal-datasync>
<journal-min-files>2</journal-min-files>
<journal-pool-files>10</journal-pool-files>
<journal-device-block-size>4096</journal-device-block-size>
<journal-file-size>10M</journal-file-size>
<!--
This value was determined through a calculation.
Your system could perform 25 writes per millisecond
on the current journal configuration.
That translates as a sync write every 40000 nanoseconds.
Note: If you specify 0 the system will perform writes directly to the disk.
We recommend this to be 0 if you are using journalType=MAPPED and journal-datasync=false.
-->
<journal-buffer-timeout>40000</journal-buffer-timeout>
<!--
When using ASYNCIO, this will determine the writing queue depth for libaio.
-->
<journal-max-io>1</journal-max-io>
<!--
You can verify the network health of a particular NIC by specifying the <network-check-NIC> element.
<network-check-NIC>theNicName</network-check-NIC>
-->
<!--
Use this to use an HTTP server to validate the network
<network-check-URL-list>http://www.apache.org</network-check-URL-list> -->
<!-- <network-check-period>10000</network-check-period> -->
<!-- <network-check-timeout>1000</network-check-timeout> -->
<!-- this is a comma separated list, no spaces, just DNS or IPs
it should accept IPV6
Warning: Make sure you understand your network topology as this is meant to validate if your network is valid.
Using IPs that could eventually disappear or be partially visible may defeat the purpose.
You can use a list of multiple IPs, and if any successful ping will make the server OK to continue running -->
<!-- <network-check-list>10.0.0.1</network-check-list> -->
<!-- use this to customize the ping used for ipv4 addresses -->
<!-- <network-check-ping-command>ping -c 1 -t %d %s</network-check-ping-command> -->
<!-- use this to customize the ping used for ipv6 addresses -->
<!-- <network-check-ping6-command>ping6 -c 1 %2$s</network-check-ping6-command> -->
<!-- how often we are looking for how many bytes are being used on the disk in ms -->
<disk-scan-period>5000</disk-scan-period>
<!-- once the disk hits this limit the system will block, or close the connection in certain protocols
that won't support flow control. -->
<max-disk-usage>90</max-disk-usage>
<!-- should the broker detect dead locks and other issues -->
<critical-analyzer>true</critical-analyzer>
<critical-analyzer-timeout>120000</critical-analyzer-timeout>
<critical-analyzer-check-period>60000</critical-analyzer-check-period>
<critical-analyzer-policy>HALT</critical-analyzer-policy>
<page-sync-timeout>40000</page-sync-timeout>
<acceptors>
<acceptor name="artemis">tcp://0.0.0.0:61616?tcpSendBufferSize=1048576;tcpReceiveBufferSize=1048576;amqpMinLargeMessageSize=102400;protocols=CORE,AMQP,STOMP,HORNETQ,MQTT,OPENWIRE;useEpoll=true;amqpCredits=1000;amqpLowCredits=300;amqpDuplicateDetection=true</acceptor>
</acceptors>
<broker-connections>
<amqp-connection uri="tcp://artemisA:61616" name="serverA">
<mirror/>
</amqp-connection>
<amqp-connection uri="tcp://artemisB:61616" name="serverB">
<mirror/>
</amqp-connection>
</broker-connections>
<security-settings>
<security-setting match="#">
<permission type="createNonDurableQueue" roles="amq"/>
<permission type="deleteNonDurableQueue" roles="amq"/>
<permission type="createDurableQueue" roles="amq"/>
<permission type="deleteDurableQueue" roles="amq"/>
<permission type="createAddress" roles="amq"/>
<permission type="deleteAddress" roles="amq"/>
<permission type="consume" roles="amq"/>
<permission type="browse" roles="amq"/>
<permission type="send" roles="amq"/>
<!-- we need this otherwise ./artemis data imp wouldn't work -->
<permission type="manage" roles="amq"/>
</security-setting>
</security-settings>
<address-settings>
<!-- if you define auto-create on certain queues, management has to be auto-create -->
<address-setting match="activemq.management#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
<!--default for catch all-->
<address-setting match="#">
<dead-letter-address>DLQ</dead-letter-address>
<expiry-address>ExpiryQueue</expiry-address>
<redelivery-delay>0</redelivery-delay>
<!-- with -1 only the global-max-size is in use for limiting -->
<max-size-bytes>-1</max-size-bytes>
<message-counter-history-day-limit>10</message-counter-history-day-limit>
<address-full-policy>PAGE</address-full-policy>
<auto-create-queues>true</auto-create-queues>
<auto-create-addresses>true</auto-create-addresses>
<auto-create-jms-queues>true</auto-create-jms-queues>
<auto-create-jms-topics>true</auto-create-jms-topics>
</address-setting>
</address-settings>
<addresses>
<address name="DLQ">
<anycast>
<queue name="DLQ" />
</anycast>
</address>
<address name="ExpiryQueue">
<anycast>
<queue name="ExpiryQueue" />
</anycast>
</address>
<address name="someQueue">
<anycast>
<queue name="someQueue" />
</anycast>
</address>
</addresses>
<!-- Uncomment the following if you want to use the Standard LoggingActiveMQServerPlugin pluging to log in events
<broker-plugins>
<broker-plugin class-name="org.apache.activemq.artemis.core.server.plugin.impl.LoggingActiveMQServerPlugin">
<property key="LOG_ALL_EVENTS" value="true"/>
<property key="LOG_CONNECTION_EVENTS" value="true"/>
<property key="LOG_SESSION_EVENTS" value="true"/>
<property key="LOG_CONSUMER_EVENTS" value="true"/>
<property key="LOG_DELIVERING_EVENTS" value="true"/>
<property key="LOG_SENDING_EVENTS" value="true"/>
<property key="LOG_INTERNAL_EVENTS" value="true"/>
</broker-plugin>
</broker-plugins>
-->
</core>
</configuration>

View File

@ -0,0 +1,86 @@
#
# 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.
#
# Additional logger names to configure (root logger is always configured)
# Root logger option
loggers=org.eclipse.jetty,org.jboss.logging,org.apache.activemq.artemis.core.server,org.apache.activemq.artemis.utils,org.apache.activemq.artemis.utils.critical,org.apache.activemq.artemis.journal,org.apache.activemq.artemis.jms.server,org.apache.activemq.artemis.integration.bootstrap,org.apache.activemq.audit.base,org.apache.activemq.audit.message,org.apache.activemq.audit.resource,org.apache.activemq.artemis.utils.pools,org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerTarget,org.apache.activemq.artemis.core.server.impl,org.apache.activemq.artemis.utils.collections,org.apache.activemq.artemis.core.postoffice.impl
# Special logger to debug mirror Security
# Root logger level
logger.level=INFO
# ActiveMQ Artemis logger levels
# These levels are candidates to eventually debug
logger.org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerTarget.level=INFO
# if you have issues with CriticalAnalyzer, setting this as TRACE would give you extra troubleshooting information.
# but do not use it regularly as it would incur in some extra CPU usage for this diagnostic.
logger.org.apache.activemq.artemis.utils.critical.level=INFO
logger.org.eclipse.jetty.level=WARN
# Root logger handlers
logger.handlers=FILE,CONSOLE
# to enable audit change the level to INFO
logger.org.apache.activemq.audit.base.level=ERROR
logger.org.apache.activemq.audit.base.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.base.useParentHandlers=false
logger.org.apache.activemq.audit.resource.level=ERROR
logger.org.apache.activemq.audit.resource.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.resource.useParentHandlers=false
logger.org.apache.activemq.audit.message.level=ERROR
logger.org.apache.activemq.audit.message.handlers=AUDIT_FILE
logger.org.apache.activemq.audit.message.useParentHandlers=false
# Console handler configuration
handler.CONSOLE=org.jboss.logmanager.handlers.ConsoleHandler
handler.CONSOLE.properties=autoFlush
handler.CONSOLE.level=TRACE
handler.CONSOLE.autoFlush=true
handler.CONSOLE.formatter=PATTERN
# File handler configuration
handler.FILE=org.jboss.logmanager.handlers.PeriodicRotatingFileHandler
handler.FILE.level=TRACE
handler.FILE.properties=suffix,append,autoFlush,fileName
handler.FILE.suffix=.yyyy-MM-dd
handler.FILE.append=true
handler.FILE.autoFlush=true
handler.FILE.fileName=${artemis.instance}/log/artemis.log
handler.FILE.formatter=PATTERN
# Formatter pattern configuration
formatter.PATTERN=org.jboss.logmanager.formatters.PatternFormatter
formatter.PATTERN.properties=pattern
formatter.PATTERN.pattern=[%t] %d %-5p [%c] %s%E%n
#Audit logger
handler.AUDIT_FILE=org.jboss.logmanager.handlers.PeriodicRotatingFileHandler
handler.AUDIT_FILE.level=INFO
handler.AUDIT_FILE.properties=suffix,append,autoFlush,fileName
handler.AUDIT_FILE.suffix=.yyyy-MM-dd
handler.AUDIT_FILE.append=true
handler.AUDIT_FILE.autoFlush=true
handler.AUDIT_FILE.fileName=${artemis.instance}/log/audit.log
handler.AUDIT_FILE.formatter=AUDIT_PATTERN
formatter.AUDIT_PATTERN=org.jboss.logmanager.formatters.PatternFormatter
formatter.AUDIT_PATTERN.properties=pattern
formatter.AUDIT_PATTERN.pattern=%d [AUDIT](%t) %s%E%n

View File

@ -0,0 +1,199 @@
/**
* 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.smoke.brokerConnection;
import javax.jms.Connection;
import javax.jms.ConnectionFactory;
import javax.jms.MessageConsumer;
import javax.jms.MessageProducer;
import javax.jms.Session;
import javax.jms.TextMessage;
import org.apache.activemq.artemis.tests.smoke.common.ContainerService;
import org.apache.activemq.artemis.tests.smoke.common.SmokeTestBase;
import org.jboss.logging.Logger;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class ChainedMirrorTest extends SmokeTestBase {
private static final Logger logger = Logger.getLogger(ChainedMirrorTest.class);
Object network;
public Object serverMainA;
public Object serverMainB;
public Object serverRoot;
ContainerService service = ContainerService.getService();
private final String SERVER_ROOT = basedir + "/target/brokerConnect/chainedMirror/serverRoot";
private final String SERVER_A = basedir + "/target/brokerConnect/chainedMirror/serverA";
private final String SERVER_B = basedir + "/target/brokerConnect/chainedMirror/serverB";
@Before
public void beforeStart() throws Exception {
disableCheckThread();
ValidateContainer.assumeArtemisContainer();
Assert.assertNotNull(basedir);
recreateBrokerDirectory(SERVER_ROOT);
recreateBrokerDirectory(SERVER_A);
recreateBrokerDirectory(SERVER_B);
network = service.newNetwork();
serverMainA = service.newBrokerImage();
serverMainB = service.newBrokerImage();
serverRoot = service.newBrokerImage();
service.setNetwork(serverMainA, network);
service.setNetwork(serverMainB, network);
service.setNetwork(serverRoot, network);
service.exposePorts(serverMainA, 61616);
service.exposePorts(serverMainB, 61616);
service.exposePorts(serverRoot, 61616);
service.prepareInstance(SERVER_ROOT);
service.prepareInstance(SERVER_A);
service.prepareInstance(SERVER_B);
service.exposeBrokerHome(serverMainA, SERVER_A);
service.exposeBrokerHome(serverMainB, SERVER_B);
service.exposeBrokerHome(serverRoot, SERVER_ROOT);
service.exposeHosts(serverRoot, "artemisTestRoot");
service.exposeHosts(serverMainA, "artemisTestA");
service.exposeHosts(serverMainB, "artemisTestB");
service.startLogging(serverMainB, "ServerB:");
service.start(serverMainB);
service.start(serverMainA);
service.start(serverRoot);
}
@After
public void afterStop() {
service.stop(serverRoot);
service.stop(serverMainA);
service.stop(serverMainB);
}
private String getQueueName() {
return "someQueue";
}
@Test
public void testChained() throws Throwable {
ConnectionFactory factory = service.createCF(serverRoot, "amqp");
ConnectionFactory factory2 = service.createCF(serverMainA, "amqp");
ConnectionFactory factory3 = service.createCF(serverMainB, "amqp");
try (Connection conn = factory.createConnection()) {
Session session = conn.createSession();
MessageProducer producer = session.createProducer(session.createQueue(getQueueName()));
for (int i = 0; i < 40; i++) {
producer.send(session.createTextMessage("message " + i));
}
}
Thread.sleep(5000); // some time to allow eventual loops
try (Connection conn = factory.createConnection()) {
Session session = conn.createSession();
conn.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(getQueueName()));
for (int i = 0; i < 10; i++) {
TextMessage message = (TextMessage) consumer.receive(1000);
Assert.assertNotNull(message);
Assert.assertEquals("message " + i, message.getText());
}
consumer.close();
}
Thread.sleep(5000); // some time to allow eventual loops
try (Connection conn = factory2.createConnection()) {
Session session = conn.createSession();
conn.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(getQueueName()));
for (int i = 10; i < 20; i++) {
TextMessage message = (TextMessage) consumer.receive(1000);
Assert.assertNotNull(message);
Assert.assertEquals("message " + i, message.getText());
}
consumer.close();
}
Thread.sleep(5000); // some time to allow eventual loops
try (Connection conn = factory3.createConnection()) {
Session session = conn.createSession();
conn.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(getQueueName()));
for (int i = 20; i < 30; i++) {
TextMessage message = (TextMessage) consumer.receive(1000);
Assert.assertNotNull(message);
Assert.assertEquals("message " + i, message.getText());
}
consumer.close();
}
Thread.sleep(5000); // some time to allow eventual loops
try (Connection conn = factory.createConnection()) {
Session session = conn.createSession();
conn.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(getQueueName()));
for (int i = 30; i < 40; i++) {
TextMessage message = (TextMessage) consumer.receive(1000);
Assert.assertNotNull(message);
Assert.assertEquals("message " + i, message.getText());
}
consumer.close();
}
Thread.sleep(5000); // some time to allow eventual loops
try (Connection conn = factory.createConnection()) {
Session session = conn.createSession();
conn.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(getQueueName()));
Assert.assertNull(consumer.receiveNoWait());
consumer.close();
}
try (Connection conn = factory2.createConnection()) {
Session session = conn.createSession();
conn.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(getQueueName()));
Assert.assertNull(consumer.receiveNoWait());
consumer.close();
}
try (Connection conn = factory3.createConnection()) {
Session session = conn.createSession();
conn.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(getQueueName()));
Assert.assertNull(consumer.receiveNoWait());
consumer.close();
}
}
}

View File

@ -0,0 +1,205 @@
/**
* 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.smoke.brokerConnection;
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 org.apache.activemq.artemis.tests.smoke.common.ContainerService;
import org.apache.activemq.artemis.tests.smoke.common.SmokeTestBase;
import org.jboss.logging.Logger;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class DualMirrorFailoverWithContainerTest extends SmokeTestBase {
private static final Logger logger = Logger.getLogger(DualMirrorFailoverWithContainerTest.class);
Object network;
public Object serverMainA;
public Object serverBackupA;
public Object serverMainB;
public Object serverBackupB;
ContainerService service = ContainerService.getService();
private final String SERVER_MAIN_A_LOCATION = basedir + "/target/brokerConnect/replicaMainServerA";
private final String SERVER_BACKUP_A_LOCATION = basedir + "/target/brokerConnect/replicaBackupServerA";
private final String SERVER_MAIN_B_LOCATION = basedir + "/target/brokerConnect/replicaMainServerB";
private final String SERVER_BACKUP_B_LOCATION = basedir + "/target/brokerConnect/replicaBackupServerB";
@Before
public void beforeStart() throws Exception {
disableCheckThread();
ValidateContainer.assumeArtemisContainer();
Assert.assertNotNull(basedir);
recreateBrokerDirectory(SERVER_MAIN_A_LOCATION);
recreateBrokerDirectory(SERVER_BACKUP_A_LOCATION);
recreateBrokerDirectory(SERVER_MAIN_B_LOCATION);
recreateBrokerDirectory(SERVER_BACKUP_B_LOCATION);
network = service.newNetwork();
serverMainA = service.newBrokerImage();
serverMainB = service.newBrokerImage();
serverBackupA = service.newBrokerImage();
serverBackupB = service.newBrokerImage();
service.setNetwork(serverMainA, network);
service.setNetwork(serverBackupA, network);
service.setNetwork(serverMainB, network);
service.setNetwork(serverBackupB, network);
service.exposePorts(serverMainA, 61616);
service.exposePorts(serverMainB, 61616);
service.exposePorts(serverBackupA, 61616);
service.exposePorts(serverBackupB, 61616);
service.prepareInstance(SERVER_MAIN_A_LOCATION);
service.prepareInstance(SERVER_MAIN_B_LOCATION);
service.prepareInstance(SERVER_BACKUP_A_LOCATION);
service.prepareInstance(SERVER_BACKUP_B_LOCATION);
service.exposeBrokerHome(serverMainA, SERVER_MAIN_A_LOCATION);
service.exposeBrokerHome(serverMainB, SERVER_MAIN_B_LOCATION);
service.exposeBrokerHome(serverBackupA, SERVER_BACKUP_A_LOCATION);
service.exposeBrokerHome(serverBackupB, SERVER_BACKUP_B_LOCATION);
service.exposeHosts(serverMainA, "mainA");
service.exposeHosts(serverBackupA, "backupA");
service.exposeHosts(serverMainB, "mainB");
service.exposeHosts(serverBackupB, "backupB");
service.logWait(serverBackupA, ".*AMQ221024.*"); // replica is synchronized
service.logWait(serverBackupB, ".*AMQ221024.*");
service.start(serverMainA);
service.start(serverMainB);
service.start(serverBackupA);
service.start(serverBackupB);
cfA = service.createCF(serverMainA, "amqp");
}
@After
public void afterStop() {
service.stop(serverBackupA);
service.stop(serverBackupB);
service.stop(serverMainA);
service.stop(serverMainB);
}
ConnectionFactory cfA;
@Test
public void testReconnectMirrorFailover() throws Throwable {
try {
roundTrip(serverMainA, serverMainB);
service.kill(serverMainB);
waitForServerToStart(service.createURI(serverBackupB, 61616), null, null, 10_000);
roundTrip(serverMainA, serverBackupB);
service.kill(serverMainA);
waitForServerToStart(service.createURI(serverBackupA, 61616), null, null, 10_000);
roundTrip(serverBackupA, serverBackupB);
} catch (Throwable e) {
// this is just so we can instant feedback in case of an assertion error, instead of having to wait tearDown the server,
// which is useful when debugging the test
logger.warn(e.getMessage(), e);
throw e;
}
}
private void roundTrip(Object serverA, Object serverB) throws Throwable {
final String QUEUE_NAME = "exampleQueue";
int NUMBER_OF_MESSAGES = 100;
{
ConnectionFactory factory = service.createCF(serverA, "amqp");
Connection connection = factory.createConnection();
Session session = connection.createSession();
Queue queue = session.createQueue(QUEUE_NAME);
MessageProducer producer = session.createProducer(queue);
for (int i = 0; i < NUMBER_OF_MESSAGES; i++) {
TextMessage msg = session.createTextMessage("message " + i);
msg.setStringProperty("body", "message " + i);
producer.send(msg);
}
connection.close();
}
Thread.sleep(1000);
{
ConnectionFactory factory = service.createCF(serverB, "amqp");
Connection connection = factory.createConnection();
connection.start();
Session session = connection.createSession();
Queue queue = session.createQueue(QUEUE_NAME);
MessageConsumer consumer = session.createConsumer(queue);
for (int i = 0; i < NUMBER_OF_MESSAGES; i++) {
TextMessage message = (TextMessage)consumer.receive(5000);
Assert.assertNotNull(message);
Assert.assertEquals("message " + i, message.getText());
}
Assert.assertNull(consumer.receiveNoWait());
// trying the way back
MessageProducer producer = session.createProducer(queue);
for (int i = 0; i < NUMBER_OF_MESSAGES; i++) {
TextMessage msg = session.createTextMessage("next-message " + i);
msg.setStringProperty("body", "next-message " + i);
producer.send(msg);
}
connection.close();
}
Thread.sleep(1000);
{
ConnectionFactory factory = service.createCF(serverA, "amqp");
Connection connection = factory.createConnection();
connection.start();
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
Queue queue = session.createQueue(QUEUE_NAME);
MessageConsumer consumer = session.createConsumer(queue);
for (int i = 0; i < NUMBER_OF_MESSAGES; i++) {
TextMessage message = (TextMessage)consumer.receive(5000);
Assert.assertNotNull(message);
Assert.assertEquals("next-message " + i, message.getText());
}
session.commit();
connection.close();
}
Thread.sleep(1000);
}
}

View File

@ -0,0 +1,370 @@
/*
* 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
* <br>
* http://www.apache.org/licenses/LICENSE-2.0
* <br>
* 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.smoke.brokerConnection;
import javax.jms.Connection;
import javax.jms.ConnectionFactory;
import javax.jms.DeliveryMode;
import javax.jms.JMSException;
import javax.jms.MessageConsumer;
import javax.jms.MessageProducer;
import javax.jms.Queue;
import javax.jms.Session;
import javax.jms.TextMessage;
import java.io.File;
import java.io.FileOutputStream;
import java.io.PrintStream;
import org.apache.activemq.artemis.cli.commands.tools.PrintData;
import org.apache.activemq.artemis.tests.smoke.common.SmokeTestBase;
import org.apache.activemq.artemis.tests.util.CFUtil;
import org.apache.activemq.artemis.util.ServerUtil;
import org.jboss.logging.Logger;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class DualMirrorNoContainerTest extends SmokeTestBase {
// Change this to true to generate a print-data in certain cases on this test
private static final boolean PRINT_DATA = false;
private static final Logger logger = Logger.getLogger(DualMirrorNoContainerTest.class);
public static final String SERVER_NAME_A = "brokerConnect/mirrorSecurityA";
public static final String SERVER_NAME_B = "brokerConnect/mirrorSecurityB";
Process processB;
Process processA;
@Before
public void beforeClass() throws Exception {
cleanupData(SERVER_NAME_A);
cleanupData(SERVER_NAME_B);
processB = startServer(SERVER_NAME_B, 0, 0);
processA = startServer(SERVER_NAME_A, 0, 0);
ServerUtil.waitForServerToStart(1, "B", "B", 30000);
ServerUtil.waitForServerToStart(0, "A", "A", 30000);
}
@Test
public void testMirrorWithTX() throws Throwable {
testMirrorOverBokerConnection(true);
}
@Test
public void testMirrorWithoutTX() throws Throwable {
testMirrorOverBokerConnection(false);
}
@Test
public void testRollback() throws Throwable {
ConnectionFactory cfA = CFUtil.createConnectionFactory("amqp", "tcp://localhost:61616");
ConnectionFactory cfB = CFUtil.createConnectionFactory("amqp", "tcp://localhost:61617");
try (Connection connectionA = cfA.createConnection("A", "A");
Connection connectionB = cfB.createConnection("B", "B")) {
// Testing things on the direction from mirroring from A to B...
Session sessionA = connectionA.createSession(true, Session.SESSION_TRANSACTED);
Queue queue = sessionA.createQueue("someQueue");
MessageProducer producerA = sessionA.createProducer(queue);
sendMessages(true, sessionA, producerA, 0, 99);
connectionA.start();
MessageConsumer consumerA = sessionA.createConsumer(queue);
receiveMessages(true, sessionA, consumerA, 0, 49);
receiveMessages(false, sessionA, consumerA, 50, 70);
// notice we will leave the messages not committed here, and we will move to the other side.. messages should still be there.
// Switching consumption to the server B
Thread.sleep(1000); // The bridge on acks is asynchronous. We need to wait some time to avoid intermittent failures
// I could replace the wait here with a Wait clause, but I would need to configure JMX or management in order to get the Queue Counts
Session sessionB = connectionB.createSession(true, Session.SESSION_TRANSACTED);
MessageConsumer consumerB = sessionB.createConsumer(queue);
MessageProducer producerB = sessionB.createProducer(queue);
connectionB.start();
receiveMessages(false, sessionB, consumerB, 50, 99);
consumerA.close();
sessionA.rollback(); // this is needed to clear up delivering state
sessionB.commit();
}
}
public void testMirrorOverBokerConnection(boolean tx) throws Throwable {
ConnectionFactory cfA = CFUtil.createConnectionFactory("amqp", "tcp://localhost:61616");
ConnectionFactory cfB = CFUtil.createConnectionFactory("amqp", "tcp://localhost:61617");
try (Connection connectionA = cfA.createConnection("A", "A");
Connection connectionB = cfB.createConnection("B", "B")) {
// Testing things on the direction from mirroring from A to B...
Session sessionA = connectionA.createSession(tx, tx ? Session.SESSION_TRANSACTED : Session.AUTO_ACKNOWLEDGE);
Queue queue = sessionA.createQueue("someQueue");
MessageProducer producerA = sessionA.createProducer(queue);
sendMessages(tx, sessionA, producerA, 0, 9);
connectionA.start();
MessageConsumer consumerA = sessionA.createConsumer(queue);
receiveMessages(tx, sessionA, consumerA, 0, 4);
consumerA.close();
// Switching consumption to the server B
Thread.sleep(1000); // The bridge on acks is asynchronous. We need to wait some time to avoid intermittent failures
// I could replace the wait here with a Wait clause, but I would need to configure JMX or management in order to get the Queue Counts
Session sessionB = connectionB.createSession(tx, tx ? Session.SESSION_TRANSACTED : Session.AUTO_ACKNOWLEDGE);
MessageConsumer consumerB = sessionB.createConsumer(queue);
MessageProducer producerB = sessionB.createProducer(queue);
connectionB.start();
receiveMessages(tx, sessionB, consumerB, 5, 9);
Assert.assertNull(consumerB.receiveNoWait());
sendMessages(tx, sessionB, producerB, 0, 19);
receiveMessages(tx, sessionB, consumerB, 0, 9);
consumerB.close();
// switching over back again to A, some sleep here
Thread.sleep(1000);
consumerA = sessionA.createConsumer(queue);
receiveMessages(tx, sessionA, consumerA, 10, 19);
Assert.assertNull(consumerA.receiveNoWait());
}
}
@Test
public void testReconnectMirror() throws Throwable {
testReconnectMirror(false);
}
@Test
public void testReconnectMirrorLargeMessage() throws Throwable {
testReconnectMirror(true);
}
private void testReconnectMirror(boolean isLarge) throws Throwable {
ConnectionFactory cfA = CFUtil.createConnectionFactory("amqp", "tcp://localhost:61616");
ConnectionFactory cfB = CFUtil.createConnectionFactory("amqp", "tcp://localhost:61617");
String largeBuffer = "";
if (isLarge) {
StringBuffer buffer = new StringBuffer();
while (buffer.length() < 200 * 1024) {
buffer.append("This is large ");
}
largeBuffer = buffer.toString();
}
int NUMBER_OF_MESSAGES = isLarge ? 100 : 1_000;
int FAILURE_INTERVAL = isLarge ? 10 : 100;
try (Connection connectionA = cfA.createConnection("A", "A")) {
// Testing things on the direction from mirroring from A to B...
Session sessionA = connectionA.createSession(true, Session.SESSION_TRANSACTED);
Queue queue = sessionA.createQueue("someQueue");
MessageProducer producerA = sessionA.createProducer(queue);
producerA.setDeliveryMode(DeliveryMode.PERSISTENT);
for (int i = 0; i < NUMBER_OF_MESSAGES; i++) {
TextMessage message = sessionA.createTextMessage("message " + i + largeBuffer);
message.setStringProperty("color", i % 2 == 0 ? "yellow" : "red");
message.setIntProperty("i", i);
producerA.send(message);
if (i % 1000 == 0 && i > 0) {
System.out.println("Message " + i);
sessionA.commit();
}
if (i % FAILURE_INTERVAL == 0 && i > 0) {
restartB();
}
}
sessionA.commit();
connectionA.start();
}
try (Connection connectionB = cfB.createConnection("B", "B")) {
// Testing things on the direction from mirroring from A to B...
Session sessionB = connectionB.createSession(true, Session.SESSION_TRANSACTED);
Queue queue = sessionB.createQueue("someQueue");
connectionB.start();
MessageConsumer consumerB = sessionB.createConsumer(queue);
for (int i = 0; i < NUMBER_OF_MESSAGES; i++) {
TextMessage message = (TextMessage)consumerB.receive(5_000);
Assert.assertNotNull("expected message at " + i, message);
Assert.assertEquals("message " + i + largeBuffer, message.getText());
}
Assert.assertNull(consumerB.receiveNoWait());
sessionB.rollback();
}
int restarted = 0;
try (Connection connectionB = cfB.createConnection("B", "B")) {
// Testing things on the direction from mirroring from A to B...
Session sessionB = connectionB.createSession(false, Session.AUTO_ACKNOWLEDGE);
Queue queue = sessionB.createQueue("someQueue");
connectionB.start();
MessageConsumer consumerB = sessionB.createConsumer(queue, "color='yellow'");
int op = 0;
for (int i = 0; i < NUMBER_OF_MESSAGES; i += 2) {
//System.out.println("Received message on i=" + i);
TextMessage message = (TextMessage)consumerB.receive(5_000);
Assert.assertNotNull("expected message at " + i, message);
Assert.assertEquals("message " + i + largeBuffer, message.getText());
if (op++ > 0 && op % FAILURE_INTERVAL == 0) {
restartA(++restarted);
}
}
Assert.assertNull(consumerB.receiveNoWait());
}
System.out.println("Restarted serverA " + restarted + " times");
Thread.sleep(1000);
try (Connection connectionA = cfA.createConnection("A", "A")) {
// Testing things on the direction from mirroring from A to B...
Session sessionA = connectionA.createSession(true, Session.SESSION_TRANSACTED);
Queue queue = sessionA.createQueue("someQueue");
connectionA.start();
MessageConsumer consumerA = sessionA.createConsumer(queue);
for (int i = 1; i < NUMBER_OF_MESSAGES; i += 2) {
TextMessage message = (TextMessage)consumerA.receive(5_000);
Assert.assertNotNull("expected message at " + i, message);
// We should only have red left
Assert.assertEquals("Unexpected message at " + i + " with i=" + message.getIntProperty("i"), "red", message.getStringProperty("color"));
Assert.assertEquals("message " + i + largeBuffer, message.getText());
}
sessionA.commit();
Assert.assertNull(consumerA.receiveNoWait());
}
Thread.sleep(5000);
try (Connection connectionB = cfB.createConnection("B", "B")) {
// Testing things on the direction from mirroring from A to B...
Session sessionB = connectionB.createSession(false, Session.AUTO_ACKNOWLEDGE);
Queue queue = sessionB.createQueue("someQueue");
connectionB.start();
MessageConsumer consumerB = sessionB.createConsumer(queue);
TextMessage message = (TextMessage)consumerB.receiveNoWait();
if (message != null) {
Assert.fail("was expected null, however received " + message.getText());
}
}
}
private void restartB() throws Exception {
processB.destroyForcibly();
Thread.sleep(500);
/*String localtionServerB = getServerLocation(SERVER_NAME_B);
File fileB = new File(localtionServerB + "/data");
PrintData.printData(new File(fileB, "bindings"), new File(fileB, "journal"), new File(fileB, "paging"), false); */
processB = startServer(SERVER_NAME_B, 0, 0);
ServerUtil.waitForServerToStart(1, "B", "B", 30000);
}
private void restartA(int restartNumber) throws Exception {
System.out.println("Restarting A");
processA.destroyForcibly();
Thread.sleep(1000);
if (PRINT_DATA) {
String localtionServerA = getServerLocation(SERVER_NAME_A);
File fileA = new File(localtionServerA + "/data");
File fileOutput = new File(localtionServerA + "/log/print-data-" + restartNumber + ".txt");
FileOutputStream fileOutputStream = new FileOutputStream(fileOutput);
PrintData.printData(new File(fileA, "bindings"), new File(fileA, "journal"), new File(fileA, "paging"), new PrintStream(fileOutputStream), false, false);
fileOutputStream.close();
}
processA = startServer(SERVER_NAME_A, 0, 0);
ServerUtil.waitForServerToStart(0, "A", "A", 30000);
Thread.sleep(1000);
}
private void receiveMessages(boolean tx, Session session, MessageConsumer consumer, int start, int end) throws JMSException {
for (int i = start; i <= end; i++) {
TextMessage message = (TextMessage) consumer.receive(1000);
Assert.assertNotNull(message);
Assert.assertEquals("message " + i, message.getText());
}
if (tx) session.commit();
}
private void sendMessages(boolean tx, Session session, MessageProducer producer, int start, int end) throws JMSException {
for (int i = start; i <= end; i++) {
producer.send(session.createTextMessage("message " + i));
}
if (tx) session.commit();
}
}

View File

@ -0,0 +1,248 @@
/**
* 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.smoke.brokerConnection;
import javax.jms.Connection;
import javax.jms.ConnectionFactory;
import javax.jms.DeliveryMode;
import javax.jms.MessageConsumer;
import javax.jms.MessageProducer;
import javax.jms.Queue;
import javax.jms.Session;
import javax.jms.TextMessage;
import org.apache.activemq.artemis.tests.smoke.common.ContainerService;
import org.apache.activemq.artemis.tests.smoke.common.SmokeTestBase;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class DualMirrorWithContainerTest extends SmokeTestBase {
Object network;
public Object serverA;
public Object serverB;
private final String SERVER_A_LOCATION = basedir + "/target/brokerConnect/serverA";
private final String SERVER_B_LOCATION = basedir + "/target/brokerConnect/serverB";
@Before
public void beforeStart() throws Exception {
disableCheckThread();
ValidateContainer.assumeArtemisContainer();
Assert.assertNotNull(basedir);
recreateBrokerDirectory(SERVER_B_LOCATION);
recreateBrokerDirectory(SERVER_A_LOCATION);
ContainerService service = ContainerService.getService();
network = service.newNetwork();
serverA = service.newBrokerImage();
serverB = service.newBrokerImage();
service.setNetwork(serverA, network);
service.setNetwork(serverB, network);
service.exposePorts(serverA, 61616);
service.exposePorts(serverB, 61616);
service.prepareInstance(SERVER_A_LOCATION);
service.prepareInstance(SERVER_B_LOCATION);
service.exposeBrokerHome(serverA, SERVER_A_LOCATION);
service.exposeBrokerHome(serverB, SERVER_B_LOCATION);
service.exposeHosts(serverA, "serverA");
service.exposeHosts(serverB, "serverB");
service.start(serverA);
service.start(serverB);
service.waitForServerToStart(serverA, "artemis", "artemis", 10_000);
service.waitForServerToStart(serverB, "artemis", "artemis", 10_000);
cfA = service.createCF(serverA, "amqp");
cfB = service.createCF(serverA, "amqp");
}
@After
public void afterStop() {
ContainerService.getService().stop(serverA);
ContainerService.getService().stop(serverB);
}
ConnectionFactory cfA;
ConnectionFactory cfB;
@Test
public void testReconnectMirror() throws Throwable {
testReconnectMirror(false);
}
@Test
public void testReconnectMirrorLarge() throws Throwable {
testReconnectMirror(true);
}
private void testReconnectMirror(boolean largemessage) throws Throwable {
int NUMBER_OF_MESSAGES = 1_000;
int FAILURE_INTERVAL = 500;
String extraBody = "message ";
if (largemessage) {
StringBuffer buffer = new StringBuffer();
while (buffer.length() < 200 * 1024) {
buffer.append("This is large ");
}
extraBody = buffer.toString();
}
try (Connection connectionA = cfA.createConnection("artemis", "artemis")) {
// Testing things on the direction from mirroring from A to B...
Session sessionA = connectionA.createSession(true, Session.SESSION_TRANSACTED);
Queue queue = sessionA.createQueue("someQueue");
MessageProducer producerA = sessionA.createProducer(queue);
producerA.setDeliveryMode(DeliveryMode.PERSISTENT);
for (int i = 0; i < NUMBER_OF_MESSAGES; i++) {
TextMessage message = sessionA.createTextMessage(extraBody + i);
message.setStringProperty("color", i % 2 == 0 ? "yellow" : "red");
message.setIntProperty("i", i);
producerA.send(message);
if (i % 100 == 0 && i > 0) {
System.out.println("Message " + i);
sessionA.commit();
}
if (i % FAILURE_INTERVAL == 0 && i > 0) {
restartB();
}
}
sessionA.commit();
connectionA.start();
}
try (Connection connectionB = cfB.createConnection("artemis", "artemis")) {
// Testing things on the direction from mirroring from A to B...
Session sessionB = connectionB.createSession(true, Session.SESSION_TRANSACTED);
Queue queue = sessionB.createQueue("someQueue");
connectionB.start();
MessageConsumer consumerB = sessionB.createConsumer(queue);
for (int i = 0; i < NUMBER_OF_MESSAGES; i++) {
TextMessage message = (TextMessage)consumerB.receive(5_000);
Assert.assertNotNull("expected message at " + i, message);
Assert.assertEquals(extraBody + i, message.getText());
}
Assert.assertNull(consumerB.receiveNoWait());
sessionB.rollback();
}
int restarted = 0;
try (Connection connectionB = cfB.createConnection("artemis", "artemis")) {
// Testing things on the direction from mirroring from A to B...
Session sessionB = connectionB.createSession(false, Session.AUTO_ACKNOWLEDGE);
Queue queue = sessionB.createQueue("someQueue");
connectionB.start();
MessageConsumer consumerB = sessionB.createConsumer(queue, "color='yellow'");
int op = 0;
for (int i = 0; i < NUMBER_OF_MESSAGES; i += 2) {
TextMessage message = (TextMessage)consumerB.receive(5_000);
Assert.assertNotNull("expected message at " + i, message);
Assert.assertEquals(extraBody + i, message.getText());
if (op++ > 0 && op % FAILURE_INTERVAL == 0) {
restartA(++restarted);
}
}
Assert.assertNull(consumerB.receiveNoWait());
}
System.out.println("Restarted serverA " + restarted + " times");
Thread.sleep(5000);
try (Connection connectionA = cfA.createConnection("artemis", "artemis")) {
// Testing things on the direction from mirroring from A to B...
Session sessionA = connectionA.createSession(true, Session.SESSION_TRANSACTED);
Queue queue = sessionA.createQueue("someQueue");
connectionA.start();
MessageConsumer consumerA = sessionA.createConsumer(queue);
for (int i = 1; i < NUMBER_OF_MESSAGES; i += 2) {
TextMessage message = (TextMessage)consumerA.receive(5_000);
Assert.assertNotNull("expected message at " + i, message);
// We should only have red left
Assert.assertEquals("Unexpected message at " + i + " with i=" + message.getIntProperty("i"), "red", message.getStringProperty("color"));
Assert.assertEquals(extraBody + i, message.getText());
}
sessionA.commit();
Assert.assertNull(consumerA.receiveNoWait());
}
Thread.sleep(5000);
try (Connection connectionB = cfB.createConnection("artemis", "artemis")) {
// Testing things on the direction from mirroring from A to B...
Session sessionB = connectionB.createSession(false, Session.AUTO_ACKNOWLEDGE);
Queue queue = sessionB.createQueue("someQueue");
connectionB.start();
MessageConsumer consumerB = sessionB.createConsumer(queue);
TextMessage message = (TextMessage)consumerB.receiveNoWait();
if (message != null) {
Assert.fail("was expected null, however received " + message.getText());
}
}
}
private void restartB() throws Exception {
ContainerService.getService().restart(serverB);
cfB = ContainerService.getService().createCF(serverB, "amqp");
}
private void restartA(int restartNumber) throws Exception {
ContainerService.getService().restart(serverA);
cfA = ContainerService.getService().createCF(serverB, "amqp");
}
}

View File

@ -30,7 +30,6 @@ import org.apache.activemq.artemis.tests.smoke.common.ContainerService;
import org.jboss.logging.Logger;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Assume;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
@ -57,35 +56,33 @@ public class QpidDispatchPeerTest extends SmokeTestBase {
private static final String QDR_HOME = basedir + "/target/brokerConnect/qdr";
@BeforeClass
public static void startServers() {
try {
Assert.assertNotNull(basedir);
public static void startServers() throws Exception {
ValidateContainer.assumeArtemisContainer();
network = service.newNetwork();
Assert.assertNotNull(basedir);
artemisServer = service.newBrokerImage();
service.setNetwork(artemisServer, network);
service.exposePorts(artemisServer, 61616);
service.prepareInstance(QDR_HOME);
service.exposeBrokerHome(artemisServer, QDR_HOME);
service.startLogging(artemisServer, "ArtemisServer:");
network = service.newNetwork();
qpidServer = service.newInterconnectImage();
service.setNetwork(qpidServer, network);
service.exposePorts(qpidServer, 5672);
service.exposeHosts(qpidServer, "qdr");
service.exposeFile(qpidServer, basedir + "/src/main/resources/servers/brokerConnect/qdr/qdrouterd.conf", "/tmp/qdrouterd.conf");
service.exposeFolder(qpidServer, basedir + "/target/brokerConnect/qdr", "/routerlog");
service.startLogging(qpidServer, "qpid-dispatch:");
service.start(qpidServer);
artemisServer = service.newBrokerImage();
service.setNetwork(artemisServer, network);
service.exposePorts(artemisServer, 61616);
service.prepareInstance(QDR_HOME);
service.exposeBrokerHome(artemisServer, QDR_HOME);
service.startLogging(artemisServer, "ArtemisServer:");
recreateBrokerDirectory(QDR_HOME);
qpidServer = service.newInterconnectImage();
service.setNetwork(qpidServer, network);
service.exposePorts(qpidServer, 5672);
service.exposeHosts(qpidServer, "qdr");
service.exposeFile(qpidServer, basedir + "/src/main/resources/servers/brokerConnect/qdr/qdrouterd.conf", "/tmp/qdrouterd.conf");
service.exposeFolder(qpidServer, basedir + "/target/brokerConnect/qdr", "/routerlog");
service.startLogging(qpidServer, "qpid-dispatch:");
service.start(qpidServer);
recreateBrokerDirectory(QDR_HOME);
service.start(artemisServer);
service.start(artemisServer);
} catch (Exception e) {
e.printStackTrace();
Assume.assumeNoException("Docker not available", e);
}
}
@AfterClass

View File

@ -0,0 +1,194 @@
/**
* 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.smoke.brokerConnection;
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 org.apache.activemq.artemis.tests.smoke.common.ContainerService;
import org.apache.activemq.artemis.tests.smoke.common.SmokeTestBase;
import org.jboss.logging.Logger;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class SplitMirrorTest extends SmokeTestBase {
private static final Logger logger = Logger.getLogger(SplitMirrorTest.class);
Object network;
public Object serverMainA;
public Object serverMainB;
public Object serverRoot;
ContainerService service = ContainerService.getService();
private final String SERVER_ROOT = basedir + "/target/brokerConnect/splitMirror/serverRoot";
private final String SERVER_A = basedir + "/target/brokerConnect/splitMirror/serverA";
private final String SERVER_B = basedir + "/target/brokerConnect/splitMirror/serverB";
@Before
public void beforeStart() throws Exception {
disableCheckThread();
ValidateContainer.assumeArtemisContainer();
Assert.assertNotNull(basedir);
recreateBrokerDirectory(SERVER_ROOT);
recreateBrokerDirectory(SERVER_A);
recreateBrokerDirectory(SERVER_B);
network = service.newNetwork();
serverMainA = service.newBrokerImage();
serverMainB = service.newBrokerImage();
serverRoot = service.newBrokerImage();
service.setNetwork(serverMainA, network);
service.setNetwork(serverMainB, network);
service.setNetwork(serverRoot, network);
service.exposePorts(serverMainA, 61616);
service.exposePorts(serverMainB, 61616);
service.exposePorts(serverRoot, 61616);
service.prepareInstance(SERVER_ROOT);
service.prepareInstance(SERVER_A);
service.prepareInstance(SERVER_B);
service.exposeBrokerHome(serverMainA, SERVER_A);
service.exposeBrokerHome(serverMainB, SERVER_B);
service.exposeBrokerHome(serverRoot, SERVER_ROOT);
service.exposeHosts(serverRoot, "artemisRoot");
service.exposeHosts(serverMainA, "artemisA");
service.exposeHosts(serverMainB, "artemisB");
service.start(serverMainA);
service.start(serverMainB);
service.start(serverRoot);
}
@After
public void afterStop() {
service.stop(serverRoot);
service.stop(serverMainA);
service.stop(serverMainB);
}
@Test
public void testSplitMirror() throws Throwable {
ConnectionFactory cfRoot = service.createCF(serverRoot, "amqp");
ConnectionFactory cfA = service.createCF(serverMainA, "amqp");
ConnectionFactory cfB = service.createCF(serverMainB, "amqp");
try (Connection connection = cfRoot.createConnection()) {
Session session = connection.createSession();
Queue queue = session.createQueue("someQueue");
MessageProducer producer = session.createProducer(queue);
for (int i = 0; i < 10; i++) {
producer.send(session.createTextMessage("sessionRoot " + i));
}
connection.start();
MessageConsumer consumer = session.createConsumer(queue);
for (int i = 0; i < 5; i++) {
Assert.assertNotNull(consumer.receive(5000));
}
consumer.close();
}
Thread.sleep(1000);
try (Connection connection = cfA.createConnection()) {
Session session = connection.createSession();
Queue queue = session.createQueue("someQueue");
MessageConsumer consumer = session.createConsumer(queue);
connection.start();
for (int i = 5; i < 10; i++) {
TextMessage message = (TextMessage) consumer.receive(5000);
Assert.assertNotNull(message);
}
Assert.assertNull(consumer.receiveNoWait());
}
try (Connection connection = cfB.createConnection()) {
Session session = connection.createSession();
Queue queue = session.createQueue("someQueue");
MessageConsumer consumer = session.createConsumer(queue);
connection.start();
for (int i = 5; i < 10; i++) {
TextMessage message = (TextMessage) consumer.receive(5000);
Assert.assertNotNull(message);
}
Assert.assertNull(consumer.receiveNoWait());
}
try (Connection connection = cfRoot.createConnection()) {
Session session = connection.createSession();
Queue queue = session.createQueue("someQueue");
MessageConsumer consumer = session.createConsumer(queue);
connection.start();
for (int i = 5; i < 10; i++) {
TextMessage message = (TextMessage) consumer.receive(5000);
Assert.assertNotNull(message);
}
Assert.assertNull(consumer.receiveNoWait());
consumer.close();
service.kill(serverMainA);
MessageProducer producer = session.createProducer(queue);
for (int i = 0; i < 33; i++) {
producer.send(session.createTextMessage("afterKill " + i));
}
}
Thread.sleep(1000);
try (Connection connection = cfB.createConnection()) {
Session session = connection.createSession();
Queue queue = session.createQueue("someQueue");
MessageConsumer consumer = session.createConsumer(queue);
connection.start();
for (int i = 0; i < 33; i++) {
TextMessage message = (TextMessage) consumer.receive(5000);
Assert.assertNotNull(message);
System.out.println("message.getText() = " + message.getText() + " i = " + i);
Assert.assertEquals("afterKill " + i, message.getText());
}
Assert.assertNull(consumer.receiveNoWait());
}
service.start(serverMainA);
cfA = service.createCF(serverMainA, "amqp");
try (Connection connection = cfA.createConnection()) {
Session session = connection.createSession();
Queue queue = session.createQueue("someQueue");
MessageConsumer consumer = session.createConsumer(queue);
connection.start();
for (int i = 0; i < 33; i++) {
TextMessage message = (TextMessage) consumer.receive(5000);
Assert.assertNotNull(message);
Assert.assertEquals("afterKill " + i, message.getText());
}
Assert.assertNull(consumer.receiveNoWait());
}
}
}

View File

@ -0,0 +1,65 @@
/**
* 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.smoke.brokerConnection;
import java.io.File;
import org.apache.activemq.artemis.tests.smoke.common.ContainerService;
import org.junit.Assume;
/** The purpose of this class is to validate if "artemis-centos" and docker (or an equivalent) is available on the environment.
* Tests can use an assume to be ignored in case the image is not available.
* The test will also cache the result by creating a file target/org.apache.activemq.artemis.tests.smoke.brokerConnection.ValidateContainer.ok
* So, we won't keep redoing the check during development on an IDE. */
public class ValidateContainer {
private static final boolean hasContainer;
static {
File fileHasImage = new File("target/" + ValidateContainer.class.getName() + ".ok");
boolean internalHasContainer = true;
if (fileHasImage.exists()) {
// this is to speed up execution when inside the IDE. Just reuse the last execution's file. If the file exists from a previous run, we know the container is available
internalHasContainer = true;
} else {
try {
ContainerService service = ContainerService.getService();
Object brokerService = service.newBrokerImage();
service.exposePorts(brokerService, 61616);
service.start(brokerService);
service.stop(brokerService);
fileHasImage.createNewFile();
} catch (Throwable e) {
e.printStackTrace();
internalHasContainer = false;
}
}
hasContainer = internalHasContainer;
}
public static boolean hasContainer() {
return hasContainer;
}
/** assume clause to validate the Artemis Container and the Container provider are available */
public static void assumeArtemisContainer() {
Assume.assumeTrue("Please build artemis.centos container image before running these tests", hasContainer());
}
}

View File

@ -122,7 +122,7 @@ public class SmokeTestBase extends ActiveMQTestBase {
protected static final void recreateBrokerDirectory(final String homeInstance) {
recreateDirectory(homeInstance + "/data");
recreateDirectory(homeInstance + "/logs");
recreateDirectory(homeInstance + "/log");
}

View File

@ -99,17 +99,17 @@ public class ConfigurationValidationTest extends ActiveMQTestBase {
Assert.assertEquals(AMQPBrokerConnectionAddressType.MIRROR, amqpBrokerConnectConfiguration.getConnectionElements().get(4).getType());
AMQPMirrorBrokerConnectionElement mirrorConnectionElement = (AMQPMirrorBrokerConnectionElement) amqpBrokerConnectConfiguration.getConnectionElements().get(4);
Assert.assertFalse(mirrorConnectionElement.isMessageAcknowledgements());
Assert.assertTrue(mirrorConnectionElement.isDurable()); // queue name passed, so this is supposed to be true
Assert.assertFalse(mirrorConnectionElement.isQueueCreation());
Assert.assertFalse(mirrorConnectionElement.isQueueRemoval());
Assert.assertEquals("TEST-REPLICA", mirrorConnectionElement.getSourceMirrorAddress().toString());
Assert.assertFalse(mirrorConnectionElement.isDurable());
amqpBrokerConnectConfiguration = fc.getAMQPConnection().get(1);
Assert.assertEquals(null, amqpBrokerConnectConfiguration.getUser()); mirrorConnectionElement = (AMQPMirrorBrokerConnectionElement) amqpBrokerConnectConfiguration.getConnectionElements().get(0);
Assert.assertEquals(null, amqpBrokerConnectConfiguration.getPassword()); Assert.assertEquals("test2", amqpBrokerConnectConfiguration.getName());
Assert.assertEquals("tcp://test2:222", amqpBrokerConnectConfiguration.getUri());
Assert.assertTrue(mirrorConnectionElement.isMessageAcknowledgements());
Assert.assertFalse(mirrorConnectionElement.isDurable()); // queue name not passed (set as null), so this is supposed to be false
Assert.assertFalse(mirrorConnectionElement.isDurable());
Assert.assertTrue(mirrorConnectionElement.isQueueCreation());
Assert.assertTrue(mirrorConnectionElement.isQueueRemoval());

View File

@ -20,7 +20,6 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Executor;
import java.util.function.ToLongFunction;
import org.apache.activemq.artemis.api.config.ActiveMQDefaultConfiguration;
import org.apache.activemq.artemis.api.core.Message;
@ -40,6 +39,7 @@ import org.apache.activemq.artemis.core.server.ServerConsumer;
import org.apache.activemq.artemis.core.server.impl.AckReason;
import org.apache.activemq.artemis.core.transaction.Transaction;
import org.apache.activemq.artemis.utils.ReferenceCounter;
import org.apache.activemq.artemis.utils.collections.NodeStore;
import org.apache.activemq.artemis.utils.collections.LinkedListIterator;
import org.apache.activemq.artemis.utils.critical.CriticalComponentImpl;
import org.apache.activemq.artemis.utils.critical.EmptyCriticalAnalyzer;
@ -142,7 +142,7 @@ public class FakeQueue extends CriticalComponentImpl implements Queue {
}
@Override
public MessageReference removeWithSuppliedID(long id, ToLongFunction<MessageReference> idSupplier) {
public MessageReference removeWithSuppliedID(String serverID, long id, NodeStore<MessageReference> nodeStore) {
return null;
}

View File

@ -200,7 +200,12 @@ public class FakePostOffice implements PostOffice {
@Override
public DuplicateIDCache getDuplicateIDCache(final SimpleString address) {
return DuplicateIDCaches.inMemory(address, 2000);
return getDuplicateIDCache(address, 2000);
}
@Override
public DuplicateIDCache getDuplicateIDCache(final SimpleString address, int idSize) {
return DuplicateIDCaches.inMemory(address, idSize);
}
@Override

View File

@ -17,13 +17,16 @@
package org.apache.activemq.artemis.tests.unit.util;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.NoSuchElementException;
import io.netty.util.collection.LongObjectHashMap;
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
import org.apache.activemq.artemis.tests.util.RandomUtil;
import org.apache.activemq.artemis.utils.collections.NodeStore;
import org.apache.activemq.artemis.utils.collections.LinkedListImpl;
import org.apache.activemq.artemis.utils.collections.LinkedListIterator;
import org.junit.Assert;
@ -110,9 +113,11 @@ public class LinkedListTest extends ActiveMQTestBase {
private static final class ObservableNode extends LinkedListImpl.Node<ObservableNode> {
public String serverID;
public int id;
ObservableNode(int id) {
ObservableNode(String serverID, int id) {
this.id = id;
this.serverID = serverID;
}
public LinkedListImpl.Node<ObservableNode> publicNext() {
@ -125,6 +130,65 @@ public class LinkedListTest extends ActiveMQTestBase {
}
static class ListNodeStore implements NodeStore<ObservableNode> {
// this is for serverID = null;
LongObjectHashMap<LinkedListImpl.Node<ObservableNode>> nodeLongObjectHashMap = new LongObjectHashMap<>();
HashMap<Object, LongObjectHashMap<LinkedListImpl.Node<ObservableNode>>> mapList = new HashMap<>();
@Override
public void storeNode(ObservableNode element, LinkedListImpl.Node<ObservableNode> node) {
LongObjectHashMap<LinkedListImpl.Node<ObservableNode>> map = getNodesMap(element.serverID);
map.put(element.id, node);
}
@Override
public LinkedListImpl.Node<ObservableNode> getNode(String listID, long id) {
LongObjectHashMap<LinkedListImpl.Node<ObservableNode>> map = getNodesMap(listID);
if (map == null) {
return null;
}
return map.get(id);
}
@Override
public void removeNode(ObservableNode element, LinkedListImpl.Node<ObservableNode> node) {
LongObjectHashMap<LinkedListImpl.Node<ObservableNode>> map = getNodesMap(element.serverID);
if (map != null) {
map.remove(element.id);
}
}
private LongObjectHashMap<LinkedListImpl.Node<ObservableNode>> getNodesMap(String listID) {
if (listID == null) {
return nodeLongObjectHashMap;
} else {
LongObjectHashMap<LinkedListImpl.Node<ObservableNode>> theMap = mapList.get(listID);
if (theMap == null) {
theMap = new LongObjectHashMap<>();
mapList.put(listID, theMap);
}
return theMap;
}
}
@Override
public void clear() {
nodeLongObjectHashMap.clear();
mapList.clear();
}
@Override
public int size() {
int size = 0;
for (LongObjectHashMap list : mapList.values()) {
size = +list.size();
}
return nodeLongObjectHashMap.size() + size;
}
}
@Test
public void testAddAndRemove() {
@ -132,7 +196,7 @@ public class LinkedListTest extends ActiveMQTestBase {
// Initial add
for (int i = 0; i < 100; i++) {
final ObservableNode o = new ObservableNode(i);
final ObservableNode o = new ObservableNode(null, i);
objs.addTail(o);
}
@ -141,7 +205,7 @@ public class LinkedListTest extends ActiveMQTestBase {
for (int i = 0; i < 500; i++) {
for (int add = 0; add < 1000; add++) {
final ObservableNode o = new ObservableNode(add);
final ObservableNode o = new ObservableNode(null, add);
objs.addTail(o);
assertNotNull("prev", o.publicPrev());
assertNull("next", o.publicNext());
@ -184,63 +248,69 @@ public class LinkedListTest extends ActiveMQTestBase {
}
private void internalAddWithID(boolean deferSupplier) {
LinkedListImpl<ObservableNode> objs = new LinkedListImpl<>();
if (!deferSupplier) {
objs.setIDSupplier(source -> source.id);
}
// Initial add
for (int i = 0; i < 1000; i++) {
final ObservableNode o = new ObservableNode(i);
objs.addTail(o);
}
Assert.assertEquals(1000, objs.size());
if (deferSupplier) {
Assert.assertEquals(0, objs.getSizeOfSuppliedIDs());
objs.setIDSupplier(source -> source.id);
} else {
// clear the ID supplier
for (int sid = 1; sid <= 2; sid++) {
LinkedListImpl<ObservableNode> objs = new LinkedListImpl<>();
objs.clearID();
// and redo it
Assert.assertEquals(0, objs.getSizeOfSuppliedIDs());
objs.setIDSupplier(source -> source.id);
Assert.assertEquals(1000, objs.size());
}
Assert.assertEquals(1000, objs.getSizeOfSuppliedIDs());
String serverID = sid == 1 ? null : "" + sid;
ListNodeStore nodeStore = new ListNodeStore();
/** remove all even items */
for (int i = 0; i < 1000; i += 2) {
objs.removeWithID(i);
}
Assert.assertEquals(500, objs.size());
Assert.assertEquals(500, objs.getSizeOfSuppliedIDs());
Iterator<ObservableNode> iterator = objs.iterator();
{
int i = 1;
while (iterator.hasNext()) {
ObservableNode value = iterator.next();
Assert.assertEquals(i, value.id);
i += 2;
if (!deferSupplier) {
objs.setNodeStore(nodeStore);
}
// Initial add
for (int i = 1; i <= 1000; i++) {
final ObservableNode o = new ObservableNode(serverID, i);
objs.addTail(o);
}
Assert.assertEquals(1000, objs.size());
if (deferSupplier) {
Assert.assertEquals(0, nodeStore.size());
objs.setNodeStore(nodeStore);
} else {
// clear the ID supplier
objs.clearID();
// and redo it
Assert.assertEquals(0, nodeStore.size());
nodeStore = new ListNodeStore();
objs.setNodeStore(nodeStore);
Assert.assertEquals(1000, objs.size());
}
Assert.assertEquals(1000, nodeStore.size());
/** remove all even items */
for (int i = 1; i <= 1000; i += 2) {
objs.removeWithID(serverID, i);
}
Assert.assertEquals(500, objs.size());
Assert.assertEquals(500, nodeStore.size());
Iterator<ObservableNode> iterator = objs.iterator();
{
int i = 2;
while (iterator.hasNext()) {
ObservableNode value = iterator.next();
Assert.assertEquals(i, value.id);
i += 2;
}
}
for (int i = 2; i <= 1000; i += 2) {
Assert.assertNotNull(objs.removeWithID(serverID, i));
}
Assert.assertEquals(0, nodeStore.size());
Assert.assertEquals(0, objs.size());
}
for (int i = 1; i < 1000; i += 2) {
objs.removeWithID(i);
}
Assert.assertEquals(0, objs.getSizeOfSuppliedIDs());
Assert.assertEquals(0, objs.size());
}
@Test
@ -249,7 +319,7 @@ public class LinkedListTest extends ActiveMQTestBase {
// Initial add
for (int i = 0; i < 1001; i++) {
final ObservableNode o = new ObservableNode(i);
final ObservableNode o = new ObservableNode(null, i);
objs.addHead(o);
}
assertEquals(1001, objs.size());
@ -884,7 +954,7 @@ public class LinkedListTest extends ActiveMQTestBase {
final int count = 100;
final LinkedListImpl<ObservableNode> list = new LinkedListImpl<>();
for (int i = 0; i < count; i++) {
final ObservableNode node = new ObservableNode(i);
final ObservableNode node = new ObservableNode(null, i);
assertNull(node.publicPrev());
assertNull(node.publicNext());
list.addTail(node);
@ -907,7 +977,7 @@ public class LinkedListTest extends ActiveMQTestBase {
final ObservableNode[] nodes = new ObservableNode[count];
final LinkedListImpl<ObservableNode> list = new LinkedListImpl<>();
for (int i = 0; i < count; i++) {
final ObservableNode node = new ObservableNode(i);
final ObservableNode node = new ObservableNode(null, i);
assertNull(node.publicPrev());
assertNull(node.publicNext());
nodes[i] = node;