ARTEMIS-4641 Support events for add of missing federation resources

When an AMQP federation instance attempts to federate an address or queue
it can fail if the remote address or queue is not present or cannot be
created based on broker policy. A federation link can also closed if the
federated resource is removed from the remote broker by management etc.
In those cases the remote broker should note the resources that were
targets of federation and send alerts to the source federation broker to
notify it that these resources become available for federation and the
source should attempt again to create federation links if demand still
exists. This allows an AMQP federation instance to heal itself based on
updates from the remote.
This commit is contained in:
Timothy Bish 2024-02-12 12:36:09 -05:00 committed by Robbie Gemmell
parent 29781bd5da
commit 94b6c0ec6e
23 changed files with 3057 additions and 64 deletions

View File

@ -70,6 +70,9 @@ public abstract class AMQPFederation implements FederationInternal {
protected final String name;
protected final ActiveMQServer server;
protected AMQPFederationEventDispatcher eventDispatcher;
protected AMQPFederationEventProcessor eventProcessor;
// Connection and Session are updated after each reconnect.
protected volatile AMQPConnectionContext connection;
protected volatile AMQPSessionContext session;
@ -168,6 +171,23 @@ public abstract class AMQPFederation implements FederationInternal {
handleFederationStopped();
signalFederationStopped();
started = false;
try {
if (eventDispatcher != null) {
eventDispatcher.close();
}
if (eventProcessor != null) {
eventProcessor.close(false);
}
} catch (ActiveMQException amqEx) {
throw amqEx;
} catch (Exception ex) {
throw (ActiveMQException) new ActiveMQException(ex.getMessage()).initCause(ex);
} finally {
eventDispatcher = null;
eventProcessor = null;
}
}
}
@ -253,6 +273,109 @@ public abstract class AMQPFederation implements FederationInternal {
return this;
}
/**
* Register an event sender instance with this federation for use in sending federation level
* events from this federation instance to the remote peer.
*
* @param dispatcher
* The event sender instance to be registered.
*/
synchronized void registerEventSender(AMQPFederationEventDispatcher dispatcher) {
if (eventDispatcher != null) {
throw new IllegalStateException("Federation event dipsatcher already registered on this federation instance.");
}
eventDispatcher = dispatcher;
}
/**
* Register an event receiver instance with this federation for use in receiving federation level
* events sent to this federation instance from the remote peer.
*
* @param dispatcher
* The event receiver instance to be registered.
*/
synchronized void registerEventReceiver(AMQPFederationEventProcessor processor) {
if (eventProcessor != null) {
throw new IllegalStateException("Federation event processor already registered on this federation instance.");
}
eventProcessor = processor;
}
/**
* Register an address by name that was either not present when an address federation consumer
* was initiated or was removed and the active address federation consumer was force closed.
* Upon (re)creation of the registered address a one time event will be sent to the remote
* federation instance which allows it to check if demand still exists and make another attempt
* at creating a consumer to federate messages from that address.
*
* @param address
* The address that is currently missing which should be watched for creation.
*/
synchronized void registerMissingAddress(String address) {
if (eventDispatcher != null) {
eventDispatcher.addAddressWatch(address);
}
}
/**
* Register a queue by name that was either not present when an queue federation consumer was
* initiated or was removed and the active queue federation consumer was force closed. Upon
* (re)creation of the registered address and queue a one time event will be sent to the remote
* federation instance which allows it to check if demand still exists and make another attempt
* at creating a consumer to federate messages from that queue.
*
* @param queue
* The queue that is currently missing which should be watched for creation.
*/
synchronized void registerMissingQueue(String queue) {
if (eventDispatcher != null) {
eventDispatcher.addQueueWatch(queue);
}
}
/**
* Triggers scan of federation address policies for local address demand on the given address
* that was added on the remote peer which was previously absent and could not be auto created
* or was removed while a federation receiver was attached and caused an existing federation
* receiver to be closed.
*
* @param addressName
* The address that has been added on the remote peer.
*/
synchronized void processRemoteAddressAdded(String addressName) {
addressMatchPolicies.values().forEach(policy -> {
try {
policy.afterRemoteAddressAdded(addressName);
} catch (Exception e) {
logger.warn("Error processing remote address added event: ", e);
signalError(e);
}
});
}
/**
* Triggers scan of federation queue policies for local queue demand on the given queue
* that was added on the remote peer which was previously absent at the time of a federation
* receiver attach or was removed and caused an existing federation receiver to be closed.
*
* @param addressName
* The address that has been added on the remote peer.
* @param queueName
* The queue that has been added on the remote peer.
*/
synchronized void processRemoteQueueAdded(String addressName, String queueName) {
queueMatchPolicies.values().forEach(policy -> {
try {
policy.afterRemoteQueueAdded(addressName, queueName);
} catch (Exception e) {
logger.warn("Error processing remote queue added event: ", e);
signalError(e);
}
});
}
/**
* Error signaling API that must be implemented by the specific federation implementation
* to handle error when creating a federation resource such as an outgoing receiver link.

View File

@ -17,12 +17,12 @@
package org.apache.activemq.artemis.protocol.amqp.connect.federation;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederation.FEDERATION_INSTANCE_RECORD;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.ADDRESS_AUTO_DELETE;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.ADDRESS_AUTO_DELETE_DELAY;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.ADDRESS_AUTO_DELETE_MSG_COUNT;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_ADDRESS_RECEIVER;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationPolicySupport.FEDERATED_ADDRESS_SOURCE_PROPERTIES;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederation.FEDERATION_INSTANCE_RECORD;
import static org.apache.activemq.artemis.protocol.amqp.proton.AmqpSupport.QUEUE_CAPABILITY;
import static org.apache.activemq.artemis.protocol.amqp.proton.AmqpSupport.TOPIC_CAPABILITY;
import static org.apache.activemq.artemis.protocol.amqp.proton.AmqpSupport.verifyOfferedCapabilities;
@ -42,6 +42,7 @@ import org.apache.activemq.artemis.core.server.QueueQueryResult;
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPException;
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPIllegalStateException;
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPInternalErrorException;
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPNotImplementedException;
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.AmqpSupport;
@ -66,7 +67,7 @@ import org.apache.qpid.proton.engine.Sender;
*/
public final class AMQPFederationAddressSenderController extends AMQPFederationBaseSenderController {
public AMQPFederationAddressSenderController(AMQPSessionContext session) {
public AMQPFederationAddressSenderController(AMQPSessionContext session) throws ActiveMQAMQPException {
super(session);
}
@ -80,10 +81,16 @@ public final class AMQPFederationAddressSenderController extends AMQPFederationB
final Connection protonConnection = sender.getSession().getConnection();
final org.apache.qpid.proton.engine.Record attachments = protonConnection.attachments();
if (attachments.get(FEDERATION_INSTANCE_RECORD, AMQPFederation.class) == null) {
AMQPFederation federation = attachments.get(FEDERATION_INSTANCE_RECORD, AMQPFederation.class);
if (federation == null) {
throw new ActiveMQAMQPIllegalStateException("Cannot create a federation link from non-federation connection");
}
if (source == null) {
throw new ActiveMQAMQPNotImplementedException("Null source lookup not supported on federation links.");
}
// Match the settlement mode of the remote instead of relying on the default of MIXED.
sender.setSenderSettleMode(sender.getRemoteSenderSettleMode());
// We don't currently support SECOND so enforce that the answer is always FIRST
@ -139,6 +146,8 @@ public final class AMQPFederationAddressSenderController extends AMQPFederationB
}
if (!addressQueryResult.isExists()) {
federation.registerMissingAddress(address.toString());
throw ActiveMQAMQPProtocolMessageBundle.BUNDLE.sourceAddressDoesntExist();
}
@ -178,6 +187,11 @@ public final class AMQPFederationAddressSenderController extends AMQPFederationB
", but it is already mapped to a different address: " + queueQuery.getAddress());
}
// Configure an action to register a watcher for this federated address to be created if it is
// removed during the lifetime of the federation receiver, if restored an event will be sent
// to the remote to prompt it to create a new receiver.
resourceDeletedAction = (e) -> federation.registerMissingAddress(address.toString());
return (Consumer) sessionSPI.createSender(senderContext, queueName, null, false);
}

View File

@ -17,10 +17,13 @@
package org.apache.activemq.artemis.protocol.amqp.connect.federation;
import java.util.function.Consumer;
import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.core.server.MessageReference;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPSessionCallback;
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPException;
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPLargeMessageWriter;
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPMessageWriter;
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPSessionContext;
@ -29,6 +32,8 @@ import org.apache.activemq.artemis.protocol.amqp.proton.AMQPTunneledCoreMessageW
import org.apache.activemq.artemis.protocol.amqp.proton.MessageWriter;
import org.apache.activemq.artemis.protocol.amqp.proton.ProtonServerSenderContext;
import org.apache.activemq.artemis.protocol.amqp.proton.SenderController;
import org.apache.qpid.proton.amqp.transport.AmqpError;
import org.apache.qpid.proton.amqp.transport.ErrorCondition;
/**
* A base class abstract {@link SenderController} implementation for use by federation address and
@ -46,7 +51,9 @@ public abstract class AMQPFederationBaseSenderController implements SenderContro
protected boolean tunnelCoreMessages; // only enabled if remote offers support.
public AMQPFederationBaseSenderController(AMQPSessionContext session) {
protected Consumer<ErrorCondition> resourceDeletedAction;
public AMQPFederationBaseSenderController(AMQPSessionContext session) throws ActiveMQAMQPException {
this.session = session;
this.sessionSPI = session.getSessionSPI();
}
@ -64,6 +71,15 @@ public abstract class AMQPFederationBaseSenderController implements SenderContro
// Currently there isn't anything needed on close of this controller.
}
@Override
public void close(ErrorCondition error) {
if (error != null && AmqpError.RESOURCE_DELETED.equals(error.getCondition())) {
if (resourceDeletedAction != null) {
resourceDeletedAction.accept(error);
}
}
}
@Override
public MessageWriter selectOutgoingMessageWriter(ProtonServerSenderContext sender, MessageReference reference) {
final MessageWriter selected;

View File

@ -41,6 +41,12 @@ public final class AMQPFederationConstants {
*/
public static final Symbol FEDERATION_CONTROL_LINK = Symbol.getSymbol("AMQ_FEDERATION_CONTROL_LINK");
/**
* A desired capability added to the federation events links that must be offered
* in return for a federation event link to be successfully established.
*/
public static final Symbol FEDERATION_EVENT_LINK = Symbol.getSymbol("AMQ_FEDERATION_EVENT_LINK");
/**
* Property name used to embed a nested map of properties meant to be applied if the federation
* resources created on the remote end of the control link if configured to do so. These properties
@ -197,4 +203,37 @@ public final class AMQPFederationConstants {
*/
public static final String TRANSFORMER_PROPERTIES_MAP = "transformer-properties-map";
/**
* Events sent across the events link will each carry an event type to indicate
* the event type which controls how the remote reacts to the given event. The type of
* event infers the payload of the structure of the message payload.
*/
public static final Symbol EVENT_TYPE = Symbol.getSymbol("x-opt-amq-federation-ev-type");
/**
* Indicates that the message carries an address and queue name that was previously
* requested but did not exist, or that was federated but the remote consumer was closed
* due to removal of the queue on the target peer.
*/
public static final String REQUESTED_QUEUE_ADDED = "REQUESTED_QUEUE_ADDED_EVENT";
/**
* Indicates that the message carries an address name that was previously requested
* but did not exist, or that was federated but the remote consumer was closed due to
* removal of the address on the target peer.
*/
public static final String REQUESTED_ADDRESS_ADDED = "REQUESTED_ADDRESS_ADDED_EVENT";
/**
* Carries the name of a Queue that was either not present when a federation consumer was
* initiated and subsequently rejected, or was removed and has been recreated.
*/
public static final String REQUESTED_QUEUE_NAME = "REQUESTED_QUEUE_NAME";
/**
* Carries the name of an Address that was either not present when a federation consumer was
* initiated and subsequently rejected, or was removed and has been recreated.
*/
public static final String REQUESTED_ADDRESS_NAME = "REQUESTED_ADDRESS_NAME";
}

View File

@ -0,0 +1,242 @@
/*
* 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.federation;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederation.FEDERATION_INSTANCE_RECORD;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_EVENT_LINK;
import java.lang.invoke.MethodHandles;
import java.util.HashSet;
import java.util.Objects;
import java.util.Set;
import org.apache.activemq.artemis.api.core.ActiveMQException;
import org.apache.activemq.artemis.api.core.RoutingType;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.core.postoffice.Binding;
import org.apache.activemq.artemis.core.postoffice.QueueBinding;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.Consumer;
import org.apache.activemq.artemis.core.server.impl.AddressInfo;
import org.apache.activemq.artemis.core.server.plugin.ActiveMQServerAddressPlugin;
import org.apache.activemq.artemis.core.server.plugin.ActiveMQServerBindingPlugin;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPSessionCallback;
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPIllegalStateException;
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPInternalErrorException;
import org.apache.activemq.artemis.protocol.amqp.logger.ActiveMQAMQPProtocolMessageBundle;
import org.apache.activemq.artemis.protocol.amqp.proton.ProtonServerSenderContext;
import org.apache.activemq.artemis.protocol.amqp.proton.SenderController;
import org.apache.qpid.proton.amqp.Symbol;
import org.apache.qpid.proton.amqp.messaging.Terminus;
import org.apache.qpid.proton.amqp.transport.ErrorCondition;
import org.apache.qpid.proton.amqp.transport.ReceiverSettleMode;
import org.apache.qpid.proton.engine.Connection;
import org.apache.qpid.proton.engine.EndpointState;
import org.apache.qpid.proton.engine.Sender;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Sender controller used to fire events from one side of an AMQP Federation connection
* to the other side.
*/
public class AMQPFederationEventDispatcher implements SenderController, ActiveMQServerBindingPlugin, ActiveMQServerAddressPlugin {
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final Sender sender;
private final AMQPFederation federation;
private final AMQPSessionCallback session;
private final ActiveMQServer server;
private final Set<String> addressWatches = new HashSet<>();
private final Set<String> queueWatches = new HashSet<>();
public AMQPFederationEventDispatcher(AMQPFederation federation, AMQPSessionCallback session, Sender sender) {
this.session = session;
this.sender = sender;
this.federation = federation;
this.server = federation.getServer();
}
private String getEventsLinkAddress() {
return sender.getName();
}
/**
* Raw event send API that accepts an {@link AMQPMessage} instance and routes it using the
* server post office instance.
*
* @param event
* The event message to send to the previously created control address.
*
* @throws Exception if an error occurs during the message send.
*/
public void sendEvent(AMQPMessage event) throws Exception {
Objects.requireNonNull(event, "Null event message is not expected and constitutes an error condition");
event.setAddress(getEventsLinkAddress());
server.getPostOffice().route(event, true);
}
@Override
public Consumer init(ProtonServerSenderContext senderContext) throws Exception {
final Connection protonConnection = senderContext.getSender().getSession().getConnection();
final org.apache.qpid.proton.engine.Record attachments = protonConnection.attachments();
AMQPFederation federation = attachments.get(FEDERATION_INSTANCE_RECORD, AMQPFederation.class);
if (federation == null) {
throw new ActiveMQAMQPIllegalStateException("Cannot create a federation link from non-federation connection");
}
// Match the settlement mode of the remote instead of relying on the default of MIXED.
sender.setSenderSettleMode(sender.getRemoteSenderSettleMode());
// We don't currently support SECOND so enforce that the answer is always FIRST
sender.setReceiverSettleMode(ReceiverSettleMode.FIRST);
// Create a temporary queue using the unique link name which is where events will
// be sent to so that they can be held until credit is granted by the remote.
final SimpleString queueName = SimpleString.toSimpleString(sender.getName());
if (sender.getLocalState() != EndpointState.ACTIVE) {
// Indicate that event link capabilities is supported.
sender.setOfferedCapabilities(new Symbol[]{FEDERATION_EVENT_LINK});
// When the federation source creates a events receiver link to receive events
// from the federation target side we land here on the target as this end should
// not be active yet, the federation source should request a dynamic source node
// to be created and we should return the address when opening this end.
final Terminus remoteTerminus = (Terminus) sender.getRemoteSource();
if (remoteTerminus == null || !remoteTerminus.getDynamic()) {
throw new ActiveMQAMQPInternalErrorException("Remote Terminus did not arrive as dynamic node: " + remoteTerminus);
}
remoteTerminus.setAddress(queueName.toString());
}
try {
session.createTemporaryQueue(queueName, RoutingType.ANYCAST);
} catch (Exception e) {
throw ActiveMQAMQPProtocolMessageBundle.BUNDLE.errorCreatingTemporaryQueue(e.getMessage());
}
// Attach to the federation instance now that we have a queue to put events onto.
federation.registerEventSender(this);
server.registerBrokerPlugin(this); // Start listening for bindings and consumer events.
return (Consumer) session.createSender(senderContext, queueName, null, false);
}
@Override
public void close() {
// Make a best effort to remove the temporary queue used for event messages on close.
final SimpleString queueName = SimpleString.toSimpleString(sender.getRemoteTarget().getAddress());
server.unRegisterBrokerPlugin(this);
try {
session.removeTemporaryQueue(queueName);
} catch (Exception e) {
// Ignored as the temporary queue should be removed on connection termination.
}
}
@Override
public void close(ErrorCondition error) {
// Ensure cleanup on force close using default close API
close();
}
/**
* Add the given address name to the set of addresses that should be watched for and
* if added to the broker send an event to the remote indicating that it now exists
* and the remote should attempt to create a new address federation consumer.
*
* This method must be called from the connection thread.
*
* @param addressName
* The address name to watch for addition.
*/
public void addAddressWatch(String addressName) {
addressWatches.add(addressName);
}
/**
* Add the given queue name to the set of queues that should be watched for and
* if added to the broker send an event to the remote indicating that it now exists
* and the remote should attempt to create a new queue federation consumer.
*
* This method must be called from the connection thread.
*
* @param queueName
* The queue name to watch for addition.
*/
public void addQueueWatch(String queueName) {
queueWatches.add(queueName);
}
@Override
public void afterAddAddress(AddressInfo addressInfo, boolean reload) throws ActiveMQException {
final String addressName = addressInfo.getName().toString();
// Run this on the connection thread so that rejection of a federation consumer
// and addition of the address can't race such that the consumer adds its intent
// concurrently with the address having been added and we miss the registration.
federation.getConnectionContext().runLater(() -> {
if (addressWatches.remove(addressName)) {
try {
sendEvent(AMQPFederationEventSupport.encodeAddressAddedEvent(addressName));
} catch (Exception e) {
logger.warn("error on send of address added event: {}", e.getMessage());
federation.signalError(
new ActiveMQAMQPInternalErrorException("Error while processing address added: " + e.getMessage() ));
}
}
});
}
@Override
public void afterAddBinding(Binding binding) throws ActiveMQException {
if (binding instanceof QueueBinding) {
final String addressName = ((QueueBinding) binding).getAddress().toString();
final String queueName = ((QueueBinding) binding).getQueue().getName().toString();
// Run this on the connection thread so that rejection of a federation consumer
// and addition of the binding can't race such that the consumer adds its intent
// concurrently with the binding having been added and we miss the registration.
federation.getConnectionContext().runLater(() -> {
if (queueWatches.remove(queueName)) {
try {
sendEvent(AMQPFederationEventSupport.encodeQueueAddedEvent(addressName, queueName));
} catch (Exception e) {
// Likely the connection failed if we get here.
logger.warn("Error on send of queue added event: {}", e.getMessage());
federation.signalError(
new ActiveMQAMQPInternalErrorException("Error while processing queue added: " + e.getMessage() ));
}
}
});
}
}
}

