Skip to content

Commit

Permalink
ARTEMIS-5090 MirrorSNF is not supposed to expiry on messages
Browse files Browse the repository at this point in the history
  • Loading branch information
clebertsuconic authored and gemmellr committed Oct 9, 2024
1 parent f648ddf commit 6186805
Show file tree
Hide file tree
Showing 3 changed files with 138 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.apache.activemq.artemis.api.core.client.ActiveMQClient;
import org.apache.activemq.artemis.core.remoting.impl.netty.NettyServerConnection;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.impl.QueueImpl;
import org.apache.activemq.artemis.core.server.management.Notification;
import org.apache.activemq.artemis.core.server.management.NotificationListener;
import org.apache.activemq.artemis.protocol.amqp.client.ProtonClientProtocolManager;
Expand Down Expand Up @@ -64,7 +65,7 @@ public class ProtonProtocolManager extends AbstractProtocolManager<AMQPMessage,

private static final List<String> websocketRegistryNames = Arrays.asList("amqp");

public static final String MIRROR_ADDRESS = "$ACTIVEMQ_ARTEMIS_MIRROR";
public static final String MIRROR_ADDRESS = QueueImpl.MIRROR_ADDRESS;

private final List<AmqpInterceptor> incomingInterceptors = new ArrayList<>();
private final List<AmqpInterceptor> outgoingInterceptors = new ArrayList<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -142,6 +142,9 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
protected static final int CRITICAL_CONSUMER = 3;
protected static final int CRITICAL_CHECK_DEPAGE = 4;

// The prefix for Mirror SNF Queues
public static final String MIRROR_ADDRESS = "$ACTIVEMQ_ARTEMIS_MIRROR";

