mirror of https://github.com/apache/activemq.git
AMQ-6422 - include the inflight count in the prefetch for positive remote credit flows. Fix and test
This commit is contained in:
parent
88af1c70d9
commit
88daeec28f
|
@ -45,8 +45,10 @@ import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import javax.jms.InvalidClientIDException;
|
import javax.jms.InvalidClientIDException;
|
||||||
|
|
||||||
import org.apache.activemq.broker.BrokerService;
|
import org.apache.activemq.broker.BrokerService;
|
||||||
|
import org.apache.activemq.broker.region.AbstractRegion;
|
||||||
import org.apache.activemq.broker.region.DurableTopicSubscription;
|
import org.apache.activemq.broker.region.DurableTopicSubscription;
|
||||||
import org.apache.activemq.broker.region.RegionBroker;
|
import org.apache.activemq.broker.region.RegionBroker;
|
||||||
|
import org.apache.activemq.broker.region.Subscription;
|
||||||
import org.apache.activemq.broker.region.TopicRegion;
|
import org.apache.activemq.broker.region.TopicRegion;
|
||||||
import org.apache.activemq.command.ActiveMQDestination;
|
import org.apache.activemq.command.ActiveMQDestination;
|
||||||
import org.apache.activemq.command.ActiveMQTempDestination;
|
import org.apache.activemq.command.ActiveMQTempDestination;
|
||||||
|
@ -712,6 +714,17 @@ public class AmqpConnection implements AmqpProtocolConverter {
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
Subscription lookupPrefetchSubscription(ConsumerInfo consumerInfo) {
|
||||||
|
Subscription subscription = null;
|
||||||
|
try {
|
||||||
|
subscription = ((AbstractRegion)((RegionBroker) brokerService.getBroker().getAdaptor(RegionBroker.class)).getRegion(consumerInfo.getDestination())).getSubscriptions().get(consumerInfo.getConsumerId());
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.warn("Error finding subscription for: " + consumerInfo + ": " + e.getMessage(), false, e);
|
||||||
|
}
|
||||||
|
return subscription;
|
||||||
|
}
|
||||||
|
|
||||||
ActiveMQDestination createTemporaryDestination(final Link link, Symbol[] capabilities) {
|
ActiveMQDestination createTemporaryDestination(final Link link, Symbol[] capabilities) {
|
||||||
ActiveMQDestination rc = null;
|
ActiveMQDestination rc = null;
|
||||||
if (contains(capabilities, TEMP_TOPIC_CAPABILITY)) {
|
if (contains(capabilities, TEMP_TOPIC_CAPABILITY)) {
|
||||||
|
|
|
@ -21,6 +21,7 @@ import static org.apache.activemq.transport.amqp.AmqpSupport.toLong;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
|
|
||||||
|
import org.apache.activemq.broker.region.Subscription;
|
||||||
import org.apache.activemq.command.ActiveMQDestination;
|
import org.apache.activemq.command.ActiveMQDestination;
|
||||||
import org.apache.activemq.command.ActiveMQMessage;
|
import org.apache.activemq.command.ActiveMQMessage;
|
||||||
import org.apache.activemq.command.ConsumerControl;
|
import org.apache.activemq.command.ConsumerControl;
|
||||||
|
@ -52,6 +53,7 @@ import org.apache.qpid.proton.amqp.transport.DeliveryState;
|
||||||
import org.apache.qpid.proton.amqp.transport.ErrorCondition;
|
import org.apache.qpid.proton.amqp.transport.ErrorCondition;
|
||||||
import org.apache.qpid.proton.amqp.transport.SenderSettleMode;
|
import org.apache.qpid.proton.amqp.transport.SenderSettleMode;
|
||||||
import org.apache.qpid.proton.engine.Delivery;
|
import org.apache.qpid.proton.engine.Delivery;
|
||||||
|
import org.apache.qpid.proton.engine.Link;
|
||||||
import org.apache.qpid.proton.engine.Sender;
|
import org.apache.qpid.proton.engine.Sender;
|
||||||
import org.fusesource.hawtbuf.Buffer;
|
import org.fusesource.hawtbuf.Buffer;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
@ -79,6 +81,7 @@ public class AmqpSender extends AmqpAbstractLink<Sender> {
|
||||||
private final String MESSAGE_FORMAT_KEY = outboundTransformer.getPrefixVendor() + "MESSAGE_FORMAT";
|
private final String MESSAGE_FORMAT_KEY = outboundTransformer.getPrefixVendor() + "MESSAGE_FORMAT";
|
||||||
|
|
||||||
private final ConsumerInfo consumerInfo;
|
private final ConsumerInfo consumerInfo;
|
||||||
|
private Subscription subscription;
|
||||||
private final boolean presettle;
|
private final boolean presettle;
|
||||||
|
|
||||||
private boolean draining;
|
private boolean draining;
|
||||||
|
@ -108,6 +111,7 @@ public class AmqpSender extends AmqpAbstractLink<Sender> {
|
||||||
public void open() {
|
public void open() {
|
||||||
if (!isClosed()) {
|
if (!isClosed()) {
|
||||||
session.registerSender(getConsumerId(), this);
|
session.registerSender(getConsumerId(), this);
|
||||||
|
subscription = session.getConnection().lookupPrefetchSubscription(consumerInfo);
|
||||||
}
|
}
|
||||||
|
|
||||||
super.open();
|
super.open();
|
||||||
|
@ -162,13 +166,14 @@ public class AmqpSender extends AmqpAbstractLink<Sender> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void flow() throws Exception {
|
public void flow() throws Exception {
|
||||||
|
Link endpoint = getEndpoint();
|
||||||
if (LOG.isTraceEnabled()) {
|
if (LOG.isTraceEnabled()) {
|
||||||
LOG.trace("Flow: draining={}, drain={} credit={}, remoteCredit={}, queued={}",
|
LOG.trace("Flow: draining={}, drain={} credit={}, remoteCredit={}, queued={}, unsettled={}",
|
||||||
draining, getEndpoint().getDrain(),
|
draining, endpoint.getDrain(),
|
||||||
getEndpoint().getCredit(), getEndpoint().getRemoteCredit(), getEndpoint().getQueued());
|
endpoint.getCredit(), endpoint.getRemoteCredit(), endpoint.getQueued(), endpoint.getUnsettled());
|
||||||
}
|
}
|
||||||
|
|
||||||
if (getEndpoint().getDrain() && !draining) {
|
if (endpoint.getDrain() && !draining) {
|
||||||
|
|
||||||
// Revert to a pull consumer.
|
// Revert to a pull consumer.
|
||||||
ConsumerControl control = new ConsumerControl();
|
ConsumerControl control = new ConsumerControl();
|
||||||
|
@ -207,7 +212,16 @@ public class AmqpSender extends AmqpAbstractLink<Sender> {
|
||||||
ConsumerControl control = new ConsumerControl();
|
ConsumerControl control = new ConsumerControl();
|
||||||
control.setConsumerId(getConsumerId());
|
control.setConsumerId(getConsumerId());
|
||||||
control.setDestination(getDestination());
|
control.setDestination(getDestination());
|
||||||
control.setPrefetch(getEndpoint().getCredit());
|
|
||||||
|
int remoteCredit = endpoint.getRemoteCredit();
|
||||||
|
if (remoteCredit > 0 && subscription != null) {
|
||||||
|
// ensure prefetch exceeds credit + inflight
|
||||||
|
if (remoteCredit + endpoint.getUnsettled() + endpoint.getQueued() > subscription.getPrefetchSize()) {
|
||||||
|
LOG.trace("Adding dispatched size to credit for sub: " + subscription);
|
||||||
|
remoteCredit += subscription.getDispatchedQueueSize();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
control.setPrefetch(remoteCredit);
|
||||||
|
|
||||||
LOG.trace("Flow: update -> consumer control with prefetch {}", control.getPrefetch());
|
LOG.trace("Flow: update -> consumer control with prefetch {}", control.getPrefetch());
|
||||||
|
|
||||||
|
|
|
@ -22,6 +22,10 @@ import static org.junit.Assert.assertNotNull;
|
||||||
import static org.junit.Assert.assertNull;
|
import static org.junit.Assert.assertNull;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
|
import java.util.LinkedList;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
import java.util.concurrent.ExecutorService;
|
||||||
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
import org.apache.activemq.broker.jmx.QueueViewMBean;
|
import org.apache.activemq.broker.jmx.QueueViewMBean;
|
||||||
|
@ -91,6 +95,108 @@ public class AmqpSendReceiveTest extends AmqpClientTestSupport {
|
||||||
connection.close();
|
connection.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test(timeout = 60000)
|
||||||
|
public void testReceiveFlowDispositionSingleCredit() throws Exception {
|
||||||
|
AmqpClient client = createAmqpClient();
|
||||||
|
AmqpConnection connection = client.connect();
|
||||||
|
AmqpSession session = connection.createSession();
|
||||||
|
|
||||||
|
AmqpSender sender = session.createSender("queue://" + getTestName());
|
||||||
|
for (int i=0;i<2; i++) {
|
||||||
|
AmqpMessage message = new AmqpMessage();
|
||||||
|
message.setMessageId("msg" + i);
|
||||||
|
sender.send(message);
|
||||||
|
}
|
||||||
|
sender.close();
|
||||||
|
connection.close();
|
||||||
|
|
||||||
|
LOG.info("Starting consumer connection");
|
||||||
|
connection = client.connect();
|
||||||
|
session = connection.createSession();
|
||||||
|
AmqpReceiver receiver = session.createReceiver("queue://" + getTestName());
|
||||||
|
receiver.flow(1);
|
||||||
|
AmqpMessage received = receiver.receive(5, TimeUnit.SECONDS);
|
||||||
|
assertNotNull(received);
|
||||||
|
|
||||||
|
receiver.flow(1);
|
||||||
|
received.accept();
|
||||||
|
|
||||||
|
received = receiver.receive(5, TimeUnit.SECONDS);
|
||||||
|
assertNotNull(received);
|
||||||
|
received.accept();
|
||||||
|
|
||||||
|
receiver.close();
|
||||||
|
connection.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout = 60000)
|
||||||
|
public void testReceiveFlowDispositionSingleCreditTopic() throws Exception {
|
||||||
|
final AmqpClient client = createAmqpClient();
|
||||||
|
final LinkedList<Throwable> errors = new LinkedList<Throwable>();
|
||||||
|
final CountDownLatch receiverReady = new CountDownLatch(1);
|
||||||
|
ExecutorService executorService = Executors.newCachedThreadPool();
|
||||||
|
|
||||||
|
executorService.submit(new Runnable() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
try {
|
||||||
|
LOG.info("Starting consumer connection");
|
||||||
|
AmqpConnection connection = client.connect();
|
||||||
|
AmqpSession session = connection.createSession();
|
||||||
|
AmqpReceiver receiver = session.createReceiver("topic://" + getTestName());
|
||||||
|
receiver.flow(1);
|
||||||
|
receiverReady.countDown();
|
||||||
|
AmqpMessage received = receiver.receive(5, TimeUnit.SECONDS);
|
||||||
|
assertNotNull(received);
|
||||||
|
|
||||||
|
receiver.flow(1);
|
||||||
|
received.accept();
|
||||||
|
|
||||||
|
received = receiver.receive(5, TimeUnit.SECONDS);
|
||||||
|
assertNotNull(received);
|
||||||
|
received.accept();
|
||||||
|
|
||||||
|
receiver.close();
|
||||||
|
connection.close();
|
||||||
|
|
||||||
|
} catch (Exception error) {
|
||||||
|
errors.add(error);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
// producer
|
||||||
|
executorService.submit(new Runnable() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
try {
|
||||||
|
|
||||||
|
receiverReady.await(20, TimeUnit.SECONDS);
|
||||||
|
AmqpConnection connection = client.connect();
|
||||||
|
AmqpSession session = connection.createSession();
|
||||||
|
|
||||||
|
AmqpSender sender = session.createSender("topic://" + getTestName());
|
||||||
|
for (int i = 0; i < 2; i++) {
|
||||||
|
AmqpMessage message = new AmqpMessage();
|
||||||
|
message.setMessageId("msg" + i);
|
||||||
|
sender.send(message);
|
||||||
|
}
|
||||||
|
sender.close();
|
||||||
|
connection.close();
|
||||||
|
} catch (Exception ignored) {
|
||||||
|
ignored.printStackTrace();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
executorService.shutdown();
|
||||||
|
executorService.awaitTermination(20, TimeUnit.SECONDS);
|
||||||
|
assertTrue("no errors: " + errors, errors.isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test(timeout = 60000)
|
@Test(timeout = 60000)
|
||||||
public void testReceiveWithJMSSelectorFilter() throws Exception {
|
public void testReceiveWithJMSSelectorFilter() throws Exception {
|
||||||
AmqpClient client = createAmqpClient();
|
AmqpClient client = createAmqpClient();
|
||||||
|
|
|
@ -484,7 +484,7 @@ public class RegionBroker extends EmptyBroker {
|
||||||
consumerExchange.getRegion().acknowledge(consumerExchange, ack);
|
consumerExchange.getRegion().acknowledge(consumerExchange, ack);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected Region getRegion(ActiveMQDestination destination) throws JMSException {
|
public Region getRegion(ActiveMQDestination destination) throws JMSException {
|
||||||
switch (destination.getDestinationType()) {
|
switch (destination.getDestinationType()) {
|
||||||
case ActiveMQDestination.QUEUE_TYPE:
|
case ActiveMQDestination.QUEUE_TYPE:
|
||||||
return queueRegion;
|
return queueRegion;
|
||||||
|
|
Loading…
Reference in New Issue