View File

@ -0,0 +1,173 @@
/*
* 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.federation;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.EVENT_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_EVENT_LINK;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.REQUESTED_ADDRESS_ADDED;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.REQUESTED_ADDRESS_NAME;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.REQUESTED_QUEUE_ADDED;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.REQUESTED_QUEUE_NAME;
import java.lang.invoke.MethodHandles;
import java.util.Map;
import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
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.exceptions.ActiveMQAMQPInternalErrorException;
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.qpid.proton.amqp.Symbol;
import org.apache.qpid.proton.amqp.messaging.Accepted;
import org.apache.qpid.proton.amqp.messaging.DeliveryAnnotations;
import org.apache.qpid.proton.amqp.messaging.Terminus;
import org.apache.qpid.proton.amqp.transport.ReceiverSettleMode;
import org.apache.qpid.proton.engine.Delivery;
import org.apache.qpid.proton.engine.EndpointState;
import org.apache.qpid.proton.engine.Receiver;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A specialized AMQP Receiver that handles events from a remote Federation connection such
* as addition of addresses or queues where federation was requested but they did not exist
* at the time and the federation consumer was rejected.
*/
public class AMQPFederationEventProcessor extends ProtonAbstractReceiver {
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final int PROCESSOR_RECEIVER_CREDITS = 10;
private static final int PROCESSOR_RECEIVER_CREDITS_LOW = 3;
private final ActiveMQServer server;
private final AMQPFederation federation;
/**
* Create the new federation event receiver
*
* @param federation
* The AMQP Federation instance that this event consumer resides in.
* @param session
* The associated session for this federation event consumer.
* @param receiver
* The proton {@link Receiver} that this event consumer reads from.
*/
public AMQPFederationEventProcessor(AMQPFederation federation, AMQPSessionContext session, Receiver receiver) {
super(session.getSessionSPI(), session.getAMQPConnectionContext(), session, receiver);
this.server = protonSession.getServer();
this.federation = federation;
}
@Override
public void initialize() throws Exception {
initialized = true;
// Match the settlement mode of the remote instead of relying on the default of MIXED.
receiver.setSenderSettleMode(receiver.getRemoteSenderSettleMode());
// We don't currently support SECOND so enforce that the answer is always FIRST
receiver.setReceiverSettleMode(ReceiverSettleMode.FIRST);
if (receiver.getLocalState() != EndpointState.ACTIVE) {
// Indicate that event link capabilities is supported.
receiver.setOfferedCapabilities(new Symbol[]{FEDERATION_EVENT_LINK});
// When the federation source creates a events sender link to send events to the
// federation target side we land here on the target as this end should not be
// active yet, the federation source should request a dynamic target node to be
// created and we should return the address when opening this end.
final Terminus remoteTerminus = (Terminus) receiver.getRemoteTarget();
if (remoteTerminus == null || !remoteTerminus.getDynamic()) {
throw new ActiveMQAMQPInternalErrorException("Remote Terminus did not arrive as dynamic node: " + remoteTerminus);
}
remoteTerminus.setAddress(receiver.getName());
}
// Inform the federation that there is an event processor in play.
federation.registerEventReceiver(this);
flow();
}
@Override
protected void actualDelivery(Message message, Delivery delivery, DeliveryAnnotations deliveryAnnotations, Receiver receiver, Transaction tx) {
logger.trace("{}::actualdelivery called for {}", server, message);
final AMQPMessage eventMessage = (AMQPMessage) message;
delivery.setContext(message);
try {
final Object eventType = AMQPMessageBrokerAccessor.getMessageAnnotationProperty(eventMessage, EVENT_TYPE);
if (REQUESTED_QUEUE_ADDED.equals(eventType)) {
final Map<String, Object> eventData = AMQPFederationEventSupport.decodeQueueAddedEvent(eventMessage);
final String addressName = eventData.get(REQUESTED_ADDRESS_NAME).toString();
final String queueName = eventData.get(REQUESTED_QUEUE_NAME).toString();
logger.trace("Remote event indicates Queue added that matched a previous request [{}::{}]", addressName, queueName);
federation.processRemoteQueueAdded(addressName, queueName);
} else if (REQUESTED_ADDRESS_ADDED.equals(eventType)) {
final Map<String, Object> eventData = AMQPFederationEventSupport.decodeAddressAddedEvent(eventMessage);
final String addressName = eventData.get(REQUESTED_ADDRESS_NAME).toString();
logger.trace("Remote event indicates Address added that matched a previous request [{}]", addressName);
federation.processRemoteAddressAdded(addressName);
} else {
federation.signalError(new ActiveMQAMQPInternalErrorException("Remote sent unknown event."));
return;
}
delivery.disposition(Accepted.getInstance());
delivery.settle();
flow();
connection.flush();
} catch (Throwable e) {
logger.warn(e.getMessage(), e);
federation.signalError(
new ActiveMQAMQPInternalErrorException("Error while processing incoming event message: " + e.getMessage() ));
}
}
@Override
protected Runnable createCreditRunnable(AMQPConnectionContext connection) {
// The events processor is not bound to the configurable credit on the connection as it could be set
// to zero if trying to create pull federation consumers so we avoid any chance of that happening as
// otherwise there would be no credit granted for the remote to send us events.
return createCreditRunnable(PROCESSOR_RECEIVER_CREDITS, PROCESSOR_RECEIVER_CREDITS_LOW, receiver, connection, this);
}
@Override
public void flow() {
creditRunnable.run();
}
}

View File

@ -0,0 +1,221 @@
/*
* 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.federation;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.EVENT_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.REQUESTED_ADDRESS_ADDED;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.REQUESTED_ADDRESS_NAME;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.REQUESTED_QUEUE_ADDED;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.REQUESTED_QUEUE_NAME;
import java.util.LinkedHashMap;
import java.util.Map;
import org.apache.activemq.artemis.api.core.ActiveMQException;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPStandardMessage;
import org.apache.activemq.artemis.protocol.amqp.logger.ActiveMQAMQPProtocolMessageBundle;
import org.apache.activemq.artemis.protocol.amqp.util.NettyWritable;
import org.apache.activemq.artemis.protocol.amqp.util.TLSEncode;
import org.apache.qpid.proton.amqp.Symbol;
import org.apache.qpid.proton.amqp.messaging.AmqpValue;
import org.apache.qpid.proton.amqp.messaging.MessageAnnotations;
import org.apache.qpid.proton.amqp.messaging.Section;
import org.apache.qpid.proton.codec.EncoderImpl;
import org.apache.qpid.proton.codec.WritableBuffer;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.PooledByteBufAllocator;
/**
* Tools used for sending and receiving events inside AMQP message instance.
*/
public final class AMQPFederationEventSupport {
/**
* Encode an event that indicates that a Queue that belongs to a federation
* request which was not present at the time of the request or was later removed
* is now present and the remote should check for demand and attempt to federate
* the resource once again.
*
* @param address
* The address that the queue is currently bound to.
* @param queue
* The queue that was part of a previous federation request.
*
* @return the AMQP message with the encoded event data.
*/
public static AMQPMessage encodeQueueAddedEvent(String address, String queue) {
final Map<Symbol, Object> annotations = new LinkedHashMap<>();
final MessageAnnotations messageAnnotations = new MessageAnnotations(annotations);
final Map<String, Object> eventMap = new LinkedHashMap<>();
final Section sectionBody = new AmqpValue(eventMap);
final ByteBuf buffer = PooledByteBufAllocator.DEFAULT.heapBuffer(1024);
annotations.put(EVENT_TYPE, REQUESTED_QUEUE_ADDED);
eventMap.put(REQUESTED_ADDRESS_NAME, address);
eventMap.put(REQUESTED_QUEUE_NAME, queue);
try {
final EncoderImpl encoder = TLSEncode.getEncoder();
encoder.setByteBuffer(new NettyWritable(buffer));
encoder.writeObject(messageAnnotations);
encoder.writeObject(sectionBody);
final byte[] data = new byte[buffer.writerIndex()];
buffer.readBytes(data);
return new AMQPStandardMessage(0, data, null);
} finally {
TLSEncode.getEncoder().setByteBuffer((WritableBuffer) null);
buffer.release();
}
}
/**
* Encode an event that indicates that an Address that belongs to a federation
* request which was not present at the time of the request or was later removed
* is now present and the remote should check for demand and attempt to federate
* the resource once again.
*
* @param address
* The address portion of the previously failed federation request
*
* @return the AMQP message with the encoded event data.
*/
public static AMQPMessage encodeAddressAddedEvent(String address) {
final Map<Symbol, Object> annotations = new LinkedHashMap<>();
final MessageAnnotations messageAnnotations = new MessageAnnotations(annotations);
final Map<String, Object> eventMap = new LinkedHashMap<>();
final Section sectionBody = new AmqpValue(eventMap);
final ByteBuf buffer = PooledByteBufAllocator.DEFAULT.heapBuffer(1024);
annotations.put(EVENT_TYPE, REQUESTED_ADDRESS_ADDED);
eventMap.put(REQUESTED_ADDRESS_NAME, address);
try {
final EncoderImpl encoder = TLSEncode.getEncoder();
encoder.setByteBuffer(new NettyWritable(buffer));
encoder.writeObject(messageAnnotations);
encoder.writeObject(sectionBody);
final byte[] data = new byte[buffer.writerIndex()];
buffer.readBytes(data);
return new AMQPStandardMessage(0, data, null);
} finally {
TLSEncode.getEncoder().setByteBuffer((WritableBuffer) null);
buffer.release();
}
}
/**
* Decode and return the Map containing the event data for a Queue that was
* the target of a previous federation request which was not present on the
* remote server or was later removed has now been (re)added.
*
* @param message
* The event message that carries the event data in its body.
*
* @return a {@link Map} containing the payload of the incoming event.
*
* @throws ActiveMQException if an error occurs while decoding the event data.
*/
@SuppressWarnings("unchecked")
public static Map<String, Object> decodeQueueAddedEvent(AMQPMessage message) throws ActiveMQException {
final Section body = message.getBody();
if (!(body instanceof AmqpValue)) {
throw ActiveMQAMQPProtocolMessageBundle.BUNDLE.malformedFederationControlMessage(
"Message body was not an AmqpValue type");
}
final AmqpValue bodyValue = (AmqpValue) body;
if (!(bodyValue.getValue() instanceof Map)) {
throw ActiveMQAMQPProtocolMessageBundle.BUNDLE.malformedFederationControlMessage(
"Message body AmqpValue did not carry an encoded Map");
}
try {
final Map<String, Object> eventMap = (Map<String, Object>) bodyValue.getValue();
if (!eventMap.containsKey(REQUESTED_ADDRESS_NAME)) {
throw ActiveMQAMQPProtocolMessageBundle.BUNDLE.malformedFederationEventMessage(
"Message body did not carry the required address name");
}
if (!eventMap.containsKey(REQUESTED_QUEUE_NAME)) {
throw ActiveMQAMQPProtocolMessageBundle.BUNDLE.malformedFederationEventMessage(
"Message body did not carry the required queue name");
}
return eventMap;
} catch (ActiveMQException amqEx) {
throw amqEx;
} catch (Exception e) {
throw ActiveMQAMQPProtocolMessageBundle.BUNDLE.malformedFederationControlMessage(
"Invalid encoded queue added event entry: " + e.getMessage());
}
}
/**
* Decode and return the Map containing the event data for an Address that was
* the target of a previous federation request which was not present on the
* remote server or was later removed has now been (re)added.
*
* @param message
* The event message that carries the event data in its body.
*
* @return a {@link Map} containing the payload of the incoming event.
*
* @throws ActiveMQException if an error occurs while decoding the event data.
*/
@SuppressWarnings("unchecked")
public static Map<String, Object> decodeAddressAddedEvent(AMQPMessage message) throws ActiveMQException {
final Section body = message.getBody();
if (!(body instanceof AmqpValue)) {
throw ActiveMQAMQPProtocolMessageBundle.BUNDLE.malformedFederationControlMessage(
"Message body was not an AmqpValue type");
}
final AmqpValue bodyValue = (AmqpValue) body;
if (!(bodyValue.getValue() instanceof Map)) {
throw ActiveMQAMQPProtocolMessageBundle.BUNDLE.malformedFederationControlMessage(
"Message body AmqpValue did not carry an encoded Map");
}
try {
final Map<String, Object> eventMap = (Map<String, Object>) bodyValue.getValue();
if (!eventMap.containsKey(REQUESTED_ADDRESS_NAME)) {
throw ActiveMQAMQPProtocolMessageBundle.BUNDLE.malformedFederationEventMessage(
"Message body did not carry the required address name");
}
return eventMap;
} catch (ActiveMQException amqEx) {
throw amqEx;
} catch (Exception e) {
throw ActiveMQAMQPProtocolMessageBundle.BUNDLE.malformedFederationControlMessage(
"Invalid encoded address added event entry: " + e.getMessage());
}
}
}

View File