private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final AtomicIntegerFieldUpdater<QueueImpl> dispatchingUpdater = AtomicIntegerFieldUpdater.newUpdater(QueueImpl.class, "dispatching");
private static final AtomicLongFieldUpdater<QueueImpl> dispatchStartTimeUpdater = AtomicLongFieldUpdater.newUpdater(QueueImpl.class, "dispatchStartTime");
Expand Down Expand Up @@ -2472,7 +2475,7 @@ public void deleteQueue(boolean removeConsumers) throws Exception {

@Override
public synchronized boolean expireReference(final long messageID) throws Exception {
if (isExpirationRedundant()) {
if (isExpiryDisabled()) {
return false;
}

Expand All @@ -2494,7 +2497,7 @@ public synchronized boolean expireReference(final long messageID) throws Excepti

@Override
public synchronized int expireReferences(final Filter filter) throws Exception {
if (isExpirationRedundant()) {
if (isExpiryDisabled()) {
return 0;
}

Expand Down Expand Up @@ -2523,14 +2526,13 @@ public synchronized int expireReferences(final Filter filter) throws Exception {

@Override
public void expireReferences(Runnable done) {
if (isExpirationRedundant()) {
if (isExpiryDisabled()) {
if (done != null) {
done.run();
}
return;
}


if (!queueDestroyed) {
getExecutor().execute(new ExpiryScanner(done));
} else {
Expand All @@ -2541,7 +2543,7 @@ public void expireReferences(Runnable done) {
}
}

public boolean isExpirationRedundant() {
private boolean isExpiryDisabled() {
final SimpleString expiryAddress = addressSettings.getExpiryAddress();
if (expiryAddress != null && expiryAddress.equals(this.address)) {
// check expire with itself would be silly (waste of time)
Expand All @@ -2550,6 +2552,11 @@ public boolean isExpirationRedundant() {
return true;
}

if (isInternalQueue() && name.toString().startsWith(MIRROR_ADDRESS)) {
logger.trace("Mirror SNF queues are not supposed to expire messages. Address={}, Queue={}", address, name);
return true;
}

return false;
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,124 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.activemq.artemis.tests.integration.amqp.connect;

import javax.jms.Connection;
import javax.jms.ConnectionFactory;
import javax.jms.MessageProducer;
import javax.jms.Session;
import java.lang.invoke.MethodHandles;
import java.util.HashMap;

import org.apache.activemq.artemis.api.core.QueueConfiguration;
import org.apache.activemq.artemis.api.core.RoutingType;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.api.core.TransportConfiguration;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPBrokerConnectConfiguration;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPBrokerConnectionAddressType;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPMirrorBrokerConnectionElement;
import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.Queue;
import org.apache.activemq.artemis.core.server.impl.QueueImpl;
import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
import org.apache.activemq.artemis.tests.util.CFUtil;
import org.apache.activemq.artemis.tests.util.RandomUtil;
import org.apache.activemq.artemis.tests.util.Wait;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import static org.junit.jupiter.api.Assertions.assertNotNull;

public class AMQPMirrorExpiryQueueTest extends ActiveMQTestBase {

private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

private static final String EXPIRY_QUEUE = AMQPMirrorExpiryQueueTest.class.getName() + "_ExpiryOut";

protected TransportConfiguration newAcceptorConfig(int port, String name) {
HashMap<String, Object> params = new HashMap<>();
params.put(TransportConstants.PORT_PROP_NAME, String.valueOf(port));
params.put(TransportConstants.PROTOCOLS_PROP_NAME, "AMQP,CORE,OPENWIRE");
HashMap<String, Object> amqpParams = new HashMap<>();
TransportConfiguration tc = new TransportConfiguration(NETTY_ACCEPTOR_FACTORY, params, name, amqpParams);
return tc;
}

protected ActiveMQServer createServer(int port, String brokerName) throws Exception {

final ActiveMQServer server = this.createServer(true, true);

server.getConfiguration().getAcceptorConfigurations().clear();
server.getConfiguration().getAcceptorConfigurations().add(newAcceptorConfig(port, "netty-acceptor"));
server.getConfiguration().setName(brokerName);
server.getConfiguration().setJournalDirectory(server.getConfiguration().getJournalDirectory() + port);
server.getConfiguration().setBindingsDirectory(server.getConfiguration().getBindingsDirectory() + port);
server.getConfiguration().setPagingDirectory(server.getConfiguration().getPagingDirectory() + port);
server.getConfiguration().setJMXManagementEnabled(true);
server.getConfiguration().setMessageExpiryScanPeriod(5);

server.getConfiguration().addAddressSetting("#", new AddressSettings().setExpiryAddress(SimpleString.of(EXPIRY_QUEUE)));
return server;
}

@Test
public void testExpiryOnMirrorSNF() throws Exception {

final long numberOfMessages = 100;
ActiveMQServer serverA = createServer(5671, getTestMethodName() + "_A");

{
AMQPBrokerConnectConfiguration amqpConnection = new AMQPBrokerConnectConfiguration(getTestMethodName() + "_willNeverConnect", "tcp://localhost:6671").setReconnectAttempts(1).setRetryInterval(10);
AMQPMirrorBrokerConnectionElement replica = new AMQPMirrorBrokerConnectionElement().setType(AMQPBrokerConnectionAddressType.MIRROR).setDurable(true);
amqpConnection.addElement(replica);
serverA.getConfiguration().addAMQPConnection(amqpConnection);
}

String queueName = getTestMethodName() + "_" + RandomUtil.randomString();

serverA.setIdentity(getTestMethodName() + "_A");
serverA.start();

serverA.createQueue(QueueConfiguration.of(queueName).setName(queueName).setRoutingType(RoutingType.ANYCAST));
Queue expiryA = serverA.createQueue(QueueConfiguration.of(EXPIRY_QUEUE).setName(EXPIRY_QUEUE).setRoutingType(RoutingType.ANYCAST));
Queue snfQueue = serverA.locateQueue(QueueImpl.MIRROR_ADDRESS + "_" + getTestMethodName() + "_willNeverConnect");
assertNotNull(snfQueue);
assertNotNull(expiryA);

ConnectionFactory factoryA = CFUtil.createConnectionFactory("CORE", "tcp://localhost:5671");
try (Connection connection = factoryA.createConnection()) {
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
MessageProducer producer = session.createProducer(session.createQueue(queueName));
producer.setTimeToLive(1);

for (int i = 0; i < numberOfMessages; i++) {
producer.send(session.createTextMessage("hello" + i));
}

session.commit();
}

Thread.sleep(10); // waiting a little more than the expiry scan period
Wait.assertEquals(numberOfMessages, expiryA::getMessageCount, 5000, 100);

// We should still have the message sends and the acks from the expired messages in the SNF
Wait.assertTrue(() -> snfQueue.getMessageCount() >= (numberOfMessages * 2), 5000, 100);
}

}

0 comments on commit 6186805

Please sign in to comment.