You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by gt...@apache.org on 2016/05/06 16:12:59 UTC

activemq git commit: validate selector aware vt can give topic semantics by dropping messages when there is no consumer. Do selector cache plugin once to avoid adaptor check on each dispatch

Repository: activemq
Updated Branches:
  refs/heads/master 5009b0c03 -> 9fc266104


validate selector aware vt can give topic semantics by dropping messages when there is no consumer. Do selector cache plugin once to avoid adaptor check on each dispatch


Project: http://git-wip-us.apache.org/repos/asf/activemq/repo
Commit: http://git-wip-us.apache.org/repos/asf/activemq/commit/9fc26610
Tree: http://git-wip-us.apache.org/repos/asf/activemq/tree/9fc26610
Diff: http://git-wip-us.apache.org/repos/asf/activemq/diff/9fc26610

Branch: refs/heads/master
Commit: 9fc2661049c32da975ae19e91d3974bdc6d13d80
Parents: 5009b0c
Author: gtully <ga...@gmail.com>
Authored: Fri May 6 17:11:50 2016 +0100
Committer: gtully <ga...@gmail.com>
Committed: Fri May 6 17:12:15 2016 +0100

----------------------------------------------------------------------
 .../SelectorAwareVirtualTopicInterceptor.java   |  25 +--
 ...rAwareVTThatDropsMessagesWhenNoConsumer.java | 154 +++++++++++++++++++
 2 files changed, 160 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/activemq/blob/9fc26610/activemq-broker/src/main/java/org/apache/activemq/broker/region/virtual/SelectorAwareVirtualTopicInterceptor.java
----------------------------------------------------------------------
diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/region/virtual/SelectorAwareVirtualTopicInterceptor.java b/activemq-broker/src/main/java/org/apache/activemq/broker/region/virtual/SelectorAwareVirtualTopicInterceptor.java
index cadf14a..d100c67 100644
--- a/activemq-broker/src/main/java/org/apache/activemq/broker/region/virtual/SelectorAwareVirtualTopicInterceptor.java
+++ b/activemq-broker/src/main/java/org/apache/activemq/broker/region/virtual/SelectorAwareVirtualTopicInterceptor.java
@@ -17,10 +17,9 @@
 package org.apache.activemq.broker.region.virtual;
 
 import org.apache.activemq.broker.Broker;
-import org.apache.activemq.broker.ProducerBrokerExchange;
 import org.apache.activemq.broker.region.Destination;
 import org.apache.activemq.broker.region.Subscription;
-import org.apache.activemq.command.ActiveMQDestination;
+import org.apache.activemq.broker.region.Topic;
 import org.apache.activemq.command.Message;
 import org.apache.activemq.filter.BooleanExpression;
 import org.apache.activemq.filter.MessageEvaluationContext;