@ -59,7 +59,7 @@ import org.apache.qpid.proton.engine.Sender;
*/
public final class AMQPFederationQueueSenderController extends AMQPFederationBaseSenderController {
public AMQPFederationQueueSenderController(AMQPSessionContext session) {
public AMQPFederationQueueSenderController(AMQPSessionContext session) throws ActiveMQAMQPException {
super(session);
}
@ -72,7 +72,9 @@ public final class AMQPFederationQueueSenderController extends AMQPFederationBas
final Connection protonConnection = sender.getSession().getConnection();
final org.apache.qpid.proton.engine.Record attachments = protonConnection.attachments();
if (attachments.get(FEDERATION_INSTANCE_RECORD, AMQPFederation.class) == null) {
AMQPFederation federation = attachments.get(FEDERATION_INSTANCE_RECORD, AMQPFederation.class);
if (federation == null) {
throw new ActiveMQAMQPIllegalStateException("Cannot create a federation link from non-federation connection");
}
@ -80,6 +82,16 @@ public final class AMQPFederationQueueSenderController extends AMQPFederationBas
throw new ActiveMQAMQPNotImplementedException("Null source lookup not supported on federation links.");
}
// Match the settlement mode of the remote instead of relying on the default of MIXED.
sender.setSenderSettleMode(sender.getRemoteSenderSettleMode());
// We don't currently support SECOND so enforce that the answer is always FIRST
sender.setReceiverSettleMode(ReceiverSettleMode.FIRST);
// We need to offer back that we support federation for the remote to complete the attach
sender.setOfferedCapabilities(new Symbol[] {FEDERATION_QUEUE_RECEIVER});
// We indicate desired to meet specification that we cannot use a capability unless we
// indicated it was desired, however unless offered by the remote we cannot use it.
sender.setDesiredCapabilities(new Symbol[] {AmqpSupport.CORE_MESSAGE_TUNNELING_SUPPORT});
// An queue receiver may supply a filter if the queue being federated had a filter attached
// to it at creation, this ensures that we only bring back message that match the original
// queue filter and not others that would simply increase traffic for no reason.
@ -110,27 +122,24 @@ public final class AMQPFederationQueueSenderController extends AMQPFederationBas
final QueueQueryResult result = sessionSPI.queueQuery(targetQueue, routingType, false, null);
if (!result.isExists()) {
federation.registerMissingQueue(targetQueue.toString());
throw new ActiveMQAMQPNotFoundException("Queue: '" + targetQueue + "' does not exist");
}
if (targetAddress != null && !result.getAddress().equals(targetAddress)) {
federation.registerMissingQueue(targetQueue.toString());
throw new ActiveMQAMQPNotFoundException("Queue: '" + targetQueue + "' is not mapped to specified address: " + targetAddress);
}
// Match the settlement mode of the remote instead of relying on the default of MIXED.
sender.setSenderSettleMode(sender.getRemoteSenderSettleMode());
// We don't currently support SECOND so enforce that the answer is always FIRST
sender.setReceiverSettleMode(ReceiverSettleMode.FIRST);
// We need to offer back that we support federation for the remote to complete the attach
sender.setOfferedCapabilities(new Symbol[] {FEDERATION_QUEUE_RECEIVER});
// We indicate desired to meet specification that we cannot use a capability unless we
// indicated it was desired, however unless offered by the remote we cannot use it.
sender.setDesiredCapabilities(new Symbol[] {AmqpSupport.CORE_MESSAGE_TUNNELING_SUPPORT});
// We need to check that the remote offers its ability to read tunneled core messages and
// if not we must not send them but instead convert all messages to AMQP messages first.
tunnelCoreMessages = verifyOfferedCapabilities(sender, AmqpSupport.CORE_MESSAGE_TUNNELING_SUPPORT);
// Configure an action to register a watcher for this federated queue to be created if it is
// removed during the lifetime of the federation receiver, if restored an event will be sent
// to the remote to prompt it to create a new receiver.
resourceDeletedAction = (e) -> federation.registerMissingQueue(targetQueue.toString());
return (Consumer) sessionSPI.createSender(senderContext, targetQueue, selector, false);
}

View File

@ -18,6 +18,7 @@
package org.apache.activemq.artemis.protocol.amqp.connect.federation;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_CONTROL_LINK;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_EVENT_LINK;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_CONFIGURATION;
import static org.apache.activemq.artemis.protocol.amqp.proton.AmqpSupport.AMQP_LINK_INITIALIZER_KEY;
@ -26,6 +27,7 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
@ -44,7 +46,6 @@ 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.AmqpSupport;
import org.apache.activemq.artemis.protocol.amqp.proton.ProtonServerSenderContext;
import org.apache.activemq.artemis.utils.UUIDGenerator;
import org.apache.qpid.proton.amqp.Symbol;
import org.apache.qpid.proton.amqp.messaging.DeleteOnClose;
import org.apache.qpid.proton.amqp.messaging.Source;
@ -55,6 +56,7 @@ import org.apache.qpid.proton.amqp.transport.ReceiverSettleMode;
import org.apache.qpid.proton.amqp.transport.SenderSettleMode;
import org.apache.qpid.proton.engine.Connection;
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;
import org.slf4j.Logger;
@ -77,6 +79,9 @@ public class AMQPFederationSource extends AMQPFederation {
// the remote federation target.
private static final Symbol[] CONTROL_LINK_CAPABILITIES = new Symbol[] {FEDERATION_CONTROL_LINK};
// Capabilities set on the events links used to react to federation resources updates
private static final Symbol[] EVENT_LINK_CAPABILITIES = new Symbol[] {FEDERATION_EVENT_LINK};
private final AMQPBrokerConnection brokerConnection;
// Remote policies that should be conveyed to the remote server for reciprocal federation operations.
@ -235,6 +240,22 @@ public class AMQPFederationSource extends AMQPFederation {
}
});
try {
eventDispatcher.close();
} catch (Exception ex) {
errorCaught.compareAndExchange(null, ex);
} finally {
eventDispatcher = null;
}
try {
eventProcessor.close(null);
} catch (Exception ex) {
errorCaught.compareAndExchange(null, ex);
} finally {
eventProcessor = null;
}
connection = null;
session = null;
@ -294,14 +315,233 @@ public class AMQPFederationSource extends AMQPFederation {
return false;
}
private void asyncCreateTargetEventsSender(AMQPFederationCommandDispatcher commandLink) {
// If no remote policies configured then we don't need an events sender link
// currently, if some other use is added for this link this code must be
// removed and tests updated to expect this link to always be created.
if (remoteAddressMatchPolicies.isEmpty() && remoteQueueMatchPolicies.isEmpty()) {
return;
}
// Schedule the outgoing event link creation on the connection event loop thread.
//
// Eventual establishment of the outgoing events link or refusal informs this side
// of the connection as to whether the remote side supports receiving events for
// resources that it attempted to federate but they did not exist at the time and
// were subsequently added or for resources that might have been later removed via
// management and then subsequently re-added.
//
// Once the outcome of the event link is known then send any remote address or queue
// federation policies so that the remote can start federation of local addresses or
// queues to itself. This ordering prevents a race on creation of the events link
// and any federation consumer creation from the remote.
connection.runLater(() -> {
if (!isStarted()) {
return;
}
try {
final Sender sender = session.getSession().sender(
"federation-events-sender:" + getName() + ":" + server.getNodeID() + ":" + UUID.randomUUID());
final Target target = new Target();
final Source source = new Source();
target.setDynamic(true);
target.setCapabilities(new Symbol[] {AmqpSupport.TEMP_TOPIC_CAPABILITY});
target.setDurable(TerminusDurability.NONE);
target.setExpiryPolicy(TerminusExpiryPolicy.LINK_DETACH);
// Set the dynamic node lifetime-policy to indicate this needs to be destroyed on close
// we don't want event links nodes remaining once a federation connection is closed.
final Map<Symbol, Object> dynamicNodeProperties = new HashMap<>();
dynamicNodeProperties.put(AmqpSupport.LIFETIME_POLICY, DeleteOnClose.getInstance());
target.setDynamicNodeProperties(dynamicNodeProperties);
sender.setSenderSettleMode(SenderSettleMode.SETTLED);
sender.setReceiverSettleMode(ReceiverSettleMode.FIRST);
sender.setDesiredCapabilities(EVENT_LINK_CAPABILITIES);
sender.setTarget(target);
sender.setSource(source);
sender.open();
final ScheduledFuture<?> futureTimeout;
final AtomicBoolean cancelled = new AtomicBoolean(false);
if (brokerConnection.getConnectionTimeout() > 0) {
futureTimeout = brokerConnection.getServer().getScheduledPool().schedule(() -> {
cancelled.set(true);
brokerConnection.connectError(ActiveMQAMQPProtocolMessageBundle.BUNDLE.brokerConnectionTimeout());
}, brokerConnection.getConnectionTimeout(), TimeUnit.MILLISECONDS);
} else {
futureTimeout = null;
}
// Using attachments to set up a Runnable that will be executed inside the remote link opened handler
sender.attachments().set(AMQP_LINK_INITIALIZER_KEY, Runnable.class, () -> {
try {
if (cancelled.get()) {
return;
}
if (futureTimeout != null) {
futureTimeout.cancel(false);
}
if (sender.getRemoteTarget() == null || !AmqpSupport.verifyOfferedCapabilities(sender)) {
// Sender rejected or not an event link endpoint so close as we will
// not support sending events to the remote but otherwise will operate
// as normal.
sender.close();
} else {
session.addFederationEventDispatcher(sender);
}
// Once we know whether the events support is active or not we can send
// the remote federation policies and allow the remote federation links
// to start forming.
remoteQueueMatchPolicies.forEach((key, policy) -> {
try {
commandLink.sendPolicy(policy);
} catch (Exception e) {
brokerConnection.error(e);
}
});
remoteAddressMatchPolicies.forEach((key, policy) -> {
try {
commandLink.sendPolicy(policy);
} catch (Exception e) {
brokerConnection.error(e);
}
});
} catch (Exception e) {
brokerConnection.error(e);
}
});
} catch (Exception e) {
brokerConnection.error(e);
}
connection.flush();
});
}
private void asnycCreateTargetEventsReceiver() {
// If no local policies configured then we don't need an events receiver link
// currently, if some other use is added for this link this code must be
// removed and tests updated to expect this link to always be created.
if (addressMatchPolicies.isEmpty() && queueMatchPolicies.isEmpty()) {
return;
}
// Schedule the incoming event link creation on the connection event loop thread.
//
// Eventual establishment of the incoming event link or refusal informs this side
// of the connection as to whether the remote will send events for addresses or
// queues that were not present when a federation consumer attempt had failed and
// were later added or an existing federation consumer was closed due to management
// action and those resource are once again available for federation.
//
// Once the outcome of the event link is known then start all the policy managers
// which will start federation from remote addresses and queues to this broker.
// This ordering prevents any races around the events receiver creation and creation
// of federation consumers on the remote.
connection.runLater(() -> {
if (!isStarted()) {
return;
}
try {
final Receiver receiver = session.getSession().receiver(
"federation-events-receiver:" + getName() + ":" + server.getNodeID() + ":" + UUID.randomUUID());
final Target target = new Target();
final Source source = new Source();
source.setDynamic(true);
source.setCapabilities(new Symbol[] {AmqpSupport.TEMP_TOPIC_CAPABILITY});
source.setDurable(TerminusDurability.NONE);
source.setExpiryPolicy(TerminusExpiryPolicy.LINK_DETACH);
// Set the dynamic node lifetime-policy to indicate this needs to be destroyed on close
// we don't want event links nodes remaining once a federation connection is closed.
final Map<Symbol, Object> dynamicNodeProperties = new HashMap<>();
dynamicNodeProperties.put(AmqpSupport.LIFETIME_POLICY, DeleteOnClose.getInstance());
source.setDynamicNodeProperties(dynamicNodeProperties);
receiver.setSenderSettleMode(SenderSettleMode.SETTLED);
receiver.setReceiverSettleMode(ReceiverSettleMode.FIRST);
receiver.setDesiredCapabilities(EVENT_LINK_CAPABILITIES);
receiver.setTarget(target);
receiver.setSource(source);
receiver.open();
final ScheduledFuture<?> futureTimeout;
final AtomicBoolean cancelled = new AtomicBoolean(false);
if (brokerConnection.getConnectionTimeout() > 0) {
futureTimeout = brokerConnection.getServer().getScheduledPool().schedule(() -> {
cancelled.set(true);
brokerConnection.connectError(ActiveMQAMQPProtocolMessageBundle.BUNDLE.brokerConnectionTimeout());
}, brokerConnection.getConnectionTimeout(), TimeUnit.MILLISECONDS);
} else {
futureTimeout = null;
}
// Using attachments to set up a Runnable that will be executed inside the remote link opened handler
receiver.attachments().set(AMQP_LINK_INITIALIZER_KEY, Runnable.class, () -> {
try {
if (cancelled.get()) {
return;
}
if (futureTimeout != null) {
futureTimeout.cancel(false);
}
if (receiver.getRemoteSource() == null || !AmqpSupport.verifyOfferedCapabilities(receiver)) {
// Receiver rejected or not an event link endpoint so close as we will
// not be receiving events from the remote but otherwise will operate
// as normal.
receiver.close();
} else {
session.addFederationEventProcessor(receiver);
}
// Once we know whether the events support is active or not we can start the
// local federation policies and allow the outgoing federation links to start
// forming.
//
// Attempt to start the policy managers in another thread to avoid blocking the IO thread
scheduler.execute(() -> {
// Sync action with federation start / stop otherwise we could get out of sync
synchronized (AMQPFederationSource.this) {
if (isStarted()) {
queueMatchPolicies.forEach((k, v) -> v.start());
addressMatchPolicies.forEach((k, v) -> v.start());
}
}
});
} catch (Exception e) {
brokerConnection.error(e);
}
});
} catch (Exception e) {
brokerConnection.error(e);
}
connection.flush();
});
}
private void asyncCreateControlLink() {
// Schedule the control link creation on the connection event loop thread
// Eventual establishment of the control link indicates successful connection
// to a remote peer that can support AMQP federation requirements.
connection.runLater(() -> {
try {
final Sender sender = session.getSession().sender("Federation:" + getName() + ":" + UUIDGenerator.getInstance().generateStringUUID());
final AMQPFederationCommandDispatcher commandLink = new AMQPFederationCommandDispatcher(sender, getServer(), session.getSessionSPI());
final Sender sender = session.getSession().sender(
"federation-control-link:" + getName() + ":" + server.getNodeID() + ":" + UUID.randomUUID());
final Target target = new Target();
// The control link should be dynamic and the node is destroyed if the connection drops
@ -383,6 +623,7 @@ public class AMQPFederationSource extends AMQPFederation {
throw new ActiveMQAMQPInternalErrorException("Error while configuring interal session metadata");
}
final AMQPFederationCommandDispatcher commandLink = new AMQPFederationCommandDispatcher(sender, getServer(), session.getSessionSPI());
final ProtonServerSenderContext senderContext =
new ProtonServerSenderContext(connection, sender, session, session.getSessionSPI(), commandLink);
@ -390,33 +631,13 @@ public class AMQPFederationSource extends AMQPFederation {
connected = true;
remoteQueueMatchPolicies.forEach((key, policy) -> {
try {
commandLink.sendPolicy(policy);
} catch (Exception e) {
brokerConnection.error(e);
}
});
remoteAddressMatchPolicies.forEach((key, policy) -> {
try {
commandLink.sendPolicy(policy);
} catch (Exception e) {
brokerConnection.error(e);
}
});
// Attempt to start the policy managers in another thread to avoid blocking the IO thread
scheduler.execute(() -> {
// Sync action with federation start / stop otherwise we could get out of sync
synchronized (AMQPFederationSource.this) {
if (isStarted()) {
queueMatchPolicies.forEach((k, v) -> v.start());
addressMatchPolicies.forEach((k, v) -> v.start());
}
}
});
// Setup events sender link to the target if there are any remote policies and
// then send those polices to start remote federation.
asyncCreateTargetEventsSender(commandLink);
// Setup events receiver link from the target if there are any local policies
// and then start the policy managers to begin tracking local demand.
asnycCreateTargetEventsReceiver();
} catch (Exception e) {
brokerConnection.error(e);
}

View File

@ -104,6 +104,22 @@ public class FederationReceiveFromQueuePolicy implements BiPredicate<String, Str
return transformerConfig;
}
public boolean testQueue(String queue) {
for (QueueMatcher matcher : excludeMatchers) {
if (matcher.testQueue(queue)) {
return false;
}
}
for (QueueMatcher matcher : includeMatchers) {
if (matcher.testQueue(queue)) {
return true;
}
}
return false;
}
@Override
public boolean test(String address, String queue) {
for (QueueMatcher matcher : excludeMatchers) {
@ -142,7 +158,15 @@ public class FederationReceiveFromQueuePolicy implements BiPredicate<String, Str
@Override
public boolean test(String address, String queue) {
return addressMatch.test(address) && queueMatch.test(queue);
return testAddress(address) && testQueue(queue);
}
public boolean testAddress(String address) {
return addressMatch.test(address);
}
public boolean testQueue(String queue) {
return queueMatch.test(queue);
}
}
}

View File

@ -23,7 +23,7 @@ import java.util.Set;
import org.apache.activemq.artemis.core.postoffice.Binding;
/**
* Am entry type class used to hold a {@link FederationConsumerInternal} and
* An entry type class used to hold a {@link FederationConsumerInternal} and
* any other state data needed by the manager that is creating them based on the
* policy configuration for the federation instance. The entry can be extended
* by federation implementation to hold additional state data for the federation

View File

@ -24,6 +24,7 @@ import java.util.Map;
import java.util.Objects;
import java.util.Set;
import org.apache.activemq.artemis.api.core.ActiveMQException;
import org.apache.activemq.artemis.api.core.RoutingType;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.core.postoffice.Binding;
import org.apache.activemq.artemis.core.postoffice.QueueBinding;
@ -390,6 +391,37 @@ public abstract class FederationAddressPolicyManager implements ActiveMQServerBi
}
}
/**
* Checks if the remote address added falls within the set of addresses that match the
* configured address policy and if so scans for local demand on that address to see
* if a new attempt to federate the address is needed.
*
* @param addressName
* The address that was added on the remote.
*
* @throws Exception if an error occurs while processing the address added event.
*/
public synchronized void afterRemoteAddressAdded(String addressName) throws Exception {
// Assume that the remote address that matched a previous federation attempt is MULTICAST
// so that we retry if current local state matches the policy and if it isn't we will once
// again record the federation attempt with the remote and but updated if the remote removes
// and adds the address again (hopefully with the correct routing type).
if (started && testIfAddressMatchesPolicy(addressName, RoutingType.MULTICAST) && !remoteConsumers.containsKey(addressName)) {
final SimpleString address = SimpleString.toSimpleString(addressName);
// Need to trigger check for all bindings that match to accumulate demand on the address
// if any and ensure an outgoing consumer is attempted.
server.getPostOffice()
.getDirectBindings(address)
.stream()
.filter(binding -> binding instanceof QueueBinding ||
(policy.isEnableDivertBindings() && binding instanceof DivertBinding))
.forEach(this::checkBindingForMatch);
}
}
/**
* Performs the test against the configured address policy to check if the target
* address is a match or not. A subclass can override this method and provide its
@ -404,6 +436,22 @@ public abstract class FederationAddressPolicyManager implements ActiveMQServerBi
return policy.test(addressInfo);
}
/**
* Performs the test against the configured address policy to check if the target
* address is a match or not. A subclass can override this method and provide its
* own match tests in combination with the configured matching policy.
*
* @param address
* The address that is being tested for a policy match.
* @param type
* The routing type of the address to test against the policy.
*
* @return <code>true</code> if the address given is a match against the policy.
*/
protected boolean testIfAddressMatchesPolicy(String address, RoutingType type) {
return policy.test(address, type);
}
/**
* Create a new {@link FederationConsumerInfo} based on the given {@link AddressInfo}
* and the configured {@link FederationReceiveFromAddressPolicy}. A subclass must override this

View File

@ -23,7 +23,7 @@ import java.util.Set;
import org.apache.activemq.artemis.core.server.ServerConsumer;
/**
* Am entry type class used to hold a {@link FederationConsumerInternal} and
* An entry type class used to hold a {@link FederationConsumerInternal} and
* any other state data needed by the manager that is creating them based on the
* policy configuration for the federation instance. The entry can be extended
* by federation implementation to hold additional state data for the federation
@ -77,16 +77,14 @@ public class FederationQueueEntry {
}
/**
* Reduce the known demand on the resource this entries consumer is associated with
* and returns true when demand reaches zero which indicates the consumer should be
* closed and the entry cleaned up.
* Remove the known demand on the resource from the given {@link ServerConsumer}.
*
* @param consumer
* The {@link ServerConsumer} that generated the demand on federated resource.
*
* @return this federation queue entry instance.
*/
public FederationQueueEntry reduceDemand(ServerConsumer consumer) {
public FederationQueueEntry removeDemand(ServerConsumer consumer) {
consumerDemand.remove(identifyConsumer(consumer));
return this;
}

View File

@ -115,7 +115,7 @@ public abstract class FederationQueuePolicyManager implements ActiveMQServerCons
return;
}
entry.reduceDemand(consumer);
entry.removeDemand(consumer);
logger.trace("Reducing demand on federated queue {}, remaining demand? {}", queueName, entry.hasDemand());
@ -204,6 +204,36 @@ public abstract class FederationQueuePolicyManager implements ActiveMQServerCons
}
}
/**
* Checks if the remote queue added falls within the set of queues that match the
* configured queue policy and if so scans for local demand on that queue to see
* if a new attempt to federate the queue is needed.
*
* @param addressName
* The address that was added on the remote.
* @param queueName
* The queue that was added on the remote.
*
* @throws Exception if an error occurs while processing the queue added event.
*/
public synchronized void afterRemoteQueueAdded(String addressName, String queueName) throws Exception {
// We ignore the remote address as locally the policy can be a wild card match and we can
// try to federate based on the Queue only, if the remote rejects the federation consumer
// binding again the request will once more be recorded and we will get another event if
// the queue were recreated such that a match could be made.
if (started && testIfQueueMatchesPolicy(queueName)) {
// Find a matching Queue with the given name and then check for demand based
// on the attached consumers and the current policy constraints.
server.getPostOffice()
.getAllBindings()
.filter(b -> b instanceof QueueBinding && ((QueueBinding) b).getQueue().getName().toString().equals(queueName))
.map(b -> (QueueBinding) b)
.forEach(b -> checkQueueForMatch(b.getQueue()));
}
}
/**
* Performs the test against the configured queue policy to check if the target
* queue and its associated address is a match or not. A subclass can override
@ -221,6 +251,21 @@ public abstract class FederationQueuePolicyManager implements ActiveMQServerCons
return policy.test(address, queueName);
}
/**
* Performs the test against the configured queue policy to check if the target
* queue minus its associated address is a match or not. A subclass can override
* this method and provide its own match tests in combination with the configured
* matching policy.
*
* @param queueName
* The name of the queue that is being tested for a policy match.
*
* @return <code>true</code> if the address given is a match against the policy.
*/
protected boolean testIfQueueMatchesPolicy(String queueName) {
return policy.testQueue(queueName);
}
/**
* Create a new {@link FederationConsumerInfo} based on the given {@link ServerConsumer}
* and the configured {@link FederationReceiveFromQueuePolicy}. A subclass can override this

View File

@ -111,4 +111,8 @@ public interface ActiveMQAMQPProtocolMessageBundle {
@Message(id = 119027, value = "Invalid AMQPConnection Remote State: {}")
ActiveMQException invalidAMQPConnectionState(Object state);
@Message(id = 119028, value = "Malformed Federation event message: {}")
ActiveMQException malformedFederationEventMessage(String message);
}

View File

@ -80,6 +80,7 @@ import java.lang.invoke.MethodHandles;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_ADDRESS_RECEIVER;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_CONTROL_LINK;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_CONTROL_LINK_VALIDATION_ADDRESS;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_EVENT_LINK;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_QUEUE_RECEIVER;
import static org.apache.activemq.artemis.protocol.amqp.proton.AmqpSupport.AMQP_LINK_INITIALIZER_KEY;
import static org.apache.activemq.artemis.protocol.amqp.proton.AmqpSupport.FAILOVER_SERVER_LIST;
@ -403,6 +404,8 @@ public class AMQPConnectionContext extends ProtonInitializable implements EventH
handleReplicaTargetLinkOpened(protonSession, receiver);
} else if (isFederationControlLink(receiver)) {
handleFederationControlLinkOpened(protonSession, receiver);
} else if (isFederationEventLink(receiver)) {
protonSession.addFederationEventProcessor(receiver);
} else {
protonSession.addReceiver(receiver);
}
@ -412,6 +415,8 @@ public class AMQPConnectionContext extends ProtonInitializable implements EventH
protonSession.addSender(sender, new AMQPFederationAddressSenderController(protonSession));
} else if (isFederationQueueReceiver(sender)) {
protonSession.addSender(sender, new AMQPFederationQueueSenderController(protonSession));
} else if (isFederationEventLink(sender)) {
protonSession.addFederationEventDispatcher(sender);
} else {
protonSession.addSender(sender);
}
@ -480,6 +485,14 @@ public class AMQPConnectionContext extends ProtonInitializable implements EventH
return verifyDesiredCapability(receiver, FEDERATION_CONTROL_LINK);
}
private static boolean isFederationEventLink(Sender sender) {
return verifyDesiredCapability(sender, FEDERATION_EVENT_LINK);
}
private static boolean isFederationEventLink(Receiver receiver) {
return verifyDesiredCapability(receiver, FEDERATION_EVENT_LINK);
}
private static boolean isFederationQueueReceiver(Sender sender) {
return verifyDesiredCapability(sender, FEDERATION_QUEUE_RECEIVER);
}

View File

@ -32,6 +32,8 @@ import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederation;
import org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationCommandProcessor;
import org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConfiguration;
import org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationEventDispatcher;
import org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationEventProcessor;
import org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationTarget;
import org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerSource;
import org.apache.activemq.artemis.protocol.amqp.connect.mirror.AMQPMirrorControllerTarget;
@ -197,20 +199,79 @@ public class AMQPSessionContext extends ProtonInitializable {
});
}
public void addFederationEventDispatcher(Sender sender) throws Exception {
addSender(sender, (c, s) -> {
final Connection protonConnection = sender.getSession().getConnection();
final org.apache.qpid.proton.engine.Record attachments = protonConnection.attachments();
final AMQPFederation federation = attachments.get(FEDERATION_INSTANCE_RECORD, AMQPFederation.class);
try {
if (federation == null) {
throw new ActiveMQAMQPIllegalStateException(
"Unexpected federation event processor opened on connection without a federation instance active");
}
final AMQPFederationEventDispatcher senderController =
new AMQPFederationEventDispatcher(federation, this.getSessionSPI(), sender);
return new ProtonServerSenderContext(connection, sender, this, this.getSessionSPI(), senderController);
} catch (ActiveMQException e) {
final ActiveMQAMQPException cause;
if (e instanceof ActiveMQAMQPException) {
cause = (ActiveMQAMQPException) e;
} else {
cause = new ActiveMQAMQPInternalErrorException(e.getMessage());
}
throw new RuntimeException(e.getMessage(), cause);
}
});
}
public void addSender(Sender sender) throws Exception {
addSender(sender, (SenderController)null);
addSender(sender, (c, s) -> {
return new ProtonServerSenderContext(connection, sender, this, sessionSPI, null);
});
}
public void addSender(Sender sender, SenderController senderController) throws Exception {
// TODO: Remove this check when we have support for global link names
boolean outgoing = (sender.getContext() != null && sender.getContext().equals(true));
ProtonServerSenderContext protonSender = outgoing ? new ProtonClientSenderContext(connection, sender, this, sessionSPI) : new ProtonServerSenderContext(connection, sender, this, sessionSPI, senderController);
addSender(sender, protonSender);
addSender(sender, (c, s) -> {
final boolean outgoing = (sender.getContext() != null && sender.getContext().equals(true));
final ProtonServerSenderContext protonSender = outgoing ?
new ProtonClientSenderContext(connection, sender, this, sessionSPI) :
new ProtonServerSenderContext(connection, sender, this, sessionSPI, senderController);
return protonSender;
});
}
public void addSender(Sender sender, ProtonServerSenderContext protonSender) throws Exception {
addSender(sender, (c, s) -> {
return protonSender;
});
}
@SuppressWarnings("unchecked")
public <T extends ProtonServerSenderContext> T addSender(Sender sender, BiFunction<AMQPSessionContext, Sender, T> senderBuilder) throws Exception {
ProtonServerSenderContext protonSender = null;
try {
try {
protonSender = senderBuilder.apply(this, sender);
} catch (RuntimeException e) {
if (e.getCause() instanceof ActiveMQAMQPException) {
throw (ActiveMQAMQPException) e.getCause();
} else if (e.getCause() != null) {
throw new ActiveMQAMQPInternalErrorException(e.getCause().getMessage(), e.getCause());
} else {
throw new ActiveMQAMQPInternalErrorException(e.getMessage(), e);
}
}
protonSender.initialize();
senders.put(sender, protonSender);
serverSenders.put(protonSender.getBrokerConsumer(), protonSender);
sender.setContext(protonSender);
@ -223,9 +284,11 @@ public class AMQPSessionContext extends ProtonInitializable {
}
protonSender.start();
return (T) protonSender;
} catch (ActiveMQAMQPException e) {
senders.remove(sender);
if (protonSender.getBrokerConsumer() != null) {
if (protonSender != null && protonSender.getBrokerConsumer() != null) {
serverSenders.remove(protonSender.getBrokerConsumer());
}
sender.setSource(null);
@ -234,6 +297,8 @@ public class AMQPSessionContext extends ProtonInitializable {
sender.close();
connection.flush();
});
return null;
}
}
@ -257,6 +322,36 @@ public class AMQPSessionContext extends ProtonInitializable {
});
}
public void addFederationEventProcessor(Receiver receiver) throws Exception {
addReceiver(receiver, (r, s) -> {
final Connection protonConnection = receiver.getSession().getConnection();
final org.apache.qpid.proton.engine.Record attachments = protonConnection.attachments();
final AMQPFederation federation = attachments.get(FEDERATION_INSTANCE_RECORD, AMQPFederation.class);
try {
if (federation == null) {
throw new ActiveMQAMQPIllegalStateException(
"Unexpected federation event processor opened on connection without a federation instance active");
}
final AMQPFederationEventProcessor eventsProcessor =
new AMQPFederationEventProcessor(federation, this, receiver);
return eventsProcessor;
} catch (ActiveMQException e) {
final ActiveMQAMQPException cause;
if (e instanceof ActiveMQAMQPException) {
cause = (ActiveMQAMQPException) e;
} else {
cause = new ActiveMQAMQPInternalErrorException(e.getMessage());
}
throw new RuntimeException(e.getMessage(), cause);
}
});
}
@SuppressWarnings("unchecked")
public void addFederationCommandProcessor(Receiver receiver) throws Exception {
addReceiver(receiver, (r, s) -> {

View File

@ -263,6 +263,7 @@ public class ProtonServerSenderContext extends ProtonInitializable implements Pr
connection.runNow(() -> {
sender.close();
controller.close(condition);
try {
sessionSPI.closeSender(brokerConsumer);
} catch (Exception e) {
@ -270,7 +271,6 @@ public class ProtonServerSenderContext extends ProtonInitializable implements Pr
} finally {
messageWriter.close();
}
sender.close();
connection.flush();
});
}

View File

@ -20,6 +20,7 @@ import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.core.server.Consumer;
import org.apache.activemq.artemis.core.server.MessageReference;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPLargeMessage;
import org.apache.qpid.proton.amqp.transport.ErrorCondition;
public interface SenderController {
@ -56,6 +57,18 @@ public interface SenderController {
*/
void close() throws Exception;
/**
* Called when the sender is being locally closed due to some error or forced
* shutdown due to resource deletion etc. The default implementation of this
* API does nothing in response to this call.
*
* @param error
* The error condition that triggered the close.
*/
default void close(ErrorCondition error) {
}
/**
* Controller selects a outgoing delivery writer that will handle the encoding and writing
* of the target {@link Message} carried in the given {@link MessageReference}. The selection

View File

@ -25,14 +25,18 @@ import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPF
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.ADDRESS_INCLUDES;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.ADDRESS_MAX_HOPS;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.ADD_ADDRESS_POLICY;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.EVENT_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_ADDRESS_RECEIVER;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_CONFIGURATION;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_CONTROL_LINK;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_EVENT_LINK;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.LARGE_MESSAGE_THRESHOLD;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.OPERATION_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.POLICY_NAME;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.RECEIVER_CREDITS;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.RECEIVER_CREDITS_LOW;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.REQUESTED_ADDRESS_ADDED;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.REQUESTED_ADDRESS_NAME;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.TRANSFORMER_CLASS_NAME;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.TRANSFORMER_PROPERTIES_MAP;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.POLICY_PROPERTIES_MAP;
@ -52,6 +56,7 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
@ -87,6 +92,7 @@ import org.apache.activemq.artemis.core.server.ComponentConfigurationRoutingType
import org.apache.activemq.artemis.core.server.Divert;
import org.apache.activemq.artemis.core.server.impl.AddressInfo;
import org.apache.activemq.artemis.core.server.transformer.Transformer;
import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
import org.apache.activemq.artemis.protocol.amqp.connect.federation.ActiveMQServerAMQPFederationPlugin;
import org.apache.activemq.artemis.protocol.amqp.federation.Federation;
@ -97,7 +103,10 @@ import org.apache.activemq.artemis.protocol.amqp.proton.AmqpSupport;
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.apache.qpid.proton.amqp.transport.AmqpError;
import org.apache.qpid.proton.amqp.transport.LinkError;
import org.apache.qpid.protonj2.test.driver.ProtonTestClient;
import org.apache.qpid.protonj2.test.driver.ProtonTestPeer;
import org.apache.qpid.protonj2.test.driver.ProtonTestServer;
import org.apache.qpid.protonj2.test.driver.matchers.messaging.HeaderMatcher;
import org.apache.qpid.protonj2.test.driver.matchers.messaging.MessageAnnotationsMatcher;
@ -142,6 +151,14 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withSenderSettleModeSettled()
.withSource().withDynamic(true)
.and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind()
.withTarget().withAddress("test-dynamic-events");
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -230,6 +247,13 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withSource().withDynamic(true)
.and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind()
.withTarget().withAddress("test-dynamic-events");
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -296,6 +320,13 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withSource().withDynamic(true)
.and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind()
.withTarget().withAddress("test-dynamic-events");
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -364,6 +395,13 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withSource().withDynamic(true)
.and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind()
.withTarget().withAddress("test-dynamic-events");
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -473,6 +511,13 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withSource().withDynamic(true)
.and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind()
.withTarget().withAddress("test-dynamic-events");
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -545,6 +590,13 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withSource().withDynamic(true)
.and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind()
.withTarget().withAddress("test-dynamic-events");
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -624,6 +676,13 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withSource().withDynamic(true)
.and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind()
.withTarget().withAddress("test-dynamic-events");
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -697,6 +756,10 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -780,6 +843,10 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -858,6 +925,10 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -944,6 +1015,10 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -1024,6 +1099,10 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -1109,6 +1188,10 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -1196,6 +1279,10 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -1274,6 +1361,10 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -1362,6 +1453,10 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -1989,6 +2084,10 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -2069,6 +2168,10 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -2334,6 +2437,10 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_ADDRESS_RECEIVER.toString())
.withName(allOf(containsString("sample-federation"),
@ -2446,6 +2553,10 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_ADDRESS_RECEIVER.toString())
.withName(allOf(containsString("sample-federation"),
@ -2555,6 +2666,10 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_ADDRESS_RECEIVER.toString())
.withName(allOf(containsString("sample-federation"),
@ -2609,6 +2724,10 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -2692,6 +2811,491 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
}
}
@Test(timeout = 20000)
public void testFederationCreatesEventSenderAndReceiverWhenLocalAndRemotePoliciesAdded() throws Exception {
final MessageAnnotationsMatcher maMatcher = new MessageAnnotationsMatcher(true);
maMatcher.withEntry(OPERATION_TYPE.toString(), Matchers.is(ADD_ADDRESS_POLICY));
final Map<String, Object> policyMap = new LinkedHashMap<>();
final List<String> includes = new ArrayList<>();
includes.add("test");
policyMap.put(POLICY_NAME, "remote-address-policy");
policyMap.put(ADDRESS_AUTO_DELETE, false);
policyMap.put(ADDRESS_AUTO_DELETE_DELAY, -1L);
policyMap.put(ADDRESS_AUTO_DELETE_MSG_COUNT, -1L);
policyMap.put(ADDRESS_MAX_HOPS, 5);
policyMap.put(ADDRESS_ENABLE_DIVERT_BINDINGS, false);
policyMap.put(ADDRESS_INCLUDES, includes);
final EncodedAmqpValueMatcher bodyMatcher = new EncodedAmqpValueMatcher(policyMap);
final TransferPayloadCompositeMatcher payloadMatcher = new TransferPayloadCompositeMatcher();
payloadMatcher.setMessageAnnotationsMatcher(maMatcher);
payloadMatcher.addMessageContentMatcher(bodyMatcher);
try (ProtonTestServer peer = new ProtonTestServer()) {
peer.expectSASLAnonymousConnect();
peer.expectOpen().respond();
peer.expectBegin().respond();
peer.expectAttach().ofSender()
.withHandle(0)
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofSender()
.withTarget().withDynamic(true)
.and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind()
.withTarget().withAddress("test-dynamic-events-sender");
peer.remoteFlow().withLinkCredit(10).queue();
peer.expectAttach().ofReceiver()
.withSource().withDynamic(true)
.and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind()
.withSource().withAddress("test-dynamic-events-receiver");
peer.expectFlow().withLinkCredit(10);
peer.remoteFlow().withLinkCredit(10).withHandle(0).queue(); // Give control link credit now to ensure ordering
peer.expectTransfer().withPayload(payloadMatcher); // Remote policy
peer.start();
final URI remoteURI = peer.getServerURI();
logger.info("Test started, peer listening on: {}", remoteURI);
final AMQPFederationAddressPolicyElement localReceiveFromAddress = new AMQPFederationAddressPolicyElement();
localReceiveFromAddress.setName("address-policy");
localReceiveFromAddress.addToIncludes("test");
localReceiveFromAddress.setAutoDelete(false);
localReceiveFromAddress.setAutoDeleteDelay(-1L);
localReceiveFromAddress.setAutoDeleteMessageCount(-1L);
final AMQPFederationAddressPolicyElement remoteReceiveFromAddress = new AMQPFederationAddressPolicyElement();
remoteReceiveFromAddress.setName("remote-address-policy");
remoteReceiveFromAddress.addToIncludes("test");
remoteReceiveFromAddress.setAutoDelete(false);
remoteReceiveFromAddress.setAutoDeleteDelay(-1L);
remoteReceiveFromAddress.setAutoDeleteMessageCount(-1L);
remoteReceiveFromAddress.setMaxHops(5);
final AMQPFederatedBrokerConnectionElement element = new AMQPFederatedBrokerConnectionElement();
element.setName("sample-federation");
element.addLocalAddressPolicy(localReceiveFromAddress);
element.addRemoteAddressPolicy(remoteReceiveFromAddress);
final AMQPBrokerConnectConfiguration amqpConnection =
new AMQPBrokerConnectConfiguration("test-address-federation", "tcp://" + remoteURI.getHost() + ":" + remoteURI.getPort());
amqpConnection.setReconnectAttempts(0);// No reconnects
amqpConnection.addElement(element);
server.getConfiguration().addAMQPConnection(amqpConnection);
server.start();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
}
}
@Test(timeout = 20000)
public void testFederationSendsRemotePolicyIfEventsSenderLinkRejected() throws Exception {
final MessageAnnotationsMatcher maMatcher = new MessageAnnotationsMatcher(true);
maMatcher.withEntry(OPERATION_TYPE.toString(), Matchers.is(ADD_ADDRESS_POLICY));
final Map<String, Object> policyMap = new LinkedHashMap<>();
final List<String> includes = new ArrayList<>();
includes.add("test");
policyMap.put(POLICY_NAME, "remote-address-policy");
policyMap.put(ADDRESS_AUTO_DELETE, false);
policyMap.put(ADDRESS_AUTO_DELETE_DELAY, -1L);
policyMap.put(ADDRESS_AUTO_DELETE_MSG_COUNT, -1L);
policyMap.put(ADDRESS_MAX_HOPS, 5);
policyMap.put(ADDRESS_ENABLE_DIVERT_BINDINGS, false);
policyMap.put(ADDRESS_INCLUDES, includes);
final EncodedAmqpValueMatcher bodyMatcher = new EncodedAmqpValueMatcher(policyMap);
final TransferPayloadCompositeMatcher payloadMatcher = new TransferPayloadCompositeMatcher();
payloadMatcher.setMessageAnnotationsMatcher(maMatcher);
payloadMatcher.addMessageContentMatcher(bodyMatcher);
try (ProtonTestServer peer = new ProtonTestServer()) {
peer.expectSASLAnonymousConnect();
peer.expectOpen().respond();
peer.expectBegin().respond();
peer.expectAttach().ofSender()
.withHandle(0)
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofSender()
.withTarget().withDynamic(true)
.and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.reject(true, LinkError.DETACH_FORCED.toString(), "Unknown error");
peer.expectDetach();
peer.remoteFlow().withLinkCredit(10).withHandle(0).queue(); // Give control link credit now to ensure ordering
peer.expectTransfer().withPayload(payloadMatcher); // Remote policy
peer.start();
final URI remoteURI = peer.getServerURI();
logger.info("Test started, peer listening on: {}", remoteURI);
final AMQPFederationAddressPolicyElement remoteReceiveFromAddress = new AMQPFederationAddressPolicyElement();
remoteReceiveFromAddress.setName("remote-address-policy");
remoteReceiveFromAddress.addToIncludes("test");
remoteReceiveFromAddress.setAutoDelete(false);
remoteReceiveFromAddress.setAutoDeleteDelay(-1L);
remoteReceiveFromAddress.setAutoDeleteMessageCount(-1L);
remoteReceiveFromAddress.setMaxHops(5);
final AMQPFederatedBrokerConnectionElement element = new AMQPFederatedBrokerConnectionElement();
element.setName("sample-federation");
element.addRemoteAddressPolicy(remoteReceiveFromAddress);
final AMQPBrokerConnectConfiguration amqpConnection =
new AMQPBrokerConnectConfiguration("test-address-federation", "tcp://" + remoteURI.getHost() + ":" + remoteURI.getPort());
amqpConnection.setReconnectAttempts(0);// No reconnects
amqpConnection.addElement(element);
server.getConfiguration().addAMQPConnection(amqpConnection);
server.start();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
}
}
@Test(timeout = 20000)
public void testRemoteBrokerSendsAddressAddedEventForInterestedPeer() throws Exception {
final AddressSettings addressSettings = new AddressSettings();
addressSettings.setAutoCreateQueues(false);
addressSettings.setAutoCreateAddresses(false);
server.getConfiguration().getAddressSettings().put("#", addressSettings);
server.start();
final Map<String, Object> remoteSourceProperties = new HashMap<>();
remoteSourceProperties.put(ADDRESS_AUTO_DELETE, true);
remoteSourceProperties.put(ADDRESS_AUTO_DELETE_DELAY, 10_000L);
remoteSourceProperties.put(ADDRESS_AUTO_DELETE_MSG_COUNT, 1L);
final MessageAnnotationsMatcher maMatcher = new MessageAnnotationsMatcher(true);
maMatcher.withEntry(EVENT_TYPE.toString(), Matchers.is(REQUESTED_ADDRESS_ADDED));
final Map<String, Object> eventMap = new LinkedHashMap<>();
eventMap.put(REQUESTED_ADDRESS_NAME, "test");
final EncodedAmqpValueMatcher bodyMatcher = new EncodedAmqpValueMatcher(eventMap);
final TransferPayloadCompositeMatcher payloadMatcher = new TransferPayloadCompositeMatcher();
payloadMatcher.setMessageAnnotationsMatcher(maMatcher);
payloadMatcher.addMessageContentMatcher(bodyMatcher);
try (ProtonTestClient peer = new ProtonTestClient()) {
scriptFederationConnectToRemote(peer, "test", false, true);
peer.connect("localhost", AMQP_PORT);
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectAttach().ofSender().withName("federation-address-receiver")
.withOfferedCapabilities(FEDERATION_ADDRESS_RECEIVER.toString())
.withTarget().also()
.withNullSource();
peer.expectDetach().respond();
// Connect to remote as if an queue had demand and matched our federation policy
peer.remoteAttach().ofReceiver()
.withDesiredCapabilities(FEDERATION_ADDRESS_RECEIVER.toString())
.withName("federation-address-receiver")
.withSenderSettleModeUnsettled()
.withReceivervSettlesFirst()
.withProperty(FEDERATED_ADDRESS_SOURCE_PROPERTIES.toString(), remoteSourceProperties)
.withSource().withDurabilityOfNone()
.withExpiryPolicyOnLinkDetach()
.withAddress("test")
.withCapabilities("topic")
.and()
.withTarget().and()
.now();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectTransfer().withPayload(payloadMatcher).accept(); // Address added event
// Manually add the address and a queue binding to create local demand.
server.addAddressInfo(new AddressInfo(SimpleString.toSimpleString("test"), RoutingType.MULTICAST));
server.createQueue(new QueueConfiguration("test").setRoutingType(RoutingType.MULTICAST)
.setAddress("test")
.setAutoCreated(false));
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectClose();
peer.remoteClose().now();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
server.stop();
}
}
@Test(timeout = 20000)
public void testFederationCreatesAddressReceiverInResponseToAddressAddedEvent() throws Exception {
try (ProtonTestServer peer = new ProtonTestServer()) {
peer.expectSASLAnonymousConnect();
peer.expectOpen().respond();
peer.expectBegin().respond();
peer.expectAttach().ofSender()
.withHandle(0)
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.remoteFlow().withLinkCredit(10);
peer.expectAttach().ofReceiver()
.withHandle(1)
.withSenderSettleModeSettled()
.withSource().withDynamic(true)
.and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind()
.withTarget().withAddress("test-dynamic-events");
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
logger.info("Test started, peer listening on: {}", remoteURI);
final AMQPFederationAddressPolicyElement receiveFromAddress = new AMQPFederationAddressPolicyElement();
receiveFromAddress.setName("address-policy");
receiveFromAddress.addToIncludes("test");
receiveFromAddress.setAutoDelete(false);
receiveFromAddress.setAutoDeleteDelay(-1L);
receiveFromAddress.setAutoDeleteMessageCount(-1L);
final AMQPFederatedBrokerConnectionElement element = new AMQPFederatedBrokerConnectionElement();
element.setName("sample-federation");
element.addLocalAddressPolicy(receiveFromAddress);
final AMQPBrokerConnectConfiguration amqpConnection =
new AMQPBrokerConnectConfiguration("test-address-federation", "tcp://" + remoteURI.getHost() + ":" + remoteURI.getPort());
amqpConnection.setReconnectAttempts(0);// No reconnects
amqpConnection.addElement(element);
server.getConfiguration().addAMQPConnection(amqpConnection);
server.start();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
final Map<String, Object> expectedSourceProperties = new HashMap<>();
expectedSourceProperties.put(ADDRESS_AUTO_DELETE, false);
expectedSourceProperties.put(ADDRESS_AUTO_DELETE_DELAY, -1L);
expectedSourceProperties.put(ADDRESS_AUTO_DELETE_MSG_COUNT, -1L);
// Reject the initial attempt
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_ADDRESS_RECEIVER.toString())
.withName(allOf(containsString("sample-federation"),
containsString("test"),
containsString("address-receiver"),
containsString(server.getNodeID().toString())))
.withProperty(FEDERATED_ADDRESS_SOURCE_PROPERTIES.toString(), expectedSourceProperties)
.respond()
.withNullSource()
.withOfferedCapabilities(FEDERATION_ADDRESS_RECEIVER.toString());
peer.remoteDetach().withClosed(true)
.withErrorCondition(AmqpError.NOT_FOUND.toString(), "Address not found")
.queue();
peer.expectFlow();
peer.expectDetach();
server.createQueue(new QueueConfiguration("test").setRoutingType(RoutingType.MULTICAST)
.setAddress("test")
.setAutoCreated(false));
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_ADDRESS_RECEIVER.toString())
.withName(allOf(containsString("sample-federation"),
containsString("test"),
containsString("address-receiver"),
containsString(server.getNodeID().toString())))
.withProperty(FEDERATED_ADDRESS_SOURCE_PROPERTIES.toString(), expectedSourceProperties)
.respond()
.withOfferedCapabilities(FEDERATION_ADDRESS_RECEIVER.toString());
peer.expectFlow().withLinkCredit(1000);
// Should not trigger attach of a federation receiver as the address doesn't match the policy..
sendAddressAddedEvent(peer, "target", 1, 0);
// Should trigger attach of federation receiver again for the test address.
sendAddressAddedEvent(peer, "test", 1, 1);
// Should not trigger attach of federation receiver as there already is one on this address
sendAddressAddedEvent(peer, "test", 1, 2);
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
}
}
@Test(timeout = 20000)
public void testAddressAddedEventIgnoredIfFederationConsumerAlreadyCreated() throws Exception {
try (ProtonTestServer peer = new ProtonTestServer()) {
peer.expectSASLAnonymousConnect();
peer.expectOpen().respond();
peer.expectBegin().respond();
peer.expectAttach().ofSender()
.withHandle(0)
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.remoteFlow().withLinkCredit(10);
peer.expectAttach().ofReceiver()
.withHandle(1)
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
logger.info("Test started, peer listening on: {}", remoteURI);
final AMQPFederationAddressPolicyElement receiveFromAddress = new AMQPFederationAddressPolicyElement();
receiveFromAddress.setName("address-policy");
receiveFromAddress.addToIncludes("test");
final AMQPFederatedBrokerConnectionElement element = new AMQPFederatedBrokerConnectionElement();
element.setName("sample-federation");
element.addLocalAddressPolicy(receiveFromAddress);
final AMQPBrokerConnectConfiguration amqpConnection =
new AMQPBrokerConnectConfiguration("test-address-federation", "tcp://" + remoteURI.getHost() + ":" + remoteURI.getPort());
amqpConnection.setReconnectAttempts(0);// No reconnects
amqpConnection.addElement(element);
server.getConfiguration().addAMQPConnection(amqpConnection);
server.start();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
// Reject the initial attempt
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_ADDRESS_RECEIVER.toString())
.respond()
.withNullSource()
.withOfferedCapabilities(FEDERATION_ADDRESS_RECEIVER.toString());
peer.remoteDetach().withClosed(true)
.withErrorCondition(AmqpError.NOT_FOUND.toString(), "Address not found")
.queue();
peer.expectFlow();
peer.expectDetach();
// Triggers the initial attach based on demand.
server.createQueue(new QueueConfiguration("test").setRoutingType(RoutingType.MULTICAST)
.setAddress("test")
.setAutoCreated(false));
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_ADDRESS_RECEIVER.toString())
.respond()
.withOfferedCapabilities(FEDERATION_ADDRESS_RECEIVER.toString());
peer.expectFlow().withLinkCredit(1000);
final ConnectionFactory factory = CFUtil.createConnectionFactory("AMQP", "tcp://localhost:" + AMQP_PORT);
final Connection connection = factory.createConnection();
final Session session = connection.createSession(Session.AUTO_ACKNOWLEDGE);
// Create demand on the Address to kick off another federation attempt.
session.createConsumer(session.createTopic("test"));
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
// Should not trigger attach of federation receiver as there already is one on this address
sendAddressAddedEvent(peer, "test", 1, 0);
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
}
}
@Test(timeout = 20000)
public void testRemoteBrokerClosesFederationReceiverAfterAddressRemoved() throws Exception {
server.start();
server.addAddressInfo(new AddressInfo(SimpleString.toSimpleString("test"), RoutingType.MULTICAST));
try (ProtonTestClient peer = new ProtonTestClient()) {
scriptFederationConnectToRemote(peer, "test", true, true);
peer.connect("localhost", AMQP_PORT);
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectAttach().ofSender().withName("federation-address-receiver")
.withOfferedCapabilities(FEDERATION_ADDRESS_RECEIVER.toString())
.withSource().withAddress("test");
// Connect to remote as if an queue had demand and matched our federation policy
peer.remoteAttach().ofReceiver()
.withDesiredCapabilities(FEDERATION_ADDRESS_RECEIVER.toString())
.withName("federation-address-receiver")
.withSenderSettleModeUnsettled()
.withReceivervSettlesFirst()
.withSource().withDurabilityOfNone()
.withExpiryPolicyOnLinkDetach()
.withAddress("test")
.withCapabilities("topic")
.and()
.withTarget().and()
.now();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectDetach().withError(AmqpError.RESOURCE_DELETED.toString());
// Force remove consumers from the address should indicate the resource was deleted.
server.removeAddressInfo(SimpleString.toSimpleString("test"), null, true);
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
final MessageAnnotationsMatcher maMatcher = new MessageAnnotationsMatcher(true);
maMatcher.withEntry(EVENT_TYPE.toString(), Matchers.is(REQUESTED_ADDRESS_ADDED));
final Map<String, Object> eventMap = new LinkedHashMap<>();
eventMap.put(REQUESTED_ADDRESS_NAME, "test");
final EncodedAmqpValueMatcher bodyMatcher = new EncodedAmqpValueMatcher(eventMap);
final TransferPayloadCompositeMatcher payloadMatcher = new TransferPayloadCompositeMatcher();
payloadMatcher.setMessageAnnotationsMatcher(maMatcher);
payloadMatcher.addMessageContentMatcher(bodyMatcher);
// Server alerts the federation event receiver that a previously federated address
// has been added once more and it could restore the previous federation state.
peer.expectTransfer().withPayload(payloadMatcher).withSettled(true);
server.addAddressInfo(new AddressInfo(SimpleString.toSimpleString("test"), RoutingType.MULTICAST));
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
// This time removing and restoring should generate no traffic as there was not
// another federation receiver added.
server.removeAddressInfo(SimpleString.toSimpleString("test"), null, true);
server.addAddressInfo(new AddressInfo(SimpleString.toSimpleString("test"), RoutingType.MULTICAST));
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectClose();
peer.remoteClose().now();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
server.stop();
}
}
private static void sendAddressAddedEvent(ProtonTestPeer peer, String address, int handle, int deliveryId) {
final Map<String, Object> eventMap = new LinkedHashMap<>();
eventMap.put(REQUESTED_ADDRESS_NAME, address);
// Should not trigger attach of federation receiver as there already is one on this address
peer.remoteTransfer().withHandle(handle)
.withDeliveryId(deliveryId)
.withSettled(true)
.withMessageAnnotations().withAnnotation(EVENT_TYPE.toString(), REQUESTED_ADDRESS_ADDED)
.also()
.withBody().withValue(eventMap)
.also()
.now();
}
public static class ApplicationPropertiesTransformer implements Transformer {
private final Map<String, String> properties = new HashMap<>();
@ -2718,7 +3322,7 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
}
}
private void sendAddresPolicyToRemote(ProtonTestClient peer, FederationReceiveFromAddressPolicy policy) {
private static void sendAddresPolicyToRemote(ProtonTestClient peer, FederationReceiveFromAddressPolicy policy) {
final Map<String, Object> policyMap = new LinkedHashMap<>();
policyMap.put(POLICY_NAME, policy.getPolicyName());
@ -2758,11 +3362,19 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
// Use this method to script the initial handshake that a broker that is establishing
// a federation connection with a remote broker instance would perform.
private void scriptFederationConnectToRemote(ProtonTestClient peer, String federationName) {
private static void scriptFederationConnectToRemote(ProtonTestClient peer, String federationName) {
scriptFederationConnectToRemote(peer, federationName, AmqpSupport.AMQP_CREDITS_DEFAULT, AmqpSupport.AMQP_LOW_CREDITS_DEFAULT);
}
private void scriptFederationConnectToRemote(ProtonTestClient peer, String federationName, int amqpCredits, int amqpLowCredits) {
private static void scriptFederationConnectToRemote(ProtonTestClient peer, String federationName, int amqpCredits, int amqpLowCredits) {
scriptFederationConnectToRemote(peer, federationName, amqpCredits, amqpLowCredits, false, false);
}
private static void scriptFederationConnectToRemote(ProtonTestClient peer, String federationName, boolean eventsSender, boolean eventsReceiver) {
scriptFederationConnectToRemote(peer, federationName, AmqpSupport.AMQP_CREDITS_DEFAULT, AmqpSupport.AMQP_LOW_CREDITS_DEFAULT, eventsSender, eventsReceiver);
}
private static void scriptFederationConnectToRemote(ProtonTestClient peer, String federationName, int amqpCredits, int amqpLowCredits, boolean eventsSender, boolean eventsReceiver ) {
final String federationControlLinkName = "Federation:control:" + UUID.randomUUID().toString();
final Map<String, Object> federationConfiguration = new HashMap<>();
@ -2798,6 +3410,58 @@ public class AMQPFederationAddressPolicyTest extends AmqpClientTestSupport {
.also()
.withOfferedCapability(FEDERATION_CONTROL_LINK.toString());
peer.expectFlow();
// Sender created when there are remote policies to send to the target
if (eventsSender) {
final String federationEventsSenderLinkName = "Federation:events-sender:test:" + UUID.randomUUID().toString();
peer.remoteAttach().ofSender()
.withName(federationEventsSenderLinkName)
.withDesiredCapabilities(FEDERATION_EVENT_LINK.toString())
.withSenderSettleModeSettled()
.withReceivervSettlesFirst()
.withSource().also()
.withTarget().withDynamic(true)
.withDurabilityOfNone()
.withExpiryPolicyOnLinkDetach()
.withLifetimePolicyOfDeleteOnClose()
.withCapabilities("temporary-topic")
.also()
.queue();
peer.expectAttach().ofReceiver()
.withName(federationEventsSenderLinkName)
.withTarget()
.withAddress(notNullValue())
.also()
.withOfferedCapability(FEDERATION_EVENT_LINK.toString());
peer.expectFlow();
}
// Receiver created when there are local policies on the source.
if (eventsReceiver) {
final String federationEventsSenderLinkName = "Federation:events-receiver:test:" + UUID.randomUUID().toString();
peer.remoteAttach().ofReceiver()
.withName(federationEventsSenderLinkName)
.withDesiredCapabilities(FEDERATION_EVENT_LINK.toString())
.withSenderSettleModeSettled()
.withReceivervSettlesFirst()
.withTarget().also()
.withSource().withDynamic(true)
.withDurabilityOfNone()
.withExpiryPolicyOnLinkDetach()
.withLifetimePolicyOfDeleteOnClose()
.withCapabilities("temporary-topic")
.also()
.queue();
peer.remoteFlow().withLinkCredit(10).queue();
peer.expectAttach().ofSender()
.withName(federationEventsSenderLinkName)
.withSource()
.withAddress(notNullValue())
.also()
.withOfferedCapability(FEDERATION_EVENT_LINK.toString());
}
}
private class AMQPTestFederationBrokerPlugin implements ActiveMQServerAMQPFederationPlugin {

View File

@ -22,6 +22,7 @@ import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPF
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.ADDRESS_AUTO_DELETE_MSG_COUNT;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_ADDRESS_RECEIVER;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_CONTROL_LINK;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_EVENT_LINK;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_QUEUE_RECEIVER;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_RECEIVER_PRIORITY;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationPolicySupport.DEFAULT_QUEUE_RECEIVER_PRIORITY_ADJUSTMENT;
@ -98,6 +99,10 @@ public class AMQPFederationConfigurationReloadTest extends AmqpClientTestSupport
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -193,6 +198,10 @@ public class AMQPFederationConfigurationReloadTest extends AmqpClientTestSupport
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -251,6 +260,10 @@ public class AMQPFederationConfigurationReloadTest extends AmqpClientTestSupport
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer2.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer2.expectFlow().withLinkCredit(10);
peer2.start();
final URI remoteURI2 = peer2.getServerURI();
@ -313,6 +326,10 @@ public class AMQPFederationConfigurationReloadTest extends AmqpClientTestSupport
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -398,6 +415,10 @@ public class AMQPFederationConfigurationReloadTest extends AmqpClientTestSupport
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -460,6 +481,10 @@ public class AMQPFederationConfigurationReloadTest extends AmqpClientTestSupport
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_QUEUE_RECEIVER.toString())
.withName(allOf(containsString("sample-federation"),

View File

@ -29,6 +29,7 @@ import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPF
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_CONFIGURATION;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_CONTROL_LINK;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_CONTROL_LINK_VALIDATION_ADDRESS;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_EVENT_LINK;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.LARGE_MESSAGE_THRESHOLD;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.LINK_ATTACH_TIMEOUT;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.OPERATION_TYPE;
@ -62,6 +63,7 @@ import org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederati
import org.apache.activemq.artemis.protocol.amqp.proton.AmqpSupport;
import org.apache.activemq.artemis.tests.integration.amqp.AmqpClientTestSupport;
import org.apache.activemq.artemis.utils.Wait;
import org.apache.qpid.proton.amqp.transport.AmqpError;
import org.apache.qpid.protonj2.test.driver.ProtonTestClient;
import org.apache.qpid.protonj2.test.driver.ProtonTestServer;
import org.apache.qpid.protonj2.test.driver.matchers.messaging.MessageAnnotationsMatcher;
@ -290,6 +292,86 @@ public class AMQPFederationConnectTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind().withTarget().withAddress("test-dynamic");
peer.remoteFlow().withLinkCredit(10).queue();
peer.expectAttach().ofSender()
.withTarget().withDynamic(true).and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind().withTarget().withAddress("test-dynamic-events");
peer.remoteFlow().withLinkCredit(10).queue();
peer.expectTransfer().withPayload(payloadMatcher);
peer.start();
final URI remoteURI = peer.getServerURI();
logger.info("Connect test started, peer listening on: {}", remoteURI);
final AMQPFederationQueuePolicyElement sendToQueue = new AMQPFederationQueuePolicyElement();
sendToQueue.setName("test-policy");
sendToQueue.setIncludeFederated(true);
sendToQueue.setPriorityAdjustment(42);
sendToQueue.addToIncludes("a", "b");
sendToQueue.addToIncludes("c", "d");
sendToQueue.addToExcludes("e", "f");
sendToQueue.addToExcludes("g", "h");
final AMQPFederatedBrokerConnectionElement element = new AMQPFederatedBrokerConnectionElement();
element.setName("test");
element.addRemoteQueuePolicy(sendToQueue);
final AMQPBrokerConnectConfiguration amqpConnection =
new AMQPBrokerConnectConfiguration("testSimpleConnect", "tcp://" + remoteURI.getHost() + ":" + remoteURI.getPort());
amqpConnection.setReconnectAttempts(0);// No reconnects
amqpConnection.addElement(element);
server.getConfiguration().addAMQPConnection(amqpConnection);
server.start();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
}
}
@Test(timeout = 20000)
public void testFederationSendsReceiveFromQueuePolicyToRemoteWhenSendToIsConfiguredAndEventSenderRejected() throws Exception {
final MessageAnnotationsMatcher maMatcher = new MessageAnnotationsMatcher(true);
maMatcher.withEntry(OPERATION_TYPE.toString(), Matchers.is(ADD_QUEUE_POLICY));
final Map<String, Object> policyMap = new LinkedHashMap<>();
final List<String> includes = new ArrayList<>();
includes.add("a");
includes.add("b");
includes.add("c");
includes.add("d");
final List<String> excludes = new ArrayList<>();
excludes.add("e");
excludes.add("f");
excludes.add("g");
excludes.add("h");
policyMap.put(POLICY_NAME, "test-policy");
policyMap.put(QUEUE_INCLUDE_FEDERATED, true);
policyMap.put(QUEUE_PRIORITY_ADJUSTMENT, 42);
policyMap.put(QUEUE_INCLUDES, includes);
policyMap.put(QUEUE_EXCLUDES, excludes);
final EncodedAmqpValueMatcher bodyMatcher = new EncodedAmqpValueMatcher(policyMap);
final TransferPayloadCompositeMatcher payloadMatcher = new TransferPayloadCompositeMatcher();
payloadMatcher.setMessageAnnotationsMatcher(maMatcher);
payloadMatcher.addMessageContentMatcher(bodyMatcher);
try (ProtonTestServer peer = new ProtonTestServer()) {
peer.expectSASLAnonymousConnect();
peer.expectOpen().respond();
peer.expectBegin().respond();
peer.expectAttach().ofSender()
.withTarget().withDynamic(true).and()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind().withTarget().withAddress("test-dynamic");
peer.expectAttach().ofSender()
.withTarget().withDynamic(true).and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respond()
.withNullSource();
peer.expectDetach().respond();
peer.remoteFlow().withHandle(0).withLinkCredit(10).queue(); // Ensure order of events
peer.expectTransfer().withPayload(payloadMatcher);
peer.start();
@ -356,6 +438,84 @@ public class AMQPFederationConnectTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind().withTarget().withAddress("test-dynamic");
peer.remoteFlow().withLinkCredit(10).queue();
peer.expectAttach().ofSender()
.withTarget().withDynamic(true).and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind().withTarget().withAddress("test-dynamic-events");
peer.remoteFlow().withLinkCredit(10).queue();
peer.expectTransfer().withPayload(payloadMatcher);
peer.start();
final URI remoteURI = peer.getServerURI();
logger.info("Connect test started, peer listening on: {}", remoteURI);
final AMQPFederationAddressPolicyElement sendToAddress = new AMQPFederationAddressPolicyElement();
sendToAddress.setName("test-policy");
sendToAddress.setAutoDelete(true);
sendToAddress.setAutoDeleteDelay(42L);
sendToAddress.setAutoDeleteMessageCount(314L);
sendToAddress.setMaxHops(5);
sendToAddress.setEnableDivertBindings(false);
sendToAddress.addToIncludes("include");
sendToAddress.addToExcludes("exclude");
final AMQPFederatedBrokerConnectionElement element = new AMQPFederatedBrokerConnectionElement();
element.setName("test");
element.addRemoteAddressPolicy(sendToAddress);
final AMQPBrokerConnectConfiguration amqpConnection =
new AMQPBrokerConnectConfiguration("test-send-policy", "tcp://" + remoteURI.getHost() + ":" + remoteURI.getPort());
amqpConnection.setReconnectAttempts(0);// No reconnects
amqpConnection.addElement(element);
server.getConfiguration().addAMQPConnection(amqpConnection);
server.start();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
}
}
@Test(timeout = 20000)
public void testFederationSendsReceiveFromAddressPolicyToRemoteWhenSendToIsConfiguredAndEventSenderRejected() throws Exception {
final MessageAnnotationsMatcher maMatcher = new MessageAnnotationsMatcher(true);
maMatcher.withEntry(OPERATION_TYPE.toString(), Matchers.is(ADD_ADDRESS_POLICY));
final Map<String, Object> policyMap = new LinkedHashMap<>();
final List<String> includes = new ArrayList<>();
includes.add("include");
final List<String> excludes = new ArrayList<>();
excludes.add("exclude");
policyMap.put(POLICY_NAME, "test-policy");
policyMap.put(ADDRESS_AUTO_DELETE, true);
policyMap.put(ADDRESS_AUTO_DELETE_DELAY, 42L);
policyMap.put(ADDRESS_AUTO_DELETE_MSG_COUNT, 314L);
policyMap.put(ADDRESS_MAX_HOPS, 5);
policyMap.put(ADDRESS_ENABLE_DIVERT_BINDINGS, false);
policyMap.put(ADDRESS_INCLUDES, includes);
policyMap.put(ADDRESS_EXCLUDES, excludes);
final EncodedAmqpValueMatcher bodyMatcher = new EncodedAmqpValueMatcher(policyMap);
final TransferPayloadCompositeMatcher payloadMatcher = new TransferPayloadCompositeMatcher();
payloadMatcher.setMessageAnnotationsMatcher(maMatcher);
payloadMatcher.addMessageContentMatcher(bodyMatcher);
try (ProtonTestServer peer = new ProtonTestServer()) {
peer.expectSASLAnonymousConnect();
peer.expectOpen().respond();
peer.expectBegin().respond();
peer.expectAttach().ofSender()
.withTarget().withDynamic(true).and()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind().withTarget().withAddress("test-dynamic");
peer.expectAttach().ofSender()
.withTarget().withDynamic(true).and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respond()
.withNullTarget();
peer.expectDetach().respond();
peer.remoteFlow().withHandle(0).withLinkCredit(10).queue(); // Ensure order of events
peer.expectTransfer().withPayload(payloadMatcher);
peer.start();
@ -410,6 +570,69 @@ public class AMQPFederationConnectTest extends AmqpClientTestSupport {
}
}
@Test(timeout = 20000)
public void testConnectToBrokerFromRemoteAsFederatedSourceAndCreateEventsSenderLink() throws Exception {
server.start();
try (ProtonTestClient peer = new ProtonTestClient()) {
scriptFederationConnectToRemote(peer, "test", false, null, null, true, false);
peer.connect("localhost", AMQP_PORT);
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectClose();
peer.remoteClose().now();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
server.stop();
logger.info("Test stopped");
}
}
@Test(timeout = 20000)
public void testConnectToBrokerFromRemoteAsFederatedSourceAndCreateEventsReceiverLink() throws Exception {
server.start();
try (ProtonTestClient peer = new ProtonTestClient()) {
scriptFederationConnectToRemote(peer, "test", false, null, null, false, true);
peer.connect("localhost", AMQP_PORT);
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectClose();
peer.remoteClose().now();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
server.stop();
logger.info("Test stopped");
}
}
@Test(timeout = 20000)
public void testConnectToBrokerFromRemoteAsFederatedSourceAndCreateEventsLinks() throws Exception {
server.start();
try (ProtonTestClient peer = new ProtonTestClient()) {
scriptFederationConnectToRemote(peer, "test", false, null, null, true, true);
peer.connect("localhost", AMQP_PORT);
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectClose();
peer.remoteClose().now();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
server.stop();
logger.info("Test stopped");
}
}
@Test(timeout = 20000)
public void testControlLinkPassesConnectAttemptWhenUserHasPrivledges() throws Exception {
enableSecurity(server, FEDERATION_CONTROL_LINK_VALIDATION_ADDRESS);
@ -450,6 +673,96 @@ public class AMQPFederationConnectTest extends AmqpClientTestSupport {
}
}
@Test(timeout = 20000)
public void testRemoteConnectionCannotAttachEventReceiverLinkWithoutControlLink() throws Exception {
server.start();
try (ProtonTestClient peer = new ProtonTestClient()) {
peer.queueClientSaslAnonymousConnect();
peer.remoteOpen().queue();
peer.expectOpen();
peer.remoteBegin().queue();
peer.expectBegin();
peer.connect("localhost", AMQP_PORT);
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
// Broker should reject the attach since there's no control link
peer.expectAttach().ofSender().withName("federation-event-receiver")
.withNullSource()
.withTarget();
peer.expectDetach().withError(AmqpError.ILLEGAL_STATE.toString()).respond();
// Attempt to create a federation event receiver link without existing control link
peer.remoteAttach().ofReceiver()
.withDesiredCapabilities(FEDERATION_EVENT_LINK.toString())
.withName("federation-event-receiver")
.withSenderSettleModeSettled()
.withReceivervSettlesFirst()
.withSource().withDurabilityOfNone()
.withExpiryPolicyOnLinkDetach()
.withCapabilities("temporary-topic")
.withDynamic(true)
.and()
.withTarget().and()
.now();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectClose();
peer.remoteClose().now();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
server.stop();
}
}
@Test(timeout = 20000)
public void testRemoteConnectionCannotAttachEventSenderLinkWithoutControlLink() throws Exception {
server.start();
try (ProtonTestClient peer = new ProtonTestClient()) {
peer.queueClientSaslAnonymousConnect();
peer.remoteOpen().queue();
peer.expectOpen();
peer.remoteBegin().queue();
peer.expectBegin();
peer.connect("localhost", AMQP_PORT);
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
// Broker should reject the attach since there's no control link
peer.expectAttach().ofReceiver().withName("federation-event-sender")
.withSource().also()
.withNullTarget();
peer.expectDetach().withError(AmqpError.ILLEGAL_STATE.toString()).respond();
// Attempt to create a federation event receiver link without existing control link
peer.remoteAttach().ofSender()
.withDesiredCapabilities(FEDERATION_EVENT_LINK.toString())
.withName("federation-event-sender")
.withSenderSettleModeSettled()
.withReceivervSettlesFirst()
.withTarget().withDurabilityOfNone()
.withExpiryPolicyOnLinkDetach()
.withCapabilities("temporary-topic")
.withDynamic(true)
.and()
.withSource().and()
.now();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectClose();
peer.remoteClose().now();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
server.stop();
}
}
// Use these methods to script the initial handshake that a broker that is establishing
// a federation connection with a remote broker instance would perform.
@ -462,6 +775,10 @@ public class AMQPFederationConnectTest extends AmqpClientTestSupport {
}
private void scriptFederationConnectToRemote(ProtonTestClient peer, String federationName, boolean auth, String username, String password) {
scriptFederationConnectToRemote(peer, federationName, auth, username, password, false, false);
}
private void scriptFederationConnectToRemote(ProtonTestClient peer, String federationName, boolean auth, String username, String password, boolean eventsSender, boolean eventsReceiver) {
final String federationControlLinkName = "Federation:test:" + UUID.randomUUID().toString();
if (auth) {
@ -488,11 +805,62 @@ public class AMQPFederationConnectTest extends AmqpClientTestSupport {
.also()
.queue();
peer.expectAttach().ofReceiver()
.withName(federationControlLinkName)
.withTarget()
.withAddress(notNullValue())
.also()
.withOfferedCapability(FEDERATION_CONTROL_LINK.toString());
peer.expectFlow();
if (eventsSender) {
final String federationEventsSenderLinkName = "Federation:events-sender:test:" + UUID.randomUUID().toString();
peer.remoteAttach().ofSender()
.withName(federationEventsSenderLinkName)
.withDesiredCapabilities(FEDERATION_EVENT_LINK.toString())
.withSenderSettleModeUnsettled()
.withReceivervSettlesFirst()
.withSource().also()
.withTarget().withDynamic(true)
.withDurabilityOfNone()
.withExpiryPolicyOnLinkDetach()
.withLifetimePolicyOfDeleteOnClose()
.withCapabilities("temporary-topic")
.also()
.queue();
peer.expectAttach().ofReceiver()
.withName(federationEventsSenderLinkName)
.withTarget()
.withAddress(notNullValue())
.also()
.withOfferedCapability(FEDERATION_EVENT_LINK.toString());
peer.expectFlow();
}
if (eventsReceiver) {
final String federationEventsSenderLinkName = "Federation:events-receiver:test:" + UUID.randomUUID().toString();
peer.remoteAttach().ofReceiver()
.withName(federationEventsSenderLinkName)
.withDesiredCapabilities(FEDERATION_EVENT_LINK.toString())
.withSenderSettleModeUnsettled()
.withReceivervSettlesFirst()
.withTarget().also()
.withSource().withDynamic(true)
.withDurabilityOfNone()
.withExpiryPolicyOnLinkDetach()
.withLifetimePolicyOfDeleteOnClose()
.withCapabilities("temporary-topic")
.also()
.queue();
peer.remoteFlow().withLinkCredit(10).queue();
peer.expectAttach().ofSender()
.withName(federationEventsSenderLinkName)
.withSource()
.withAddress(notNullValue())
.also()
.withOfferedCapability(FEDERATION_EVENT_LINK.toString());
}
}
private void scriptFederationConnectToRemoteNotAuthorizedForControlAddress(ProtonTestClient peer, String federationName, String username, String password) {

View File

@ -18,8 +18,10 @@
package org.apache.activemq.artemis.tests.integration.amqp.connect;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.ADD_QUEUE_POLICY;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.EVENT_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_CONFIGURATION;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_CONTROL_LINK;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_EVENT_LINK;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_QUEUE_RECEIVER;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.FEDERATION_RECEIVER_PRIORITY;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.LARGE_MESSAGE_THRESHOLD;
@ -31,6 +33,9 @@ import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPF
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.QUEUE_PRIORITY_ADJUSTMENT;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.RECEIVER_CREDITS;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.RECEIVER_CREDITS_LOW;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.REQUESTED_ADDRESS_NAME;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.REQUESTED_QUEUE_ADDED;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.REQUESTED_QUEUE_NAME;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.TRANSFORMER_CLASS_NAME;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.TRANSFORMER_PROPERTIES_MAP;
import static org.apache.activemq.artemis.protocol.amqp.connect.federation.AMQPFederationConstants.POLICY_PROPERTIES_MAP;
@ -75,13 +80,18 @@ import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPBroker
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPFederatedBrokerConnectionElement;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPFederationQueuePolicyElement;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.impl.AddressInfo;
import org.apache.activemq.artemis.core.server.transformer.Transformer;
import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
import org.apache.activemq.artemis.protocol.amqp.federation.FederationReceiveFromQueuePolicy;
import org.apache.activemq.artemis.protocol.amqp.proton.AmqpSupport;
import org.apache.activemq.artemis.tests.integration.amqp.AmqpClientTestSupport;
import org.apache.activemq.artemis.tests.util.CFUtil;
import org.apache.qpid.proton.amqp.transport.AmqpError;
import org.apache.qpid.proton.amqp.transport.LinkError;
import org.apache.qpid.protonj2.test.driver.ProtonTestClient;
import org.apache.qpid.protonj2.test.driver.ProtonTestPeer;
import org.apache.qpid.protonj2.test.driver.ProtonTestServer;
import org.apache.qpid.protonj2.test.driver.matchers.messaging.ApplicationPropertiesMatcher;
import org.apache.qpid.protonj2.test.driver.matchers.messaging.HeaderMatcher;
@ -89,6 +99,7 @@ import org.apache.qpid.protonj2.test.driver.matchers.messaging.MessageAnnotation
import org.apache.qpid.protonj2.test.driver.matchers.messaging.PropertiesMatcher;
import org.apache.qpid.protonj2.test.driver.matchers.transport.TransferPayloadCompositeMatcher;
import org.apache.qpid.protonj2.test.driver.matchers.types.EncodedAmqpValueMatcher;
import org.hamcrest.Matchers;
import org.jgroups.util.UUID;
import org.junit.Test;
import org.slf4j.Logger;
@ -135,6 +146,11 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withSenderSettleModeSettled()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -200,6 +216,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -262,6 +282,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -325,6 +349,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -385,6 +413,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -469,6 +501,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -530,6 +566,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -607,6 +647,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -667,6 +711,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -730,6 +778,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -819,6 +871,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -897,6 +953,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -954,6 +1014,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_QUEUE_RECEIVER.toString())
.withName(allOf(containsString("sample-federation"),
@ -964,7 +1028,7 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
peer.expectFlow().withLinkCredit(1000);
// Trigger the error that should cause the broker to drop and reconnect
peer.remoteDetach().withErrorCondition("amqp:internal-error", "the resource suffered an internal error").afterDelay(10).now();
peer.remoteDetach().withErrorCondition("amqp:internal-error", "the resource suffered an internal error").afterDelay(15).now();
peer.waitForScriptToComplete(50, TimeUnit.SECONDS);
peer.expectDetach(); // demand will be gone and receiver link should close.
@ -984,6 +1048,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -1071,6 +1139,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -1132,6 +1204,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -1933,6 +2009,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
target.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
target.expectFlow().withLinkCredit(10);
target.start();
final URI remoteURI = target.getServerURI();
@ -2038,6 +2118,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
target.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
target.expectFlow().withLinkCredit(10);
target.start();
final URI remoteURI = target.getServerURI();
@ -2131,6 +2215,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -2212,6 +2300,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -2271,6 +2363,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -2343,6 +2439,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
peer.expectAttach().ofSender()
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
@ -2662,6 +2762,10 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_QUEUE_RECEIVER.toString())
.withName(allOf(containsString("sample-federation"),
@ -2687,6 +2791,480 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
}
}
@Test(timeout = 20000)
public void testFederationCreatesEventSenderAndReceiverWhenLocalAndRemotePoliciesAdded() throws Exception {
final MessageAnnotationsMatcher maMatcher = new MessageAnnotationsMatcher(true);
maMatcher.withEntry(OPERATION_TYPE.toString(), Matchers.is(ADD_QUEUE_POLICY));
final Map<String, Object> policyMap = new LinkedHashMap<>();
final List<String> includes = new ArrayList<>();
includes.add("*");
includes.add("test");
policyMap.put(POLICY_NAME, "test-policy");
policyMap.put(QUEUE_INCLUDE_FEDERATED, false);
policyMap.put(QUEUE_PRIORITY_ADJUSTMENT, 64);
policyMap.put(QUEUE_INCLUDES, includes);
final EncodedAmqpValueMatcher bodyMatcher = new EncodedAmqpValueMatcher(policyMap);
final TransferPayloadCompositeMatcher payloadMatcher = new TransferPayloadCompositeMatcher();
payloadMatcher.setMessageAnnotationsMatcher(maMatcher);
payloadMatcher.addMessageContentMatcher(bodyMatcher);
try (ProtonTestServer peer = new ProtonTestServer()) {
peer.expectSASLAnonymousConnect();
peer.expectOpen().respond();
peer.expectBegin().respond();
peer.expectAttach().ofSender()
.withHandle(0)
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofSender()
.withTarget().withDynamic(true)
.and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind()
.withTarget().withAddress("test-dynamic-events-sender");
peer.remoteFlow().withLinkCredit(10).queue();
peer.expectAttach().ofReceiver()
.withSource().withDynamic(true)
.and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind()
.withSource().withAddress("test-dynamic-events-receiver");
peer.expectFlow().withLinkCredit(10);
peer.remoteFlow().withLinkCredit(10).withHandle(0).queue(); // Give control link credit now to ensure ordering
peer.expectTransfer().withPayload(payloadMatcher); // Remote policy
peer.start();
final URI remoteURI = peer.getServerURI();
logger.info("Test started, peer listening on: {}", remoteURI);
final AMQPFederationQueuePolicyElement localReceiveFromQueue = new AMQPFederationQueuePolicyElement();
localReceiveFromQueue.setName("test-policy");
localReceiveFromQueue.setIncludeFederated(true);
localReceiveFromQueue.setPriorityAdjustment(42);
localReceiveFromQueue.addToIncludes("*", "test");
final AMQPFederationQueuePolicyElement remoteReceiveFromQueue = new AMQPFederationQueuePolicyElement();
remoteReceiveFromQueue.setName("test-policy");
remoteReceiveFromQueue.setIncludeFederated(false);
remoteReceiveFromQueue.setPriorityAdjustment(64);
remoteReceiveFromQueue.addToIncludes("*", "test");
final AMQPFederatedBrokerConnectionElement element = new AMQPFederatedBrokerConnectionElement();
element.setName("sample-federation");
element.addLocalQueuePolicy(localReceiveFromQueue);
element.addRemoteQueuePolicy(remoteReceiveFromQueue);
final AMQPBrokerConnectConfiguration amqpConnection =
new AMQPBrokerConnectConfiguration("test-address-federation", "tcp://" + remoteURI.getHost() + ":" + remoteURI.getPort());
amqpConnection.setReconnectAttempts(0);// No reconnects
amqpConnection.addElement(element);
server.getConfiguration().addAMQPConnection(amqpConnection);
server.start();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
}
}
@Test(timeout = 20000)
public void testFederationSendsRemotePolicyIfEventsSenderLinkRejected() throws Exception {
final MessageAnnotationsMatcher maMatcher = new MessageAnnotationsMatcher(true);
maMatcher.withEntry(OPERATION_TYPE.toString(), Matchers.is(ADD_QUEUE_POLICY));
final Map<String, Object> policyMap = new LinkedHashMap<>();
final List<String> includes = new ArrayList<>();
includes.add("*");
includes.add("test");
policyMap.put(POLICY_NAME, "test-policy");
policyMap.put(QUEUE_INCLUDE_FEDERATED, false);
policyMap.put(QUEUE_PRIORITY_ADJUSTMENT, 64);
policyMap.put(QUEUE_INCLUDES, includes);
final EncodedAmqpValueMatcher bodyMatcher = new EncodedAmqpValueMatcher(policyMap);
final TransferPayloadCompositeMatcher payloadMatcher = new TransferPayloadCompositeMatcher();
payloadMatcher.setMessageAnnotationsMatcher(maMatcher);
payloadMatcher.addMessageContentMatcher(bodyMatcher);
try (ProtonTestServer peer = new ProtonTestServer()) {
peer.expectSASLAnonymousConnect();
peer.expectOpen().respond();
peer.expectBegin().respond();
peer.expectAttach().ofSender()
.withHandle(0)
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.expectAttach().ofSender()
.withTarget().withDynamic(true)
.and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.reject(true, LinkError.DETACH_FORCED.toString(), "Unknown error");
peer.expectDetach();
peer.remoteFlow().withLinkCredit(10).withHandle(0).queue(); // Give control link credit now to ensure ordering
peer.expectTransfer().withPayload(payloadMatcher); // Remote policy
peer.start();
final URI remoteURI = peer.getServerURI();
logger.info("Test started, peer listening on: {}", remoteURI);
final AMQPFederationQueuePolicyElement remoteReceiveFromQueue = new AMQPFederationQueuePolicyElement();
remoteReceiveFromQueue.setName("test-policy");
remoteReceiveFromQueue.setIncludeFederated(false);
remoteReceiveFromQueue.setPriorityAdjustment(64);
remoteReceiveFromQueue.addToIncludes("*", "test");
final AMQPFederatedBrokerConnectionElement element = new AMQPFederatedBrokerConnectionElement();
element.setName("sample-federation");
element.addRemoteQueuePolicy(remoteReceiveFromQueue);
final AMQPBrokerConnectConfiguration amqpConnection =
new AMQPBrokerConnectConfiguration("test-address-federation", "tcp://" + remoteURI.getHost() + ":" + remoteURI.getPort());
amqpConnection.setReconnectAttempts(0);// No reconnects
amqpConnection.addElement(element);
server.getConfiguration().addAMQPConnection(amqpConnection);
server.start();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
}
}
@Test(timeout = 20000)
public void testRemoteBrokerSendsQueueAddedEventForInterestedPeer() throws Exception {
final AddressSettings addressSettings = new AddressSettings();
addressSettings.setAutoCreateQueues(false);
addressSettings.setAutoCreateAddresses(false);
server.getConfiguration().getAddressSettings().put("#", addressSettings);
server.start();
final MessageAnnotationsMatcher maMatcher = new MessageAnnotationsMatcher(true);
maMatcher.withEntry(EVENT_TYPE.toString(), Matchers.is(REQUESTED_QUEUE_ADDED));
final Map<String, Object> eventMap = new LinkedHashMap<>();
eventMap.put(REQUESTED_ADDRESS_NAME, "test");
eventMap.put(REQUESTED_QUEUE_NAME, "test");
final EncodedAmqpValueMatcher bodyMatcher = new EncodedAmqpValueMatcher(eventMap);
final TransferPayloadCompositeMatcher payloadMatcher = new TransferPayloadCompositeMatcher();
payloadMatcher.setMessageAnnotationsMatcher(maMatcher);
payloadMatcher.addMessageContentMatcher(bodyMatcher);
try (ProtonTestClient peer = new ProtonTestClient()) {
scriptFederationConnectToRemote(peer, "test", false, true);
peer.connect("localhost", AMQP_PORT);
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectAttach().ofSender().withName("federation-queue-receiver")
.withOfferedCapabilities(FEDERATION_QUEUE_RECEIVER.toString())
.withTarget().also()
.withNullSource();
peer.expectDetach().respond();
// Connect to remote as if an queue had demand and matched our federation policy
peer.remoteAttach().ofReceiver()
.withDesiredCapabilities(FEDERATION_QUEUE_RECEIVER.toString())
.withName("federation-queue-receiver")
.withProperty(FEDERATION_RECEIVER_PRIORITY.toString(), DEFAULT_QUEUE_RECEIVER_PRIORITY_ADJUSTMENT)
.withSenderSettleModeUnsettled()
.withReceivervSettlesFirst()
.withSource().withDurabilityOfNone()
.withExpiryPolicyOnLinkDetach()
.withAddress("test::test")
.withCapabilities("queue")
.and()
.withTarget().and()
.now();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectTransfer().withPayload(payloadMatcher).accept(); // Address added event
// Manually add the address and a queue binding to create local demand.
server.addAddressInfo(new AddressInfo(SimpleString.toSimpleString("test"), RoutingType.MULTICAST));
server.createQueue(new QueueConfiguration("test").setRoutingType(RoutingType.MULTICAST)
.setAddress("test")
.setAutoCreated(false));
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectClose();
peer.remoteClose().now();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
server.stop();
}
}
@Test(timeout = 20000)
public void testFederationCreatesAddressReceiverInResponseToAddressAddedEvent() throws Exception {
try (ProtonTestServer peer = new ProtonTestServer()) {
peer.expectSASLAnonymousConnect();
peer.expectOpen().respond();
peer.expectBegin().respond();
peer.expectAttach().ofSender()
.withHandle(0)
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respond()
.withOfferedCapabilities(FEDERATION_CONTROL_LINK.toString());
peer.remoteFlow().withLinkCredit(10);
peer.expectAttach().ofReceiver()
.withHandle(1)
.withSenderSettleModeSettled()
.withSource().withDynamic(true)
.and()
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind()
.withTarget().withAddress("test-dynamic-events");
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
logger.info("Test started, peer listening on: {}", remoteURI);
final AMQPFederationQueuePolicyElement receiveFromQueue = new AMQPFederationQueuePolicyElement();
receiveFromQueue.setName("test-policy");
receiveFromQueue.setIncludeFederated(false);
receiveFromQueue.addToIncludes("*", "test");
final AMQPFederatedBrokerConnectionElement element = new AMQPFederatedBrokerConnectionElement();
element.setName("sample-federation");
element.addLocalQueuePolicy(receiveFromQueue);
final AMQPBrokerConnectConfiguration amqpConnection =
new AMQPBrokerConnectConfiguration("test-queue-federation", "tcp://" + remoteURI.getHost() + ":" + remoteURI.getPort());
amqpConnection.setReconnectAttempts(0);// No reconnects
amqpConnection.addElement(element);
server.getConfiguration().addAMQPConnection(amqpConnection);
server.start();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
// Reject the initial attempt
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_QUEUE_RECEIVER.toString())
.withName(allOf(containsString("sample-federation"),
containsString("test::test"),
containsString("queue-receiver"),
containsString(server.getNodeID().toString())))
.respond()
.withNullSource()
.withOfferedCapabilities(FEDERATION_QUEUE_RECEIVER.toString());
peer.remoteDetach().withClosed(true)
.withErrorCondition(AmqpError.NOT_FOUND.toString(), "Queue not found")
.queue();
peer.expectFlow();
peer.expectDetach();
final ConnectionFactory factory = CFUtil.createConnectionFactory("AMQP", "tcp://localhost:" + AMQP_PORT);
final Connection connection = factory.createConnection();
final Session session = connection.createSession(Session.AUTO_ACKNOWLEDGE);
// Create demand on the Queue to kick off federation.
session.createConsumer(session.createQueue("test"));
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_QUEUE_RECEIVER.toString())
.withName(allOf(containsString("sample-federation"),
containsString("test::test"),
containsString("queue-receiver"),
containsString(server.getNodeID().toString())))
.respond()
.withOfferedCapabilities(FEDERATION_QUEUE_RECEIVER.toString());
peer.expectFlow().withLinkCredit(1000);
// Should not trigger attach of a federation receiver as the queue doesn't match the policy..
sendQueueAddedEvent(peer, "target", "target", 1, 0);
// Should trigger attach of federation receiver again for the test queue.
sendQueueAddedEvent(peer, "test", "test", 1, 1);
// Should not trigger another attach of federation receiver as there already is one.
sendQueueAddedEvent(peer, "test", "test", 1, 2);
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
}
}
@Test(timeout = 20000)
public void testAddressAddedEventIgnoredIfFederationConsumerAlreadyCreated() throws Exception {
try (ProtonTestServer peer = new ProtonTestServer()) {
peer.expectSASLAnonymousConnect();
peer.expectOpen().respond();
peer.expectBegin().respond();
peer.expectAttach().ofSender()
.withHandle(0)
.withDesiredCapability(FEDERATION_CONTROL_LINK.toString())
.respondInKind();
peer.remoteFlow().withLinkCredit(10);
peer.expectAttach().ofReceiver()
.withHandle(1)
.withDesiredCapability(FEDERATION_EVENT_LINK.toString())
.respondInKind();
peer.expectFlow().withLinkCredit(10);
peer.start();
final URI remoteURI = peer.getServerURI();
logger.info("Test started, peer listening on: {}", remoteURI);
final AMQPFederationQueuePolicyElement receiveFromQueue = new AMQPFederationQueuePolicyElement();
receiveFromQueue.setName("test-policy");
receiveFromQueue.setIncludeFederated(false);
receiveFromQueue.addToIncludes("*", "test");
final AMQPFederatedBrokerConnectionElement element = new AMQPFederatedBrokerConnectionElement();
element.setName("sample-federation");
element.addLocalQueuePolicy(receiveFromQueue);
final AMQPBrokerConnectConfiguration amqpConnection =
new AMQPBrokerConnectConfiguration("test-queue-federation", "tcp://" + remoteURI.getHost() + ":" + remoteURI.getPort());
amqpConnection.setReconnectAttempts(0);// No reconnects
amqpConnection.addElement(element);
server.getConfiguration().addAMQPConnection(amqpConnection);
server.start();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
// Reject the initial attempt
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_QUEUE_RECEIVER.toString())
.respond()
.withNullSource()
.withOfferedCapabilities(FEDERATION_QUEUE_RECEIVER.toString());
peer.remoteDetach().withClosed(true)
.withErrorCondition(AmqpError.NOT_FOUND.toString(), "Queue not found")
.queue();
peer.expectFlow();
peer.expectDetach();
final ConnectionFactory factory = CFUtil.createConnectionFactory("AMQP", "tcp://localhost:" + AMQP_PORT);
final Connection connection = factory.createConnection();
final Session session = connection.createSession(Session.AUTO_ACKNOWLEDGE);
// Create demand on the Queue to kick off the first federation attempt.
session.createConsumer(session.createQueue("test"));
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectAttach().ofReceiver()
.withDesiredCapability(FEDERATION_QUEUE_RECEIVER.toString())
.respond()
.withOfferedCapabilities(FEDERATION_QUEUE_RECEIVER.toString());
peer.expectFlow().withLinkCredit(1000);
// Create demand on the Queue again to kick off another federation attempt.
session.createConsumer(session.createQueue("test"));
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
// Should not trigger attach of federation receiver as there already is one on this queue
sendQueueAddedEvent(peer, "test", "test", 1, 0);
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
}
}
@Test(timeout = 20000)
public void testRemoteBrokerClosesFederationReceiverAfterQueueRemoved() throws Exception {
server.start();
server.createQueue(new QueueConfiguration("test").setRoutingType(RoutingType.ANYCAST)
.setAddress("test")
.setAutoCreated(false));
try (ProtonTestClient peer = new ProtonTestClient()) {
scriptFederationConnectToRemote(peer, "test", true, true);
peer.connect("localhost", AMQP_PORT);
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectAttach().ofSender().withName("test::test")
.withOfferedCapabilities(FEDERATION_QUEUE_RECEIVER.toString())
.withSource().withAddress("test::test");
// Connect to remote as if an queue had demand and matched our federation policy
peer.remoteAttach().ofReceiver()
.withDesiredCapabilities(FEDERATION_QUEUE_RECEIVER.toString())
.withName("test::test")
.withProperty(FEDERATION_RECEIVER_PRIORITY.toString(), DEFAULT_QUEUE_RECEIVER_PRIORITY_ADJUSTMENT)
.withSenderSettleModeUnsettled()
.withReceivervSettlesFirst()
.withSource().withDurabilityOfNone()
.withExpiryPolicyOnLinkDetach()
.withAddress("test::test")
.withCapabilities("queue")
.and()
.withTarget().and()
.now();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectDetach().withError(AmqpError.RESOURCE_DELETED.toString());
// Force remove consumers from the queue should indicate the resource was deleted.
server.destroyQueue(SimpleString.toSimpleString("test"), null, false, true);
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
final MessageAnnotationsMatcher maMatcher = new MessageAnnotationsMatcher(true);
maMatcher.withEntry(EVENT_TYPE.toString(), Matchers.is(REQUESTED_QUEUE_ADDED));
final Map<String, Object> eventMap = new LinkedHashMap<>();
eventMap.put(REQUESTED_ADDRESS_NAME, "test");
eventMap.put(REQUESTED_QUEUE_NAME, "test");
final EncodedAmqpValueMatcher bodyMatcher = new EncodedAmqpValueMatcher(eventMap);
final TransferPayloadCompositeMatcher payloadMatcher = new TransferPayloadCompositeMatcher();
payloadMatcher.setMessageAnnotationsMatcher(maMatcher);
payloadMatcher.addMessageContentMatcher(bodyMatcher);
// Server alerts the federation event receiver that a previously federated queue
// has been added once more and it could restore the previous federation state.
peer.expectTransfer().withPayload(payloadMatcher).withSettled(true);
server.createQueue(new QueueConfiguration("test").setRoutingType(RoutingType.ANYCAST)
.setAddress("test")
.setAutoCreated(false));
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
// This time removing and restoring should generate no traffic as there was not
// another federation receiver added.
server.destroyQueue(SimpleString.toSimpleString("test"), null, false, true);
server.createQueue(new QueueConfiguration("test").setRoutingType(RoutingType.ANYCAST)
.setAddress("test")
.setAutoCreated(false));
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.expectClose();
peer.remoteClose().now();
peer.waitForScriptToComplete(5, TimeUnit.SECONDS);
peer.close();
server.stop();
}
}
private static void sendQueueAddedEvent(ProtonTestPeer peer, String address, String queue, int handle, int deliveryId) {
final Map<String, Object> eventMap = new LinkedHashMap<>();
eventMap.put(REQUESTED_ADDRESS_NAME, address);
eventMap.put(REQUESTED_QUEUE_NAME, queue);
// Should not trigger another attach of federation receiver as there already is one.
peer.remoteTransfer().withHandle(handle)
.withDeliveryId(deliveryId)
.withSettled(true)
.withMessageAnnotations().withAnnotation(EVENT_TYPE.toString(), REQUESTED_QUEUE_ADDED)
.also()
.withBody().withValue(eventMap)
.also()
.now();
}
public static class ApplicationPropertiesTransformer implements Transformer {
private final Map<String, String> properties = new HashMap<>();
@ -2768,6 +3346,14 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
}
private void scriptFederationConnectToRemote(ProtonTestClient peer, String federationName, int amqpCredits, int amqpLowCredits) {
scriptFederationConnectToRemote(peer, federationName, amqpCredits, amqpLowCredits, false, false);
}
private void scriptFederationConnectToRemote(ProtonTestClient peer, String federationName, boolean eventsSender, boolean eventsReceiver) {
scriptFederationConnectToRemote(peer, federationName, AmqpSupport.AMQP_CREDITS_DEFAULT, AmqpSupport.AMQP_LOW_CREDITS_DEFAULT, eventsSender, eventsReceiver);
}
private void scriptFederationConnectToRemote(ProtonTestClient peer, String federationName, int amqpCredits, int amqpLowCredits, boolean eventsSender, boolean eventsReceiver ) {
final String federationControlLinkName = "Federation:control:" + UUID.randomUUID().toString();
final Map<String, Object> federationConfiguration = new HashMap<>();
@ -2803,5 +3389,57 @@ public class AMQPFederationQueuePolicyTest extends AmqpClientTestSupport {
.also()
.withOfferedCapability(FEDERATION_CONTROL_LINK.toString());
peer.expectFlow();
// Sender created when there are remote policies to send to the target
if (eventsSender) {
final String federationEventsSenderLinkName = "Federation:events-sender:test:" + UUID.randomUUID().toString();
peer.remoteAttach().ofSender()
.withName(federationEventsSenderLinkName)
.withDesiredCapabilities(FEDERATION_EVENT_LINK.toString())
.withSenderSettleModeSettled()
.withReceivervSettlesFirst()
.withSource().also()
.withTarget().withDynamic(true)
.withDurabilityOfNone()
.withExpiryPolicyOnLinkDetach()
.withLifetimePolicyOfDeleteOnClose()
.withCapabilities("temporary-topic")
.also()
.queue();
peer.expectAttach().ofReceiver()
.withName(federationEventsSenderLinkName)
.withTarget()
.withAddress(notNullValue())
.also()
.withOfferedCapability(FEDERATION_EVENT_LINK.toString());
peer.expectFlow();
}
// Receiver created when there are local policies on the source.
if (eventsReceiver) {
final String federationEventsSenderLinkName = "Federation:events-receiver:test:" + UUID.randomUUID().toString();
peer.remoteAttach().ofReceiver()
.withName(federationEventsSenderLinkName)
.withDesiredCapabilities(FEDERATION_EVENT_LINK.toString())
.withSenderSettleModeSettled()
.withReceivervSettlesFirst()
.withTarget().also()
.withSource().withDynamic(true)
.withDurabilityOfNone()
.withExpiryPolicyOnLinkDetach()
.withLifetimePolicyOfDeleteOnClose()
.withCapabilities("temporary-topic")
.also()
.queue();
peer.remoteFlow().withLinkCredit(10).queue();
peer.expectAttach().ofSender()
.withName(federationEventsSenderLinkName)
.withSource()
.withAddress(notNullValue())
.also()
.withOfferedCapability(FEDERATION_EVENT_LINK.toString());
}
}
}