mirror of https://github.com/apache/activemq.git
Add expanded transmit callback interface so that a failure to transmit can be distinguished from normal operation and allow for no further attempts at dispatch fixing the current NPE when async dispatch is enabled. git-svn-id: https://svn.apache.org/repos/asf/activemq/trunk@1432487 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
2a5d0821b1
commit
ae61847d02
|
@ -36,10 +36,44 @@ import java.util.concurrent.atomic.AtomicReference;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
|
|
||||||
import javax.transaction.xa.XAResource;
|
import javax.transaction.xa.XAResource;
|
||||||
|
|
||||||
import org.apache.activemq.advisory.AdvisorySupport;
|
import org.apache.activemq.advisory.AdvisorySupport;
|
||||||
import org.apache.activemq.broker.region.ConnectionStatistics;
|
import org.apache.activemq.broker.region.ConnectionStatistics;
|
||||||
import org.apache.activemq.broker.region.RegionBroker;
|
import org.apache.activemq.broker.region.RegionBroker;
|
||||||
import org.apache.activemq.command.*;
|
import org.apache.activemq.command.ActiveMQDestination;
|
||||||
|
import org.apache.activemq.command.BrokerInfo;
|
||||||
|
import org.apache.activemq.command.Command;
|
||||||
|
import org.apache.activemq.command.CommandTypes;
|
||||||
|
import org.apache.activemq.command.ConnectionControl;
|
||||||
|
import org.apache.activemq.command.ConnectionError;
|
||||||
|
import org.apache.activemq.command.ConnectionId;
|
||||||
|
import org.apache.activemq.command.ConnectionInfo;
|
||||||
|
import org.apache.activemq.command.ConsumerControl;
|
||||||
|
import org.apache.activemq.command.ConsumerId;
|
||||||
|
import org.apache.activemq.command.ConsumerInfo;
|
||||||
|
import org.apache.activemq.command.ControlCommand;
|
||||||
|
import org.apache.activemq.command.DataArrayResponse;
|
||||||
|
import org.apache.activemq.command.DestinationInfo;
|
||||||
|
import org.apache.activemq.command.ExceptionResponse;
|
||||||
|
import org.apache.activemq.command.FlushCommand;
|
||||||
|
import org.apache.activemq.command.IntegerResponse;
|
||||||
|
import org.apache.activemq.command.KeepAliveInfo;
|
||||||
|
import org.apache.activemq.command.Message;
|
||||||
|
import org.apache.activemq.command.MessageAck;
|
||||||
|
import org.apache.activemq.command.MessageDispatch;
|
||||||
|
import org.apache.activemq.command.MessageDispatchNotification;
|
||||||
|
import org.apache.activemq.command.MessagePull;
|
||||||
|
import org.apache.activemq.command.ProducerAck;
|
||||||
|
import org.apache.activemq.command.ProducerId;
|
||||||
|
import org.apache.activemq.command.ProducerInfo;
|
||||||
|
import org.apache.activemq.command.RemoveSubscriptionInfo;
|
||||||
|
import org.apache.activemq.command.Response;
|
||||||
|
import org.apache.activemq.command.SessionId;
|
||||||
|
import org.apache.activemq.command.SessionInfo;
|
||||||
|
import org.apache.activemq.command.ShutdownInfo;
|
||||||
|
import org.apache.activemq.command.TransactionId;
|
||||||
|
import org.apache.activemq.command.TransactionInfo;
|
||||||
|
import org.apache.activemq.command.WireFormatInfo;
|
||||||
import org.apache.activemq.network.DemandForwardingBridge;
|
import org.apache.activemq.network.DemandForwardingBridge;
|
||||||
import org.apache.activemq.network.MBeanNetworkListener;
|
import org.apache.activemq.network.MBeanNetworkListener;
|
||||||
import org.apache.activemq.network.NetworkBridgeConfiguration;
|
import org.apache.activemq.network.NetworkBridgeConfiguration;
|
||||||
|
@ -57,12 +91,12 @@ import org.apache.activemq.thread.TaskRunnerFactory;
|
||||||
import org.apache.activemq.transaction.Transaction;
|
import org.apache.activemq.transaction.Transaction;
|
||||||
import org.apache.activemq.transport.DefaultTransportListener;
|
import org.apache.activemq.transport.DefaultTransportListener;
|
||||||
import org.apache.activemq.transport.ResponseCorrelator;
|
import org.apache.activemq.transport.ResponseCorrelator;
|
||||||
|
import org.apache.activemq.transport.TransmitCallback;
|
||||||
import org.apache.activemq.transport.Transport;
|
import org.apache.activemq.transport.Transport;
|
||||||
import org.apache.activemq.transport.TransportDisposedIOException;
|
import org.apache.activemq.transport.TransportDisposedIOException;
|
||||||
import org.apache.activemq.transport.TransportFactory;
|
import org.apache.activemq.transport.TransportFactory;
|
||||||
import org.apache.activemq.util.IntrospectionSupport;
|
import org.apache.activemq.util.IntrospectionSupport;
|
||||||
import org.apache.activemq.util.MarshallingSupport;
|
import org.apache.activemq.util.MarshallingSupport;
|
||||||
import org.apache.activemq.util.ServiceSupport;
|
|
||||||
import org.apache.activemq.util.URISupport;
|
import org.apache.activemq.util.URISupport;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
@ -174,6 +208,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
*
|
*
|
||||||
* @return size of dispatch queue
|
* @return size of dispatch queue
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public int getDispatchQueueSize() {
|
public int getDispatchQueueSize() {
|
||||||
synchronized (dispatchQueue) {
|
synchronized (dispatchQueue) {
|
||||||
return dispatchQueue.size();
|
return dispatchQueue.size();
|
||||||
|
@ -207,8 +242,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean expected(IOException e) {
|
private boolean expected(IOException e) {
|
||||||
return isStomp() &&
|
return isStomp() && ((e instanceof SocketException && e.getMessage().indexOf("reset") != -1) || e instanceof EOFException);
|
||||||
((e instanceof SocketException && e.getMessage().indexOf("reset") != -1) || e instanceof EOFException);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean isStomp() {
|
private boolean isStomp() {
|
||||||
|
@ -221,6 +255,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
* service exception closes a socket, we should not tie up broker threads
|
* service exception closes a socket, we should not tie up broker threads
|
||||||
* since client sockets may hang or cause deadlocks.
|
* since client sockets may hang or cause deadlocks.
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public void serviceExceptionAsync(final IOException e) {
|
public void serviceExceptionAsync(final IOException e) {
|
||||||
if (asyncException.compareAndSet(false, true)) {
|
if (asyncException.compareAndSet(false, true)) {
|
||||||
new Thread("Async Exception Handler") {
|
new Thread("Async Exception Handler") {
|
||||||
|
@ -237,6 +272,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
* if: the client is closing or broker is closing. Otherwise, the connection
|
* if: the client is closing or broker is closing. Otherwise, the connection
|
||||||
* error transmitted to the client before stopping it's transport.
|
* error transmitted to the client before stopping it's transport.
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public void serviceException(Throwable e) {
|
public void serviceException(Throwable e) {
|
||||||
// are we a transport exception such as not being able to dispatch
|
// are we a transport exception such as not being able to dispatch
|
||||||
// synchronously to a transport
|
// synchronously to a transport
|
||||||
|
@ -282,6 +318,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response service(Command command) {
|
public Response service(Command command) {
|
||||||
MDC.put("activemq.connector", connector.getUri().toString());
|
MDC.put("activemq.connector", connector.getUri().toString());
|
||||||
Response response = null;
|
Response response = null;
|
||||||
|
@ -324,30 +361,36 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return response;
|
return response;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processKeepAlive(KeepAliveInfo info) throws Exception {
|
public Response processKeepAlive(KeepAliveInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processRemoveSubscription(RemoveSubscriptionInfo info) throws Exception {
|
public Response processRemoveSubscription(RemoveSubscriptionInfo info) throws Exception {
|
||||||
broker.removeSubscription(lookupConnectionState(info.getConnectionId()).getContext(), info);
|
broker.removeSubscription(lookupConnectionState(info.getConnectionId()).getContext(), info);
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processWireFormat(WireFormatInfo info) throws Exception {
|
public Response processWireFormat(WireFormatInfo info) throws Exception {
|
||||||
wireFormatInfo = info;
|
wireFormatInfo = info;
|
||||||
protocolVersion.set(info.getVersion());
|
protocolVersion.set(info.getVersion());
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processShutdown(ShutdownInfo info) throws Exception {
|
public Response processShutdown(ShutdownInfo info) throws Exception {
|
||||||
stopAsync();
|
stopAsync();
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processFlush(FlushCommand command) throws Exception {
|
public Response processFlush(FlushCommand command) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processBeginTransaction(TransactionInfo info) throws Exception {
|
public Response processBeginTransaction(TransactionInfo info) throws Exception {
|
||||||
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
||||||
context = null;
|
context = null;
|
||||||
|
@ -365,6 +408,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processEndTransaction(TransactionInfo info) throws Exception {
|
public Response processEndTransaction(TransactionInfo info) throws Exception {
|
||||||
// No need to do anything. This packet is just sent by the client
|
// No need to do anything. This packet is just sent by the client
|
||||||
// make sure he is synced with the server as commit command could
|
// make sure he is synced with the server as commit command could
|
||||||
|
@ -372,6 +416,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processPrepareTransaction(TransactionInfo info) throws Exception {
|
public Response processPrepareTransaction(TransactionInfo info) throws Exception {
|
||||||
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
||||||
context = null;
|
context = null;
|
||||||
|
@ -403,6 +448,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processCommitTransactionOnePhase(TransactionInfo info) throws Exception {
|
public Response processCommitTransactionOnePhase(TransactionInfo info) throws Exception {
|
||||||
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
||||||
context = cs.getContext();
|
context = cs.getContext();
|
||||||
|
@ -411,6 +457,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processCommitTransactionTwoPhase(TransactionInfo info) throws Exception {
|
public Response processCommitTransactionTwoPhase(TransactionInfo info) throws Exception {
|
||||||
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
||||||
context = cs.getContext();
|
context = cs.getContext();
|
||||||
|
@ -419,6 +466,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processRollbackTransaction(TransactionInfo info) throws Exception {
|
public Response processRollbackTransaction(TransactionInfo info) throws Exception {
|
||||||
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
||||||
context = cs.getContext();
|
context = cs.getContext();
|
||||||
|
@ -427,6 +475,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processForgetTransaction(TransactionInfo info) throws Exception {
|
public Response processForgetTransaction(TransactionInfo info) throws Exception {
|
||||||
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
||||||
context = cs.getContext();
|
context = cs.getContext();
|
||||||
|
@ -434,6 +483,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processRecoverTransactions(TransactionInfo info) throws Exception {
|
public Response processRecoverTransactions(TransactionInfo info) throws Exception {
|
||||||
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
||||||
context = cs.getContext();
|
context = cs.getContext();
|
||||||
|
@ -441,6 +491,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return new DataArrayResponse(preparedTransactions);
|
return new DataArrayResponse(preparedTransactions);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processMessage(Message messageSend) throws Exception {
|
public Response processMessage(Message messageSend) throws Exception {
|
||||||
ProducerId producerId = messageSend.getProducerId();
|
ProducerId producerId = messageSend.getProducerId();
|
||||||
ProducerBrokerExchange producerExchange = getProducerBrokerExchange(producerId);
|
ProducerBrokerExchange producerExchange = getProducerBrokerExchange(producerId);
|
||||||
|
@ -450,6 +501,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processMessageAck(MessageAck ack) throws Exception {
|
public Response processMessageAck(MessageAck ack) throws Exception {
|
||||||
ConsumerBrokerExchange consumerExchange = getConsumerBrokerExchange(ack.getConsumerId());
|
ConsumerBrokerExchange consumerExchange = getConsumerBrokerExchange(ack.getConsumerId());
|
||||||
if (consumerExchange != null) {
|
if (consumerExchange != null) {
|
||||||
|
@ -458,15 +510,18 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processMessagePull(MessagePull pull) throws Exception {
|
public Response processMessagePull(MessagePull pull) throws Exception {
|
||||||
return broker.messagePull(lookupConnectionState(pull.getConsumerId()).getContext(), pull);
|
return broker.messagePull(lookupConnectionState(pull.getConsumerId()).getContext(), pull);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processMessageDispatchNotification(MessageDispatchNotification notification) throws Exception {
|
public Response processMessageDispatchNotification(MessageDispatchNotification notification) throws Exception {
|
||||||
broker.processDispatchNotification(notification);
|
broker.processDispatchNotification(notification);
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processAddDestination(DestinationInfo info) throws Exception {
|
public Response processAddDestination(DestinationInfo info) throws Exception {
|
||||||
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
||||||
broker.addDestinationInfo(cs.getContext(), info);
|
broker.addDestinationInfo(cs.getContext(), info);
|
||||||
|
@ -476,6 +531,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processRemoveDestination(DestinationInfo info) throws Exception {
|
public Response processRemoveDestination(DestinationInfo info) throws Exception {
|
||||||
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
TransportConnectionState cs = lookupConnectionState(info.getConnectionId());
|
||||||
broker.removeDestinationInfo(cs.getContext(), info);
|
broker.removeDestinationInfo(cs.getContext(), info);
|
||||||
|
@ -485,6 +541,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processAddProducer(ProducerInfo info) throws Exception {
|
public Response processAddProducer(ProducerInfo info) throws Exception {
|
||||||
SessionId sessionId = info.getProducerId().getParentId();
|
SessionId sessionId = info.getProducerId().getParentId();
|
||||||
ConnectionId connectionId = sessionId.getParentId();
|
ConnectionId connectionId = sessionId.getParentId();
|
||||||
|
@ -517,6 +574,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processRemoveProducer(ProducerId id) throws Exception {
|
public Response processRemoveProducer(ProducerId id) throws Exception {
|
||||||
SessionId sessionId = id.getParentId();
|
SessionId sessionId = id.getParentId();
|
||||||
ConnectionId connectionId = sessionId.getParentId();
|
ConnectionId connectionId = sessionId.getParentId();
|
||||||
|
@ -535,6 +593,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processAddConsumer(ConsumerInfo info) throws Exception {
|
public Response processAddConsumer(ConsumerInfo info) throws Exception {
|
||||||
SessionId sessionId = info.getConsumerId().getParentId();
|
SessionId sessionId = info.getConsumerId().getParentId();
|
||||||
ConnectionId connectionId = sessionId.getParentId();
|
ConnectionId connectionId = sessionId.getParentId();
|
||||||
|
@ -569,6 +628,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processRemoveConsumer(ConsumerId id, long lastDeliveredSequenceId) throws Exception {
|
public Response processRemoveConsumer(ConsumerId id, long lastDeliveredSequenceId) throws Exception {
|
||||||
SessionId sessionId = id.getParentId();
|
SessionId sessionId = id.getParentId();
|
||||||
ConnectionId connectionId = sessionId.getParentId();
|
ConnectionId connectionId = sessionId.getParentId();
|
||||||
|
@ -593,6 +653,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processAddSession(SessionInfo info) throws Exception {
|
public Response processAddSession(SessionInfo info) throws Exception {
|
||||||
ConnectionId connectionId = info.getSessionId().getParentId();
|
ConnectionId connectionId = info.getSessionId().getParentId();
|
||||||
TransportConnectionState cs = lookupConnectionState(connectionId);
|
TransportConnectionState cs = lookupConnectionState(connectionId);
|
||||||
|
@ -609,6 +670,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processRemoveSession(SessionId id, long lastDeliveredSequenceId) throws Exception {
|
public Response processRemoveSession(SessionId id, long lastDeliveredSequenceId) throws Exception {
|
||||||
ConnectionId connectionId = id.getParentId();
|
ConnectionId connectionId = id.getParentId();
|
||||||
TransportConnectionState cs = lookupConnectionState(connectionId);
|
TransportConnectionState cs = lookupConnectionState(connectionId);
|
||||||
|
@ -642,6 +704,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processAddConnection(ConnectionInfo info) throws Exception {
|
public Response processAddConnection(ConnectionInfo info) throws Exception {
|
||||||
// Older clients should have been defaulting this field to true.. but
|
// Older clients should have been defaulting this field to true.. but
|
||||||
// they were not.
|
// they were not.
|
||||||
|
@ -728,6 +791,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public synchronized Response processRemoveConnection(ConnectionId id, long lastDeliveredSequenceId)
|
public synchronized Response processRemoveConnection(ConnectionId id, long lastDeliveredSequenceId)
|
||||||
throws InterruptedException {
|
throws InterruptedException {
|
||||||
LOG.debug("remove connection id: " + id);
|
LOG.debug("remove connection id: " + id);
|
||||||
|
@ -776,15 +840,18 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processProducerAck(ProducerAck ack) throws Exception {
|
public Response processProducerAck(ProducerAck ack) throws Exception {
|
||||||
// A broker should not get ProducerAck messages.
|
// A broker should not get ProducerAck messages.
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Connector getConnector() {
|
public Connector getConnector() {
|
||||||
return connector;
|
return connector;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public void dispatchSync(Command message) {
|
public void dispatchSync(Command message) {
|
||||||
try {
|
try {
|
||||||
processDispatch(message);
|
processDispatch(message);
|
||||||
|
@ -793,6 +860,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public void dispatchAsync(Command message) {
|
public void dispatchAsync(Command message) {
|
||||||
if (!stopping.get()) {
|
if (!stopping.get()) {
|
||||||
if (taskRunner == null) {
|
if (taskRunner == null) {
|
||||||
|
@ -810,17 +878,17 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
} else {
|
} else {
|
||||||
if (message.isMessageDispatch()) {
|
if (message.isMessageDispatch()) {
|
||||||
MessageDispatch md = (MessageDispatch) message;
|
MessageDispatch md = (MessageDispatch) message;
|
||||||
Runnable sub = md.getTransmitCallback();
|
TransmitCallback sub = md.getTransmitCallback();
|
||||||
broker.postProcessDispatch(md);
|
broker.postProcessDispatch(md);
|
||||||
if (sub != null) {
|
if (sub != null) {
|
||||||
sub.run();
|
sub.onFailure();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void processDispatch(Command command) throws IOException {
|
protected void processDispatch(Command command) throws IOException {
|
||||||
final MessageDispatch messageDispatch = (MessageDispatch) (command.isMessageDispatch() ? command : null);
|
MessageDispatch messageDispatch = (MessageDispatch) (command.isMessageDispatch() ? command : null);
|
||||||
try {
|
try {
|
||||||
if (!stopping.get()) {
|
if (!stopping.get()) {
|
||||||
if (messageDispatch != null) {
|
if (messageDispatch != null) {
|
||||||
|
@ -828,17 +896,27 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
}
|
}
|
||||||
dispatch(command);
|
dispatch(command);
|
||||||
}
|
}
|
||||||
} finally {
|
} catch (Throwable e) {
|
||||||
if (messageDispatch != null) {
|
if (messageDispatch != null) {
|
||||||
Runnable sub = messageDispatch.getTransmitCallback();
|
TransmitCallback sub = messageDispatch.getTransmitCallback();
|
||||||
broker.postProcessDispatch(messageDispatch);
|
broker.postProcessDispatch(messageDispatch);
|
||||||
if (sub != null) {
|
if (sub != null) {
|
||||||
sub.run();
|
sub.onFailure();
|
||||||
|
}
|
||||||
|
messageDispatch = null;
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
if (messageDispatch != null) {
|
||||||
|
TransmitCallback sub = messageDispatch.getTransmitCallback();
|
||||||
|
broker.postProcessDispatch(messageDispatch);
|
||||||
|
if (sub != null) {
|
||||||
|
sub.onSuccess();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public boolean iterate() {
|
public boolean iterate() {
|
||||||
try {
|
try {
|
||||||
if (pendingStop || stopping.get()) {
|
if (pendingStop || stopping.get()) {
|
||||||
|
@ -877,6 +955,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
/**
|
/**
|
||||||
* Returns the statistics for this connection
|
* Returns the statistics for this connection
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public ConnectionStatistics getStatistics() {
|
public ConnectionStatistics getStatistics() {
|
||||||
return statistics;
|
return statistics;
|
||||||
}
|
}
|
||||||
|
@ -889,10 +968,12 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
this.messageAuthorizationPolicy = messageAuthorizationPolicy;
|
this.messageAuthorizationPolicy = messageAuthorizationPolicy;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public boolean isManageable() {
|
public boolean isManageable() {
|
||||||
return manageable;
|
return manageable;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public void start() throws Exception {
|
public void start() throws Exception {
|
||||||
try {
|
try {
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
|
@ -931,6 +1012,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public void stop() throws Exception {
|
public void stop() throws Exception {
|
||||||
// do not stop task the task runner factories (taskRunnerFactory, stopTaskRunnerFactory)
|
// do not stop task the task runner factories (taskRunnerFactory, stopTaskRunnerFactory)
|
||||||
// as their lifecycle is handled elsewhere
|
// as their lifecycle is handled elsewhere
|
||||||
|
@ -949,6 +1031,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
stopTaskRunnerFactory.execute(new Runnable() {
|
stopTaskRunnerFactory.execute(new Runnable() {
|
||||||
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
try {
|
try {
|
||||||
Thread.sleep(waitTime);
|
Thread.sleep(waitTime);
|
||||||
|
@ -985,6 +1068,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
stopTaskRunnerFactory.execute(new Runnable() {
|
stopTaskRunnerFactory.execute(new Runnable() {
|
||||||
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
serviceLock.writeLock().lock();
|
serviceLock.writeLock().lock();
|
||||||
try {
|
try {
|
||||||
|
@ -1039,10 +1123,10 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
Command command = iter.next();
|
Command command = iter.next();
|
||||||
if (command.isMessageDispatch()) {
|
if (command.isMessageDispatch()) {
|
||||||
MessageDispatch md = (MessageDispatch) command;
|
MessageDispatch md = (MessageDispatch) command;
|
||||||
Runnable sub = md.getTransmitCallback();
|
TransmitCallback sub = md.getTransmitCallback();
|
||||||
broker.postProcessDispatch(md);
|
broker.postProcessDispatch(md);
|
||||||
if (sub != null) {
|
if (sub != null) {
|
||||||
sub.run();
|
sub.onFailure();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1109,6 +1193,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
/**
|
/**
|
||||||
* @return true if the Connection is slow
|
* @return true if the Connection is slow
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public boolean isSlow() {
|
public boolean isSlow() {
|
||||||
return slow;
|
return slow;
|
||||||
}
|
}
|
||||||
|
@ -1132,6 +1217,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
/**
|
/**
|
||||||
* @return if after being marked, the Connection is still writing
|
* @return if after being marked, the Connection is still writing
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public boolean isBlocked() {
|
public boolean isBlocked() {
|
||||||
return blocked;
|
return blocked;
|
||||||
}
|
}
|
||||||
|
@ -1139,6 +1225,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
/**
|
/**
|
||||||
* @return true if the Connection is connected
|
* @return true if the Connection is connected
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public boolean isConnected() {
|
public boolean isConnected() {
|
||||||
return connected;
|
return connected;
|
||||||
}
|
}
|
||||||
|
@ -1160,6 +1247,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
/**
|
/**
|
||||||
* @return true if the Connection is active
|
* @return true if the Connection is active
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public boolean isActive() {
|
public boolean isActive() {
|
||||||
return active;
|
return active;
|
||||||
}
|
}
|
||||||
|
@ -1178,10 +1266,12 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return starting;
|
return starting;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public synchronized boolean isNetworkConnection() {
|
public synchronized boolean isNetworkConnection() {
|
||||||
return networkConnection;
|
return networkConnection;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public boolean isFaultTolerantConnection() {
|
public boolean isFaultTolerantConnection() {
|
||||||
return this.faultTolerantConnection;
|
return this.faultTolerantConnection;
|
||||||
}
|
}
|
||||||
|
@ -1201,9 +1291,9 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
this.pendingStop = pendingStop;
|
this.pendingStop = pendingStop;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processBrokerInfo(BrokerInfo info) {
|
public Response processBrokerInfo(BrokerInfo info) {
|
||||||
if (info.isSlaveBroker()) {
|
if (info.isSlaveBroker()) {
|
||||||
BrokerService bService = connector.getBrokerService();
|
|
||||||
LOG.error(" Slave Brokers are no longer supported - slave trying to attach is: " + info.getBrokerName());
|
LOG.error(" Slave Brokers are no longer supported - slave trying to attach is: " + info.getBrokerName());
|
||||||
} else if (info.isNetworkConnection() && info.isDuplexConnection()) {
|
} else if (info.isNetworkConnection() && info.isDuplexConnection()) {
|
||||||
// so this TransportConnection is the rear end of a network bridge
|
// so this TransportConnection is the rear end of a network bridge
|
||||||
|
@ -1291,10 +1381,12 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public String getRemoteAddress() {
|
public String getRemoteAddress() {
|
||||||
return transport.getRemoteAddress();
|
return transport.getRemoteAddress();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public String getConnectionId() {
|
public String getConnectionId() {
|
||||||
List<TransportConnectionState> connectionStates = listConnectionStates();
|
List<TransportConnectionState> connectionStates = listConnectionStates();
|
||||||
for (TransportConnectionState cs : connectionStates) {
|
for (TransportConnectionState cs : connectionStates) {
|
||||||
|
@ -1306,6 +1398,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public void updateClient(ConnectionControl control) {
|
public void updateClient(ConnectionControl control) {
|
||||||
if (isActive() && isBlocked() == false && isFaultTolerantConnection() && this.wireFormatInfo != null
|
if (isActive() && isBlocked() == false && isFaultTolerantConnection() && this.wireFormatInfo != null
|
||||||
&& this.wireFormatInfo.getVersion() >= 6) {
|
&& this.wireFormatInfo.getVersion() >= 6) {
|
||||||
|
@ -1388,6 +1481,7 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return protocolVersion.get();
|
return protocolVersion.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processControlCommand(ControlCommand command) throws Exception {
|
public Response processControlCommand(ControlCommand command) throws Exception {
|
||||||
String control = command.getCommand();
|
String control = command.getCommand();
|
||||||
if (control != null && control.equals("shutdown")) {
|
if (control != null && control.equals("shutdown")) {
|
||||||
|
@ -1396,10 +1490,12 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processMessageDispatch(MessageDispatch dispatch) throws Exception {
|
public Response processMessageDispatch(MessageDispatch dispatch) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processConnectionControl(ConnectionControl control) throws Exception {
|
public Response processConnectionControl(ConnectionControl control) throws Exception {
|
||||||
if (control != null) {
|
if (control != null) {
|
||||||
faultTolerantConnection = control.isFaultTolerant();
|
faultTolerantConnection = control.isFaultTolerant();
|
||||||
|
@ -1407,10 +1503,12 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processConnectionError(ConnectionError error) throws Exception {
|
public Response processConnectionError(ConnectionError error) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processConsumerControl(ConsumerControl control) throws Exception {
|
public Response processConsumerControl(ConsumerControl control) throws Exception {
|
||||||
ConsumerBrokerExchange consumerExchange = getConsumerBrokerExchange(control.getConsumerId());
|
ConsumerBrokerExchange consumerExchange = getConsumerBrokerExchange(control.getConsumerId());
|
||||||
broker.processConsumerControl(consumerExchange, control);
|
broker.processConsumerControl(consumerExchange, control);
|
||||||
|
|
|
@ -24,7 +24,6 @@ import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
import javax.jms.InvalidSelectorException;
|
|
||||||
import javax.jms.JMSException;
|
import javax.jms.JMSException;
|
||||||
|
|
||||||
import org.apache.activemq.broker.Broker;
|
import org.apache.activemq.broker.Broker;
|
||||||
|
@ -43,6 +42,7 @@ import org.apache.activemq.command.MessagePull;
|
||||||
import org.apache.activemq.command.Response;
|
import org.apache.activemq.command.Response;
|
||||||
import org.apache.activemq.thread.Scheduler;
|
import org.apache.activemq.thread.Scheduler;
|
||||||
import org.apache.activemq.transaction.Synchronization;
|
import org.apache.activemq.transaction.Synchronization;
|
||||||
|
import org.apache.activemq.transport.TransmitCallback;
|
||||||
import org.apache.activemq.usage.SystemUsage;
|
import org.apache.activemq.usage.SystemUsage;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
@ -88,6 +88,7 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
|
||||||
/**
|
/**
|
||||||
* Allows a message to be pulled on demand by a client
|
* Allows a message to be pulled on demand by a client
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public Response pullMessage(ConnectionContext context, MessagePull pull) throws Exception {
|
public Response pullMessage(ConnectionContext context, MessagePull pull) throws Exception {
|
||||||
// The slave should not deliver pull messages.
|
// The slave should not deliver pull messages.
|
||||||
// TODO: when the slave becomes a master, He should send a NULL message to all the
|
// TODO: when the slave becomes a master, He should send a NULL message to all the
|
||||||
|
@ -143,6 +144,7 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public void add(MessageReference node) throws Exception {
|
public void add(MessageReference node) throws Exception {
|
||||||
synchronized (pendingLock) {
|
synchronized (pendingLock) {
|
||||||
// The destination may have just been removed...
|
// The destination may have just been removed...
|
||||||
|
@ -160,6 +162,7 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
|
||||||
dispatchPending();
|
dispatchPending();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public void processMessageDispatchNotification(MessageDispatchNotification mdn) throws Exception {
|
public void processMessageDispatchNotification(MessageDispatchNotification mdn) throws Exception {
|
||||||
synchronized(pendingLock) {
|
synchronized(pendingLock) {
|
||||||
try {
|
try {
|
||||||
|
@ -189,6 +192,7 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
|
||||||
+ mdn.getConsumerId() + " on " + mdn.getDestination().getPhysicalName());
|
+ mdn.getConsumerId() + " on " + mdn.getDestination().getPhysicalName());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public final void acknowledge(final ConnectionContext context,final MessageAck ack) throws Exception {
|
public final void acknowledge(final ConnectionContext context,final MessageAck ack) throws Exception {
|
||||||
// Handle the standard acknowledgment case.
|
// Handle the standard acknowledgment case.
|
||||||
boolean callDispatchMatched = false;
|
boolean callDispatchMatched = false;
|
||||||
|
@ -305,7 +309,7 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
|
||||||
Destination nodeDest = (Destination) node.getRegionDestination();
|
Destination nodeDest = (Destination) node.getRegionDestination();
|
||||||
if (node.isExpired()) {
|
if (node.isExpired()) {
|
||||||
if (broker.isExpired(node)) {
|
if (broker.isExpired(node)) {
|
||||||
Destination regionDestination = (Destination) nodeDest;
|
Destination regionDestination = nodeDest;
|
||||||
regionDestination.messageExpired(context, this, node);
|
regionDestination.messageExpired(context, this, node);
|
||||||
}
|
}
|
||||||
iter.remove();
|
iter.remove();
|
||||||
|
@ -500,6 +504,7 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
|
||||||
broker.getRoot().sendToDeadLetterQueue(context, node, this);
|
broker.getRoot().sendToDeadLetterQueue(context, node, this);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public int getInFlightSize() {
|
public int getInFlightSize() {
|
||||||
return dispatched.size();
|
return dispatched.size();
|
||||||
}
|
}
|
||||||
|
@ -509,6 +514,7 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
|
||||||
*
|
*
|
||||||
* @return
|
* @return
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public boolean isFull() {
|
public boolean isFull() {
|
||||||
return dispatched.size() - prefetchExtension.get() >= info.getPrefetchSize();
|
return dispatched.size() - prefetchExtension.get() >= info.getPrefetchSize();
|
||||||
}
|
}
|
||||||
|
@ -516,6 +522,7 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
|
||||||
/**
|
/**
|
||||||
* @return true when 60% or more room is left for dispatching messages
|
* @return true when 60% or more room is left for dispatching messages
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public boolean isLowWaterMark() {
|
public boolean isLowWaterMark() {
|
||||||
return (dispatched.size() - prefetchExtension.get()) <= (info.getPrefetchSize() * .4);
|
return (dispatched.size() - prefetchExtension.get()) <= (info.getPrefetchSize() * .4);
|
||||||
}
|
}
|
||||||
|
@ -523,6 +530,7 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
|
||||||
/**
|
/**
|
||||||
* @return true when 10% or less room is left for dispatching messages
|
* @return true when 10% or less room is left for dispatching messages
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public boolean isHighWaterMark() {
|
public boolean isHighWaterMark() {
|
||||||
return (dispatched.size() - prefetchExtension.get()) >= (info.getPrefetchSize() * .9);
|
return (dispatched.size() - prefetchExtension.get()) >= (info.getPrefetchSize() * .9);
|
||||||
}
|
}
|
||||||
|
@ -532,22 +540,27 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
|
||||||
return info.getPrefetchSize() + prefetchExtension.get() - dispatched.size();
|
return info.getPrefetchSize() + prefetchExtension.get() - dispatched.size();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public int getPendingQueueSize() {
|
public int getPendingQueueSize() {
|
||||||
return pending.size();
|
return pending.size();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public int getDispatchedQueueSize() {
|
public int getDispatchedQueueSize() {
|
||||||
return dispatched.size();
|
return dispatched.size();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public long getDequeueCounter() {
|
public long getDequeueCounter() {
|
||||||
return dequeueCounter;
|
return dequeueCounter;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public long getDispatchedCounter() {
|
public long getDispatchedCounter() {
|
||||||
return dispatchCounter;
|
return dispatchCounter;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public long getEnqueueCounter() {
|
public long getEnqueueCounter() {
|
||||||
return enqueueCounter;
|
return enqueueCounter;
|
||||||
}
|
}
|
||||||
|
@ -613,8 +626,7 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
|
||||||
setPendingBatchSize(pending, numberToDispatch);
|
setPendingBatchSize(pending, numberToDispatch);
|
||||||
int count = 0;
|
int count = 0;
|
||||||
pending.reset();
|
pending.reset();
|
||||||
while (pending.hasNext() && !isFull()
|
while (pending.hasNext() && !isFull() && count < numberToDispatch) {
|
||||||
&& count < numberToDispatch) {
|
|
||||||
MessageReference node = pending.next();
|
MessageReference node = pending.next();
|
||||||
if (node == null) {
|
if (node == null) {
|
||||||
break;
|
break;
|
||||||
|
@ -683,15 +695,29 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (info.isDispatchAsync()) {
|
if (info.isDispatchAsync()) {
|
||||||
md.setTransmitCallback(new Runnable() {
|
md.setTransmitCallback(new TransmitCallback() {
|
||||||
|
|
||||||
public void run() {
|
@Override
|
||||||
// Since the message gets queued up in async dispatch,
|
public void onSuccess() {
|
||||||
// we don't want to
|
// Since the message gets queued up in async dispatch, we don't want to
|
||||||
// decrease the reference count until it gets put on the
|
// decrease the reference count until it gets put on the wire.
|
||||||
// wire.
|
|
||||||
onDispatch(node, message);
|
onDispatch(node, message);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onFailure() {
|
||||||
|
Destination nodeDest = (Destination) node.getRegionDestination();
|
||||||
|
if (nodeDest != null) {
|
||||||
|
if (node != QueueMessageReference.NULL_MESSAGE) {
|
||||||
|
nodeDest.getDestinationStatistics().getDispatched().increment();
|
||||||
|
nodeDest.getDestinationStatistics().getInflight().increment();
|
||||||
|
if (LOG.isTraceEnabled()) {
|
||||||
|
LOG.trace(info.getConsumerId() + " failed to dispatch: " + message.getMessageId() + " - "
|
||||||
|
+ message.getDestination() + ", dispatched: " + dispatchCounter + ", inflight: " + dispatched.size());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
});
|
});
|
||||||
context.getConnection().dispatchAsync(md);
|
context.getConnection().dispatchAsync(md);
|
||||||
} else {
|
} else {
|
||||||
|
@ -728,6 +754,7 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
|
||||||
*
|
*
|
||||||
* @param newPrefetch
|
* @param newPrefetch
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public void updateConsumerPrefetch(int newPrefetch) {
|
public void updateConsumerPrefetch(int newPrefetch) {
|
||||||
if (context != null && context.getConnection() != null && context.getConnection().isManageable()) {
|
if (context != null && context.getConnection() != null && context.getConnection().isManageable()) {
|
||||||
ConsumerControl cc = new ConsumerControl();
|
ConsumerControl cc = new ConsumerControl();
|
||||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.activemq.command.MessagePull;
|
||||||
import org.apache.activemq.command.Response;
|
import org.apache.activemq.command.Response;
|
||||||
import org.apache.activemq.thread.Scheduler;
|
import org.apache.activemq.thread.Scheduler;
|
||||||
import org.apache.activemq.transaction.Synchronization;
|
import org.apache.activemq.transaction.Synchronization;
|
||||||
|
import org.apache.activemq.transport.TransmitCallback;
|
||||||
import org.apache.activemq.usage.SystemUsage;
|
import org.apache.activemq.usage.SystemUsage;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
@ -96,6 +97,7 @@ public class TopicSubscription extends AbstractSubscription {
|
||||||
this.active=true;
|
this.active=true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public void add(MessageReference node) throws Exception {
|
public void add(MessageReference node) throws Exception {
|
||||||
if (isDuplicate(node)) {
|
if (isDuplicate(node)) {
|
||||||
return;
|
return;
|
||||||
|
@ -236,6 +238,7 @@ public class TopicSubscription extends AbstractSubscription {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public void processMessageDispatchNotification(MessageDispatchNotification mdn) {
|
public void processMessageDispatchNotification(MessageDispatchNotification mdn) {
|
||||||
synchronized (matchedListMutex) {
|
synchronized (matchedListMutex) {
|
||||||
try {
|
try {
|
||||||
|
@ -256,6 +259,7 @@ public class TopicSubscription extends AbstractSubscription {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public synchronized void acknowledge(final ConnectionContext context, final MessageAck ack) throws Exception {
|
public synchronized void acknowledge(final ConnectionContext context, final MessageAck ack) throws Exception {
|
||||||
// Handle the standard acknowledgment case.
|
// Handle the standard acknowledgment case.
|
||||||
if (ack.isStandardAck() || ack.isPoisonAck() || ack.isIndividualAck()) {
|
if (ack.isStandardAck() || ack.isPoisonAck() || ack.isIndividualAck()) {
|
||||||
|
@ -299,6 +303,7 @@ public class TopicSubscription extends AbstractSubscription {
|
||||||
throw new JMSException("Invalid acknowledgment: " + ack);
|
throw new JMSException("Invalid acknowledgment: " + ack);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response pullMessage(ConnectionContext context, MessagePull pull) throws Exception {
|
public Response pullMessage(ConnectionContext context, MessagePull pull) throws Exception {
|
||||||
|
|
||||||
// The slave should not deliver pull messages.
|
// The slave should not deliver pull messages.
|
||||||
|
@ -320,6 +325,7 @@ public class TopicSubscription extends AbstractSubscription {
|
||||||
if (pull.getTimeout() > 0) {
|
if (pull.getTimeout() > 0) {
|
||||||
scheduler.executeAfterDelay(new Runnable() {
|
scheduler.executeAfterDelay(new Runnable() {
|
||||||
|
|
||||||
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
pullTimeout();
|
pullTimeout();
|
||||||
}
|
}
|
||||||
|
@ -346,10 +352,12 @@ public class TopicSubscription extends AbstractSubscription {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public int getPendingQueueSize() {
|
public int getPendingQueueSize() {
|
||||||
return matched();
|
return matched();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public int getDispatchedQueueSize() {
|
public int getDispatchedQueueSize() {
|
||||||
return (int)(dispatchedCounter.get() - dequeueCounter.get());
|
return (int)(dispatchedCounter.get() - dequeueCounter.get());
|
||||||
}
|
}
|
||||||
|
@ -358,14 +366,17 @@ public class TopicSubscription extends AbstractSubscription {
|
||||||
return maximumPendingMessages;
|
return maximumPendingMessages;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public long getDispatchedCounter() {
|
public long getDispatchedCounter() {
|
||||||
return dispatchedCounter.get();
|
return dispatchedCounter.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public long getEnqueueCounter() {
|
public long getEnqueueCounter() {
|
||||||
return enqueueCounter.get();
|
return enqueueCounter.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public long getDequeueCounter() {
|
public long getDequeueCounter() {
|
||||||
return dequeueCounter.get();
|
return dequeueCounter.get();
|
||||||
}
|
}
|
||||||
|
@ -445,10 +456,12 @@ public class TopicSubscription extends AbstractSubscription {
|
||||||
|
|
||||||
// Implementation methods
|
// Implementation methods
|
||||||
// -------------------------------------------------------------------------
|
// -------------------------------------------------------------------------
|
||||||
|
@Override
|
||||||
public boolean isFull() {
|
public boolean isFull() {
|
||||||
return getDispatchedQueueSize() >= info.getPrefetchSize() && !prefetchWindowOpen.get();
|
return getDispatchedQueueSize() >= info.getPrefetchSize() && !prefetchWindowOpen.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public int getInFlightSize() {
|
public int getInFlightSize() {
|
||||||
return getDispatchedQueueSize();
|
return getDispatchedQueueSize();
|
||||||
}
|
}
|
||||||
|
@ -456,6 +469,7 @@ public class TopicSubscription extends AbstractSubscription {
|
||||||
/**
|
/**
|
||||||
* @return true when 60% or more room is left for dispatching messages
|
* @return true when 60% or more room is left for dispatching messages
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public boolean isLowWaterMark() {
|
public boolean isLowWaterMark() {
|
||||||
return getDispatchedQueueSize() <= (info.getPrefetchSize() * .4);
|
return getDispatchedQueueSize() <= (info.getPrefetchSize() * .4);
|
||||||
}
|
}
|
||||||
|
@ -463,6 +477,7 @@ public class TopicSubscription extends AbstractSubscription {
|
||||||
/**
|
/**
|
||||||
* @return true when 10% or less room is left for dispatching messages
|
* @return true when 10% or less room is left for dispatching messages
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public boolean isHighWaterMark() {
|
public boolean isHighWaterMark() {
|
||||||
return getDispatchedQueueSize() >= (info.getPrefetchSize() * .9);
|
return getDispatchedQueueSize() >= (info.getPrefetchSize() * .9);
|
||||||
}
|
}
|
||||||
|
@ -507,6 +522,7 @@ public class TopicSubscription extends AbstractSubscription {
|
||||||
*
|
*
|
||||||
* @param newPrefetch
|
* @param newPrefetch
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public void updateConsumerPrefetch(int newPrefetch) {
|
public void updateConsumerPrefetch(int newPrefetch) {
|
||||||
if (context != null && context.getConnection() != null && context.getConnection().isManageable()) {
|
if (context != null && context.getConnection() != null && context.getConnection().isManageable()) {
|
||||||
ConsumerControl cc = new ConsumerControl();
|
ConsumerControl cc = new ConsumerControl();
|
||||||
|
@ -567,9 +583,18 @@ public class TopicSubscription extends AbstractSubscription {
|
||||||
}
|
}
|
||||||
if (info.isDispatchAsync()) {
|
if (info.isDispatchAsync()) {
|
||||||
if (node != null) {
|
if (node != null) {
|
||||||
md.setTransmitCallback(new Runnable() {
|
md.setTransmitCallback(new TransmitCallback() {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void onSuccess() {
|
||||||
|
Destination regionDestination = (Destination) node.getRegionDestination();
|
||||||
|
regionDestination.getDestinationStatistics().getDispatched().increment();
|
||||||
|
regionDestination.getDestinationStatistics().getInflight().increment();
|
||||||
|
node.decrementReferenceCount();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onFailure() {
|
||||||
Destination regionDestination = (Destination) node.getRegionDestination();
|
Destination regionDestination = (Destination) node.getRegionDestination();
|
||||||
regionDestination.getDestinationStatistics().getDispatched().increment();
|
regionDestination.getDestinationStatistics().getDispatched().increment();
|
||||||
regionDestination.getDestinationStatistics().getInflight().increment();
|
regionDestination.getDestinationStatistics().getInflight().increment();
|
||||||
|
@ -612,6 +637,7 @@ public class TopicSubscription extends AbstractSubscription {
|
||||||
+ getDequeueCounter() + ", matched=" + matched() + ", discarded=" + discarded();
|
+ getDequeueCounter() + ", matched=" + matched() + ", discarded=" + discarded();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public void destroy() {
|
public void destroy() {
|
||||||
this.active=false;
|
this.active=false;
|
||||||
synchronized (matchedListMutex) {
|
synchronized (matchedListMutex) {
|
||||||
|
|
|
@ -21,6 +21,7 @@ import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.CopyOnWriteArrayList;
|
import java.util.concurrent.CopyOnWriteArrayList;
|
||||||
|
|
||||||
import org.apache.activemq.advisory.AdvisorySupport;
|
import org.apache.activemq.advisory.AdvisorySupport;
|
||||||
import org.apache.activemq.broker.Broker;
|
import org.apache.activemq.broker.Broker;
|
||||||
import org.apache.activemq.broker.ConnectionContext;
|
import org.apache.activemq.broker.ConnectionContext;
|
||||||
|
@ -36,8 +37,6 @@ import org.slf4j.LoggerFactory;
|
||||||
/**
|
/**
|
||||||
* persist pending messages pending message (messages awaiting dispatch to a
|
* persist pending messages pending message (messages awaiting dispatch to a
|
||||||
* consumer) cursor
|
* consumer) cursor
|
||||||
*
|
|
||||||
*
|
|
||||||
*/
|
*/
|
||||||
public class StoreDurableSubscriberCursor extends AbstractPendingMessageCursor {
|
public class StoreDurableSubscriberCursor extends AbstractPendingMessageCursor {
|
||||||
|
|
||||||
|
@ -50,6 +49,7 @@ public class StoreDurableSubscriberCursor extends AbstractPendingMessageCursor {
|
||||||
private PendingMessageCursor currentCursor;
|
private PendingMessageCursor currentCursor;
|
||||||
private final DurableTopicSubscription subscription;
|
private final DurableTopicSubscription subscription;
|
||||||
private boolean immediatePriorityDispatch = true;
|
private boolean immediatePriorityDispatch = true;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param broker Broker for this cursor
|
* @param broker Broker for this cursor
|
||||||
* @param clientId clientId for this cursor
|
* @param clientId clientId for this cursor
|
||||||
|
@ -67,7 +67,7 @@ public class StoreDurableSubscriberCursor extends AbstractPendingMessageCursor {
|
||||||
} else {
|
} else {
|
||||||
this.nonPersistent = new VMPendingMessageCursor(this.prioritizedMessages);
|
this.nonPersistent = new VMPendingMessageCursor(this.prioritizedMessages);
|
||||||
}
|
}
|
||||||
|
|
||||||
this.nonPersistent.setMaxBatchSize(maxBatchSize);
|
this.nonPersistent.setMaxBatchSize(maxBatchSize);
|
||||||
this.nonPersistent.setSystemUsage(systemUsage);
|
this.nonPersistent.setSystemUsage(systemUsage);
|
||||||
this.storePrefetches.add(this.nonPersistent);
|
this.storePrefetches.add(this.nonPersistent);
|
||||||
|
@ -82,7 +82,7 @@ public class StoreDurableSubscriberCursor extends AbstractPendingMessageCursor {
|
||||||
if (!isStarted()) {
|
if (!isStarted()) {
|
||||||
super.start();
|
super.start();
|
||||||
for (PendingMessageCursor tsp : storePrefetches) {
|
for (PendingMessageCursor tsp : storePrefetches) {
|
||||||
tsp.setMessageAudit(getMessageAudit());
|
tsp.setMessageAudit(getMessageAudit());
|
||||||
tsp.start();
|
tsp.start();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -108,7 +108,7 @@ public class StoreDurableSubscriberCursor extends AbstractPendingMessageCursor {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Add a destination
|
* Add a destination
|
||||||
*
|
*
|
||||||
* @param context
|
* @param context
|
||||||
* @param destination
|
* @param destination
|
||||||
* @throws Exception
|
* @throws Exception
|
||||||
|
@ -134,7 +134,7 @@ public class StoreDurableSubscriberCursor extends AbstractPendingMessageCursor {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* remove a destination
|
* remove a destination
|
||||||
*
|
*
|
||||||
* @param context
|
* @param context
|
||||||
* @param destination
|
* @param destination
|
||||||
* @throws Exception
|
* @throws Exception
|
||||||
|
@ -173,7 +173,7 @@ public class StoreDurableSubscriberCursor extends AbstractPendingMessageCursor {
|
||||||
/**
|
/**
|
||||||
* Informs the Broker if the subscription needs to intervention to recover
|
* Informs the Broker if the subscription needs to intervention to recover
|
||||||
* it's state e.g. DurableTopicSubscriber may do
|
* it's state e.g. DurableTopicSubscriber may do
|
||||||
*
|
*
|
||||||
* @see org.apache.activemq.broker.region.cursors.AbstractPendingMessageCursor
|
* @see org.apache.activemq.broker.region.cursors.AbstractPendingMessageCursor
|
||||||
* @return true if recovery required
|
* @return true if recovery required
|
||||||
*/
|
*/
|
||||||
|
@ -290,6 +290,7 @@ public class StoreDurableSubscriberCursor extends AbstractPendingMessageCursor {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized void release() {
|
public synchronized void release() {
|
||||||
|
this.currentCursor = null;
|
||||||
for (PendingMessageCursor storePrefetch : storePrefetches) {
|
for (PendingMessageCursor storePrefetch : storePrefetches) {
|
||||||
storePrefetch.release();
|
storePrefetch.release();
|
||||||
}
|
}
|
||||||
|
@ -326,7 +327,7 @@ public class StoreDurableSubscriberCursor extends AbstractPendingMessageCursor {
|
||||||
tsp.setSystemUsage(usageManager);
|
tsp.setSystemUsage(usageManager);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setMemoryUsageHighWaterMark(int memoryUsageHighWaterMark) {
|
public void setMemoryUsageHighWaterMark(int memoryUsageHighWaterMark) {
|
||||||
super.setMemoryUsageHighWaterMark(memoryUsageHighWaterMark);
|
super.setMemoryUsageHighWaterMark(memoryUsageHighWaterMark);
|
||||||
|
@ -334,7 +335,7 @@ public class StoreDurableSubscriberCursor extends AbstractPendingMessageCursor {
|
||||||
cursor.setMemoryUsageHighWaterMark(memoryUsageHighWaterMark);
|
cursor.setMemoryUsageHighWaterMark(memoryUsageHighWaterMark);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setMaxProducersToAudit(int maxProducersToAudit) {
|
public void setMaxProducersToAudit(int maxProducersToAudit) {
|
||||||
super.setMaxProducersToAudit(maxProducersToAudit);
|
super.setMaxProducersToAudit(maxProducersToAudit);
|
||||||
|
@ -350,7 +351,7 @@ public class StoreDurableSubscriberCursor extends AbstractPendingMessageCursor {
|
||||||
cursor.setMaxAuditDepth(maxAuditDepth);
|
cursor.setMaxAuditDepth(maxAuditDepth);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setEnableAudit(boolean enableAudit) {
|
public void setEnableAudit(boolean enableAudit) {
|
||||||
super.setEnableAudit(enableAudit);
|
super.setEnableAudit(enableAudit);
|
||||||
|
@ -358,7 +359,7 @@ public class StoreDurableSubscriberCursor extends AbstractPendingMessageCursor {
|
||||||
cursor.setEnableAudit(enableAudit);
|
cursor.setEnableAudit(enableAudit);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setUseCache(boolean useCache) {
|
public void setUseCache(boolean useCache) {
|
||||||
super.setUseCache(useCache);
|
super.setUseCache(useCache);
|
||||||
|
@ -366,7 +367,7 @@ public class StoreDurableSubscriberCursor extends AbstractPendingMessageCursor {
|
||||||
cursor.setUseCache(useCache);
|
cursor.setUseCache(useCache);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected synchronized PendingMessageCursor getNextCursor() throws Exception {
|
protected synchronized PendingMessageCursor getNextCursor() throws Exception {
|
||||||
if (currentCursor == null || currentCursor.isEmpty()) {
|
if (currentCursor == null || currentCursor.isEmpty()) {
|
||||||
currentCursor = null;
|
currentCursor = null;
|
||||||
|
@ -384,7 +385,7 @@ public class StoreDurableSubscriberCursor extends AbstractPendingMessageCursor {
|
||||||
}
|
}
|
||||||
return currentCursor;
|
return currentCursor;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "StoreDurableSubscriber(" + clientId + ":" + subscriberName + ")";
|
return "StoreDurableSubscriber(" + clientId + ":" + subscriberName + ")";
|
||||||
|
|
|
@ -17,11 +17,12 @@
|
||||||
package org.apache.activemq.command;
|
package org.apache.activemq.command;
|
||||||
|
|
||||||
import org.apache.activemq.state.CommandVisitor;
|
import org.apache.activemq.state.CommandVisitor;
|
||||||
|
import org.apache.activemq.transport.TransmitCallback;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
* @openwire:marshaller code="21"
|
* @openwire:marshaller code="21"
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
public class MessageDispatch extends BaseCommand {
|
public class MessageDispatch extends BaseCommand {
|
||||||
|
|
||||||
|
@ -34,13 +35,15 @@ public class MessageDispatch extends BaseCommand {
|
||||||
|
|
||||||
protected transient long deliverySequenceId;
|
protected transient long deliverySequenceId;
|
||||||
protected transient Object consumer;
|
protected transient Object consumer;
|
||||||
protected transient Runnable transmitCallback;
|
protected transient TransmitCallback transmitCallback;
|
||||||
protected transient Throwable rollbackCause;
|
protected transient Throwable rollbackCause;
|
||||||
|
|
||||||
|
@Override
|
||||||
public byte getDataStructureType() {
|
public byte getDataStructureType() {
|
||||||
return DATA_STRUCTURE_TYPE;
|
return DATA_STRUCTURE_TYPE;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public boolean isMessageDispatch() {
|
public boolean isMessageDispatch() {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
@ -105,15 +108,16 @@ public class MessageDispatch extends BaseCommand {
|
||||||
this.consumer = consumer;
|
this.consumer = consumer;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response visit(CommandVisitor visitor) throws Exception {
|
public Response visit(CommandVisitor visitor) throws Exception {
|
||||||
return visitor.processMessageDispatch(this);
|
return visitor.processMessageDispatch(this);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Runnable getTransmitCallback() {
|
public TransmitCallback getTransmitCallback() {
|
||||||
return transmitCallback;
|
return transmitCallback;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setTransmitCallback(Runnable transmitCallback) {
|
public void setTransmitCallback(TransmitCallback transmitCallback) {
|
||||||
this.transmitCallback = transmitCallback;
|
this.transmitCallback = transmitCallback;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,25 @@
|
||||||
|
/**
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
public interface TransmitCallback {
|
||||||
|
|
||||||
|
void onSuccess();
|
||||||
|
|
||||||
|
void onFailure();
|
||||||
|
|
||||||
|
}
|
Loading…
Reference in New Issue