@@ -38,10 +37,12 @@ import java.util.Set;
 public class SelectorAwareVirtualTopicInterceptor extends VirtualTopicInterceptor {
     private static final Logger LOG = LoggerFactory.getLogger(SelectorAwareVirtualTopicInterceptor.class);
     LRUCache<String,BooleanExpression> expressionCache = new LRUCache<String,BooleanExpression>();
-    private SubQueueSelectorCacheBroker selectorCachePlugin;
+    private final SubQueueSelectorCacheBroker selectorCachePlugin;
 
     public SelectorAwareVirtualTopicInterceptor(Destination next, VirtualTopic virtualTopic) {
         super(next, virtualTopic);
+        selectorCachePlugin = (SubQueueSelectorCacheBroker)
+                ((Topic)next).createConnectionContext().getBroker().getAdaptor(SubQueueSelectorCacheBroker.class);
     }
 
     /**
@@ -75,11 +76,8 @@ public class SelectorAwareVirtualTopicInterceptor extends VirtualTopicIntercepto
         boolean matches = false;
         LOG.debug("No active consumer match found. Will try cache if configured...");
 
-        //retrieve the specific plugin class and lookup the selector for the destination.
-        final SubQueueSelectorCacheBroker cache = getSubQueueSelectorCacheBrokerPlugin(broker);
-
-        if (cache != null) {
-            final Set<String> selectors = cache.getSelector(dest.getActiveMQDestination().getQualifiedName());
+        if (selectorCachePlugin != null) {
+            final Set<String> selectors = selectorCachePlugin.getSelector(dest.getActiveMQDestination().getQualifiedName());
             if (selectors != null) {
                 for (String selector : selectors) {
                     try {
@@ -110,17 +108,6 @@ public class SelectorAwareVirtualTopicInterceptor extends VirtualTopicIntercepto
     }
 
     /**
-     * @return The SubQueueSelectorCacheBroker instance or null if no such broker is available.
-     */
-    private SubQueueSelectorCacheBroker getSubQueueSelectorCacheBrokerPlugin(final Broker broker) {
-        if (selectorCachePlugin == null) {
-            selectorCachePlugin = (SubQueueSelectorCacheBroker) broker.getAdaptor(SubQueueSelectorCacheBroker.class);
-        } //if
-
-        return selectorCachePlugin;
-    }
-
-    /**
      * Pre-compile the JMS selector.
      *
      * @param selectorExpression The non-null JMS selector expression.

http://git-wip-us.apache.org/repos/asf/activemq/blob/9fc26610/activemq-unit-tests/src/test/java/org/apache/activemq/usecases/SelectorAwareVTThatDropsMessagesWhenNoConsumer.java
----------------------------------------------------------------------
diff --git a/activemq-unit-tests/src/test/java/org/apache/activemq/usecases/SelectorAwareVTThatDropsMessagesWhenNoConsumer.java b/activemq-unit-tests/src/test/java/org/apache/activemq/usecases/SelectorAwareVTThatDropsMessagesWhenNoConsumer.java
new file mode 100644
index 0000000..836570e
--- /dev/null
+++ b/activemq-unit-tests/src/test/java/org/apache/activemq/usecases/SelectorAwareVTThatDropsMessagesWhenNoConsumer.java
@@ -0,0 +1,154 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.activemq.usecases;
+
+import org.apache.activemq.ActiveMQConnectionFactory;
+import org.apache.activemq.broker.BrokerFactory;
+import org.apache.activemq.broker.BrokerService;
+import org.apache.activemq.broker.region.DestinationInterceptor;
+import org.apache.activemq.broker.region.virtual.VirtualDestination;
+import org.apache.activemq.broker.region.virtual.VirtualDestinationInterceptor;
+import org.apache.activemq.broker.region.virtual.VirtualTopic;
+import org.apache.activemq.util.Wait;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.jms.Connection;
+import javax.jms.Destination;
+import javax.jms.Message;
+import javax.jms.MessageConsumer;
+import javax.jms.MessageListener;
+import javax.jms.MessageProducer;
+import javax.jms.Queue;
+import javax.jms.Session;
+import javax.jms.TextMessage;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.Assert.assertEquals;
+
+public class SelectorAwareVTThatDropsMessagesWhenNoConsumer {
+    protected static final Logger LOG = LoggerFactory.getLogger(SelectorAwareVTThatDropsMessagesWhenNoConsumer.class);
+    private static final String QUEUE_NAME="AMQ4899TestQueue";
+    private static final String CONSUMER_QUEUE="Consumer.Orders.VirtualOrders." + QUEUE_NAME;
+    private static final String PRODUCER_DESTINATION_NAME = "VirtualOrders." + QUEUE_NAME;
+
+    final AtomicInteger receivedCount = new AtomicInteger(0);
+
+    private BrokerService broker;
+
+    @Before
+    public void setUp() {
+        setupBroker("broker://()/localhost?");
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        if (broker != null) {
+            broker.stop();
+            broker.waitUntilStopped();
+        }
+    }
+
+    @Test(timeout = 60 * 1000)
+    public void testVirtualTopicMultipleSelectors() throws Exception{
+        ActiveMQConnectionFactory factory = new ActiveMQConnectionFactory("vm://localhost");
+        Connection connection = factory.createConnection();
+        connection.start();
+        Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
+
+        Queue consumerQueue = session.createQueue(CONSUMER_QUEUE);
+        MessageListener listenerA = new CountingListener(receivedCount);
+        MessageConsumer consumerA = session.createConsumer(consumerQueue);
+        consumerA.setMessageListener(listenerA);
+
+        Destination producerDestination = session.createTopic(PRODUCER_DESTINATION_NAME);
+        MessageProducer producer = session.createProducer(producerDestination);
+        TextMessage message = session.createTextMessage("bla");
+        producer.send(message);
+        producer.send(message);
+
+        Wait.waitFor(new Wait.Condition() {
+            @Override
+            public boolean isSatisified() throws Exception {
+                return receivedCount.get() == 2;
+            }
+        });
+
+        consumerA.close();
+
+        producer.send(message);
+        producer.send(message);
+
+        assertEquals(2, receivedCount.get());
+
+        LOG.debug("Restarting consumerA");
+        consumerA = session.createConsumer(consumerQueue);
+        consumerA.setMessageListener(listenerA);
+
+        producer.send(message);
+
+        Wait.waitFor(new Wait.Condition() {
+            @Override
+            public boolean isSatisified() throws Exception {
+                return receivedCount.get() == 3;
+            }
+        });
+
+        assertEquals(3, receivedCount.get());
+        connection.close();
+    }
+
+    /**
+     * Setup broker with VirtualTopic configured
+     */
+    private void setupBroker(String uri) {
+        try {
+            broker = BrokerFactory.createBroker(uri);
+
+            VirtualDestinationInterceptor interceptor = new VirtualDestinationInterceptor();
+            VirtualTopic virtualTopic = new VirtualTopic();
+            virtualTopic.setName("VirtualOrders.>");
+            virtualTopic.setSelectorAware(true);
+            VirtualDestination[] virtualDestinations = { virtualTopic };
+            interceptor.setVirtualDestinations(virtualDestinations);
+            broker.setDestinationInterceptors(new DestinationInterceptor[]{interceptor});
+
+            broker.setUseJmx(false);
+            broker.start();
+            broker.waitUntilStarted();
+        } catch (Exception e) {
+            LOG.error("Failed creating broker", e);
+        }
+    }
+
+    class CountingListener implements MessageListener {
+        AtomicInteger counter;
+
+        public CountingListener(AtomicInteger counter) {
+            this.counter = counter;
+        }
+
+        @Override
+        public void onMessage(Message message) {
+            counter.incrementAndGet();
+        }
+    }
+}
+