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
This commit is contained in:
Gary Tully 2012-12-21 14:41:23 +00:00
parent 0704edab2e
commit bc45bf8065
6 changed files with 199 additions and 31 deletions

View File

@ -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();

View File

@ -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() {

View File

@ -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();

View File

@ -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);

View File

@ -129,6 +129,10 @@ public class StompConnection {
if (client != null) {
headers.put("client-id", client);
}
connect(headers);
}
public void connect(HashMap<String, String> headers) throws Exception {
StompFrame frame = new StompFrame("CONNECT", headers);
sendFrame(frame.format());

View File

@ -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<String,String> headers = new HashMap<String, String>();
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<String,String> headers = new HashMap<String, String>();
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;
}
}));
}
}