From bc45bf806569fbcc21a4765b69f46a463dd8b2bf Mon Sep 17 00:00:00 2001 From: Gary Tully Date: Fri, 21 Dec 2012 14:41:23 +0000 Subject: [PATCH] https://issues.apache.org/jira/browse/AMQ-4234 https://issues.apache.org/jira/browse/AMQ-4235 - ensure prefetch set via consumer control gets interepreted by policy such that defaults can be overidden, tidy up default prefetch values choices for stomp so they can pick up broker overrides git-svn-id: https://svn.apache.org/repos/asf/activemq/trunk@1424936 13f79535-47bb-0310-9956-ffa450edef68 --- .../broker/region/AbstractRegion.java | 9 +- .../broker/region/policy/PolicyEntry.java | 60 ++++----- .../activemq/ZeroPrefetchConsumerTest.java | 29 +++++ .../transport/stomp/ProtocolConverter.java | 9 +- .../transport/stomp/StompConnection.java | 4 + .../transport/stomp/StompPrefetchTest.java | 119 ++++++++++++++++++ 6 files changed, 199 insertions(+), 31 deletions(-) create mode 100644 activemq-stomp/src/test/java/org/apache/activemq/transport/stomp/StompPrefetchTest.java diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/region/AbstractRegion.java b/activemq-broker/src/main/java/org/apache/activemq/broker/region/AbstractRegion.java index 894a211420..19f734614f 100755 --- a/activemq-broker/src/main/java/org/apache/activemq/broker/region/AbstractRegion.java +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/region/AbstractRegion.java @@ -31,6 +31,7 @@ import org.apache.activemq.broker.ConsumerBrokerExchange; import org.apache.activemq.broker.DestinationAlreadyExistsException; import org.apache.activemq.broker.ProducerBrokerExchange; import org.apache.activemq.broker.TransportConnection; +import org.apache.activemq.broker.region.policy.PolicyEntry; import org.apache.activemq.command.ActiveMQDestination; import org.apache.activemq.command.ConsumerControl; import org.apache.activemq.command.ConsumerId; @@ -582,9 +583,15 @@ public abstract class AbstractRegion implements Region { Subscription sub = subscriptions.get(control.getConsumerId()); if (sub != null && sub instanceof AbstractSubscription) { ((AbstractSubscription) sub).setPrefetchSize(control.getPrefetch()); + if (broker.getDestinationPolicy() != null) { + PolicyEntry entry = broker.getDestinationPolicy().getEntryFor(control.getDestination()); + if (entry != null) { + entry.configurePrefetch(sub); + } + } if (LOG.isDebugEnabled()) { LOG.debug("setting prefetch: " + control.getPrefetch() + ", on subscription: " - + control.getConsumerId()); + + control.getConsumerId() + "; resulting value: " + sub.getConsumerInfo().getCurrentPrefetchSize()); } try { lookup(consumerExchange.getConnectionContext(), control.getDestination(),false).wakeup(); diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/region/policy/PolicyEntry.java b/activemq-broker/src/main/java/org/apache/activemq/broker/region/policy/PolicyEntry.java index 458d6749bc..3f9ce63237 100644 --- a/activemq-broker/src/main/java/org/apache/activemq/broker/region/policy/PolicyEntry.java +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/region/policy/PolicyEntry.java @@ -24,6 +24,7 @@ import org.apache.activemq.broker.region.DurableTopicSubscription; import org.apache.activemq.broker.region.Queue; import org.apache.activemq.broker.region.QueueBrowserSubscription; import org.apache.activemq.broker.region.QueueSubscription; +import org.apache.activemq.broker.region.Subscription; import org.apache.activemq.broker.region.Topic; import org.apache.activemq.broker.region.TopicSubscription; import org.apache.activemq.broker.region.cursors.PendingMessageCursor; @@ -182,11 +183,7 @@ public class PolicyEntry extends DestinationMapEntry { } public void configure(Broker broker, SystemUsage memoryManager, TopicSubscription subscription) { - //override prefetch size if not set by the Consumer - int prefetch=subscription.getConsumerInfo().getPrefetchSize(); - if (prefetch == ActiveMQPrefetchPolicy.DEFAULT_TOPIC_PREFETCH){ - subscription.getConsumerInfo().setPrefetchSize(getTopicPrefetch()); - } + configurePrefetch(subscription); if (pendingMessageLimitStrategy != null) { int value = pendingMessageLimitStrategy.getMaximumPendingMessageLimit(subscription); int consumerLimit = subscription.getInfo().getMaximumPendingMessageLimit(); @@ -220,12 +217,8 @@ public class PolicyEntry extends DestinationMapEntry { public void configure(Broker broker, SystemUsage memoryManager, DurableTopicSubscription sub) { String clientId = sub.getSubscriptionKey().getClientId(); String subName = sub.getSubscriptionKey().getSubscriptionName(); - int prefetch = sub.getPrefetchSize(); sub.setCursorMemoryHighWaterMark(getCursorMemoryHighWaterMark()); - //override prefetch size if not set by the Consumer - if (prefetch == ActiveMQPrefetchPolicy.DEFAULT_DURABLE_TOPIC_PREFETCH || prefetch == ActiveMQPrefetchPolicy.DEFAULT_OPTIMIZE_DURABLE_TOPIC_PREFETCH){ - sub.setPrefetchSize(getDurableTopicPrefetch()); - } + configurePrefetch(sub); if (pendingDurableSubscriberPolicy != null) { PendingMessageCursor cursor = pendingDurableSubscriberPolicy.getSubscriberPendingMessageCursor(broker,clientId, subName,sub.getPrefetchSize(),sub); cursor.setSystemUsage(memoryManager); @@ -242,33 +235,44 @@ public class PolicyEntry extends DestinationMapEntry { } public void configure(Broker broker, SystemUsage memoryManager, QueueBrowserSubscription sub) { - - int prefetch = sub.getPrefetchSize(); - //override prefetch size if not set by the Consumer - - if (prefetch == ActiveMQPrefetchPolicy.DEFAULT_QUEUE_BROWSER_PREFETCH){ - sub.setPrefetchSize(getQueueBrowserPrefetch()); - } + configurePrefetch(sub); sub.setCursorMemoryHighWaterMark(getCursorMemoryHighWaterMark()); sub.setUsePrefetchExtension(isUsePrefetchExtension()); } public void configure(Broker broker, SystemUsage memoryManager, QueueSubscription sub) { - - int prefetch = sub.getPrefetchSize(); - //override prefetch size if not set by the Consumer - - if (prefetch == ActiveMQPrefetchPolicy.DEFAULT_QUEUE_PREFETCH){ - sub.setPrefetchSize(getQueuePrefetch()); - if (sub.getPrefetchSize() == 0) { - // tell the sub so that it can issue a pull request - sub.updateConsumerPrefetch(0); - } - } + configurePrefetch(sub); sub.setCursorMemoryHighWaterMark(getCursorMemoryHighWaterMark()); sub.setUsePrefetchExtension(isUsePrefetchExtension()); } + public void configurePrefetch(Subscription subscription) { + + final int currentPrefetch = subscription.getConsumerInfo().getPrefetchSize(); + if (subscription instanceof QueueBrowserSubscription) { + if (currentPrefetch == ActiveMQPrefetchPolicy.DEFAULT_QUEUE_BROWSER_PREFETCH) { + ((QueueBrowserSubscription) subscription).setPrefetchSize(getQueueBrowserPrefetch()); + } + } else if (subscription instanceof QueueSubscription) { + if (currentPrefetch == ActiveMQPrefetchPolicy.DEFAULT_QUEUE_PREFETCH) { + ((QueueSubscription) subscription).setPrefetchSize(getQueuePrefetch()); + } + } else if (subscription instanceof DurableTopicSubscription) { + if (currentPrefetch == ActiveMQPrefetchPolicy.DEFAULT_DURABLE_TOPIC_PREFETCH || + subscription.getConsumerInfo().getPrefetchSize() == ActiveMQPrefetchPolicy.DEFAULT_OPTIMIZE_DURABLE_TOPIC_PREFETCH) { + ((DurableTopicSubscription)subscription).setPrefetchSize(getDurableTopicPrefetch()); + } + } else if (subscription instanceof TopicSubscription) { + if (currentPrefetch == ActiveMQPrefetchPolicy.DEFAULT_TOPIC_PREFETCH) { + ((TopicSubscription) subscription).setPrefetchSize(getTopicPrefetch()); + } + } + if (currentPrefetch != 0 && subscription.getPrefetchSize() == 0) { + // tell the sub so that it can issue a pull request + subscription.updateConsumerPrefetch(0); + } + } + // Properties // ------------------------------------------------------------------------- public DispatchPolicy getDispatchPolicy() { diff --git a/activemq-core/src/test/java/org/apache/activemq/ZeroPrefetchConsumerTest.java b/activemq-core/src/test/java/org/apache/activemq/ZeroPrefetchConsumerTest.java index 8b39e78523..fd0167313e 100644 --- a/activemq-core/src/test/java/org/apache/activemq/ZeroPrefetchConsumerTest.java +++ b/activemq-core/src/test/java/org/apache/activemq/ZeroPrefetchConsumerTest.java @@ -27,10 +27,14 @@ import javax.jms.Session; import javax.jms.TextMessage; import org.apache.activemq.broker.BrokerService; +import org.apache.activemq.broker.region.Subscription; import org.apache.activemq.broker.region.policy.PolicyEntry; import org.apache.activemq.broker.region.policy.PolicyMap; import org.apache.activemq.command.ActiveMQDestination; import org.apache.activemq.command.ActiveMQQueue; +import org.apache.activemq.command.ConsumerControl; +import org.apache.activemq.command.ConsumerInfo; +import org.apache.activemq.command.ExceptionResponse; import org.apache.activemq.spring.SpringConsumer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -332,6 +336,31 @@ public class ZeroPrefetchConsumerTest extends EmbeddedBrokerTestSupport { assertEquals("Should have received a message!", answer.getText(), "Msg1"); } + // https://issues.apache.org/jira/browse/AMQ-4234 + // https://issues.apache.org/jira/browse/AMQ-4235 + public void testBrokerZeroPrefetchConfigWithConsumerControl() throws Exception { + Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE); + + ActiveMQMessageConsumer consumer = (ActiveMQMessageConsumer) session.createConsumer(brokerZeroQueue); + assertEquals("broker config prefetch in effect", 0, consumer.info.getCurrentPrefetchSize()); + + // verify sub view broker + Subscription sub = + broker.getRegionBroker().getDestinationMap().get(ActiveMQDestination.transform(brokerZeroQueue)).getConsumers().get(0); + assertEquals("broker sub prefetch is correct", 0, sub.getConsumerInfo().getCurrentPrefetchSize()); + + // manipulate Prefetch (like failover and stomp) + ConsumerControl consumerControl = new ConsumerControl(); + consumerControl.setConsumerId(consumer.info.getConsumerId()); + consumerControl.setDestination(ActiveMQDestination.transform(brokerZeroQueue)); + consumerControl.setPrefetch(1000); // default for a q + + Object reply = ((ActiveMQConnection) connection).getTransport().request(consumerControl); + assertTrue("good request", !(reply instanceof ExceptionResponse)); + assertEquals("broker config prefetch in effect", 0, consumer.info.getCurrentPrefetchSize()); + assertEquals("broker sub prefetch is correct", 0, sub.getConsumerInfo().getCurrentPrefetchSize()); + } + @Override protected BrokerService createBroker() throws Exception { BrokerService brokerService = super.createBroker(); diff --git a/activemq-stomp/src/main/java/org/apache/activemq/transport/stomp/ProtocolConverter.java b/activemq-stomp/src/main/java/org/apache/activemq/transport/stomp/ProtocolConverter.java index 1da70cad24..a6a22f17f2 100644 --- a/activemq-stomp/src/main/java/org/apache/activemq/transport/stomp/ProtocolConverter.java +++ b/activemq-stomp/src/main/java/org/apache/activemq/transport/stomp/ProtocolConverter.java @@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import javax.jms.JMSException; +import org.apache.activemq.ActiveMQPrefetchPolicy; import org.apache.activemq.broker.BrokerContext; import org.apache.activemq.broker.BrokerContextAware; import org.apache.activemq.command.ActiveMQDestination; @@ -553,12 +554,15 @@ public class ProtocolConverter { final ActiveMQDestination actualDest = translator.convertDestination(this, destination, true); if (actualDest == null) { - throw new ProtocolException("Invalid Destination."); + throw new ProtocolException("Invalid 'null' Destination."); } final ConsumerId id = new ConsumerId(sessionId, consumerIdGenerator.getNextSequenceId()); ConsumerInfo consumerInfo = new ConsumerInfo(id); - consumerInfo.setPrefetchSize(1000); + consumerInfo.setPrefetchSize(actualDest.isQueue() ? + ActiveMQPrefetchPolicy.DEFAULT_QUEUE_PREFETCH : + headers.containsKey("activemq.subscriptionName") ? + ActiveMQPrefetchPolicy.DEFAULT_DURABLE_TOPIC_PREFETCH : ActiveMQPrefetchPolicy.DEFAULT_TOPIC_PREFETCH); consumerInfo.setDispatchAsync(true); String browser = headers.get(Stomp.Headers.Subscribe.BROWSER); @@ -569,6 +573,7 @@ public class ProtocolConverter { } consumerInfo.setBrowser(true); + consumerInfo.setPrefetchSize(ActiveMQPrefetchPolicy.DEFAULT_QUEUE_BROWSER_PREFETCH); } String selector = headers.remove(Stomp.Headers.Subscribe.SELECTOR); diff --git a/activemq-stomp/src/main/java/org/apache/activemq/transport/stomp/StompConnection.java b/activemq-stomp/src/main/java/org/apache/activemq/transport/stomp/StompConnection.java index a21d0fde9b..7a4ba5e701 100644 --- a/activemq-stomp/src/main/java/org/apache/activemq/transport/stomp/StompConnection.java +++ b/activemq-stomp/src/main/java/org/apache/activemq/transport/stomp/StompConnection.java @@ -129,6 +129,10 @@ public class StompConnection { if (client != null) { headers.put("client-id", client); } + connect(headers); + } + + public void connect(HashMap headers) throws Exception { StompFrame frame = new StompFrame("CONNECT", headers); sendFrame(frame.format()); diff --git a/activemq-stomp/src/test/java/org/apache/activemq/transport/stomp/StompPrefetchTest.java b/activemq-stomp/src/test/java/org/apache/activemq/transport/stomp/StompPrefetchTest.java new file mode 100644 index 0000000000..8618a2873e --- /dev/null +++ b/activemq-stomp/src/test/java/org/apache/activemq/transport/stomp/StompPrefetchTest.java @@ -0,0 +1,119 @@ +/** + * 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.transport.stomp; + +import java.util.HashMap; +import javax.jms.Connection; +import javax.jms.Destination; +import javax.jms.Message; +import javax.jms.MessageConsumer; +import javax.jms.MessageProducer; +import javax.jms.Session; +import org.apache.activemq.ActiveMQConnection; +import org.apache.activemq.broker.region.Subscription; +import org.apache.activemq.broker.region.policy.ConstantPendingMessageLimitStrategy; +import org.apache.activemq.broker.region.policy.PolicyEntry; +import org.apache.activemq.broker.region.policy.PolicyMap; +import org.apache.activemq.command.ActiveMQDestination; +import org.apache.activemq.command.ActiveMQQueue; +import org.apache.activemq.command.ActiveMQTopic; +import org.apache.activemq.util.Wait; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +import static org.junit.Assert.*; + +public class StompPrefetchTest extends StompTestSupport { + private static final Logger LOG = LoggerFactory.getLogger(StompPrefetchTest.class); + + @Override + protected void applyBrokerPolicies() throws Exception { + + PolicyEntry policy = new PolicyEntry(); + policy.setQueuePrefetch(10); + policy.setTopicPrefetch(10); + policy.setDurableTopicPrefetch(10); + policy.setQueueBrowserPrefetch(10); + PolicyMap pMap = new PolicyMap(); + pMap.setDefaultEntry(policy); + + brokerService.setDestinationPolicy(pMap); + brokerService.setAdvisorySupport(true); + } + + @Test + public void testTopicSubPrefetch() throws Exception { + + stompConnection.connect("system", "manager"); + stompConnection.subscribe("/topic/T", Stomp.Headers.Subscribe.AckModeValues.AUTO); + + verifyPrefetch(10, new ActiveMQTopic("T")); + } + + @Test + public void testDurableSubPrefetch() throws Exception { + stompConnection.connect("system", "manager"); + HashMap headers = new HashMap(); + headers.put("id", "durablesub"); + stompConnection.subscribe("/topic/T", Stomp.Headers.Subscribe.AckModeValues.AUTO, headers); + + verifyPrefetch(10, new ActiveMQTopic("T")); + } + + @Test + public void testQBrowserSubPrefetch() throws Exception { + HashMap headers = new HashMap(); + headers.put("login","system"); + headers.put("passcode","manager"); + headers.put("id", "aBrowser"); + headers.put("browser", "true"); + headers.put("accept-version","1.1"); + + stompConnection.connect(headers); + stompConnection.subscribe("/queue/Q", Stomp.Headers.Subscribe.AckModeValues.AUTO, headers); + + verifyPrefetch(10, new ActiveMQQueue("Q")); + } + + @Test + public void testQueueSubPrefetch() throws Exception { + stompConnection.connect("system", "manager"); + stompConnection.subscribe("/queue/Q", Stomp.Headers.Subscribe.AckModeValues.AUTO); + + verifyPrefetch(10, new ActiveMQQueue("Q")); + } + + private void verifyPrefetch(final int val, final Destination dest) throws Exception { + assertTrue("success in time", Wait.waitFor(new Wait.Condition() { + @Override + public boolean isSatisified() throws Exception { + try { + Subscription sub = + brokerService.getRegionBroker().getDestinationMap().get(ActiveMQDestination.transform(dest)).getConsumers().get(0); + LOG.info("sub prefetch: " + sub.getConsumerInfo().getPrefetchSize()); + return val == sub.getConsumerInfo().getPrefetchSize(); + } catch (Exception ignored) { + } + return false; + } + })); + } + +}