ARTEMIS-463 More simplifications on the openwire head
https://issues.apache.org/jira/browse/ARTEMIS-463 This will have some extra refactoring on the protocol head, transferring responsibility to the broker classes in a lot of cases and removing some duplicated code This was a team effort from Clebert Suconic and Howard Gao
This commit is contained in:
parent
6ddf486f8f
commit
3aedf27386
|
@ -1044,35 +1044,31 @@ public class JMSServerManagerImpl implements JMSServerManager, ActivateCallback
|
|||
}
|
||||
}
|
||||
|
||||
private boolean internalCreateQueue(final String queueName,
|
||||
private synchronized boolean internalCreateQueue(final String queueName,
|
||||
final String selectorString,
|
||||
final boolean durable) throws Exception {
|
||||
// TODO: there was an openwire test failng because of this
|
||||
// is this really needed for FailoverClusterTest ?
|
||||
synchronized (queues) {
|
||||
if (queues.get(queueName) != null) {
|
||||
return false;
|
||||
if (queues.get(queueName) != null) {
|
||||
return false;
|
||||
}
|
||||
else {
|
||||
ActiveMQQueue activeMQQueue = ActiveMQDestination.createQueue(queueName);
|
||||
|
||||
// Convert from JMS selector to core filter
|
||||
String coreFilterString = null;
|
||||
|
||||
if (selectorString != null) {
|
||||
coreFilterString = SelectorTranslator.convertToActiveMQFilterString(selectorString);
|
||||
}
|
||||
else {
|
||||
ActiveMQQueue activeMQQueue = ActiveMQDestination.createQueue(queueName);
|
||||
|
||||
// Convert from JMS selector to core filter
|
||||
String coreFilterString = null;
|
||||
Queue queue = server.deployQueue(SimpleString.toSimpleString(activeMQQueue.getAddress()), SimpleString.toSimpleString(activeMQQueue.getAddress()), SimpleString.toSimpleString(coreFilterString), durable, false);
|
||||
|
||||
if (selectorString != null) {
|
||||
coreFilterString = SelectorTranslator.convertToActiveMQFilterString(selectorString);
|
||||
}
|
||||
queues.put(queueName, activeMQQueue);
|
||||
|
||||
Queue queue = server.deployQueue(SimpleString.toSimpleString(activeMQQueue.getAddress()), SimpleString.toSimpleString(activeMQQueue.getAddress()), SimpleString.toSimpleString(coreFilterString), durable, false);
|
||||
this.recoverregistryBindings(queueName, PersistedType.Queue);
|
||||
|
||||
queues.put(queueName, activeMQQueue);
|
||||
jmsManagementService.registerQueue(activeMQQueue, queue);
|
||||
|
||||
this.recoverregistryBindings(queueName, PersistedType.Queue);
|
||||
|
||||
jmsManagementService.registerQueue(activeMQQueue, queue);
|
||||
|
||||
return true;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1084,7 +1080,7 @@ public class JMSServerManagerImpl implements JMSServerManager, ActivateCallback
|
|||
* @return
|
||||
* @throws Exception
|
||||
*/
|
||||
private boolean internalCreateTopic(final String topicName) throws Exception {
|
||||
private synchronized boolean internalCreateTopic(final String topicName) throws Exception {
|
||||
|
||||
if (topics.get(topicName) != null) {
|
||||
return false;
|
||||
|
|
|
@ -20,6 +20,8 @@ import java.util.concurrent.Executor;
|
|||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import org.apache.activemq.artemis.core.io.IOCallback;
|
||||
import org.apache.activemq.artemis.core.server.MessageReference;
|
||||
import org.apache.activemq.artemis.core.transaction.Transaction;
|
||||
import org.apache.activemq.artemis.spi.core.remoting.Connection;
|
||||
import org.apache.activemq.artemis.spi.core.remoting.ReadyListener;
|
||||
import org.apache.qpid.proton.amqp.Binary;
|
||||
|
@ -117,7 +119,7 @@ public class ProtonSessionIntegrationCallback implements AMQPSessionCallback, Se
|
|||
false, // boolean autoCommitAcks,
|
||||
false, // boolean preAcknowledge,
|
||||
true, //boolean xa,
|
||||
(String) null, this, null, true);
|
||||
(String) null, this, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -214,7 +216,12 @@ public class ProtonSessionIntegrationCallback implements AMQPSessionCallback, Se
|
|||
|
||||
@Override
|
||||
public Binary getCurrentTXID() {
|
||||
return new Binary(ByteUtil.longToBytes(serverSession.getCurrentTransaction().getID()));
|
||||
Transaction tx = serverSession.getCurrentTransaction();
|
||||
if (tx == null) {
|
||||
tx = serverSession.newTransaction();
|
||||
serverSession.resetTX(tx);
|
||||
}
|
||||
return new Binary(ByteUtil.longToBytes(tx.getID()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -341,7 +348,7 @@ public class ProtonSessionIntegrationCallback implements AMQPSessionCallback, Se
|
|||
}
|
||||
|
||||
@Override
|
||||
public int sendMessage(ServerMessage message, ServerConsumer consumer, int deliveryCount) {
|
||||
public int sendMessage(MessageReference ref, ServerMessage message, ServerConsumer consumer, int deliveryCount) {
|
||||
|
||||
ProtonPlugSender plugSender = (ProtonPlugSender) consumer.getProtocolContext();
|
||||
|
||||
|
@ -359,7 +366,7 @@ public class ProtonSessionIntegrationCallback implements AMQPSessionCallback, Se
|
|||
}
|
||||
|
||||
@Override
|
||||
public int sendLargeMessage(ServerMessage message, ServerConsumer consumer, long bodySize, int deliveryCount) {
|
||||
public int sendLargeMessage(MessageReference ref, ServerMessage message, ServerConsumer consumer, long bodySize, int deliveryCount) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
|
|
@ -96,7 +96,11 @@ public class MQTTConnectionManager {
|
|||
String id = UUIDGenerator.getInstance().generateStringUUID();
|
||||
ActiveMQServer server = session.getServer();
|
||||
|
||||
ServerSession serverSession = server.createSession(id, username, password, ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE, session.getConnection(), MQTTUtil.SESSION_AUTO_COMMIT_SENDS, MQTTUtil.SESSION_AUTO_COMMIT_ACKS, MQTTUtil.SESSION_PREACKNOWLEDGE, MQTTUtil.SESSION_XA, null, session.getSessionCallback(), null, // Session factory
|
||||
ServerSession serverSession = server.createSession(id, username, password,
|
||||
ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE,
|
||||
session.getConnection(), MQTTUtil.SESSION_AUTO_COMMIT_SENDS,
|
||||
MQTTUtil.SESSION_AUTO_COMMIT_ACKS, MQTTUtil.SESSION_PREACKNOWLEDGE,
|
||||
MQTTUtil.SESSION_XA, null, session.getSessionCallback(),
|
||||
MQTTUtil.SESSION_AUTO_CREATE_QUEUE);
|
||||
return (ServerSessionImpl) serverSession;
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
package org.apache.activemq.artemis.core.protocol.mqtt;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.server.MessageReference;
|
||||
import org.apache.activemq.artemis.core.server.ServerConsumer;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.SessionCallback;
|
||||
|
@ -41,7 +42,7 @@ public class MQTTSessionCallback implements SessionCallback {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int sendMessage(ServerMessage message, ServerConsumer consumer, int deliveryCount) {
|
||||
public int sendMessage(MessageReference referece, ServerMessage message, ServerConsumer consumer, int deliveryCount) {
|
||||
try {
|
||||
session.getMqttPublishManager().sendMessage(message, consumer, deliveryCount);
|
||||
}
|
||||
|
@ -62,8 +63,8 @@ public class MQTTSessionCallback implements SessionCallback {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int sendLargeMessage(ServerMessage message, ServerConsumer consumer, long bodySize, int deliveryCount) {
|
||||
return sendMessage(message, consumer, deliveryCount);
|
||||
public int sendLargeMessage(MessageReference reference, ServerMessage message, ServerConsumer consumer, long bodySize, int deliveryCount) {
|
||||
return sendMessage(reference, message, consumer, deliveryCount);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -1,59 +0,0 @@
|
|||
/*
|
||||
* 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.artemis.core.protocol.openwire;
|
||||
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.core.server.Queue;
|
||||
import org.apache.activemq.artemis.core.server.impl.RefsOperation;
|
||||
import org.apache.activemq.artemis.core.transaction.Transaction;
|
||||
import org.apache.activemq.artemis.core.transaction.impl.TransactionImpl;
|
||||
|
||||
import javax.transaction.xa.Xid;
|
||||
|
||||
public class AMQTransactionImpl extends TransactionImpl {
|
||||
|
||||
private boolean rollbackForClose = false;
|
||||
|
||||
public AMQTransactionImpl(Xid xid, StorageManager storageManager, int timeoutSeconds) {
|
||||
super(xid, storageManager, timeoutSeconds);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RefsOperation createRefsOperation(Queue queue) {
|
||||
return new AMQrefsOperation(queue, storageManager);
|
||||
}
|
||||
|
||||
public class AMQrefsOperation extends RefsOperation {
|
||||
|
||||
public AMQrefsOperation(Queue queue, StorageManager storageManager) {
|
||||
super(queue, storageManager);
|
||||
}
|
||||
|
||||
|
||||
// This is because the Rollbacks happen through the consumer, not through the server's
|
||||
@Override
|
||||
public void afterRollback(Transaction tx) {
|
||||
if (rollbackForClose) {
|
||||
super.afterRollback(tx);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void setRollbackForClose() {
|
||||
this.rollbackForClose = true;
|
||||
}
|
||||
}
|
|
@ -16,15 +16,18 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.core.protocol.openwire;
|
||||
|
||||
import javax.jms.IllegalStateException;
|
||||
import javax.jms.InvalidClientIDException;
|
||||
import javax.jms.InvalidDestinationException;
|
||||
import javax.jms.JMSSecurityException;
|
||||
import javax.transaction.xa.XAException;
|
||||
import javax.transaction.xa.XAResource;
|
||||
import javax.transaction.xa.Xid;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.ListIterator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
@ -45,22 +48,30 @@ import org.apache.activemq.artemis.core.protocol.openwire.amq.AMQConnectionConte
|
|||
import org.apache.activemq.artemis.core.protocol.openwire.amq.AMQConsumer;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.amq.AMQConsumerBrokerExchange;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.amq.AMQProducerBrokerExchange;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.amq.AMQServerConsumer;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.amq.AMQSession;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.amq.AMQSingleConsumerBrokerExchange;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.util.OpenWireUtil;
|
||||
import org.apache.activemq.artemis.core.remoting.FailureListener;
|
||||
import org.apache.activemq.artemis.core.security.CheckType;
|
||||
import org.apache.activemq.artemis.core.security.SecurityAuth;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQMessageBundle;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServer;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
|
||||
import org.apache.activemq.artemis.core.server.BindingQueryResult;
|
||||
import org.apache.activemq.artemis.core.server.MessageReference;
|
||||
import org.apache.activemq.artemis.core.server.Queue;
|
||||
import org.apache.activemq.artemis.core.server.ServerConsumer;
|
||||
import org.apache.activemq.artemis.core.server.ServerSession;
|
||||
import org.apache.activemq.artemis.core.server.SlowConsumerDetectionListener;
|
||||
import org.apache.activemq.artemis.core.server.impl.RefsOperation;
|
||||
import org.apache.activemq.artemis.core.transaction.Transaction;
|
||||
import org.apache.activemq.artemis.core.transaction.TransactionOperationAbstract;
|
||||
import org.apache.activemq.artemis.core.transaction.TransactionPropertyIndexes;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.AbstractRemotingConnection;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
|
||||
import org.apache.activemq.artemis.spi.core.remoting.Connection;
|
||||
import org.apache.activemq.artemis.utils.ConcurrentHashSet;
|
||||
import org.apache.activemq.artemis.utils.UUIDGenerator;
|
||||
import org.apache.activemq.command.ActiveMQDestination;
|
||||
import org.apache.activemq.command.ActiveMQMessage;
|
||||
import org.apache.activemq.command.ActiveMQTopic;
|
||||
|
@ -96,6 +107,7 @@ 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.command.XATransactionId;
|
||||
import org.apache.activemq.openwire.OpenWireFormat;
|
||||
import org.apache.activemq.state.CommandVisitor;
|
||||
import org.apache.activemq.state.ConnectionState;
|
||||
|
@ -130,29 +142,42 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
// Clebert: Artemis session has meta-data support, perhaps we could reuse it here
|
||||
private Map<String, SessionId> sessionIdMap = new ConcurrentHashMap<>();
|
||||
|
||||
|
||||
private final Map<ConsumerId, AMQConsumerBrokerExchange> consumerExchanges = new HashMap<>();
|
||||
private final Map<ProducerId, AMQProducerBrokerExchange> producerExchanges = new HashMap<>();
|
||||
private final Map<ConsumerId, AMQConsumerBrokerExchange> consumerExchanges = new ConcurrentHashMap<>();
|
||||
private final Map<ProducerId, AMQProducerBrokerExchange> producerExchanges = new ConcurrentHashMap<>();
|
||||
|
||||
// Clebert TODO: Artemis already stores the Session. Why do we need a different one here
|
||||
private Map<SessionId, AMQSession> sessions = new ConcurrentHashMap<>();
|
||||
|
||||
|
||||
|
||||
private ConnectionState state;
|
||||
|
||||
private final Set<ActiveMQDestination> tempQueues = new ConcurrentHashSet<>();
|
||||
|
||||
private Map<TransactionId, TransactionInfo> txMap = new ConcurrentHashMap<>();
|
||||
/**
|
||||
* Openwire doesn't sen transactions associated with any sessions.
|
||||
* It will however send beingTX / endTX as it would be doing it with XA Transactions.
|
||||
* But always without any association with Sessions.
|
||||
* This collection will hold nonXA transactions. Hopefully while they are in transit only.
|
||||
*/
|
||||
private Map<TransactionId, Transaction> txMap = new ConcurrentHashMap<>();
|
||||
|
||||
private volatile AMQSession advisorySession;
|
||||
|
||||
private final ActiveMQServer server;
|
||||
|
||||
/**
|
||||
* This is to be used with connection operations that don't have a session.
|
||||
* Such as TM operations.
|
||||
*/
|
||||
private ServerSession internalSession;
|
||||
|
||||
// TODO-NOW: check on why there are two connections created for every createConnection on the client.
|
||||
public OpenWireConnection(Connection connection,
|
||||
ActiveMQServer server,
|
||||
Executor executor,
|
||||
OpenWireProtocolManager openWireProtocolManager,
|
||||
OpenWireFormat wf) {
|
||||
super(connection, executor);
|
||||
this.server = server;
|
||||
this.protocolManager = openWireProtocolManager;
|
||||
this.wireFormat = wf;
|
||||
}
|
||||
|
@ -206,7 +231,6 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
boolean responseRequired = command.isResponseRequired();
|
||||
int commandId = command.getCommandId();
|
||||
|
||||
|
||||
// TODO-NOW: the server should send packets to the client based on the requested times
|
||||
// need to look at what Andy did on AMQP
|
||||
|
||||
|
@ -227,8 +251,10 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
response = command.visit(commandProcessorInstance);
|
||||
}
|
||||
catch (Exception e) {
|
||||
// TODO: logging
|
||||
e.printStackTrace();
|
||||
if (responseRequired) {
|
||||
response = new ExceptionResponse(e);
|
||||
response = convertException(e);
|
||||
}
|
||||
}
|
||||
finally {
|
||||
|
@ -276,6 +302,16 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
}
|
||||
|
||||
public void sendException(Exception e) {
|
||||
Response resp = convertException(e);
|
||||
try {
|
||||
dispatch(resp);
|
||||
}
|
||||
catch (IOException e2) {
|
||||
ActiveMQServerLogger.LOGGER.warn(e.getMessage(), e2);
|
||||
}
|
||||
}
|
||||
|
||||
private Response convertException(Exception e) {
|
||||
Response resp;
|
||||
if (e instanceof ActiveMQSecurityException) {
|
||||
resp = new ExceptionResponse(new JMSSecurityException(e.getMessage()));
|
||||
|
@ -286,12 +322,7 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
else {
|
||||
resp = new ExceptionResponse(e);
|
||||
}
|
||||
try {
|
||||
dispatch(resp);
|
||||
}
|
||||
catch (IOException e2) {
|
||||
ActiveMQServerLogger.LOGGER.warn(e.getMessage(), e2);
|
||||
}
|
||||
return resp;
|
||||
}
|
||||
|
||||
private void setLastCommand(Command command) {
|
||||
|
@ -426,9 +457,7 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
}
|
||||
}
|
||||
|
||||
private void addConsumerBrokerExchange(ConsumerId id,
|
||||
AMQSession amqSession,
|
||||
List<AMQConsumer> consumerList) {
|
||||
private void addConsumerBrokerExchange(ConsumerId id, AMQSession amqSession, List<AMQConsumer> consumerList) {
|
||||
AMQConsumerBrokerExchange result = consumerExchanges.get(id);
|
||||
if (result == null) {
|
||||
if (consumerList.size() == 1) {
|
||||
|
@ -471,12 +500,6 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
return result;
|
||||
}
|
||||
|
||||
private void removeConsumerBrokerExchange(ConsumerId id) {
|
||||
synchronized (consumerExchanges) {
|
||||
consumerExchanges.remove(id);
|
||||
}
|
||||
}
|
||||
|
||||
public void deliverMessage(MessageDispatch dispatch) {
|
||||
Message m = dispatch.getMessage();
|
||||
if (m != null) {
|
||||
|
@ -504,7 +527,7 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
}
|
||||
}
|
||||
|
||||
private void disconnect(ActiveMQException me, String reason, boolean fail) {
|
||||
private void disconnect(ActiveMQException me, String reason, boolean fail) {
|
||||
|
||||
if (context == null || destroyed) {
|
||||
return;
|
||||
|
@ -576,7 +599,7 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
}
|
||||
}
|
||||
|
||||
public AMQConnectionContext initContext(ConnectionInfo info) {
|
||||
public AMQConnectionContext initContext(ConnectionInfo info) throws Exception {
|
||||
WireFormatInfo wireFormatInfo = wireFormat.getPreferedWireFormatInfo();
|
||||
// Older clients should have been defaulting this field to true.. but
|
||||
// they were not.
|
||||
|
@ -608,9 +631,15 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
info.setClientIp(getRemoteAddress());
|
||||
}
|
||||
|
||||
createInternalSession(info);
|
||||
|
||||
return context;
|
||||
}
|
||||
|
||||
private void createInternalSession(ConnectionInfo info) throws Exception {
|
||||
internalSession = server.createSession(UUIDGenerator.getInstance().generateStringUUID(), context.getUserName(), info.getPassword(), -1, this, true, false, false, false, null, null, true);
|
||||
}
|
||||
|
||||
//raise the refCount of context
|
||||
public void reconnect(AMQConnectionContext existingContext, ConnectionInfo info) {
|
||||
this.context = existingContext;
|
||||
|
@ -663,17 +692,17 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
ActiveMQDestination dest = info.getDestination();
|
||||
if (dest.isQueue()) {
|
||||
SimpleString qName = OpenWireUtil.toCoreAddress(dest);
|
||||
QueueBinding binding = (QueueBinding) protocolManager.getServer().getPostOffice().getBinding(qName);
|
||||
QueueBinding binding = (QueueBinding) server.getPostOffice().getBinding(qName);
|
||||
if (binding == null) {
|
||||
if (getState().getInfo() != null) {
|
||||
|
||||
CheckType checkType = dest.isTemporary() ? CheckType.CREATE_NON_DURABLE_QUEUE : CheckType.CREATE_DURABLE_QUEUE;
|
||||
protocolManager.getServer().getSecurityStore().check(qName, checkType, this);
|
||||
server.getSecurityStore().check(qName, checkType, this);
|
||||
|
||||
protocolManager.getServer().checkQueueCreationLimit(getUsername());
|
||||
server.checkQueueCreationLimit(getUsername());
|
||||
}
|
||||
ConnectionInfo connInfo = getState().getInfo();
|
||||
protocolManager.getServer().createQueue(qName, qName, null, connInfo == null ? null : SimpleString.toSimpleString(connInfo.getUserName()), false, dest.isTemporary());
|
||||
server.createQueue(qName, qName, null, connInfo == null ? null : SimpleString.toSimpleString(connInfo.getUserName()), false, dest.isTemporary());
|
||||
}
|
||||
|
||||
if (dest.isTemporary()) {
|
||||
|
@ -690,11 +719,12 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
public void updateConsumer(ConsumerControl consumerControl) {
|
||||
SessionId sessionId = consumerControl.getConsumerId().getParentId();
|
||||
AMQSession amqSession = sessions.get(sessionId);
|
||||
amqSession.updateConsumerPrefetchSize(consumerControl.getConsumerId(), consumerControl.getPrefetch());
|
||||
ConsumerId consumerId = consumerControl.getConsumerId();
|
||||
AMQConsumerBrokerExchange exchange = this.consumerExchanges.get(consumerId);
|
||||
if (exchange != null) {
|
||||
exchange.updateConsumerPrefetchSize(consumerControl.getPrefetch());
|
||||
}
|
||||
}
|
||||
|
||||
public void addConsumer(ConsumerInfo info) throws Exception {
|
||||
|
@ -707,7 +737,7 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
}
|
||||
SessionState ss = cs.getSessionState(sessionId);
|
||||
if (ss == null) {
|
||||
throw new IllegalStateException(protocolManager.getServer() + " Cannot add a consumer to a session that had not been registered: " + sessionId);
|
||||
throw new IllegalStateException(server + " Cannot add a consumer to a session that had not been registered: " + sessionId);
|
||||
}
|
||||
// Avoid replaying dup commands
|
||||
if (!ss.getConsumerIds().contains(info.getConsumerId())) {
|
||||
|
@ -729,13 +759,13 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
|
||||
@Override
|
||||
public void onSlowConsumer(ServerConsumer consumer) {
|
||||
if (consumer instanceof AMQServerConsumer) {
|
||||
AMQServerConsumer serverConsumer = (AMQServerConsumer)consumer;
|
||||
ActiveMQTopic topic = AdvisorySupport.getSlowConsumerAdvisoryTopic(serverConsumer.getAmqConsumer().getOpenwireDestination());
|
||||
if (consumer.getProtocolData() != null && consumer.getProtocolData() instanceof AMQConsumer) {
|
||||
AMQConsumer amqConsumer = (AMQConsumer) consumer.getProtocolData();
|
||||
ActiveMQTopic topic = AdvisorySupport.getSlowConsumerAdvisoryTopic(amqConsumer.getOpenwireDestination());
|
||||
ActiveMQMessage advisoryMessage = new ActiveMQMessage();
|
||||
try {
|
||||
advisoryMessage.setStringProperty(AdvisorySupport.MSG_PROPERTY_CONSUMER_ID, serverConsumer.getAmqConsumer().getId().toString());
|
||||
protocolManager.fireAdvisory(context, topic, advisoryMessage, serverConsumer.getAmqConsumer().getId());
|
||||
advisoryMessage.setStringProperty(AdvisorySupport.MSG_PROPERTY_CONSUMER_ID, amqConsumer.getId().toString());
|
||||
protocolManager.fireAdvisory(context, topic, advisoryMessage, amqConsumer.getId());
|
||||
}
|
||||
catch (Exception e) {
|
||||
// TODO-NOW: LOGGING
|
||||
|
@ -758,9 +788,13 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
}
|
||||
|
||||
public AMQSession addSession(SessionInfo ss, boolean internal) {
|
||||
AMQSession amqSession = new AMQSession(getState().getInfo(), ss, protocolManager.getServer(), this, protocolManager.getScheduledPool(), protocolManager);
|
||||
AMQSession amqSession = new AMQSession(getState().getInfo(), ss, server, this, protocolManager.getScheduledPool());
|
||||
amqSession.initialize();
|
||||
amqSession.setInternal(internal);
|
||||
|
||||
if (internal) {
|
||||
amqSession.disableSecurity();
|
||||
}
|
||||
|
||||
sessions.put(ss.getSessionId(), amqSession);
|
||||
sessionIdMap.put(amqSession.getCoreSession().getName(), ss.getSessionId());
|
||||
return amqSession;
|
||||
|
@ -780,10 +814,10 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
public void removeDestination(ActiveMQDestination dest) throws Exception {
|
||||
if (dest.isQueue()) {
|
||||
SimpleString qName = new SimpleString("jms.queue." + dest.getPhysicalName());
|
||||
protocolManager.getServer().destroyQueue(qName);
|
||||
server.destroyQueue(qName);
|
||||
}
|
||||
else {
|
||||
Bindings bindings = protocolManager.getServer().getPostOffice().getBindingsForAddress(SimpleString.toSimpleString("jms.topic." + dest.getPhysicalName()));
|
||||
Bindings bindings = server.getPostOffice().getBindingsForAddress(SimpleString.toSimpleString("jms.topic." + dest.getPhysicalName()));
|
||||
Iterator<Binding> iterator = bindings.getBindings().iterator();
|
||||
|
||||
while (iterator.hasNext()) {
|
||||
|
@ -815,17 +849,15 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
private void validateDestination(ActiveMQDestination destination) throws Exception {
|
||||
if (destination.isQueue()) {
|
||||
SimpleString physicalName = OpenWireUtil.toCoreAddress(destination);
|
||||
BindingQueryResult result = protocolManager.getServer().bindingQuery(physicalName);
|
||||
BindingQueryResult result = server.bindingQuery(physicalName);
|
||||
if (!result.isExists() && !result.isAutoCreateJmsQueues()) {
|
||||
throw ActiveMQMessageBundle.BUNDLE.noSuchQueue(physicalName);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
CommandProcessor commandProcessorInstance = new CommandProcessor();
|
||||
|
||||
|
||||
// This will listen for commands throught the protocolmanager
|
||||
public class CommandProcessor implements CommandVisitor {
|
||||
|
||||
|
@ -934,18 +966,71 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
|
||||
@Override
|
||||
public Response processRemoveSubscription(RemoveSubscriptionInfo subInfo) throws Exception {
|
||||
protocolManager.removeSubscription(subInfo);
|
||||
SimpleString subQueueName = new SimpleString(org.apache.activemq.artemis.jms.client.ActiveMQDestination.createQueueNameForDurableSubscription(true, subInfo.getClientId(), subInfo.getSubscriptionName()));
|
||||
server.destroyQueue(subQueueName);
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Response processRollbackTransaction(TransactionInfo info) throws Exception {
|
||||
protocolManager.rollbackTransaction(info);
|
||||
TransactionId txId = info.getTransactionId();
|
||||
txMap.remove(txId);
|
||||
Transaction tx = lookupTX(info.getTransactionId(), null);
|
||||
if (info.getTransactionId().isXATransaction() && tx == null) {
|
||||
throw newXAException("Transaction '" + info.getTransactionId() + "' has not been started.", XAException.XAER_NOTA);
|
||||
}
|
||||
else if (tx != null) {
|
||||
|
||||
AMQSession amqSession = (AMQSession) tx.getProtocolData();
|
||||
|
||||
if (amqSession != null) {
|
||||
amqSession.getCoreSession().resetTX(tx);
|
||||
|
||||
try {
|
||||
returnReferences(tx, amqSession);
|
||||
}
|
||||
finally {
|
||||
amqSession.getCoreSession().resetTX(null);
|
||||
}
|
||||
}
|
||||
tx.rollback();
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Openwire will redeliver rolled back references.
|
||||
* We need to return those here.
|
||||
*/
|
||||
private void returnReferences(Transaction tx, AMQSession session) throws Exception {
|
||||
if (session == null || session.isClosed()) {
|
||||
return;
|
||||
}
|
||||
|
||||
RefsOperation oper = (RefsOperation) tx.getProperty(TransactionPropertyIndexes.REFS_OPERATION);
|
||||
|
||||
if (oper != null) {
|
||||
List<MessageReference> ackRefs = oper.getReferencesToAcknowledge();
|
||||
|
||||
for (ListIterator<MessageReference> referenceIterator = ackRefs.listIterator(ackRefs.size()); referenceIterator.hasPrevious(); ) {
|
||||
MessageReference ref = referenceIterator.previous();
|
||||
|
||||
Long consumerID = ref.getConsumerId();
|
||||
|
||||
ServerConsumer consumer = null;
|
||||
if (consumerID != null) {
|
||||
consumer = session.getCoreSession().locateConsumer(consumerID);
|
||||
}
|
||||
|
||||
if (consumer != null) {
|
||||
referenceIterator.remove();
|
||||
ref.incrementDeliveryCount();
|
||||
consumer.backToDelivering(ref);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Response processShutdown(ShutdownInfo info) throws Exception {
|
||||
OpenWireConnection.this.shutdown(false);
|
||||
|
@ -989,43 +1074,134 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
|
||||
@Override
|
||||
public Response processBeginTransaction(TransactionInfo info) throws Exception {
|
||||
TransactionId txId = info.getTransactionId();
|
||||
final TransactionId txID = info.getTransactionId();
|
||||
|
||||
if (!txMap.containsKey(txId)) {
|
||||
txMap.put(txId, info);
|
||||
try {
|
||||
internalSession.resetTX(null);
|
||||
if (txID.isXATransaction()) {
|
||||
Xid xid = OpenWireUtil.toXID(txID);
|
||||
internalSession.xaStart(xid);
|
||||
}
|
||||
else {
|
||||
Transaction transaction = internalSession.newTransaction();
|
||||
txMap.put(txID, transaction);
|
||||
transaction.addOperation(new TransactionOperationAbstract() {
|
||||
@Override
|
||||
public void afterCommit(Transaction tx) {
|
||||
txMap.remove(txID);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
finally {
|
||||
internalSession.resetTX(null);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Response processBrokerInfo(BrokerInfo arg0) throws Exception {
|
||||
throw new IllegalStateException("not implemented! ");
|
||||
public Response processCommitTransactionOnePhase(TransactionInfo info) throws Exception {
|
||||
return processCommit(info, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Response processCommitTransactionOnePhase(TransactionInfo info) throws Exception {
|
||||
try {
|
||||
protocolManager.commitTransactionOnePhase(info);
|
||||
TransactionId txId = info.getTransactionId();
|
||||
txMap.remove(txId);
|
||||
}
|
||||
catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
throw e;
|
||||
}
|
||||
private Response processCommit(TransactionInfo info, boolean onePhase) throws Exception {
|
||||
TransactionId txID = info.getTransactionId();
|
||||
|
||||
Transaction tx = lookupTX(txID, null);
|
||||
|
||||
AMQSession session = (AMQSession) tx.getProtocolData();
|
||||
|
||||
tx.commit(onePhase);
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Response processCommitTransactionTwoPhase(TransactionInfo info) throws Exception {
|
||||
protocolManager.commitTransactionTwoPhase(info);
|
||||
TransactionId txId = info.getTransactionId();
|
||||
txMap.remove(txId);
|
||||
return processCommit(info, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Response processForgetTransaction(TransactionInfo info) throws Exception {
|
||||
TransactionId txID = info.getTransactionId();
|
||||
|
||||
if (txID.isXATransaction()) {
|
||||
try {
|
||||
Xid xid = OpenWireUtil.toXID(info.getTransactionId());
|
||||
internalSession.xaForget(xid);
|
||||
}
|
||||
catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
else {
|
||||
txMap.remove(txID);
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Response processPrepareTransaction(TransactionInfo info) throws Exception {
|
||||
TransactionId txID = info.getTransactionId();
|
||||
|
||||
try {
|
||||
if (txID.isXATransaction()) {
|
||||
try {
|
||||
Xid xid = OpenWireUtil.toXID(info.getTransactionId());
|
||||
internalSession.xaPrepare(xid);
|
||||
}
|
||||
catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
else {
|
||||
Transaction tx = lookupTX(txID, null);
|
||||
tx.prepare();
|
||||
}
|
||||
}
|
||||
finally {
|
||||
internalSession.resetTX(null);
|
||||
}
|
||||
|
||||
return new IntegerResponse(XAResource.XA_RDONLY);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Response processEndTransaction(TransactionInfo info) throws Exception {
|
||||
TransactionId txID = info.getTransactionId();
|
||||
|
||||
if (txID.isXATransaction()) {
|
||||
try {
|
||||
Transaction tx = lookupTX(txID, null);
|
||||
internalSession.resetTX(tx);
|
||||
try {
|
||||
Xid xid = OpenWireUtil.toXID(info.getTransactionId());
|
||||
internalSession.xaEnd(xid);
|
||||
}
|
||||
finally {
|
||||
internalSession.resetTX(null);
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
else {
|
||||
txMap.remove(info);
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Response processBrokerInfo(BrokerInfo arg0) throws Exception {
|
||||
throw new IllegalStateException("not implemented! ");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Response processConnectionControl(ConnectionControl connectionControl) throws Exception {
|
||||
//activemq5 keeps a var to remember only the faultTolerant flag
|
||||
|
@ -1057,31 +1233,11 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
throw new IllegalStateException("not implemented! ");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Response processEndTransaction(TransactionInfo info) throws Exception {
|
||||
protocolManager.endTransaction(info);
|
||||
TransactionId txId = info.getTransactionId();
|
||||
|
||||
if (!txMap.containsKey(txId)) {
|
||||
txMap.put(txId, info);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Response processFlush(FlushCommand arg0) throws Exception {
|
||||
throw new IllegalStateException("not implemented! ");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Response processForgetTransaction(TransactionInfo info) throws Exception {
|
||||
TransactionId txId = info.getTransactionId();
|
||||
txMap.remove(txId);
|
||||
|
||||
protocolManager.forgetTransaction(info.getTransactionId());
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Response processKeepAlive(KeepAliveInfo arg0) throws Exception {
|
||||
throw new IllegalStateException("not implemented! ");
|
||||
|
@ -1097,15 +1253,32 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
|
||||
AMQSession session = getSession(producerId.getParentId());
|
||||
|
||||
session.send(producerInfo, messageSend, sendProducerAck);
|
||||
Transaction tx = lookupTX(messageSend.getTransactionId(), session);
|
||||
|
||||
session.getCoreSession().resetTX(tx);
|
||||
try {
|
||||
session.send(producerInfo, messageSend, sendProducerAck);
|
||||
}
|
||||
finally {
|
||||
session.getCoreSession().resetTX(null);
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Response processMessageAck(MessageAck ack) throws Exception {
|
||||
AMQConsumerBrokerExchange consumerBrokerExchange = consumerExchanges.get(ack.getConsumerId());
|
||||
consumerBrokerExchange.acknowledge(ack);
|
||||
AMQSession session = getSession(ack.getConsumerId().getParentId());
|
||||
Transaction tx = lookupTX(ack.getTransactionId(), session);
|
||||
session.getCoreSession().resetTX(tx);
|
||||
|
||||
try {
|
||||
AMQConsumerBrokerExchange consumerBrokerExchange = consumerExchanges.get(ack.getConsumerId());
|
||||
consumerBrokerExchange.acknowledge(ack);
|
||||
}
|
||||
finally {
|
||||
session.getCoreSession().resetTX(null);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -1129,13 +1302,6 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Response processPrepareTransaction(TransactionInfo info) throws Exception {
|
||||
protocolManager.prepareTransaction(info);
|
||||
//activemq needs a rdonly response
|
||||
return new IntegerResponse(XAResource.XA_RDONLY);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Response processProducerAck(ProducerAck arg0) throws Exception {
|
||||
// a broker doesn't do producers.. this shouldn't happen
|
||||
|
@ -1144,20 +1310,13 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
|
||||
@Override
|
||||
public Response processRecoverTransactions(TransactionInfo info) throws Exception {
|
||||
Set<SessionId> sIds = state.getSessionIds();
|
||||
|
||||
|
||||
List<Xid> xids = server.getResourceManager().getInDoubtTransactions();
|
||||
List<TransactionId> recovered = new ArrayList<>();
|
||||
if (sIds != null) {
|
||||
for (SessionId sid : sIds) {
|
||||
AMQSession s = sessions.get(sid);
|
||||
if (s != null) {
|
||||
s.recover(recovered);
|
||||
}
|
||||
}
|
||||
for (Xid xid : xids) {
|
||||
XATransactionId amqXid = new XATransactionId(xid);
|
||||
recovered.add(amqXid);
|
||||
}
|
||||
|
||||
return new DataArrayResponse(recovered.toArray(new TransactionId[0]));
|
||||
return new DataArrayResponse(recovered.toArray(new TransactionId[recovered.size()]));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1186,15 +1345,45 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
|
|||
ConsumerInfo info = consumerState.getInfo();
|
||||
info.setLastDeliveredSequenceId(lastDeliveredSequenceId);
|
||||
|
||||
AMQConsumerBrokerExchange consumerBrokerExchange = consumerExchanges.get(id);
|
||||
AMQConsumerBrokerExchange consumerBrokerExchange = consumerExchanges.remove(id);
|
||||
|
||||
consumerBrokerExchange.removeConsumer();
|
||||
|
||||
removeConsumerBrokerExchange(id);
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private Transaction lookupTX(TransactionId txID, AMQSession session) throws IllegalStateException {
|
||||
if (txID == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
Xid xid = null;
|
||||
Transaction transaction;
|
||||
if (txID.isXATransaction()) {
|
||||
xid = OpenWireUtil.toXID(txID);
|
||||
transaction = server.getResourceManager().getTransaction(xid);
|
||||
}
|
||||
else {
|
||||
transaction = txMap.get(txID);
|
||||
}
|
||||
|
||||
if (transaction == null) {
|
||||
throw new IllegalStateException("cannot find transactionInfo::" + txID + " xid=" + xid);
|
||||
}
|
||||
|
||||
if (session != null && transaction.getProtocolData() != session) {
|
||||
transaction.setProtocolData(session);
|
||||
}
|
||||
|
||||
return transaction;
|
||||
}
|
||||
|
||||
public static XAException newXAException(String s, int errorCode) {
|
||||
XAException xaException = new XAException(s + " " + "xaErrorCode:" + errorCode);
|
||||
xaException.errorCode = errorCode;
|
||||
return xaException;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
|||
import org.apache.activemq.artemis.api.core.ActiveMQPropertyConversionException;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.amq.AMQConsumer;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.util.OpenWireUtil;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.core.server.impl.ServerMessageImpl;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.MessageConverter;
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
package org.apache.activemq.artemis.core.protocol.openwire;
|
||||
|
||||
import javax.jms.InvalidClientIDException;
|
||||
import javax.transaction.xa.XAException;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
|
@ -33,7 +32,6 @@ import org.apache.activemq.advisory.AdvisorySupport;
|
|||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.BaseInterceptor;
|
||||
import org.apache.activemq.artemis.api.core.Interceptor;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.api.core.client.ClusterTopologyListener;
|
||||
import org.apache.activemq.artemis.api.core.client.TopologyMember;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.amq.AMQConnectionContext;
|
||||
|
@ -65,11 +63,7 @@ import org.apache.activemq.command.MessageDispatch;
|
|||
import org.apache.activemq.command.MessageId;
|
||||
import org.apache.activemq.command.ProducerId;
|
||||
import org.apache.activemq.command.ProducerInfo;
|
||||
import org.apache.activemq.command.RemoveSubscriptionInfo;
|
||||
import org.apache.activemq.command.TransactionId;
|
||||
import org.apache.activemq.command.TransactionInfo;
|
||||
import org.apache.activemq.command.WireFormatInfo;
|
||||
import org.apache.activemq.command.XATransactionId;
|
||||
import org.apache.activemq.openwire.OpenWireFormat;
|
||||
import org.apache.activemq.openwire.OpenWireFormatFactory;
|
||||
import org.apache.activemq.state.ProducerState;
|
||||
|
@ -96,15 +90,10 @@ public class OpenWireProtocolManager implements ProtocolManager<Interceptor>, Cl
|
|||
|
||||
private final CopyOnWriteArrayList<OpenWireConnection> connections = new CopyOnWriteArrayList<>();
|
||||
|
||||
// TODO-NOW: this can probably go away
|
||||
private final Map<String, AMQConnectionContext> clientIdSet = new HashMap<String, AMQConnectionContext>();
|
||||
|
||||
private String brokerName;
|
||||
|
||||
// Clebert: Artemis already has a Resource Manager. Need to remove this..
|
||||
// The TransactionID extends XATransactionID, so all we need is to convert the XID here
|
||||
private Map<TransactionId, AMQSession> transactions = new ConcurrentHashMap<>();
|
||||
|
||||
private final Map<String, TopologyMember> topologyMap = new ConcurrentHashMap<>();
|
||||
|
||||
private final LinkedList<TopologyMember> members = new LinkedList<>();
|
||||
|
@ -140,7 +129,7 @@ public class OpenWireProtocolManager implements ProtocolManager<Interceptor>, Cl
|
|||
}
|
||||
|
||||
public OpenWireFormat getNewWireFormat() {
|
||||
return (OpenWireFormat)wireFactory.createWireFormat();
|
||||
return (OpenWireFormat) wireFactory.createWireFormat();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -156,9 +145,7 @@ public class OpenWireProtocolManager implements ProtocolManager<Interceptor>, Cl
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
public void removeConnection(ConnectionInfo info,
|
||||
Throwable error) throws InvalidClientIDException {
|
||||
public void removeConnection(ConnectionInfo info, Throwable error) throws InvalidClientIDException {
|
||||
synchronized (clientIdSet) {
|
||||
String clientId = info.getClientId();
|
||||
if (clientId != null) {
|
||||
|
@ -176,7 +163,6 @@ public class OpenWireProtocolManager implements ProtocolManager<Interceptor>, Cl
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
public ScheduledExecutorService getScheduledPool() {
|
||||
return scheduledPool;
|
||||
}
|
||||
|
@ -223,7 +209,7 @@ public class OpenWireProtocolManager implements ProtocolManager<Interceptor>, Cl
|
|||
@Override
|
||||
public ConnectionEntry createConnectionEntry(Acceptor acceptorUsed, Connection connection) {
|
||||
OpenWireFormat wf = (OpenWireFormat) wireFactory.createWireFormat();
|
||||
OpenWireConnection owConn = new OpenWireConnection(connection, server.getExecutorFactory().getExecutor(), this, wf);
|
||||
OpenWireConnection owConn = new OpenWireConnection(connection, server, server.getExecutorFactory().getExecutor(), this, wf);
|
||||
owConn.sendHandshake();
|
||||
|
||||
// TODO CLEBERT What is this constant here? we should get it from TTL initial pings
|
||||
|
@ -323,7 +309,7 @@ public class OpenWireProtocolManager implements ProtocolManager<Interceptor>, Cl
|
|||
fireAdvisory(context, topic, copy);
|
||||
|
||||
// init the conn
|
||||
context.getConnection().addSessions( context.getConnectionState().getSessionIds());
|
||||
context.getConnection().addSessions(context.getConnectionState().getSessionIds());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -343,9 +329,9 @@ public class OpenWireProtocolManager implements ProtocolManager<Interceptor>, Cl
|
|||
* See AdvisoryBroker.fireAdvisory()
|
||||
*/
|
||||
public void fireAdvisory(AMQConnectionContext context,
|
||||
ActiveMQTopic topic,
|
||||
Command command,
|
||||
ConsumerId targetConsumerId) throws Exception {
|
||||
ActiveMQTopic topic,
|
||||
Command command,
|
||||
ConsumerId targetConsumerId) throws Exception {
|
||||
ActiveMQMessage advisoryMessage = new ActiveMQMessage();
|
||||
advisoryMessage.setStringProperty(AdvisorySupport.MSG_PROPERTY_ORIGIN_BROKER_NAME, getBrokerName());
|
||||
String id = getBrokerId() != null ? getBrokerId().getValue() : "NOT_SET";
|
||||
|
@ -448,55 +434,6 @@ public class OpenWireProtocolManager implements ProtocolManager<Interceptor>, Cl
|
|||
public boolean isStopping() {
|
||||
return false;
|
||||
}
|
||||
public void endTransaction(TransactionInfo info) throws Exception {
|
||||
AMQSession txSession = transactions.get(info.getTransactionId());
|
||||
|
||||
if (txSession != null) {
|
||||
txSession.endTransaction(info);
|
||||
}
|
||||
}
|
||||
|
||||
public void commitTransactionOnePhase(TransactionInfo info) throws Exception {
|
||||
AMQSession txSession = transactions.get(info.getTransactionId());
|
||||
|
||||
if (txSession != null) {
|
||||
txSession.commitOnePhase(info);
|
||||
}
|
||||
transactions.remove(info.getTransactionId());
|
||||
}
|
||||
|
||||
public void prepareTransaction(TransactionInfo info) throws Exception {
|
||||
XATransactionId xid = (XATransactionId) info.getTransactionId();
|
||||
AMQSession txSession = transactions.get(xid);
|
||||
if (txSession != null) {
|
||||
txSession.prepareTransaction(xid);
|
||||
}
|
||||
}
|
||||
|
||||
public void commitTransactionTwoPhase(TransactionInfo info) throws Exception {
|
||||
XATransactionId xid = (XATransactionId) info.getTransactionId();
|
||||
AMQSession txSession = transactions.get(xid);
|
||||
if (txSession != null) {
|
||||
txSession.commitTwoPhase(xid);
|
||||
}
|
||||
transactions.remove(xid);
|
||||
}
|
||||
|
||||
public void rollbackTransaction(TransactionInfo info) throws Exception {
|
||||
AMQSession txSession = transactions.get(info.getTransactionId());
|
||||
if (txSession != null) {
|
||||
txSession.rollback(info);
|
||||
}
|
||||
else if (info.getTransactionId().isLocalTransaction()) {
|
||||
//during a broker restart, recovered local transaction may not be registered
|
||||
//in that case we ignore and let the tx removed silently by connection.
|
||||
//see AMQ1925Test.testAMQ1925_TXBegin
|
||||
}
|
||||
else {
|
||||
throw newXAException("Transaction '" + info.getTransactionId() + "' has not been started.", XAException.XAER_NOTA);
|
||||
}
|
||||
transactions.remove(info.getTransactionId());
|
||||
}
|
||||
|
||||
public boolean validateUser(String login, String passcode) {
|
||||
boolean validated = true;
|
||||
|
@ -510,26 +447,6 @@ public class OpenWireProtocolManager implements ProtocolManager<Interceptor>, Cl
|
|||
return validated;
|
||||
}
|
||||
|
||||
public void forgetTransaction(TransactionId xid) throws Exception {
|
||||
AMQSession txSession = transactions.get(xid);
|
||||
if (txSession != null) {
|
||||
txSession.forget(xid);
|
||||
}
|
||||
transactions.remove(xid);
|
||||
}
|
||||
|
||||
/**
|
||||
* TODO: remove this, use the regular ResourceManager from the Server's
|
||||
*/
|
||||
public void registerTx(TransactionId txId, AMQSession amqSession) {
|
||||
transactions.put(txId, amqSession);
|
||||
}
|
||||
|
||||
public void removeSubscription(RemoveSubscriptionInfo subInfo) throws Exception {
|
||||
SimpleString subQueueName = new SimpleString(org.apache.activemq.artemis.jms.client.ActiveMQDestination.createQueueNameForDurableSubscription(true, subInfo.getClientId(), subInfo.getSubscriptionName()));
|
||||
server.destroyQueue(subQueueName);
|
||||
}
|
||||
|
||||
public void sendBrokerInfo(OpenWireConnection connection) throws Exception {
|
||||
BrokerInfo brokerInfo = new BrokerInfo();
|
||||
brokerInfo.setBrokerName(getBrokerName());
|
||||
|
@ -543,14 +460,26 @@ public class OpenWireProtocolManager implements ProtocolManager<Interceptor>, Cl
|
|||
connection.dispatch(brokerInfo);
|
||||
}
|
||||
|
||||
/**
|
||||
* URI property
|
||||
*/
|
||||
@SuppressWarnings("unused")
|
||||
public void setRebalanceClusterClients(boolean rebalance) {
|
||||
this.rebalanceClusterClients = rebalance;
|
||||
}
|
||||
|
||||
/**
|
||||
* URI property
|
||||
*/
|
||||
@SuppressWarnings("unused")
|
||||
public boolean isRebalanceClusterClients() {
|
||||
return this.rebalanceClusterClients;
|
||||
}
|
||||
|
||||
/**
|
||||
* URI property
|
||||
*/
|
||||
@SuppressWarnings("unused")
|
||||
public void setUpdateClusterClients(boolean updateClusterClients) {
|
||||
this.updateClusterClients = updateClusterClients;
|
||||
}
|
||||
|
@ -559,10 +488,18 @@ public class OpenWireProtocolManager implements ProtocolManager<Interceptor>, Cl
|
|||
return this.updateClusterClients;
|
||||
}
|
||||
|
||||
/**
|
||||
* URI property
|
||||
*/
|
||||
@SuppressWarnings("unused")
|
||||
public void setUpdateClusterClientsOnRemove(boolean updateClusterClientsOnRemove) {
|
||||
this.updateClusterClientsOnRemove = updateClusterClientsOnRemove;
|
||||
}
|
||||
|
||||
/**
|
||||
* URI property
|
||||
*/
|
||||
@SuppressWarnings("unused")
|
||||
public boolean isUpdateClusterClientsOnRemove() {
|
||||
return this.updateClusterClientsOnRemove;
|
||||
}
|
||||
|
@ -571,10 +508,4 @@ public class OpenWireProtocolManager implements ProtocolManager<Interceptor>, Cl
|
|||
this.brokerName = name;
|
||||
}
|
||||
|
||||
public static XAException newXAException(String s, int errorCode) {
|
||||
XAException xaException = new XAException(s + " " + "xaErrorCode:" + errorCode);
|
||||
xaException.errorCode = errorCode;
|
||||
return xaException;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1,73 +0,0 @@
|
|||
/*
|
||||
* 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.artemis.core.protocol.openwire;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffers;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.command.ActiveMQDestination;
|
||||
import org.apache.activemq.command.ActiveMQQueue;
|
||||
import org.apache.activemq.command.ActiveMQTopic;
|
||||
import org.apache.activemq.util.ByteSequence;
|
||||
|
||||
public class OpenWireUtil {
|
||||
|
||||
public static ActiveMQBuffer toActiveMQBuffer(ByteSequence bytes) {
|
||||
ActiveMQBuffer buffer = ActiveMQBuffers.fixedBuffer(bytes.length);
|
||||
|
||||
buffer.writeBytes(bytes.data, bytes.offset, bytes.length);
|
||||
return buffer;
|
||||
}
|
||||
|
||||
public static SimpleString toCoreAddress(ActiveMQDestination dest) {
|
||||
if (dest.isQueue()) {
|
||||
return new SimpleString("jms.queue." + dest.getPhysicalName());
|
||||
}
|
||||
else {
|
||||
return new SimpleString("jms.topic." + dest.getPhysicalName());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* We convert the core address to an ActiveMQ Destination. We use the actual address on the message rather than the
|
||||
* destination set on the consumer because it maybe different and the JMS spec says that it should be what ever was
|
||||
* set on publish/send so a divert or wildcard may mean thats its different to the destination subscribed to by the
|
||||
* consumer
|
||||
*/
|
||||
public static ActiveMQDestination toAMQAddress(ServerMessage message, ActiveMQDestination actualDestination) {
|
||||
String address = message.getAddress().toString();
|
||||
String strippedAddress = address.replace("jms.queue.", "").replace("jms.topic.", "");
|
||||
if (actualDestination.isQueue()) {
|
||||
return new ActiveMQQueue(strippedAddress);
|
||||
}
|
||||
else {
|
||||
return new ActiveMQTopic(strippedAddress);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
*This util converts amq wildcards to compatible core wildcards
|
||||
*The conversion is like this:
|
||||
*AMQ * wildcard --> Core * wildcard (no conversion)
|
||||
*AMQ > wildcard --> Core # wildcard
|
||||
*/
|
||||
public static String convertWildcard(String physicalName) {
|
||||
return physicalName.replaceAll("(\\.>)+", ".#");
|
||||
}
|
||||
|
||||
}
|
|
@ -48,7 +48,7 @@ public class AMQCompositeConsumerBrokerExchange extends AMQConsumerBrokerExchang
|
|||
public void acknowledge(MessageAck ack) throws Exception {
|
||||
AMQConsumer amqConsumer = consumerMap.get(ack.getDestination());
|
||||
if (amqConsumer != null) {
|
||||
amqSession.acknowledge(ack, amqConsumer);
|
||||
amqConsumer.acknowledge(ack);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -58,4 +58,11 @@ public class AMQCompositeConsumerBrokerExchange extends AMQConsumerBrokerExchang
|
|||
amqConsumer.removeConsumer();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateConsumerPrefetchSize(int prefetch) {
|
||||
for (AMQConsumer amqConsumer : consumerMap.values()) {
|
||||
amqConsumer.setPrefetchSize(prefetch);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,10 +17,8 @@
|
|||
package org.apache.activemq.artemis.core.protocol.openwire.amq;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.Set;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
|
@ -30,11 +28,14 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.client.impl.ClientConsumerImpl;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.OpenWireMessageConverter;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.OpenWireUtil;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.util.OpenWireUtil;
|
||||
import org.apache.activemq.artemis.core.server.MessageReference;
|
||||
import org.apache.activemq.artemis.core.server.QueueQueryResult;
|
||||
import org.apache.activemq.artemis.core.server.ServerConsumer;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.core.server.SlowConsumerDetectionListener;
|
||||
import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
|
||||
import org.apache.activemq.artemis.core.transaction.Transaction;
|
||||
import org.apache.activemq.command.ConsumerControl;
|
||||
import org.apache.activemq.command.ConsumerId;
|
||||
import org.apache.activemq.command.ConsumerInfo;
|
||||
|
@ -42,7 +43,6 @@ import org.apache.activemq.command.MessageAck;
|
|||
import org.apache.activemq.command.MessageDispatch;
|
||||
import org.apache.activemq.command.MessageId;
|
||||
import org.apache.activemq.command.MessagePull;
|
||||
import org.apache.activemq.command.TransactionId;
|
||||
import org.apache.activemq.wireformat.WireFormat;
|
||||
|
||||
public class AMQConsumer {
|
||||
|
@ -50,11 +50,10 @@ public class AMQConsumer {
|
|||
private org.apache.activemq.command.ActiveMQDestination openwireDestination;
|
||||
private ConsumerInfo info;
|
||||
private final ScheduledExecutorService scheduledPool;
|
||||
private long nativeId = -1;
|
||||
private ServerConsumer serverConsumer;
|
||||
|
||||
private int prefetchSize;
|
||||
private AtomicInteger windowAvailable;
|
||||
private final java.util.Queue<MessageInfo> deliveringRefs = new ConcurrentLinkedQueue<>();
|
||||
private AtomicInteger currentWindow;
|
||||
private long messagePullSequence = 0;
|
||||
private MessagePullHandler messagePullHandler;
|
||||
|
||||
|
@ -67,20 +66,13 @@ public class AMQConsumer {
|
|||
this.info = info;
|
||||
this.scheduledPool = scheduledPool;
|
||||
this.prefetchSize = info.getPrefetchSize();
|
||||
this.windowAvailable = new AtomicInteger(prefetchSize);
|
||||
this.currentWindow = new AtomicInteger(prefetchSize);
|
||||
if (prefetchSize == 0) {
|
||||
messagePullHandler = new MessagePullHandler();
|
||||
}
|
||||
}
|
||||
|
||||
public void init(SlowConsumerDetectionListener slowConsumerDetectionListener, long nativeId) throws Exception {
|
||||
this.nativeId = nativeId;
|
||||
AMQServerConsumer serverConsumer = createServerConsumer(info, slowConsumerDetectionListener);
|
||||
serverConsumer.setAmqConsumer(this);
|
||||
}
|
||||
|
||||
|
||||
private AMQServerConsumer createServerConsumer(ConsumerInfo info, SlowConsumerDetectionListener slowConsumerDetectionListener) throws Exception {
|
||||
|
||||
SimpleString selector = info.getSelector() == null ? null : new SimpleString(info.getSelector());
|
||||
|
||||
|
@ -93,13 +85,13 @@ public class AMQConsumer {
|
|||
|
||||
SimpleString queueName = createTopicSubscription(info.isDurable(), info.getClientId(), physicalName, info.getSubscriptionName(), selector, address);
|
||||
|
||||
AMQServerConsumer serverConsumer = (AMQServerConsumer) session.getCoreSession().createConsumer(nativeId, queueName, null, info.isBrowser(), false, -1);
|
||||
serverConsumer = session.getCoreSession().createConsumer(nativeId, queueName, null, info.isBrowser(), false, -1);
|
||||
serverConsumer.setlowConsumerDetection(slowConsumerDetectionListener);
|
||||
return serverConsumer;
|
||||
}
|
||||
else {
|
||||
SimpleString queueName = new SimpleString("jms.queue." + physicalName);
|
||||
AMQServerConsumer serverConsumer = (AMQServerConsumer) session.getCoreSession().createConsumer(nativeId, queueName, selector, info.isBrowser(), false, -1);
|
||||
SimpleString queueName = OpenWireUtil.toCoreAddress(openwireDestination);
|
||||
session.getCoreServer().getJMSQueueCreator().create(queueName);
|
||||
serverConsumer = session.getCoreSession().createConsumer(nativeId, queueName, selector, info.isBrowser(), false, -1);
|
||||
serverConsumer.setlowConsumerDetection(slowConsumerDetectionListener);
|
||||
AddressSettings addrSettings = session.getCoreServer().getAddressSettingsRepository().getMatch(queueName.toString());
|
||||
if (addrSettings != null) {
|
||||
|
@ -113,10 +105,10 @@ public class AMQConsumer {
|
|||
}
|
||||
}
|
||||
|
||||
return serverConsumer;
|
||||
|
||||
}
|
||||
|
||||
serverConsumer.setProtocolData(this);
|
||||
|
||||
}
|
||||
|
||||
private SimpleString createTopicSubscription(boolean isDurable,
|
||||
|
@ -167,12 +159,6 @@ public class AMQConsumer {
|
|||
return queueName;
|
||||
}
|
||||
|
||||
|
||||
|
||||
public long getNativeId() {
|
||||
return this.nativeId;
|
||||
}
|
||||
|
||||
public ConsumerId getId() {
|
||||
return info.getConsumerId();
|
||||
}
|
||||
|
@ -182,16 +168,17 @@ public class AMQConsumer {
|
|||
}
|
||||
|
||||
public void acquireCredit(int n) throws Exception {
|
||||
boolean promptDelivery = windowAvailable.get() == 0;
|
||||
if (windowAvailable.get() < prefetchSize) {
|
||||
this.windowAvailable.addAndGet(n);
|
||||
}
|
||||
int oldwindow = currentWindow.getAndAdd(n);
|
||||
|
||||
boolean promptDelivery = oldwindow < prefetchSize;
|
||||
|
||||
if (promptDelivery) {
|
||||
session.getCoreSession().promptDelivery(nativeId);
|
||||
serverConsumer.promptDelivery();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public int handleDeliver(ServerMessage message, int deliveryCount) {
|
||||
public int handleDeliver(MessageReference reference, ServerMessage message, int deliveryCount) {
|
||||
MessageDispatch dispatch;
|
||||
try {
|
||||
if (messagePullHandler != null && !messagePullHandler.checkForcedConsumer(message)) {
|
||||
|
@ -200,9 +187,9 @@ public class AMQConsumer {
|
|||
//decrement deliveryCount as AMQ client tends to add 1.
|
||||
dispatch = OpenWireMessageConverter.createMessageDispatch(message, deliveryCount - 1, this);
|
||||
int size = dispatch.getMessage().getSize();
|
||||
this.deliveringRefs.add(new MessageInfo(dispatch.getMessage().getMessageId(), message.getMessageID(), size));
|
||||
reference.setProtocolData(dispatch.getMessage().getMessageId());
|
||||
session.deliverMessage(dispatch);
|
||||
windowAvailable.decrementAndGet();
|
||||
currentWindow.decrementAndGet();
|
||||
return size;
|
||||
}
|
||||
catch (IOException e) {
|
||||
|
@ -218,114 +205,59 @@ public class AMQConsumer {
|
|||
md.setConsumerId(getId());
|
||||
md.setDestination(openwireDestination);
|
||||
session.deliverMessage(md);
|
||||
windowAvailable.decrementAndGet();
|
||||
}
|
||||
|
||||
/** The acknowledgement in openwire is done based on intervals.
|
||||
* We will iterate through the list of delivering messages at {@link ServerConsumer#getDeliveringReferencesBasedOnProtocol(boolean, Object, Object)}
|
||||
* and add those to the Transaction.
|
||||
* Notice that we will start a new transaction on the cases where there is no transaction. */
|
||||
public void acknowledge(MessageAck ack) throws Exception {
|
||||
|
||||
|
||||
MessageId first = ack.getFirstMessageId();
|
||||
MessageId lastm = ack.getLastMessageId();
|
||||
TransactionId tid = ack.getTransactionId();
|
||||
boolean isLocalTx = (tid != null) && tid.isLocalTransaction();
|
||||
boolean single = lastm.equals(first);
|
||||
MessageId last = ack.getLastMessageId();
|
||||
|
||||
MessageInfo mi = null;
|
||||
int n = 0;
|
||||
|
||||
if (ack.isIndividualAck()) {
|
||||
Iterator<MessageInfo> iter = deliveringRefs.iterator();
|
||||
while (iter.hasNext()) {
|
||||
mi = iter.next();
|
||||
if (mi.amqId.equals(lastm)) {
|
||||
n++;
|
||||
if (!isLocalTx) {
|
||||
iter.remove();
|
||||
session.getCoreSession().individualAcknowledge(nativeId, mi.nativeId);
|
||||
}
|
||||
else {
|
||||
mi.setLocalAcked(true);
|
||||
}
|
||||
if (tid == null) {
|
||||
session.getCoreSession().commit();
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (ack.isRedeliveredAck()) {
|
||||
//client tells that this message is for redlivery.
|
||||
//do nothing until poisoned.
|
||||
n = ack.getMessageCount();
|
||||
}
|
||||
else if (ack.isPoisonAck()) {
|
||||
//send to dlq
|
||||
Iterator<MessageInfo> iter = deliveringRefs.iterator();
|
||||
boolean firstFound = false;
|
||||
while (iter.hasNext()) {
|
||||
mi = iter.next();
|
||||
if (mi.amqId.equals(first)) {
|
||||
n++;
|
||||
iter.remove();
|
||||
session.getCoreSession().moveToDeadLetterAddress(nativeId, mi.nativeId, ack.getPoisonCause());
|
||||
session.getCoreSession().commit();
|
||||
if (single) {
|
||||
break;
|
||||
}
|
||||
firstFound = true;
|
||||
}
|
||||
else if (firstFound || first == null) {
|
||||
n++;
|
||||
iter.remove();
|
||||
session.getCoreSession().moveToDeadLetterAddress(nativeId, mi.nativeId, ack.getPoisonCause());
|
||||
session.getCoreSession().commit();
|
||||
if (mi.amqId.equals(lastm)) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (ack.isDeliveredAck() || ack.isExpiredAck()) {
|
||||
//ToDo: implement with tests
|
||||
n = ack.getMessageCount();
|
||||
}
|
||||
else {
|
||||
Iterator<MessageInfo> iter = deliveringRefs.iterator();
|
||||
boolean firstFound = false;
|
||||
while (iter.hasNext()) {
|
||||
MessageInfo ami = iter.next();
|
||||
if (ami.amqId.equals(first)) {
|
||||
n++;
|
||||
if (!isLocalTx) {
|
||||
iter.remove();
|
||||
}
|
||||
else {
|
||||
ami.setLocalAcked(true);
|
||||
}
|
||||
if (single) {
|
||||
mi = ami;
|
||||
break;
|
||||
}
|
||||
firstFound = true;
|
||||
}
|
||||
else if (firstFound || first == null) {
|
||||
n++;
|
||||
if (!isLocalTx) {
|
||||
iter.remove();
|
||||
}
|
||||
else {
|
||||
ami.setLocalAcked(true);
|
||||
}
|
||||
if (ami.amqId.equals(lastm)) {
|
||||
mi = ami;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (mi != null && !isLocalTx) {
|
||||
session.getCoreSession().acknowledge(nativeId, mi.nativeId);
|
||||
}
|
||||
if (first == null) {
|
||||
first = last;
|
||||
}
|
||||
|
||||
acquireCredit(n);
|
||||
boolean removeReferences = !serverConsumer.isBrowseOnly(); // if it's browse only, nothing to be acked, we just remove the lists
|
||||
|
||||
if (ack.isRedeliveredAck() || ack.isDeliveredAck() || ack.isExpiredAck()) {
|
||||
removeReferences = false;
|
||||
}
|
||||
|
||||
List<MessageReference> ackList = serverConsumer.getDeliveringReferencesBasedOnProtocol(removeReferences, first, last);
|
||||
|
||||
acquireCredit(ack.getMessageCount());
|
||||
|
||||
if (removeReferences) {
|
||||
|
||||
Transaction originalTX = session.getCoreSession().getCurrentTransaction();
|
||||
Transaction transaction;
|
||||
|
||||
if (originalTX == null) {
|
||||
transaction = session.getCoreSession().newTransaction();
|
||||
}
|
||||
else {
|
||||
transaction = originalTX;
|
||||
}
|
||||
|
||||
if (ack.isIndividualAck() || ack.isStandardAck()) {
|
||||
for (MessageReference ref : ackList) {
|
||||
ref.acknowledge(transaction);
|
||||
}
|
||||
}
|
||||
else if (ack.isPoisonAck()) {
|
||||
for (MessageReference ref : ackList) {
|
||||
ref.getQueue().sendToDeadLetterAddress(transaction, ref);
|
||||
}
|
||||
}
|
||||
|
||||
if (originalTX == null) {
|
||||
transaction.commit(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void browseFinished() {
|
||||
|
@ -337,61 +269,23 @@ public class AMQConsumer {
|
|||
session.deliverMessage(md);
|
||||
}
|
||||
|
||||
//this is called before session commit a local tx
|
||||
public void finishTx() throws Exception {
|
||||
MessageInfo lastMi = null;
|
||||
|
||||
MessageInfo mi = null;
|
||||
Iterator<MessageInfo> iter = deliveringRefs.iterator();
|
||||
while (iter.hasNext()) {
|
||||
mi = iter.next();
|
||||
if (mi.isLocalAcked()) {
|
||||
iter.remove();
|
||||
lastMi = mi;
|
||||
}
|
||||
}
|
||||
|
||||
if (lastMi != null) {
|
||||
session.getCoreSession().acknowledge(nativeId, lastMi.nativeId);
|
||||
}
|
||||
}
|
||||
|
||||
public void rollbackTx(Set<Long> acked) throws Exception {
|
||||
MessageInfo lastMi = null;
|
||||
|
||||
MessageInfo mi = null;
|
||||
Iterator<MessageInfo> iter = deliveringRefs.iterator();
|
||||
while (iter.hasNext()) {
|
||||
mi = iter.next();
|
||||
if (mi.isLocalAcked()) {
|
||||
acked.add(mi.nativeId);
|
||||
lastMi = mi;
|
||||
}
|
||||
}
|
||||
|
||||
if (lastMi != null) {
|
||||
session.getCoreSession().acknowledge(nativeId, lastMi.nativeId);
|
||||
}
|
||||
}
|
||||
|
||||
public ConsumerInfo getInfo() {
|
||||
return info;
|
||||
}
|
||||
|
||||
public boolean hasCredits() {
|
||||
return windowAvailable.get() > 0;
|
||||
return currentWindow.get() > 0;
|
||||
}
|
||||
|
||||
public void processMessagePull(MessagePull messagePull) throws Exception {
|
||||
windowAvailable.incrementAndGet();
|
||||
|
||||
currentWindow.incrementAndGet();
|
||||
if (messagePullHandler != null) {
|
||||
messagePullHandler.nextSequence(messagePullSequence++, messagePull.getTimeout());
|
||||
}
|
||||
}
|
||||
|
||||
public void removeConsumer() throws Exception {
|
||||
session.removeConsumer(nativeId);
|
||||
serverConsumer.close(false);
|
||||
}
|
||||
|
||||
public org.apache.activemq.command.ActiveMQDestination getOpenwireDestination() {
|
||||
|
@ -400,10 +294,10 @@ public class AMQConsumer {
|
|||
|
||||
public void setPrefetchSize(int prefetchSize) {
|
||||
this.prefetchSize = prefetchSize;
|
||||
this.windowAvailable.set(prefetchSize);
|
||||
this.currentWindow.set(prefetchSize);
|
||||
this.info.setPrefetchSize(prefetchSize);
|
||||
if (this.prefetchSize > 0) {
|
||||
session.getCoreSession().promptDelivery(nativeId);
|
||||
serverConsumer.promptDelivery();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -421,7 +315,7 @@ public class AMQConsumer {
|
|||
this.next = next;
|
||||
this.timeout = timeout;
|
||||
latch = new CountDownLatch(1);
|
||||
session.getCoreSession().forceConsumerDelivery(nativeId, messagePullSequence);
|
||||
serverConsumer.forceDelivery(messagePullSequence);
|
||||
//if we are 0 timeout or less we need to wait to get either the forced message or a real message.
|
||||
if (timeout <= 0) {
|
||||
latch.await(10, TimeUnit.SECONDS);
|
||||
|
@ -434,7 +328,6 @@ public class AMQConsumer {
|
|||
|
||||
public boolean checkForcedConsumer(ServerMessage message) {
|
||||
if (message.containsProperty(ClientConsumerImpl.FORCED_DELIVERY_MESSAGE)) {
|
||||
System.out.println("MessagePullHandler.checkForcedConsumer");
|
||||
if (next >= 0) {
|
||||
if (timeout <= 0) {
|
||||
latch.countDown();
|
||||
|
|
|
@ -32,4 +32,6 @@ public abstract class AMQConsumerBrokerExchange {
|
|||
public abstract void processMessagePull(MessagePull messagePull) throws Exception;
|
||||
|
||||
public abstract void removeConsumer() throws Exception;
|
||||
|
||||
public abstract void updateConsumerPrefetchSize(int prefetch);
|
||||
}
|
||||
|
|
|
@ -1,102 +0,0 @@
|
|||
/*
|
||||
* 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.artemis.core.protocol.openwire.amq;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.activemq.artemis.core.filter.Filter;
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.core.postoffice.QueueBinding;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.OpenWireMessageConverter;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServer;
|
||||
import org.apache.activemq.artemis.core.server.MessageReference;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.core.server.impl.QueueImpl;
|
||||
import org.apache.activemq.artemis.core.server.impl.ServerConsumerImpl;
|
||||
import org.apache.activemq.artemis.core.server.management.ManagementService;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.SessionCallback;
|
||||
|
||||
public class AMQServerConsumer extends ServerConsumerImpl {
|
||||
|
||||
// TODO-NOW: remove this once unified
|
||||
AMQConsumer amqConsumer;
|
||||
|
||||
public AMQConsumer getAmqConsumer() {
|
||||
return amqConsumer;
|
||||
}
|
||||
|
||||
/** TODO-NOW: remove this once unified */
|
||||
public void setAmqConsumer(AMQConsumer amqConsumer) {
|
||||
this.amqConsumer = amqConsumer;
|
||||
}
|
||||
|
||||
public AMQServerConsumer(long consumerID,
|
||||
AMQServerSession serverSession,
|
||||
QueueBinding binding,
|
||||
Filter filter,
|
||||
boolean started,
|
||||
boolean browseOnly,
|
||||
StorageManager storageManager,
|
||||
SessionCallback callback,
|
||||
boolean preAcknowledge,
|
||||
boolean strictUpdateDeliveryCount,
|
||||
ManagementService managementService,
|
||||
boolean supportLargeMessage,
|
||||
Integer credits,
|
||||
final ActiveMQServer server) throws Exception {
|
||||
super(consumerID, serverSession, binding, filter, started, browseOnly, storageManager, callback, preAcknowledge, strictUpdateDeliveryCount, managementService, supportLargeMessage, credits, server);
|
||||
}
|
||||
|
||||
public void amqPutBackToDeliveringList(final List<MessageReference> refs) {
|
||||
synchronized (this.deliveringRefs) {
|
||||
for (MessageReference ref : refs) {
|
||||
ref.incrementDeliveryCount();
|
||||
deliveringRefs.add(ref);
|
||||
}
|
||||
//adjust the order. Suppose deliveringRefs has 2 existing
|
||||
//refs m1, m2, and refs has 3 m3, m4, m5
|
||||
//new order must be m3, m4, m5, m1, m2
|
||||
if (refs.size() > 0) {
|
||||
long first = refs.get(0).getMessage().getMessageID();
|
||||
MessageReference m = deliveringRefs.peek();
|
||||
while (m.getMessage().getMessageID() != first) {
|
||||
deliveringRefs.poll();
|
||||
deliveringRefs.add(m);
|
||||
m = deliveringRefs.peek();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void moveToDeadLetterAddress(long mid, Throwable cause) throws Exception {
|
||||
MessageReference ref = removeReferenceByID(mid);
|
||||
|
||||
if (ref == null) {
|
||||
throw new IllegalStateException("Cannot find ref to ack " + mid);
|
||||
}
|
||||
|
||||
ServerMessage coreMsg = ref.getMessage();
|
||||
coreMsg.putStringProperty(OpenWireMessageConverter.AMQ_MSG_DLQ_DELIVERY_FAILURE_CAUSE_PROPERTY, cause.toString());
|
||||
|
||||
QueueImpl queue = (QueueImpl) ref.getQueue();
|
||||
synchronized (queue) {
|
||||
queue.sendToDeadLetterAddress(ref);
|
||||
queue.decDelivering();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -1,391 +0,0 @@
|
|||
/*
|
||||
* 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.artemis.core.protocol.openwire.amq;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.Pair;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.api.core.management.CoreNotificationType;
|
||||
import org.apache.activemq.artemis.api.core.management.ManagementHelper;
|
||||
import org.apache.activemq.artemis.core.filter.Filter;
|
||||
import org.apache.activemq.artemis.core.filter.impl.FilterImpl;
|
||||
import org.apache.activemq.artemis.core.persistence.OperationContext;
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.core.postoffice.Binding;
|
||||
import org.apache.activemq.artemis.core.postoffice.BindingType;
|
||||
import org.apache.activemq.artemis.core.postoffice.PostOffice;
|
||||
import org.apache.activemq.artemis.core.postoffice.QueueBinding;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.AMQTransactionImpl;
|
||||
import org.apache.activemq.artemis.core.security.SecurityStore;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQMessageBundle;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
|
||||
import org.apache.activemq.artemis.core.server.MessageReference;
|
||||
import org.apache.activemq.artemis.core.server.Queue;
|
||||
import org.apache.activemq.artemis.core.server.QueueCreator;
|
||||
import org.apache.activemq.artemis.core.server.ServerConsumer;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl;
|
||||
import org.apache.activemq.artemis.core.server.impl.RefsOperation;
|
||||
import org.apache.activemq.artemis.core.server.impl.ServerConsumerImpl;
|
||||
import org.apache.activemq.artemis.core.server.impl.ServerSessionImpl;
|
||||
import org.apache.activemq.artemis.core.server.management.ManagementService;
|
||||
import org.apache.activemq.artemis.core.server.management.Notification;
|
||||
import org.apache.activemq.artemis.core.transaction.ResourceManager;
|
||||
import org.apache.activemq.artemis.core.transaction.TransactionPropertyIndexes;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.SessionCallback;
|
||||
import org.apache.activemq.artemis.utils.TypedProperties;
|
||||
import org.apache.activemq.artemis.utils.UUID;
|
||||
|
||||
public class AMQServerSession extends ServerSessionImpl {
|
||||
|
||||
private boolean internal;
|
||||
|
||||
public AMQServerSession(String name,
|
||||
String username,
|
||||
String password,
|
||||
int minLargeMessageSize,
|
||||
boolean autoCommitSends,
|
||||
boolean autoCommitAcks,
|
||||
boolean preAcknowledge,
|
||||
boolean persistDeliveryCountBeforeDelivery,
|
||||
boolean xa,
|
||||
RemotingConnection connection,
|
||||
StorageManager storageManager,
|
||||
PostOffice postOffice,
|
||||
ResourceManager resourceManager,
|
||||
SecurityStore securityStore,
|
||||
ManagementService managementService,
|
||||
ActiveMQServerImpl activeMQServerImpl,
|
||||
SimpleString managementAddress,
|
||||
SimpleString simpleString,
|
||||
SessionCallback callback,
|
||||
QueueCreator queueCreator,
|
||||
OperationContext context) throws Exception {
|
||||
super(name, username, password, minLargeMessageSize, autoCommitSends, autoCommitAcks, preAcknowledge, persistDeliveryCountBeforeDelivery, xa, connection, storageManager, postOffice, resourceManager, securityStore, managementService, activeMQServerImpl, managementAddress, simpleString, callback, context, new AMQTransactionFactory(), queueCreator);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doClose(final boolean failed) throws Exception {
|
||||
Set<ServerConsumer> consumersClone = new HashSet<>(consumers.values());
|
||||
for (ServerConsumer consumer : consumersClone) {
|
||||
AMQServerConsumer amqConsumer = (AMQServerConsumer)consumer;
|
||||
amqConsumer.setStarted(false);
|
||||
}
|
||||
|
||||
synchronized (this) {
|
||||
if (tx != null && tx.getXid() == null) {
|
||||
((AMQTransactionImpl) tx).setRollbackForClose();
|
||||
}
|
||||
}
|
||||
super.doClose(failed);
|
||||
}
|
||||
|
||||
public AtomicInteger getConsumerCredits(final long consumerID) {
|
||||
ServerConsumer consumer = consumers.get(consumerID);
|
||||
|
||||
if (consumer == null) {
|
||||
ActiveMQServerLogger.LOGGER.debug("There is no consumer with id " + consumerID);
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
return ((ServerConsumerImpl) consumer).getAvailableCredits();
|
||||
}
|
||||
|
||||
public void enableXA() throws Exception {
|
||||
if (!this.xa) {
|
||||
if (this.tx != null) {
|
||||
//that's not expected, maybe a warning.
|
||||
this.tx.rollback();
|
||||
this.tx = null;
|
||||
}
|
||||
|
||||
this.autoCommitAcks = false;
|
||||
this.autoCommitSends = false;
|
||||
|
||||
this.xa = true;
|
||||
}
|
||||
}
|
||||
|
||||
public void enableTx() throws Exception {
|
||||
if (this.xa) {
|
||||
throw new IllegalStateException("Session is XA");
|
||||
}
|
||||
|
||||
this.autoCommitAcks = false;
|
||||
this.autoCommitSends = false;
|
||||
|
||||
if (this.tx != null) {
|
||||
//that's not expected, maybe a warning.
|
||||
this.tx.rollback();
|
||||
this.tx = null;
|
||||
}
|
||||
|
||||
this.tx = newTransaction();
|
||||
}
|
||||
|
||||
//amq specific behavior
|
||||
|
||||
// TODO: move this to AMQSession
|
||||
public void amqRollback(Set<Long> acked) throws Exception {
|
||||
if (tx == null) {
|
||||
// Might be null if XA
|
||||
|
||||
tx = newTransaction();
|
||||
}
|
||||
|
||||
RefsOperation oper = (RefsOperation) tx.getProperty(TransactionPropertyIndexes.REFS_OPERATION);
|
||||
|
||||
if (oper != null) {
|
||||
List<MessageReference> ackRefs = oper.getReferencesToAcknowledge();
|
||||
Map<Long, List<MessageReference>> toAcks = new HashMap<>();
|
||||
for (MessageReference ref : ackRefs) {
|
||||
Long consumerId = ref.getConsumerId();
|
||||
|
||||
if (this.consumers.containsKey(consumerId)) {
|
||||
if (acked.contains(ref.getMessage().getMessageID())) {
|
||||
List<MessageReference> ackList = toAcks.get(consumerId);
|
||||
if (ackList == null) {
|
||||
ackList = new ArrayList<>();
|
||||
toAcks.put(consumerId, ackList);
|
||||
}
|
||||
ackList.add(ref);
|
||||
}
|
||||
}
|
||||
else {
|
||||
//consumer must have been closed, cancel to queue
|
||||
ref.getQueue().cancel(tx, ref);
|
||||
}
|
||||
}
|
||||
//iterate consumers
|
||||
if (toAcks.size() > 0) {
|
||||
Iterator<Entry<Long, List<MessageReference>>> iter = toAcks.entrySet().iterator();
|
||||
while (iter.hasNext()) {
|
||||
Entry<Long, List<MessageReference>> entry = iter.next();
|
||||
ServerConsumer consumer = consumers.get(entry.getKey());
|
||||
((AMQServerConsumer) consumer).amqPutBackToDeliveringList(entry.getValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
tx.rollback();
|
||||
|
||||
if (xa) {
|
||||
tx = null;
|
||||
}
|
||||
else {
|
||||
tx = newTransaction();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* The failed flag is used here to control delivery count.
|
||||
* If set to true the delivery count won't decrement.
|
||||
*/
|
||||
public void amqCloseConsumer(long consumerID, boolean failed) throws Exception {
|
||||
final ServerConsumer consumer = consumers.get(consumerID);
|
||||
|
||||
if (consumer != null) {
|
||||
consumer.close(failed);
|
||||
}
|
||||
else {
|
||||
ActiveMQServerLogger.LOGGER.cannotFindConsumer(consumerID);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServerConsumer createConsumer(final long consumerID,
|
||||
final SimpleString queueName,
|
||||
final SimpleString filterString,
|
||||
final boolean browseOnly,
|
||||
final boolean supportLargeMessage,
|
||||
final Integer credits) throws Exception {
|
||||
if (this.internal) {
|
||||
// Clebert TODO: PQP!!!!!!!!!!!!!!!!!!!!
|
||||
|
||||
//internal sessions doesn't check security:: Why??? //// what's the reason for that? Where a link?
|
||||
|
||||
Binding binding = postOffice.getBinding(queueName);
|
||||
|
||||
if (binding == null || binding.getType() != BindingType.LOCAL_QUEUE) {
|
||||
throw ActiveMQMessageBundle.BUNDLE.noSuchQueue(queueName);
|
||||
}
|
||||
|
||||
Filter filter = FilterImpl.createFilter(filterString);
|
||||
|
||||
ServerConsumer consumer = newConsumer(consumerID, this, (QueueBinding) binding, filter, started, browseOnly, storageManager, callback, preAcknowledge, strictUpdateDeliveryCount, managementService, supportLargeMessage, credits);
|
||||
consumers.put(consumer.getID(), consumer);
|
||||
|
||||
if (!browseOnly) {
|
||||
TypedProperties props = new TypedProperties();
|
||||
|
||||
props.putSimpleStringProperty(ManagementHelper.HDR_ADDRESS, binding.getAddress());
|
||||
|
||||
props.putSimpleStringProperty(ManagementHelper.HDR_CLUSTER_NAME, binding.getClusterName());
|
||||
|
||||
props.putSimpleStringProperty(ManagementHelper.HDR_ROUTING_NAME, binding.getRoutingName());
|
||||
|
||||
props.putIntProperty(ManagementHelper.HDR_DISTANCE, binding.getDistance());
|
||||
|
||||
Queue theQueue = (Queue) binding.getBindable();
|
||||
|
||||
props.putIntProperty(ManagementHelper.HDR_CONSUMER_COUNT, theQueue.getConsumerCount());
|
||||
|
||||
// HORNETQ-946
|
||||
props.putSimpleStringProperty(ManagementHelper.HDR_USER, SimpleString.toSimpleString(username));
|
||||
|
||||
props.putSimpleStringProperty(ManagementHelper.HDR_REMOTE_ADDRESS, SimpleString.toSimpleString(this.remotingConnection.getRemoteAddress()));
|
||||
|
||||
props.putSimpleStringProperty(ManagementHelper.HDR_SESSION_NAME, SimpleString.toSimpleString(name));
|
||||
|
||||
if (filterString != null) {
|
||||
props.putSimpleStringProperty(ManagementHelper.HDR_FILTERSTRING, filterString);
|
||||
}
|
||||
|
||||
Notification notification = new Notification(null, CoreNotificationType.CONSUMER_CREATED, props);
|
||||
|
||||
if (ActiveMQServerLogger.LOGGER.isDebugEnabled()) {
|
||||
ActiveMQServerLogger.LOGGER.debug("Session with user=" + username + ", connection=" + this.remotingConnection + " created a consumer on queue " + queueName + ", filter = " + filterString);
|
||||
}
|
||||
|
||||
managementService.sendNotification(notification);
|
||||
}
|
||||
|
||||
return consumer;
|
||||
}
|
||||
else {
|
||||
return super.createConsumer(consumerID, queueName, filterString, browseOnly, supportLargeMessage, credits);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Queue createQueue(final SimpleString address,
|
||||
final SimpleString name,
|
||||
final SimpleString filterString,
|
||||
final boolean temporary,
|
||||
final boolean durable) throws Exception {
|
||||
if (!this.internal) {
|
||||
return super.createQueue(address, name, filterString, temporary, durable);
|
||||
}
|
||||
|
||||
Queue queue = server.createQueue(address, name, filterString, SimpleString.toSimpleString(getUsername()), durable, temporary);
|
||||
|
||||
if (temporary) {
|
||||
// Temporary queue in core simply means the queue will be deleted if
|
||||
// the remoting connection
|
||||
// dies. It does not mean it will get deleted automatically when the
|
||||
// session is closed.
|
||||
// It is up to the user to delete the queue when finished with it
|
||||
|
||||
TempQueueCleanerUpper cleaner = new TempQueueCleanerUpper(server, name);
|
||||
|
||||
remotingConnection.addCloseListener(cleaner);
|
||||
remotingConnection.addFailureListener(cleaner);
|
||||
|
||||
tempQueueCleannerUppers.put(name, cleaner);
|
||||
}
|
||||
|
||||
if (ActiveMQServerLogger.LOGGER.isDebugEnabled()) {
|
||||
ActiveMQServerLogger.LOGGER.debug("Queue " + name + " created on address " + name +
|
||||
" with filter=" + filterString + " temporary = " +
|
||||
temporary + " durable=" + durable + " on session user=" + this.username + ", connection=" + this.remotingConnection);
|
||||
}
|
||||
|
||||
return queue;
|
||||
}
|
||||
|
||||
|
||||
// Clebert TODO: Get rid of these mthods
|
||||
@Override
|
||||
protected void doSend(final ServerMessage msg, final boolean direct) throws Exception {
|
||||
if (!this.internal) {
|
||||
super.doSend(msg, direct);
|
||||
return;
|
||||
}
|
||||
|
||||
//bypass security check for internal sessions
|
||||
if (tx == null || autoCommitSends) {
|
||||
}
|
||||
else {
|
||||
routingContext.setTransaction(tx);
|
||||
}
|
||||
|
||||
try {
|
||||
postOffice.route(msg, getQueueCreator(), routingContext, direct);
|
||||
|
||||
Pair<UUID, AtomicLong> value = targetAddressInfos.get(msg.getAddress());
|
||||
|
||||
if (value == null) {
|
||||
targetAddressInfos.put(msg.getAddress(), new Pair<>(msg.getUserID(), new AtomicLong(1)));
|
||||
}
|
||||
else {
|
||||
value.setA(msg.getUserID());
|
||||
value.getB().incrementAndGet();
|
||||
}
|
||||
}
|
||||
finally {
|
||||
routingContext.clear();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ServerConsumer newConsumer(long consumerID,
|
||||
ServerSessionImpl serverSessionImpl,
|
||||
QueueBinding binding,
|
||||
Filter filter,
|
||||
boolean started2,
|
||||
boolean browseOnly,
|
||||
StorageManager storageManager2,
|
||||
SessionCallback callback2,
|
||||
boolean preAcknowledge2,
|
||||
boolean strictUpdateDeliveryCount2,
|
||||
ManagementService managementService2,
|
||||
boolean supportLargeMessage,
|
||||
Integer credits) throws Exception {
|
||||
return new AMQServerConsumer(consumerID, this, binding, filter, started, browseOnly, storageManager, callback, preAcknowledge, strictUpdateDeliveryCount, managementService, supportLargeMessage, credits, this.server);
|
||||
}
|
||||
|
||||
public AMQServerConsumer getConsumer(long nativeId) {
|
||||
return (AMQServerConsumer) this.consumers.get(nativeId);
|
||||
}
|
||||
|
||||
public void setInternal(boolean internal) {
|
||||
this.internal = internal;
|
||||
}
|
||||
|
||||
public boolean isInternal() {
|
||||
return this.internal;
|
||||
}
|
||||
|
||||
public void moveToDeadLetterAddress(long consumerId, long mid, Throwable cause) throws Exception {
|
||||
AMQServerConsumer consumer = getConsumer(consumerId);
|
||||
consumer.moveToDeadLetterAddress(mid, cause);
|
||||
}
|
||||
|
||||
}
|
|
@ -1,69 +0,0 @@
|
|||
/*
|
||||
* 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.artemis.core.protocol.openwire.amq;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.persistence.OperationContext;
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.core.postoffice.PostOffice;
|
||||
import org.apache.activemq.artemis.core.security.SecurityStore;
|
||||
import org.apache.activemq.artemis.core.server.QueueCreator;
|
||||
import org.apache.activemq.artemis.core.server.ServerSessionFactory;
|
||||
import org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl;
|
||||
import org.apache.activemq.artemis.core.server.impl.ServerSessionImpl;
|
||||
import org.apache.activemq.artemis.core.server.management.ManagementService;
|
||||
import org.apache.activemq.artemis.core.transaction.ResourceManager;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.SessionCallback;
|
||||
|
||||
public class AMQServerSessionFactory implements ServerSessionFactory {
|
||||
|
||||
private static final AMQServerSessionFactory singleInstance = new AMQServerSessionFactory();
|
||||
|
||||
public static AMQServerSessionFactory getInstance() {
|
||||
return singleInstance;
|
||||
}
|
||||
|
||||
private AMQServerSessionFactory() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServerSessionImpl createCoreSession(String name,
|
||||
String username,
|
||||
String password,
|
||||
int minLargeMessageSize,
|
||||
boolean autoCommitSends,
|
||||
boolean autoCommitAcks,
|
||||
boolean preAcknowledge,
|
||||
boolean persistDeliveryCountBeforeDelivery,
|
||||
boolean xa,
|
||||
RemotingConnection connection,
|
||||
StorageManager storageManager,
|
||||
PostOffice postOffice,
|
||||
ResourceManager resourceManager,
|
||||
SecurityStore securityStore,
|
||||
ManagementService managementService,
|
||||
ActiveMQServerImpl activeMQServerImpl,
|
||||
SimpleString managementAddress,
|
||||
SimpleString simpleString,
|
||||
SessionCallback callback,
|
||||
QueueCreator queueCreator,
|
||||
OperationContext context) throws Exception {
|
||||
return new AMQServerSession(name, username, password, minLargeMessageSize, autoCommitSends, autoCommitAcks, preAcknowledge, persistDeliveryCountBeforeDelivery, xa, connection, storageManager, postOffice, resourceManager, securityStore, managementService, activeMQServerImpl, managementAddress, simpleString, callback, queueCreator, context);
|
||||
}
|
||||
|
||||
}
|
|
@ -17,13 +17,7 @@
|
|||
package org.apache.activemq.artemis.core.protocol.openwire.amq;
|
||||
|
||||
import javax.jms.ResourceAllocationException;
|
||||
import javax.transaction.xa.Xid;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
@ -32,15 +26,15 @@ import org.apache.activemq.artemis.api.core.SimpleString;
|
|||
import org.apache.activemq.artemis.core.paging.PagingStore;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.OpenWireConnection;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.OpenWireMessageConverter;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.OpenWireProtocolManager;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.OpenWireUtil;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.util.OpenWireUtil;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServer;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
|
||||
import org.apache.activemq.artemis.core.server.MessageReference;
|
||||
import org.apache.activemq.artemis.core.server.ServerConsumer;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.core.server.ServerSession;
|
||||
import org.apache.activemq.artemis.core.server.SlowConsumerDetectionListener;
|
||||
import org.apache.activemq.artemis.core.settings.impl.AddressFullMessagePolicy;
|
||||
import org.apache.activemq.artemis.core.transaction.impl.XidImpl;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.SessionCallback;
|
||||
import org.apache.activemq.artemis.spi.core.remoting.Connection;
|
||||
import org.apache.activemq.artemis.spi.core.remoting.ReadyListener;
|
||||
|
@ -48,43 +42,30 @@ import org.apache.activemq.artemis.utils.IDGenerator;
|
|||
import org.apache.activemq.artemis.utils.SimpleIDGenerator;
|
||||
import org.apache.activemq.command.ActiveMQDestination;
|
||||
import org.apache.activemq.command.ConnectionInfo;
|
||||
import org.apache.activemq.command.ConsumerId;
|
||||
import org.apache.activemq.command.ConsumerInfo;
|
||||
import org.apache.activemq.command.Message;
|
||||
import org.apache.activemq.command.MessageAck;
|
||||
import org.apache.activemq.command.MessageDispatch;
|
||||
import org.apache.activemq.command.ProducerAck;
|
||||
import org.apache.activemq.command.ProducerInfo;
|
||||
import org.apache.activemq.command.SessionInfo;
|
||||
import org.apache.activemq.command.TransactionId;
|
||||
import org.apache.activemq.command.TransactionInfo;
|
||||
import org.apache.activemq.command.XATransactionId;
|
||||
import org.apache.activemq.openwire.OpenWireFormat;
|
||||
import org.apache.activemq.wireformat.WireFormat;
|
||||
|
||||
public class AMQSession implements SessionCallback {
|
||||
|
||||
// ConsumerID is generated inside the session, 0, 1, 2, ... as many consumers as you have on the session
|
||||
protected final IDGenerator idGenerator = new SimpleIDGenerator(0);
|
||||
protected final IDGenerator consumerIDGenerator = new SimpleIDGenerator(0);
|
||||
|
||||
private ConnectionInfo connInfo;
|
||||
private AMQServerSession coreSession;
|
||||
private ServerSession coreSession;
|
||||
private SessionInfo sessInfo;
|
||||
private ActiveMQServer server;
|
||||
private OpenWireConnection connection;
|
||||
|
||||
private Map<Long, AMQConsumer> consumers = new ConcurrentHashMap<>();
|
||||
|
||||
private AtomicBoolean started = new AtomicBoolean(false);
|
||||
|
||||
private TransactionId txId = null;
|
||||
|
||||
private boolean isTx;
|
||||
|
||||
private final ScheduledExecutorService scheduledPool;
|
||||
|
||||
private OpenWireProtocolManager manager;
|
||||
|
||||
// The sessionWireformat used by the session
|
||||
// this object is meant to be used per thread / session
|
||||
// so we make a new one per AMQSession
|
||||
|
@ -94,20 +75,22 @@ public class AMQSession implements SessionCallback {
|
|||
SessionInfo sessInfo,
|
||||
ActiveMQServer server,
|
||||
OpenWireConnection connection,
|
||||
ScheduledExecutorService scheduledPool,
|
||||
OpenWireProtocolManager manager) {
|
||||
ScheduledExecutorService scheduledPool) {
|
||||
this.connInfo = connInfo;
|
||||
this.sessInfo = sessInfo;
|
||||
|
||||
this.server = server;
|
||||
this.connection = connection;
|
||||
this.scheduledPool = scheduledPool;
|
||||
this.manager = manager;
|
||||
OpenWireFormat marshaller = (OpenWireFormat) connection.getMarshaller();
|
||||
|
||||
this.converter = new OpenWireMessageConverter(marshaller.copy());
|
||||
}
|
||||
|
||||
public boolean isClosed() {
|
||||
return coreSession.isClosed();
|
||||
}
|
||||
|
||||
public OpenWireMessageConverter getConverter() {
|
||||
return converter;
|
||||
}
|
||||
|
@ -122,7 +105,7 @@ public class AMQSession implements SessionCallback {
|
|||
// now
|
||||
|
||||
try {
|
||||
coreSession = (AMQServerSession) server.createSession(name, username, password, minLargeMessageSize, connection, true, false, false, false, null, this, AMQServerSessionFactory.getInstance(), true);
|
||||
coreSession = server.createSession(name, username, password, minLargeMessageSize, connection, true, false, false, false, null, this, true);
|
||||
|
||||
long sessionId = sessInfo.getSessionId().getValue();
|
||||
if (sessionId == -1) {
|
||||
|
@ -136,8 +119,8 @@ public class AMQSession implements SessionCallback {
|
|||
}
|
||||
|
||||
public List<AMQConsumer> createConsumer(ConsumerInfo info,
|
||||
AMQSession amqSession,
|
||||
SlowConsumerDetectionListener slowConsumerDetectionListener) throws Exception {
|
||||
AMQSession amqSession,
|
||||
SlowConsumerDetectionListener slowConsumerDetectionListener) throws Exception {
|
||||
//check destination
|
||||
ActiveMQDestination dest = info.getDestination();
|
||||
ActiveMQDestination[] dests = null;
|
||||
|
@ -147,7 +130,7 @@ public class AMQSession implements SessionCallback {
|
|||
else {
|
||||
dests = new ActiveMQDestination[]{dest};
|
||||
}
|
||||
// Map<ActiveMQDestination, AMQConsumer> consumerMap = new HashMap<>();
|
||||
|
||||
List<AMQConsumer> consumersList = new java.util.LinkedList<>();
|
||||
|
||||
for (ActiveMQDestination openWireDest : dests) {
|
||||
|
@ -157,9 +140,9 @@ public class AMQSession implements SessionCallback {
|
|||
}
|
||||
AMQConsumer consumer = new AMQConsumer(this, openWireDest, info, scheduledPool);
|
||||
|
||||
consumer.init(slowConsumerDetectionListener, idGenerator.generateID());
|
||||
long nativeID = consumerIDGenerator.generateID();
|
||||
consumer.init(slowConsumerDetectionListener, nativeID);
|
||||
consumersList.add(consumer);
|
||||
consumers.put(consumer.getNativeId(), consumer);
|
||||
}
|
||||
|
||||
return consumersList;
|
||||
|
@ -180,7 +163,7 @@ public class AMQSession implements SessionCallback {
|
|||
|
||||
@Override
|
||||
public void browserFinished(ServerConsumer consumer) {
|
||||
AMQConsumer theConsumer = ((AMQServerConsumer) consumer).getAmqConsumer();
|
||||
AMQConsumer theConsumer = (AMQConsumer) consumer.getProtocolData();
|
||||
if (theConsumer != null) {
|
||||
theConsumer.browseFinished();
|
||||
}
|
||||
|
@ -204,13 +187,20 @@ public class AMQSession implements SessionCallback {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int sendMessage(ServerMessage message, ServerConsumer consumerID, int deliveryCount) {
|
||||
AMQConsumer consumer = consumers.get(consumerID.getID());
|
||||
return consumer.handleDeliver(message, deliveryCount);
|
||||
public int sendMessage(MessageReference reference,
|
||||
ServerMessage message,
|
||||
ServerConsumer consumer,
|
||||
int deliveryCount) {
|
||||
AMQConsumer theConsumer = (AMQConsumer) consumer.getProtocolData();
|
||||
return theConsumer.handleDeliver(reference, message, deliveryCount);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int sendLargeMessage(ServerMessage message, ServerConsumer consumerID, long bodySize, int deliveryCount) {
|
||||
public int sendLargeMessage(MessageReference reference,
|
||||
ServerMessage message,
|
||||
ServerConsumer consumerID,
|
||||
long bodySize,
|
||||
int deliveryCount) {
|
||||
// TODO Auto-generated method stub
|
||||
return 0;
|
||||
}
|
||||
|
@ -231,16 +221,15 @@ public class AMQSession implements SessionCallback {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean hasCredits(ServerConsumer consumerID) {
|
||||
public boolean hasCredits(ServerConsumer consumer) {
|
||||
|
||||
AMQConsumer amqConsumer;
|
||||
AMQConsumer amqConsumer = null;
|
||||
|
||||
amqConsumer = consumers.get(consumerID.getID());
|
||||
|
||||
if (amqConsumer != null) {
|
||||
return amqConsumer.hasCredits();
|
||||
if (consumer.getProtocolData() != null) {
|
||||
amqConsumer = (AMQConsumer) consumer.getProtocolData();
|
||||
}
|
||||
return false;
|
||||
|
||||
return amqConsumer != null && amqConsumer.hasCredits();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -252,11 +241,6 @@ public class AMQSession implements SessionCallback {
|
|||
public void send(final ProducerInfo producerInfo,
|
||||
final Message messageSend,
|
||||
boolean sendProducerAck) throws Exception {
|
||||
TransactionId tid = messageSend.getTransactionId();
|
||||
if (tid != null) {
|
||||
resetSessionTx(tid);
|
||||
}
|
||||
|
||||
messageSend.setBrokerInTime(System.currentTimeMillis());
|
||||
|
||||
ActiveMQDestination destination = messageSend.getDestination();
|
||||
|
@ -376,7 +360,7 @@ public class AMQSession implements SessionCallback {
|
|||
}
|
||||
}
|
||||
|
||||
public AMQServerSession getCoreSession() {
|
||||
public ServerSession getCoreSession() {
|
||||
return this.coreSession;
|
||||
}
|
||||
|
||||
|
@ -384,160 +368,16 @@ public class AMQSession implements SessionCallback {
|
|||
return this.server;
|
||||
}
|
||||
|
||||
public void removeConsumer(long consumerId) throws Exception {
|
||||
boolean failed = !(this.txId != null || this.isTx);
|
||||
|
||||
coreSession.amqCloseConsumer(consumerId, failed);
|
||||
consumers.remove(consumerId);
|
||||
}
|
||||
|
||||
public WireFormat getMarshaller() {
|
||||
return this.connection.getMarshaller();
|
||||
}
|
||||
|
||||
public void acknowledge(MessageAck ack, AMQConsumer consumer) throws Exception {
|
||||
TransactionId tid = ack.getTransactionId();
|
||||
if (tid != null) {
|
||||
this.resetSessionTx(ack.getTransactionId());
|
||||
}
|
||||
consumer.acknowledge(ack);
|
||||
|
||||
if (tid == null && ack.getAckType() == MessageAck.STANDARD_ACK_TYPE) {
|
||||
this.coreSession.commit();
|
||||
}
|
||||
}
|
||||
|
||||
//AMQ session and transactions are create separately. Whether a session
|
||||
//is transactional or not is known only when a TransactionInfo command
|
||||
//comes in.
|
||||
public void resetSessionTx(TransactionId xid) throws Exception {
|
||||
if ((this.txId != null) && (!this.txId.equals(xid))) {
|
||||
throw new IllegalStateException("Session already associated with a tx");
|
||||
}
|
||||
|
||||
this.isTx = true;
|
||||
if (this.txId == null) {
|
||||
//now reset session
|
||||
this.txId = xid;
|
||||
|
||||
if (xid.isXATransaction()) {
|
||||
XATransactionId xaXid = (XATransactionId) xid;
|
||||
coreSession.enableXA();
|
||||
XidImpl coreXid = new XidImpl(xaXid.getBranchQualifier(), xaXid.getFormatId(), xaXid.getGlobalTransactionId());
|
||||
coreSession.xaStart(coreXid);
|
||||
}
|
||||
else {
|
||||
coreSession.enableTx();
|
||||
}
|
||||
|
||||
this.manager.registerTx(this.txId, this);
|
||||
}
|
||||
}
|
||||
|
||||
private void checkTx(TransactionId inId) {
|
||||
if (this.txId == null) {
|
||||
throw new IllegalStateException("Session has no transaction associated with it");
|
||||
}
|
||||
|
||||
if (!this.txId.equals(inId)) {
|
||||
throw new IllegalStateException("Session already associated with another tx");
|
||||
}
|
||||
|
||||
this.isTx = true;
|
||||
}
|
||||
|
||||
public void endTransaction(TransactionInfo info) throws Exception {
|
||||
checkTx(info.getTransactionId());
|
||||
|
||||
if (txId.isXATransaction()) {
|
||||
XATransactionId xid = (XATransactionId) txId;
|
||||
XidImpl coreXid = new XidImpl(xid.getBranchQualifier(), xid.getFormatId(), xid.getGlobalTransactionId());
|
||||
this.coreSession.xaEnd(coreXid);
|
||||
}
|
||||
}
|
||||
|
||||
public void commitOnePhase(TransactionInfo info) throws Exception {
|
||||
checkTx(info.getTransactionId());
|
||||
|
||||
if (txId.isXATransaction()) {
|
||||
XATransactionId xid = (XATransactionId) txId;
|
||||
XidImpl coreXid = new XidImpl(xid.getBranchQualifier(), xid.getFormatId(), xid.getGlobalTransactionId());
|
||||
this.coreSession.xaCommit(coreXid, true);
|
||||
}
|
||||
else {
|
||||
Iterator<AMQConsumer> iter = consumers.values().iterator();
|
||||
while (iter.hasNext()) {
|
||||
AMQConsumer consumer = iter.next();
|
||||
consumer.finishTx();
|
||||
}
|
||||
this.coreSession.commit();
|
||||
}
|
||||
|
||||
this.txId = null;
|
||||
}
|
||||
|
||||
public void prepareTransaction(XATransactionId xid) throws Exception {
|
||||
checkTx(xid);
|
||||
XidImpl coreXid = new XidImpl(xid.getBranchQualifier(), xid.getFormatId(), xid.getGlobalTransactionId());
|
||||
this.coreSession.xaPrepare(coreXid);
|
||||
}
|
||||
|
||||
public void commitTwoPhase(XATransactionId xid) throws Exception {
|
||||
checkTx(xid);
|
||||
XidImpl coreXid = new XidImpl(xid.getBranchQualifier(), xid.getFormatId(), xid.getGlobalTransactionId());
|
||||
this.coreSession.xaCommit(coreXid, false);
|
||||
|
||||
this.txId = null;
|
||||
}
|
||||
|
||||
public void rollback(TransactionInfo info) throws Exception {
|
||||
checkTx(info.getTransactionId());
|
||||
if (this.txId.isXATransaction()) {
|
||||
XATransactionId xid = (XATransactionId) txId;
|
||||
XidImpl coreXid = new XidImpl(xid.getBranchQualifier(), xid.getFormatId(), xid.getGlobalTransactionId());
|
||||
this.coreSession.xaRollback(coreXid);
|
||||
}
|
||||
else {
|
||||
Iterator<AMQConsumer> iter = consumers.values().iterator();
|
||||
Set<Long> acked = new HashSet<>();
|
||||
while (iter.hasNext()) {
|
||||
AMQConsumer consumer = iter.next();
|
||||
consumer.rollbackTx(acked);
|
||||
}
|
||||
//on local rollback, amq broker doesn't do anything about the delivered
|
||||
//messages, which stay at clients until next time
|
||||
this.coreSession.amqRollback(acked);
|
||||
}
|
||||
|
||||
this.txId = null;
|
||||
}
|
||||
|
||||
public void recover(List<TransactionId> recovered) {
|
||||
List<Xid> xids = this.coreSession.xaGetInDoubtXids();
|
||||
for (Xid xid : xids) {
|
||||
XATransactionId amqXid = new XATransactionId(xid);
|
||||
recovered.add(amqXid);
|
||||
}
|
||||
}
|
||||
|
||||
public void forget(final TransactionId tid) throws Exception {
|
||||
checkTx(tid);
|
||||
XATransactionId xid = (XATransactionId) tid;
|
||||
XidImpl coreXid = new XidImpl(xid.getBranchQualifier(), xid.getFormatId(), xid.getGlobalTransactionId());
|
||||
this.coreSession.xaForget(coreXid);
|
||||
this.txId = null;
|
||||
}
|
||||
|
||||
public ConnectionInfo getConnectionInfo() {
|
||||
return this.connInfo;
|
||||
}
|
||||
|
||||
public void setInternal(boolean internal) {
|
||||
this.coreSession.setInternal(internal);
|
||||
}
|
||||
|
||||
public boolean isInternal() {
|
||||
return this.coreSession.isInternal();
|
||||
public void disableSecurity() {
|
||||
this.coreSession.disableSecurity();
|
||||
}
|
||||
|
||||
public void deliverMessage(MessageDispatch dispatch) {
|
||||
|
@ -548,20 +388,6 @@ public class AMQSession implements SessionCallback {
|
|||
this.coreSession.close(false);
|
||||
}
|
||||
|
||||
public AMQConsumer getConsumer(Long coreConsumerId) {
|
||||
return consumers.get(coreConsumerId);
|
||||
}
|
||||
|
||||
public void updateConsumerPrefetchSize(ConsumerId consumerId, int prefetch) {
|
||||
Iterator<AMQConsumer> iterator = consumers.values().iterator();
|
||||
while (iterator.hasNext()) {
|
||||
AMQConsumer consumer = iterator.next();
|
||||
if (consumer.getId().equals(consumerId)) {
|
||||
consumer.setPrefetchSize(prefetch);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public OpenWireConnection getConnection() {
|
||||
return connection;
|
||||
}
|
||||
|
|
|
@ -40,6 +40,12 @@ public class AMQSingleConsumerBrokerExchange extends AMQConsumerBrokerExchange {
|
|||
|
||||
@Override
|
||||
public void acknowledge(MessageAck ack) throws Exception {
|
||||
amqSession.acknowledge(ack, consumer);
|
||||
consumer.acknowledge(ack);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateConsumerPrefetchSize(int prefetch) {
|
||||
consumer.setPrefetchSize(prefetch);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1,32 +0,0 @@
|
|||
/*
|
||||
* 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.artemis.core.protocol.openwire.amq;
|
||||
|
||||
import javax.transaction.xa.Xid;
|
||||
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.AMQTransactionImpl;
|
||||
import org.apache.activemq.artemis.core.transaction.Transaction;
|
||||
import org.apache.activemq.artemis.core.transaction.TransactionFactory;
|
||||
|
||||
public class AMQTransactionFactory implements TransactionFactory {
|
||||
|
||||
@Override
|
||||
public Transaction newTransaction(Xid xid, StorageManager storageManager, int timeoutSeconds) {
|
||||
return new AMQTransactionImpl(xid, storageManager, timeoutSeconds);
|
||||
}
|
||||
}
|
|
@ -1,47 +0,0 @@
|
|||
/*
|
||||
* 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.artemis.core.protocol.openwire.amq;
|
||||
|
||||
import org.apache.activemq.command.MessageId;
|
||||
|
||||
public class MessageInfo {
|
||||
|
||||
public MessageId amqId;
|
||||
public long nativeId;
|
||||
public int size;
|
||||
//mark message that is acked within a local tx
|
||||
public boolean localAcked;
|
||||
|
||||
public MessageInfo(MessageId amqId, long nativeId, int size) {
|
||||
this.amqId = amqId;
|
||||
this.nativeId = nativeId;
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "native mid: " + this.nativeId + " amqId: " + amqId + " local acked: " + localAcked;
|
||||
}
|
||||
|
||||
public void setLocalAcked(boolean ack) {
|
||||
localAcked = ack;
|
||||
}
|
||||
|
||||
public boolean isLocalAcked() {
|
||||
return localAcked;
|
||||
}
|
||||
}
|
|
@ -226,7 +226,7 @@ class StompProtocolManager implements ProtocolManager<StompFrameInterceptor> {
|
|||
if (stompSession == null) {
|
||||
stompSession = new StompSession(connection, this, server.getStorageManager().newContext(server.getExecutorFactory().getExecutor()));
|
||||
String name = UUIDGenerator.getInstance().generateStringUUID();
|
||||
ServerSession session = server.createSession(name, connection.getLogin(), connection.getPasscode(), ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE, connection, true, false, false, false, null, stompSession, null, true);
|
||||
ServerSession session = server.createSession(name, connection.getLogin(), connection.getPasscode(), ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE, connection, true, false, false, false, null, stompSession, true);
|
||||
stompSession.setServerSession(session);
|
||||
sessions.put(connection.getID(), stompSession);
|
||||
}
|
||||
|
@ -239,7 +239,7 @@ class StompProtocolManager implements ProtocolManager<StompFrameInterceptor> {
|
|||
if (stompSession == null) {
|
||||
stompSession = new StompSession(connection, this, server.getStorageManager().newContext(executor));
|
||||
String name = UUIDGenerator.getInstance().generateStringUUID();
|
||||
ServerSession session = server.createSession(name, connection.getLogin(), connection.getPasscode(), ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE, connection, false, false, false, false, null, stompSession, null, true);
|
||||
ServerSession session = server.createSession(name, connection.getLogin(), connection.getPasscode(), ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE, connection, false, false, false, false, null, stompSession, true);
|
||||
stompSession.setServerSession(session);
|
||||
transactedSessions.put(txID, stompSession);
|
||||
}
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.activemq.artemis.core.persistence.StorageManager;
|
|||
import org.apache.activemq.artemis.core.persistence.impl.journal.LargeServerMessageImpl;
|
||||
import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants;
|
||||
import org.apache.activemq.artemis.core.server.LargeServerMessage;
|
||||
import org.apache.activemq.artemis.core.server.MessageReference;
|
||||
import org.apache.activemq.artemis.core.server.QueueQueryResult;
|
||||
import org.apache.activemq.artemis.core.server.ServerConsumer;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
|
@ -118,7 +119,7 @@ public class StompSession implements SessionCallback {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int sendMessage(ServerMessage serverMessage, final ServerConsumer consumer, int deliveryCount) {
|
||||
public int sendMessage(MessageReference ref, ServerMessage serverMessage, final ServerConsumer consumer, int deliveryCount) {
|
||||
LargeServerMessageImpl largeMessage = null;
|
||||
ServerMessage newServerMessage = serverMessage;
|
||||
try {
|
||||
|
@ -207,7 +208,7 @@ public class StompSession implements SessionCallback {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int sendLargeMessage(ServerMessage msg, ServerConsumer consumer, long bodySize, int deliveryCount) {
|
||||
public int sendLargeMessage(MessageReference ref, ServerMessage msg, ServerConsumer consumer, long bodySize, int deliveryCount) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
|
|
@ -1301,7 +1301,6 @@ public class ConfigurationImpl implements Configuration, Serializable {
|
|||
public TransportConfiguration[] getTransportConfigurations(final List<String> connectorNames) {
|
||||
TransportConfiguration[] tcConfigs = (TransportConfiguration[]) Array.newInstance(TransportConfiguration.class, connectorNames.size());
|
||||
int count = 0;
|
||||
System.out.println(debugConnectors());
|
||||
|
||||
for (String connectorName : connectorNames) {
|
||||
TransportConfiguration connector = getConnectorConfigurations().get(connectorName);
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
|
|||
import org.apache.activemq.artemis.core.server.MessageReference;
|
||||
import org.apache.activemq.artemis.core.server.Queue;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.core.transaction.Transaction;
|
||||
|
||||
public class PagedReferenceImpl implements PagedReference {
|
||||
|
||||
|
@ -48,6 +49,18 @@ public class PagedReferenceImpl implements PagedReference {
|
|||
|
||||
private boolean alreadyAcked;
|
||||
|
||||
private Object protocolData;
|
||||
|
||||
@Override
|
||||
public Object getProtocolData() {
|
||||
return protocolData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setProtocolData(Object protocolData) {
|
||||
this.protocolData = protocolData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServerMessage getMessage() {
|
||||
return getPagedMessage().getMessage();
|
||||
|
@ -199,9 +212,19 @@ public class PagedReferenceImpl implements PagedReference {
|
|||
subscription.ack(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void acknowledge(Transaction tx) throws Exception {
|
||||
if (tx == null) {
|
||||
getQueue().acknowledge(this);
|
||||
}
|
||||
else {
|
||||
getQueue().acknowledge(tx, this);
|
||||
}
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see java.lang.Object#toString()
|
||||
*/
|
||||
* @see java.lang.Object#toString()
|
||||
*/
|
||||
@Override
|
||||
public String toString() {
|
||||
String msgToString;
|
||||
|
|
|
@ -149,7 +149,7 @@ public class ActiveMQPacketHandler implements ChannelHandler {
|
|||
}
|
||||
|
||||
ServerSession session = server.createSession(request.getName(), activeMQPrincipal == null ? request.getUsername() : activeMQPrincipal.getUserName(), activeMQPrincipal == null ? request.getPassword() : activeMQPrincipal.getPassword(), request.getMinLargeMessageSize(), connection, request.isAutoCommitSends(), request.isAutoCommitAcks(), request.isPreAcknowledge(), request.isXA(), request.getDefaultAddress(),
|
||||
new CoreSessionCallback(request.getName(), protocolManager, channel, connection), null, true);
|
||||
new CoreSessionCallback(request.getName(), protocolManager, channel, connection), true);
|
||||
|
||||
ServerSessionPacketHandler handler = new ServerSessionPacketHandler(session, server.getStorageManager(), channel);
|
||||
channel.setHandler(handler);
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionRec
|
|||
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionReceiveLargeMessage;
|
||||
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionReceiveMessage;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
|
||||
import org.apache.activemq.artemis.core.server.MessageReference;
|
||||
import org.apache.activemq.artemis.core.server.ServerConsumer;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.ProtocolManager;
|
||||
|
@ -56,7 +57,7 @@ public final class CoreSessionCallback implements SessionCallback {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int sendLargeMessage(ServerMessage message, ServerConsumer consumer, long bodySize, int deliveryCount) {
|
||||
public int sendLargeMessage(MessageReference ref, ServerMessage message, ServerConsumer consumer, long bodySize, int deliveryCount) {
|
||||
Packet packet = new SessionReceiveLargeMessage(consumer.getID(), message, bodySize, deliveryCount);
|
||||
|
||||
channel.send(packet);
|
||||
|
@ -79,7 +80,7 @@ public final class CoreSessionCallback implements SessionCallback {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int sendMessage(ServerMessage message, ServerConsumer consumer, int deliveryCount) {
|
||||
public int sendMessage(MessageReference ref, ServerMessage message, ServerConsumer consumer, int deliveryCount) {
|
||||
Packet packet = new SessionReceiveMessage(consumer.getID(), message, deliveryCount);
|
||||
|
||||
int size = 0;
|
||||
|
|
|
@ -139,7 +139,6 @@ public interface ActiveMQServer extends ActiveMQComponent {
|
|||
boolean xa,
|
||||
String defaultAddress,
|
||||
SessionCallback callback,
|
||||
ServerSessionFactory sessionFactory,
|
||||
boolean autoCreateQueues) throws Exception;
|
||||
|
||||
SecurityStore getSecurityStore();
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.core.server;
|
||||
|
||||
import org.apache.activemq.artemis.core.transaction.Transaction;
|
||||
|
||||
/**
|
||||
* A reference to a message.
|
||||
*
|
||||
|
@ -35,6 +37,14 @@ public interface MessageReference {
|
|||
*/
|
||||
int getMessageMemoryEstimate();
|
||||
|
||||
/** To be used on holding protocol specific data during the delivery.
|
||||
* This will be only valid while the message is on the delivering queue at the consumer */
|
||||
Object getProtocolData();
|
||||
|
||||
/** To be used on holding protocol specific data during the delivery.
|
||||
* This will be only valid while the message is on the delivering queue at the consumer */
|
||||
void setProtocolData(Object data);
|
||||
|
||||
MessageReference copy(Queue queue);
|
||||
|
||||
/**
|
||||
|
@ -61,6 +71,8 @@ public interface MessageReference {
|
|||
|
||||
void acknowledge() throws Exception;
|
||||
|
||||
void acknowledge(Transaction tx) throws Exception;
|
||||
|
||||
void setConsumerId(Long consumerID);
|
||||
|
||||
Long getConsumerId();
|
||||
|
|
|
@ -150,6 +150,8 @@ public interface Queue extends Bindable {
|
|||
|
||||
int sendMessagesToDeadLetterAddress(Filter filter) throws Exception;
|
||||
|
||||
void sendToDeadLetterAddress(final Transaction tx, final MessageReference ref) throws Exception;
|
||||
|
||||
boolean changeReferencePriority(long messageID, byte newPriority) throws Exception;
|
||||
|
||||
int changeReferencesPriority(Filter filter, byte newPriority) throws Exception;
|
||||
|
|
|
@ -31,6 +31,12 @@ public interface ServerConsumer extends Consumer {
|
|||
|
||||
void fireSlowConsumer();
|
||||
|
||||
/** this is to be used with anything specific on a protocol head. */
|
||||
Object getProtocolData();
|
||||
|
||||
/** this is to be used with anything specific on a protocol head. */
|
||||
void setProtocolData(Object protocolData);
|
||||
|
||||
/**
|
||||
* @param protocolContext
|
||||
* @see #getProtocolContext()
|
||||
|
@ -68,6 +74,12 @@ public interface ServerConsumer extends Consumer {
|
|||
|
||||
MessageReference removeReferenceByID(long messageID) throws Exception;
|
||||
|
||||
/** Some protocols may choose to send the message back to delivering instead of redeliver.
|
||||
* For example openwire will redeliver through the client, so messages will go back to delivering list after rollback. */
|
||||
void backToDelivering(MessageReference reference);
|
||||
|
||||
List<MessageReference> getDeliveringReferencesBasedOnProtocol(boolean remove, Object protocolDataStart, Object protocolDataEnd);
|
||||
|
||||
void acknowledge(Transaction tx, long messageID) throws Exception;
|
||||
|
||||
void individualAcknowledge(Transaction tx, long messageID) throws Exception;
|
||||
|
|
|
@ -36,9 +36,19 @@ public interface ServerSession extends SecurityAuth {
|
|||
|
||||
Object getConnectionID();
|
||||
|
||||
/**
|
||||
* Certain protocols may create an internal session that shouldn't go through security checks.
|
||||
* make sure you don't expose this property through any protocol layer as that would be a security breach
|
||||
*/
|
||||
void enableSecurity();
|
||||
|
||||
void disableSecurity();
|
||||
|
||||
@Override
|
||||
RemotingConnection getRemotingConnection();
|
||||
|
||||
Transaction newTransaction();
|
||||
|
||||
boolean removeConsumer(long consumerID) throws Exception;
|
||||
|
||||
void acknowledge(long consumerID, long messageID) throws Exception;
|
||||
|
@ -87,6 +97,11 @@ public interface ServerSession extends SecurityAuth {
|
|||
|
||||
void stop();
|
||||
|
||||
/**
|
||||
* To be used by protocol heads that needs to control the transaction outside the session context.
|
||||
*/
|
||||
void resetTX(Transaction transaction);
|
||||
|
||||
Queue createQueue(SimpleString address,
|
||||
SimpleString name,
|
||||
SimpleString filterString,
|
||||
|
@ -100,6 +115,13 @@ public interface ServerSession extends SecurityAuth {
|
|||
SimpleString filterString,
|
||||
boolean browseOnly) throws Exception;
|
||||
|
||||
ServerConsumer createConsumer(final long consumerID,
|
||||
final SimpleString queueName,
|
||||
final SimpleString filterString,
|
||||
final boolean browseOnly,
|
||||
final boolean supportLargeMessage,
|
||||
final Integer credits) throws Exception;
|
||||
|
||||
QueueQueryResult executeQueueQuery(SimpleString name) throws Exception;
|
||||
|
||||
BindingQueryResult executeBindingQuery(SimpleString address) throws Exception;
|
||||
|
@ -151,6 +173,10 @@ public interface ServerSession extends SecurityAuth {
|
|||
|
||||
Transaction getCurrentTransaction();
|
||||
|
||||
ServerConsumer locateConsumer(long consumerID) throws Exception;
|
||||
|
||||
boolean isClosed();
|
||||
|
||||
void createSharedQueue(SimpleString address,
|
||||
SimpleString name,
|
||||
boolean durable,
|
||||
|
|
|
@ -1,55 +0,0 @@
|
|||
/*
|
||||
* 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.artemis.core.server;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.persistence.OperationContext;
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.core.postoffice.PostOffice;
|
||||
import org.apache.activemq.artemis.core.security.SecurityStore;
|
||||
import org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl;
|
||||
import org.apache.activemq.artemis.core.server.impl.ServerSessionImpl;
|
||||
import org.apache.activemq.artemis.core.server.management.ManagementService;
|
||||
import org.apache.activemq.artemis.core.transaction.ResourceManager;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.SessionCallback;
|
||||
|
||||
public interface ServerSessionFactory {
|
||||
|
||||
ServerSessionImpl createCoreSession(String name,
|
||||
String username,
|
||||
String password,
|
||||
int minLargeMessageSize,
|
||||
boolean autoCommitSends,
|
||||
boolean autoCommitAcks,
|
||||
boolean preAcknowledge,
|
||||
boolean persistDeliveryCountBeforeDelivery,
|
||||
boolean xa,
|
||||
RemotingConnection connection,
|
||||
StorageManager storageManager,
|
||||
PostOffice postOffice,
|
||||
ResourceManager resourceManager,
|
||||
SecurityStore securityStore,
|
||||
ManagementService managementService,
|
||||
ActiveMQServerImpl activeMQServerImpl,
|
||||
SimpleString managementAddress,
|
||||
SimpleString simpleString,
|
||||
SessionCallback callback,
|
||||
QueueCreator queueCreator,
|
||||
OperationContext context) throws Exception;
|
||||
|
||||
}
|
|
@ -112,7 +112,6 @@ import org.apache.activemq.artemis.core.server.QueueFactory;
|
|||
import org.apache.activemq.artemis.core.server.QueueQueryResult;
|
||||
import org.apache.activemq.artemis.core.server.SecuritySettingPlugin;
|
||||
import org.apache.activemq.artemis.core.server.ServerSession;
|
||||
import org.apache.activemq.artemis.core.server.ServerSessionFactory;
|
||||
import org.apache.activemq.artemis.core.server.ServiceRegistry;
|
||||
import org.apache.activemq.artemis.core.server.cluster.BackupManager;
|
||||
import org.apache.activemq.artemis.core.server.cluster.ClusterManager;
|
||||
|
@ -1091,7 +1090,6 @@ public class ActiveMQServerImpl implements ActiveMQServer {
|
|||
final boolean xa,
|
||||
final String defaultAddress,
|
||||
final SessionCallback callback,
|
||||
final ServerSessionFactory sessionFactory,
|
||||
final boolean autoCreateQueues) throws Exception {
|
||||
|
||||
if (securityStore != null) {
|
||||
|
@ -1105,7 +1103,7 @@ public class ActiveMQServerImpl implements ActiveMQServer {
|
|||
checkSessionLimit(username);
|
||||
|
||||
final OperationContext context = storageManager.newContext(getExecutorFactory().getExecutor());
|
||||
final ServerSessionImpl session = internalCreateSession(name, username, password, minLargeMessageSize, connection, autoCommitSends, autoCommitAcks, preAcknowledge, xa, defaultAddress, callback, context, sessionFactory, autoCreateQueues);
|
||||
final ServerSessionImpl session = internalCreateSession(name, username, password, minLargeMessageSize, connection, autoCommitSends, autoCommitAcks, preAcknowledge, xa, defaultAddress, callback, context, autoCreateQueues);
|
||||
|
||||
sessions.put(name, session);
|
||||
|
||||
|
@ -1178,14 +1176,8 @@ public class ActiveMQServerImpl implements ActiveMQServer {
|
|||
String defaultAddress,
|
||||
SessionCallback callback,
|
||||
OperationContext context,
|
||||
ServerSessionFactory sessionFactory,
|
||||
boolean autoCreateJMSQueues) throws Exception {
|
||||
if (sessionFactory == null) {
|
||||
return new ServerSessionImpl(name, username, password, minLargeMessageSize, autoCommitSends, autoCommitAcks, preAcknowledge, configuration.isPersistDeliveryCountBeforeDelivery(), xa, connection, storageManager, postOffice, resourceManager, securityStore, managementService, this, configuration.getManagementAddress(), defaultAddress == null ? null : new SimpleString(defaultAddress), callback, context, autoCreateJMSQueues ? jmsQueueCreator : null);
|
||||
}
|
||||
else {
|
||||
return sessionFactory.createCoreSession(name, username, password, minLargeMessageSize, autoCommitSends, autoCommitAcks, preAcknowledge, configuration.isPersistDeliveryCountBeforeDelivery(), xa, connection, storageManager, postOffice, resourceManager, securityStore, managementService, this, configuration.getManagementAddress(), defaultAddress == null ? null : new SimpleString(defaultAddress), callback, jmsQueueCreator, context);
|
||||
}
|
||||
return new ServerSessionImpl(name, username, password, minLargeMessageSize, autoCommitSends, autoCommitAcks, preAcknowledge, configuration.isPersistDeliveryCountBeforeDelivery(), xa, connection, storageManager, postOffice, resourceManager, securityStore, managementService, this, configuration.getManagementAddress(), defaultAddress == null ? null : new SimpleString(defaultAddress), callback, context, autoCreateJMSQueues ? jmsQueueCreator : null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.activemq.artemis.core.server.Queue;
|
|||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.core.settings.HierarchicalRepository;
|
||||
import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
|
||||
import org.apache.activemq.artemis.core.transaction.Transaction;
|
||||
|
||||
/**
|
||||
* A queue that will discard messages if a newer message with the same
|
||||
|
@ -187,6 +188,16 @@ public class LastValueQueue extends QueueImpl {
|
|||
map.remove(prop);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getProtocolData() {
|
||||
return ref.getProtocolData();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setProtocolData(Object data) {
|
||||
ref.setProtocolData(data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setAlreadyAcked() {
|
||||
ref.setAlreadyAcked();
|
||||
|
@ -246,6 +257,11 @@ public class LastValueQueue extends QueueImpl {
|
|||
ref.setScheduledDeliveryTime(scheduledDeliveryTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void acknowledge(Transaction tx) throws Exception {
|
||||
ref.acknowledge(tx);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPersistedCount(int count) {
|
||||
ref.setPersistedCount(count);
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
import org.apache.activemq.artemis.core.server.MessageReference;
|
||||
import org.apache.activemq.artemis.core.server.Queue;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.core.transaction.Transaction;
|
||||
import org.apache.activemq.artemis.utils.MemorySize;
|
||||
|
||||
/**
|
||||
|
@ -42,6 +43,8 @@ public class MessageReferenceImpl implements MessageReference {
|
|||
|
||||
private boolean alreadyAcked;
|
||||
|
||||
private Object protocolData;
|
||||
|
||||
// Static --------------------------------------------------------
|
||||
|
||||
private static final int memoryOffset;
|
||||
|
@ -86,6 +89,16 @@ public class MessageReferenceImpl implements MessageReference {
|
|||
|
||||
// MessageReference implementation -------------------------------
|
||||
|
||||
@Override
|
||||
public Object getProtocolData() {
|
||||
return protocolData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setProtocolData(Object protocolData) {
|
||||
this.protocolData = protocolData;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the persistedCount
|
||||
*/
|
||||
|
@ -174,7 +187,16 @@ public class MessageReferenceImpl implements MessageReference {
|
|||
|
||||
@Override
|
||||
public void acknowledge() throws Exception {
|
||||
queue.acknowledge(this);
|
||||
this.acknowledge(null);
|
||||
}
|
||||
|
||||
public void acknowledge(Transaction tx) throws Exception {
|
||||
if (tx == null) {
|
||||
getQueue().acknowledge(this);
|
||||
}
|
||||
else {
|
||||
getQueue().acknowledge(tx, this);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -1074,7 +1074,7 @@ public class QueueImpl implements Queue {
|
|||
if (isTrace) {
|
||||
ActiveMQServerLogger.LOGGER.trace("moving expired reference " + ref + " to address = " + expiryAddress + " from queue=" + this.getName());
|
||||
}
|
||||
move(expiryAddress, ref, true, false);
|
||||
move(null, expiryAddress, ref, true, false);
|
||||
}
|
||||
else {
|
||||
if (isTrace) {
|
||||
|
@ -1461,7 +1461,7 @@ public class QueueImpl implements Queue {
|
|||
MessageReference ref = iter.next();
|
||||
if (ref.getMessage().getMessageID() == messageID) {
|
||||
incDelivering();
|
||||
sendToDeadLetterAddress(ref);
|
||||
sendToDeadLetterAddress(null, ref);
|
||||
iter.remove();
|
||||
refRemoved(ref);
|
||||
return true;
|
||||
|
@ -1480,7 +1480,7 @@ public class QueueImpl implements Queue {
|
|||
MessageReference ref = iter.next();
|
||||
if (filter == null || filter.match(ref.getMessage())) {
|
||||
incDelivering();
|
||||
sendToDeadLetterAddress(ref);
|
||||
sendToDeadLetterAddress(null, ref);
|
||||
iter.remove();
|
||||
refRemoved(ref);
|
||||
count++;
|
||||
|
@ -1507,7 +1507,7 @@ public class QueueImpl implements Queue {
|
|||
refRemoved(ref);
|
||||
incDelivering();
|
||||
try {
|
||||
move(toAddress, ref, false, rejectDuplicate);
|
||||
move(null, toAddress, ref, false, rejectDuplicate);
|
||||
}
|
||||
catch (Exception e) {
|
||||
decDelivering();
|
||||
|
@ -2120,7 +2120,7 @@ public class QueueImpl implements Queue {
|
|||
if (isTrace) {
|
||||
ActiveMQServerLogger.LOGGER.trace("Sending reference " + reference + " to DLA = " + addressSettings.getDeadLetterAddress() + " since ref.getDeliveryCount=" + reference.getDeliveryCount() + "and maxDeliveries=" + maxDeliveries + " from queue=" + this.getName());
|
||||
}
|
||||
sendToDeadLetterAddress(reference, addressSettings.getDeadLetterAddress());
|
||||
sendToDeadLetterAddress(null, reference, addressSettings.getDeadLetterAddress());
|
||||
|
||||
return false;
|
||||
}
|
||||
|
@ -2337,36 +2337,45 @@ public class QueueImpl implements Queue {
|
|||
}
|
||||
}
|
||||
|
||||
public void sendToDeadLetterAddress(final MessageReference ref) throws Exception {
|
||||
sendToDeadLetterAddress(ref, addressSettingsRepository.getMatch(address.toString()).getDeadLetterAddress());
|
||||
public void sendToDeadLetterAddress(final Transaction tx, final MessageReference ref) throws Exception {
|
||||
sendToDeadLetterAddress(tx, ref, addressSettingsRepository.getMatch(address.toString()).getDeadLetterAddress());
|
||||
}
|
||||
|
||||
private void sendToDeadLetterAddress(final MessageReference ref,
|
||||
private void sendToDeadLetterAddress(final Transaction tx, final MessageReference ref,
|
||||
final SimpleString deadLetterAddress) throws Exception {
|
||||
if (deadLetterAddress != null) {
|
||||
Bindings bindingList = postOffice.getBindingsForAddress(deadLetterAddress);
|
||||
|
||||
if (bindingList.getBindings().isEmpty()) {
|
||||
ActiveMQServerLogger.LOGGER.messageExceededMaxDelivery(ref, deadLetterAddress);
|
||||
acknowledge(ref);
|
||||
ref.acknowledge(tx);
|
||||
}
|
||||
else {
|
||||
ActiveMQServerLogger.LOGGER.messageExceededMaxDeliverySendtoDLA(ref, deadLetterAddress, name);
|
||||
move(deadLetterAddress, ref, false, false);
|
||||
move(tx, deadLetterAddress, ref, false, false);
|
||||
}
|
||||
}
|
||||
else {
|
||||
ActiveMQServerLogger.LOGGER.messageExceededMaxDeliveryNoDLA(name);
|
||||
|
||||
acknowledge(ref);
|
||||
ref.acknowledge(tx);
|
||||
}
|
||||
}
|
||||
|
||||
private void move(final SimpleString address,
|
||||
private void move(final Transaction originalTX,
|
||||
final SimpleString address,
|
||||
final MessageReference ref,
|
||||
final boolean expiry,
|
||||
final boolean rejectDuplicate) throws Exception {
|
||||
Transaction tx = new TransactionImpl(storageManager);
|
||||
Transaction tx;
|
||||
|
||||
if (originalTX != null) {
|
||||
tx = originalTX;
|
||||
}
|
||||
else {
|
||||
// if no TX we create a new one to commit at the end
|
||||
tx = new TransactionImpl(storageManager);
|
||||
}
|
||||
|
||||
ServerMessage copyMessage = makeCopy(ref, expiry);
|
||||
|
||||
|
@ -2376,7 +2385,9 @@ public class QueueImpl implements Queue {
|
|||
|
||||
acknowledge(tx, ref);
|
||||
|
||||
tx.commit();
|
||||
if (originalTX == null) {
|
||||
tx.commit();
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
|
@ -20,7 +20,7 @@ import java.math.BigDecimal;
|
|||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
import java.util.concurrent.ConcurrentLinkedDeque;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
@ -83,10 +83,12 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
|||
|
||||
private final ServerSession session;
|
||||
|
||||
private final Object lock = new Object();
|
||||
protected final Object lock = new Object();
|
||||
|
||||
private final boolean supportLargeMessage;
|
||||
|
||||
private Object protocolData;
|
||||
|
||||
private Object protocolContext;
|
||||
|
||||
private final ActiveMQServer server;
|
||||
|
@ -123,7 +125,7 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
|||
|
||||
private final StorageManager storageManager;
|
||||
|
||||
protected final java.util.Queue<MessageReference> deliveringRefs = new ConcurrentLinkedQueue<>();
|
||||
protected final java.util.Deque<MessageReference> deliveringRefs = new ConcurrentLinkedDeque<>();
|
||||
|
||||
private final SessionCallback callback;
|
||||
|
||||
|
@ -230,6 +232,16 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
|||
// ServerConsumer implementation
|
||||
// ----------------------------------------------------------------------
|
||||
|
||||
@Override
|
||||
public Object getProtocolData() {
|
||||
return protocolData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setProtocolData(Object protocolData) {
|
||||
this.protocolData = protocolData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setlowConsumerDetection(SlowConsumerDetectionListener listener) {
|
||||
this.slowConsumerListener = listener;
|
||||
|
@ -524,7 +536,7 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
|||
forcedDeliveryMessage.putLongProperty(ClientConsumerImpl.FORCED_DELIVERY_MESSAGE, sequence);
|
||||
forcedDeliveryMessage.setAddress(messageQueue.getName());
|
||||
|
||||
callback.sendMessage(forcedDeliveryMessage, ServerConsumerImpl.this, 0);
|
||||
callback.sendMessage(null, forcedDeliveryMessage, ServerConsumerImpl.this, 0);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -560,7 +572,7 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
|||
if (!deliveringRefs.isEmpty()) {
|
||||
for (MessageReference ref : deliveringRefs) {
|
||||
if (performACK) {
|
||||
ackReference(tx, ref);
|
||||
ref.acknowledge(tx);
|
||||
|
||||
performACK = false;
|
||||
}
|
||||
|
@ -713,6 +725,44 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
|||
return messageQueue;
|
||||
}
|
||||
|
||||
|
||||
/** Remove references based on the protocolData.
|
||||
* there will be an interval defined between protocolDataStart and protocolDataEnd.
|
||||
* This method will fetch the delivering references, remove them from the delivering list and return a list.
|
||||
*
|
||||
* This will be useful for other protocols that will need this such as openWire or MQTT. */
|
||||
public List<MessageReference> getDeliveringReferencesBasedOnProtocol(boolean remove, Object protocolDataStart, Object protocolDataEnd) {
|
||||
LinkedList<MessageReference> retReferences = new LinkedList<>();
|
||||
boolean hit = false;
|
||||
synchronized (lock) {
|
||||
Iterator<MessageReference> referenceIterator = deliveringRefs.iterator();
|
||||
|
||||
while (referenceIterator.hasNext()) {
|
||||
MessageReference reference = referenceIterator.next();
|
||||
|
||||
if (!hit) {
|
||||
hit = reference.getProtocolData() != null && reference.getProtocolData().equals(protocolDataStart);
|
||||
}
|
||||
|
||||
// notice: this is not an else clause, this is also valid for the first hit
|
||||
if (hit) {
|
||||
if (remove) {
|
||||
referenceIterator.remove();
|
||||
}
|
||||
retReferences.add(reference);
|
||||
|
||||
// Whenever this is met we interrupt the loop
|
||||
// even on the first hit
|
||||
if (reference.getProtocolData() != null && reference.getProtocolData().equals(protocolDataEnd)) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return retReferences;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void acknowledge(Transaction tx, final long messageID) throws Exception {
|
||||
if (browseOnly) {
|
||||
|
@ -750,7 +800,8 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
|||
throw ils;
|
||||
}
|
||||
|
||||
ackReference(tx, ref);
|
||||
ref.acknowledge(tx);
|
||||
|
||||
acks++;
|
||||
} while (ref.getMessage().getMessageID() != messageID);
|
||||
|
||||
|
@ -780,15 +831,6 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
|||
}
|
||||
}
|
||||
|
||||
private void ackReference(Transaction tx, MessageReference ref) throws Exception {
|
||||
if (tx == null) {
|
||||
ref.getQueue().acknowledge(ref);
|
||||
}
|
||||
else {
|
||||
ref.getQueue().acknowledge(tx, ref);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void individualAcknowledge(Transaction tx,
|
||||
final long messageID) throws Exception {
|
||||
|
@ -818,7 +860,7 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
|||
throw ils;
|
||||
}
|
||||
|
||||
ackReference(tx, ref);
|
||||
ref.acknowledge(tx);
|
||||
|
||||
if (startedTransaction) {
|
||||
tx.commit();
|
||||
|
@ -866,6 +908,12 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
|||
ref.getQueue().cancel(ref, System.currentTimeMillis());
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void backToDelivering(MessageReference reference) {
|
||||
deliveringRefs.addFirst(reference);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MessageReference removeReferenceByID(final long messageID) throws Exception {
|
||||
if (browseOnly) {
|
||||
|
@ -965,7 +1013,7 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
|||
* @param message
|
||||
*/
|
||||
private void deliverStandardMessage(final MessageReference ref, final ServerMessage message) {
|
||||
int packetSize = callback.sendMessage(message, ServerConsumerImpl.this, ref.getDeliveryCount());
|
||||
int packetSize = callback.sendMessage(ref, message, ServerConsumerImpl.this, ref.getDeliveryCount());
|
||||
|
||||
if (availableCredits != null) {
|
||||
availableCredits.addAndGet(-packetSize);
|
||||
|
@ -1057,7 +1105,7 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
|||
|
||||
sentInitialPacket = true;
|
||||
|
||||
int packetSize = callback.sendLargeMessage(currentLargeMessage, ServerConsumerImpl.this, context.getLargeBodySize(), ref.getDeliveryCount());
|
||||
int packetSize = callback.sendLargeMessage(ref, currentLargeMessage, ServerConsumerImpl.this, context.getLargeBodySize(), ref.getDeliveryCount());
|
||||
|
||||
if (availableCredits != null) {
|
||||
availableCredits.addAndGet(-packetSize);
|
||||
|
|
|
@ -55,6 +55,7 @@ import org.apache.activemq.artemis.core.postoffice.QueueBinding;
|
|||
import org.apache.activemq.artemis.core.remoting.CloseListener;
|
||||
import org.apache.activemq.artemis.core.remoting.FailureListener;
|
||||
import org.apache.activemq.artemis.core.security.CheckType;
|
||||
import org.apache.activemq.artemis.core.security.SecurityAuth;
|
||||
import org.apache.activemq.artemis.core.security.SecurityStore;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQMessageBundle;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServer;
|
||||
|
@ -74,7 +75,6 @@ import org.apache.activemq.artemis.core.server.management.Notification;
|
|||
import org.apache.activemq.artemis.core.transaction.ResourceManager;
|
||||
import org.apache.activemq.artemis.core.transaction.Transaction;
|
||||
import org.apache.activemq.artemis.core.transaction.Transaction.State;
|
||||
import org.apache.activemq.artemis.core.transaction.TransactionFactory;
|
||||
import org.apache.activemq.artemis.core.transaction.TransactionOperationAbstract;
|
||||
import org.apache.activemq.artemis.core.transaction.TransactionPropertyIndexes;
|
||||
import org.apache.activemq.artemis.core.transaction.impl.TransactionImpl;
|
||||
|
@ -97,6 +97,8 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
|
||||
// Attributes ----------------------------------------------------------------------------
|
||||
|
||||
private boolean securityEnabled = true;
|
||||
|
||||
protected final String username;
|
||||
|
||||
protected final String password;
|
||||
|
@ -169,8 +171,6 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
// concurrently.
|
||||
private volatile boolean closed = false;
|
||||
|
||||
private final TransactionFactory transactionFactory;
|
||||
|
||||
public ServerSessionImpl(final String name,
|
||||
final String username,
|
||||
final String password,
|
||||
|
@ -192,31 +192,6 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
final SessionCallback callback,
|
||||
final OperationContext context,
|
||||
final QueueCreator queueCreator) throws Exception {
|
||||
this(name, username, password, minLargeMessageSize, autoCommitSends, autoCommitAcks, preAcknowledge, strictUpdateDeliveryCount, xa, remotingConnection, storageManager, postOffice, resourceManager, securityStore, managementService, server, managementAddress, defaultAddress, callback, context, null, queueCreator);
|
||||
}
|
||||
|
||||
public ServerSessionImpl(final String name,
|
||||
final String username,
|
||||
final String password,
|
||||
final int minLargeMessageSize,
|
||||
final boolean autoCommitSends,
|
||||
final boolean autoCommitAcks,
|
||||
final boolean preAcknowledge,
|
||||
final boolean strictUpdateDeliveryCount,
|
||||
final boolean xa,
|
||||
final RemotingConnection remotingConnection,
|
||||
final StorageManager storageManager,
|
||||
final PostOffice postOffice,
|
||||
final ResourceManager resourceManager,
|
||||
final SecurityStore securityStore,
|
||||
final ManagementService managementService,
|
||||
final ActiveMQServer server,
|
||||
final SimpleString managementAddress,
|
||||
final SimpleString defaultAddress,
|
||||
final SessionCallback callback,
|
||||
final OperationContext context,
|
||||
TransactionFactory transactionFactory,
|
||||
final QueueCreator queueCreator) throws Exception {
|
||||
this.username = username;
|
||||
|
||||
this.password = password;
|
||||
|
@ -261,13 +236,6 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
|
||||
this.queueCreator = queueCreator;
|
||||
|
||||
if (transactionFactory == null) {
|
||||
this.transactionFactory = new DefaultTransactionFactory();
|
||||
}
|
||||
else {
|
||||
this.transactionFactory = transactionFactory;
|
||||
}
|
||||
|
||||
if (!xa) {
|
||||
tx = newTransaction();
|
||||
}
|
||||
|
@ -275,6 +243,19 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
|
||||
// ServerSession implementation ----------------------------------------------------------------------------
|
||||
|
||||
@Override
|
||||
public void enableSecurity() {
|
||||
this.securityEnabled = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void disableSecurity() {
|
||||
this.securityEnabled = false;
|
||||
}
|
||||
|
||||
public boolean isClosed() {
|
||||
return closed;
|
||||
}
|
||||
/**
|
||||
* @return the sessionContext
|
||||
*/
|
||||
|
@ -386,7 +367,9 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
|
||||
remotingConnection.removeFailureListener(this);
|
||||
|
||||
callback.closed();
|
||||
if (callback != null) {
|
||||
callback.closed();
|
||||
}
|
||||
|
||||
closed = true;
|
||||
}
|
||||
|
@ -397,6 +380,12 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
return queueCreator;
|
||||
}
|
||||
|
||||
protected void securityCheck(SimpleString address, CheckType checkType, SecurityAuth auth) throws Exception {
|
||||
if (securityEnabled) {
|
||||
securityStore.check(address, checkType, auth);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServerConsumer createConsumer(final long consumerID,
|
||||
final SimpleString queueName,
|
||||
|
@ -417,11 +406,11 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
throw ActiveMQMessageBundle.BUNDLE.noSuchQueue(queueName);
|
||||
}
|
||||
|
||||
securityStore.check(binding.getAddress(), CheckType.CONSUME, this);
|
||||
securityCheck(binding.getAddress(), CheckType.CONSUME, this);
|
||||
|
||||
Filter filter = FilterImpl.createFilter(filterString);
|
||||
|
||||
ServerConsumer consumer = newConsumer(consumerID, this, (QueueBinding) binding, filter, started, browseOnly, storageManager, callback, preAcknowledge, strictUpdateDeliveryCount, managementService, supportLargeMessage, credits);
|
||||
ServerConsumer consumer = new ServerConsumerImpl(consumerID, this, (QueueBinding)binding, filter, started, browseOnly, storageManager, callback, preAcknowledge, strictUpdateDeliveryCount, managementService, supportLargeMessage, credits, server);
|
||||
consumers.put(consumer.getID(), consumer);
|
||||
|
||||
if (!browseOnly) {
|
||||
|
@ -465,20 +454,13 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
return consumer;
|
||||
}
|
||||
|
||||
protected ServerConsumer newConsumer(long consumerID,
|
||||
ServerSessionImpl serverSessionImpl,
|
||||
QueueBinding binding,
|
||||
Filter filter,
|
||||
boolean started2,
|
||||
boolean browseOnly,
|
||||
StorageManager storageManager2,
|
||||
SessionCallback callback2,
|
||||
boolean preAcknowledge2,
|
||||
boolean strictUpdateDeliveryCount2,
|
||||
ManagementService managementService2,
|
||||
boolean supportLargeMessage,
|
||||
Integer credits) throws Exception {
|
||||
return new ServerConsumerImpl(consumerID, this, binding, filter, started, browseOnly, storageManager, callback, preAcknowledge, strictUpdateDeliveryCount, managementService, supportLargeMessage, credits, server);
|
||||
/** Some protocols may chose to hold their transactions outside of the ServerSession.
|
||||
* This can be used to replace the transaction.
|
||||
* Notice that we set autoCommitACK and autoCommitSends to true if tx == null */
|
||||
public void resetTX(Transaction transaction) {
|
||||
this.tx = transaction;
|
||||
this.autoCommitAcks = transaction == null;
|
||||
this.autoCommitSends = transaction == null;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -489,10 +471,10 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
final boolean durable) throws Exception {
|
||||
if (durable) {
|
||||
// make sure the user has privileges to create this queue
|
||||
securityStore.check(address, CheckType.CREATE_DURABLE_QUEUE, this);
|
||||
securityCheck(address, CheckType.CREATE_DURABLE_QUEUE, this);
|
||||
}
|
||||
else {
|
||||
securityStore.check(address, CheckType.CREATE_NON_DURABLE_QUEUE, this);
|
||||
securityCheck(address, CheckType.CREATE_NON_DURABLE_QUEUE, this);
|
||||
}
|
||||
|
||||
server.checkQueueCreationLimit(getUsername());
|
||||
|
@ -537,7 +519,7 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
final SimpleString name,
|
||||
boolean durable,
|
||||
final SimpleString filterString) throws Exception {
|
||||
securityStore.check(address, CheckType.CREATE_NON_DURABLE_QUEUE, this);
|
||||
securityCheck(address, CheckType.CREATE_NON_DURABLE_QUEUE, this);
|
||||
|
||||
server.checkQueueCreationLimit(getUsername());
|
||||
|
||||
|
@ -632,7 +614,7 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
|
||||
@Override
|
||||
public void forceConsumerDelivery(final long consumerID, final long sequence) throws Exception {
|
||||
ServerConsumer consumer = consumers.get(consumerID);
|
||||
ServerConsumer consumer = locateConsumer(consumerID);
|
||||
|
||||
// this would be possible if the server consumer was closed by pings/pongs.. etc
|
||||
if (consumer != null) {
|
||||
|
@ -640,15 +622,6 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
}
|
||||
}
|
||||
|
||||
public void promptDelivery(long consumerID) {
|
||||
ServerConsumer consumer = consumers.get(consumerID);
|
||||
|
||||
// this would be possible if the server consumer was closed by pings/pongs.. etc
|
||||
if (consumer != null) {
|
||||
consumer.promptDelivery();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void acknowledge(final long consumerID, final long messageID) throws Exception {
|
||||
ServerConsumer consumer = findConsumer(consumerID);
|
||||
|
@ -674,8 +647,12 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
}
|
||||
}
|
||||
|
||||
public ServerConsumer locateConsumer(long consumerID) {
|
||||
return consumers.get(consumerID);
|
||||
}
|
||||
|
||||
private ServerConsumer findConsumer(long consumerID) throws Exception {
|
||||
ServerConsumer consumer = consumers.get(consumerID);
|
||||
ServerConsumer consumer = locateConsumer(consumerID);
|
||||
|
||||
if (consumer == null) {
|
||||
Transaction currentTX = tx;
|
||||
|
@ -710,7 +687,7 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
|
||||
@Override
|
||||
public void individualCancel(final long consumerID, final long messageID, boolean failed) throws Exception {
|
||||
ServerConsumer consumer = consumers.get(consumerID);
|
||||
ServerConsumer consumer = locateConsumer(consumerID);
|
||||
|
||||
if (consumer != null) {
|
||||
consumer.individualCancel(messageID, failed);
|
||||
|
@ -720,7 +697,7 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
|
||||
@Override
|
||||
public void expire(final long consumerID, final long messageID) throws Exception {
|
||||
MessageReference ref = consumers.get(consumerID).removeReferenceByID(messageID);
|
||||
MessageReference ref = locateConsumer(consumerID).removeReferenceByID(messageID);
|
||||
|
||||
if (ref != null) {
|
||||
ref.getQueue().expire(ref);
|
||||
|
@ -778,8 +755,8 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
/**
|
||||
* @return
|
||||
*/
|
||||
protected Transaction newTransaction() {
|
||||
return transactionFactory.newTransaction(null, storageManager, timeoutSeconds);
|
||||
public Transaction newTransaction() {
|
||||
return new TransactionImpl(null, storageManager, timeoutSeconds);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -787,7 +764,7 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
* @return
|
||||
*/
|
||||
private Transaction newTransaction(final Xid xid) {
|
||||
return transactionFactory.newTransaction(xid, storageManager, timeoutSeconds);
|
||||
return new TransactionImpl(xid, storageManager, timeoutSeconds);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1122,13 +1099,7 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
|
||||
@Override
|
||||
public List<Xid> xaGetInDoubtXids() {
|
||||
List<Xid> xids = new ArrayList<>();
|
||||
|
||||
xids.addAll(resourceManager.getPreparedTransactions());
|
||||
xids.addAll(resourceManager.getHeuristicCommittedTransactions());
|
||||
xids.addAll(resourceManager.getHeuristicRolledbackTransactions());
|
||||
|
||||
return xids;
|
||||
return resourceManager.getInDoubtTransactions();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1189,7 +1160,7 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
|
||||
@Override
|
||||
public void closeConsumer(final long consumerID) throws Exception {
|
||||
final ServerConsumer consumer = consumers.get(consumerID);
|
||||
final ServerConsumer consumer = locateConsumer(consumerID);
|
||||
|
||||
if (consumer != null) {
|
||||
consumer.close(false);
|
||||
|
@ -1201,7 +1172,7 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
|
||||
@Override
|
||||
public void receiveConsumerCredits(final long consumerID, final int credits) throws Exception {
|
||||
ServerConsumer consumer = consumers.get(consumerID);
|
||||
ServerConsumer consumer = locateConsumer(consumerID);
|
||||
|
||||
if (consumer == null) {
|
||||
ActiveMQServerLogger.LOGGER.debug("There is no consumer with id " + consumerID);
|
||||
|
@ -1214,9 +1185,6 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
|
||||
@Override
|
||||
public Transaction getCurrentTransaction() {
|
||||
if (tx == null) {
|
||||
tx = newTransaction();
|
||||
}
|
||||
return tx;
|
||||
}
|
||||
|
||||
|
@ -1489,7 +1457,7 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
|
||||
private void handleManagementMessage(final ServerMessage message, final boolean direct) throws Exception {
|
||||
try {
|
||||
securityStore.check(message.getAddress(), CheckType.MANAGE, this);
|
||||
securityCheck(message.getAddress(), CheckType.MANAGE, this);
|
||||
}
|
||||
catch (ActiveMQException e) {
|
||||
if (!autoCommitSends) {
|
||||
|
@ -1564,7 +1532,7 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
protected void doSend(final ServerMessage msg, final boolean direct) throws Exception {
|
||||
// check the user has write access to this address.
|
||||
try {
|
||||
securityStore.check(msg.getAddress(), CheckType.SEND, this);
|
||||
securityCheck(msg.getAddress(), CheckType.SEND, this);
|
||||
}
|
||||
catch (ActiveMQException e) {
|
||||
if (!autoCommitSends && tx != null) {
|
||||
|
@ -1613,12 +1581,4 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
|
|||
return Collections.emptyList();
|
||||
}
|
||||
}
|
||||
|
||||
private static class DefaultTransactionFactory implements TransactionFactory {
|
||||
|
||||
@Override
|
||||
public Transaction newTransaction(Xid xid, StorageManager storageManager, int timeoutSeconds) {
|
||||
return new TransactionImpl(xid, storageManager, timeoutSeconds);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -45,4 +45,7 @@ public interface ResourceManager extends ActiveMQComponent {
|
|||
|
||||
List<Xid> getHeuristicRolledbackTransactions();
|
||||
|
||||
List<Xid> getInDoubtTransactions();
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -33,6 +33,12 @@ public interface Transaction {
|
|||
ACTIVE, PREPARED, COMMITTED, ROLLEDBACK, SUSPENDED, ROLLBACK_ONLY
|
||||
}
|
||||
|
||||
Object getProtocolData();
|
||||
|
||||
/** Protocol managers can use this field to store any object needed.
|
||||
* An example would be the Session used by the transaction on openwire */
|
||||
void setProtocolData(Object data);
|
||||
|
||||
boolean isEffective();
|
||||
|
||||
void prepare() throws Exception;
|
||||
|
|
|
@ -1,26 +0,0 @@
|
|||
/*
|
||||
* 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.artemis.core.transaction;
|
||||
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
|
||||
import javax.transaction.xa.Xid;
|
||||
|
||||
public interface TransactionFactory {
|
||||
|
||||
Transaction newTransaction(Xid xid, StorageManager storageManager, int timeoutSeconds);
|
||||
}
|
|
@ -21,6 +21,7 @@ import java.util.ArrayList;
|
|||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -161,6 +162,17 @@ public class ResourceManagerImpl implements ResourceManager {
|
|||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Xid> getInDoubtTransactions() {
|
||||
List<Xid> xids = new LinkedList<>();
|
||||
|
||||
xids.addAll(getPreparedTransactions());
|
||||
xids.addAll(getHeuristicCommittedTransactions());
|
||||
xids.addAll(getHeuristicRolledbackTransactions());
|
||||
|
||||
return xids;
|
||||
}
|
||||
|
||||
private List<Xid> getHeuristicCompletedTransactions(final boolean isCommit) {
|
||||
List<Xid> xids = new ArrayList<>();
|
||||
for (HeuristicCompletionHolder holder : heuristicCompletions) {
|
||||
|
@ -207,6 +219,7 @@ public class ResourceManagerImpl implements ResourceManager {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
synchronized void setFuture(final Future<?> future) {
|
||||
this.future = future;
|
||||
}
|
||||
|
@ -221,7 +234,6 @@ public class ResourceManagerImpl implements ResourceManager {
|
|||
|
||||
}
|
||||
|
||||
|
||||
private static final class HeuristicCompletionHolder {
|
||||
|
||||
public final boolean isCommit;
|
||||
|
|
|
@ -59,6 +59,18 @@ public class TransactionImpl implements Transaction {
|
|||
|
||||
private int timeoutSeconds = -1;
|
||||
|
||||
private Object protocolData;
|
||||
|
||||
@Override
|
||||
public Object getProtocolData() {
|
||||
return protocolData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setProtocolData(Object protocolData) {
|
||||
this.protocolData = protocolData;
|
||||
}
|
||||
|
||||
public TransactionImpl(final StorageManager storageManager, final int timeoutSeconds) {
|
||||
this.storageManager = storageManager;
|
||||
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.activemq.artemis.spi.core.protocol;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.server.MessageReference;
|
||||
import org.apache.activemq.artemis.core.server.ServerConsumer;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.spi.core.remoting.ReadyListener;
|
||||
|
@ -36,9 +37,15 @@ public interface SessionCallback {
|
|||
|
||||
void sendProducerCreditsFailMessage(int credits, SimpleString address);
|
||||
|
||||
int sendMessage(ServerMessage message, ServerConsumer consumerID, int deliveryCount);
|
||||
// Note: don't be tempted to remove the parameter message
|
||||
// Even though ref will contain the message in certain cases
|
||||
// such as paging the message could be a SoftReference or WeakReference
|
||||
// and I wanted to avoid re-fetching paged data in case of GCs on this specific case.
|
||||
//
|
||||
// Future developments may change this, but beware why I have chosen to keep the parameter separated here
|
||||
int sendMessage(MessageReference ref, ServerMessage message, ServerConsumer consumerID, int deliveryCount);
|
||||
|
||||
int sendLargeMessage(ServerMessage message, ServerConsumer consumerID, long bodySize, int deliveryCount);
|
||||
int sendLargeMessage(MessageReference reference, ServerMessage message, ServerConsumer consumerID, long bodySize, int deliveryCount);
|
||||
|
||||
int sendLargeMessageContinuation(ServerConsumer consumerID,
|
||||
byte[] body,
|
||||
|
|
|
@ -1105,6 +1105,10 @@ public class ScheduledDeliveryHandlerTest extends Assert {
|
|||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sendToDeadLetterAddress(Transaction tx, MessageReference ref) throws Exception {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean sendMessageToDeadLetterAddress(long messageID) throws Exception {
|
||||
return false;
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.junit.rules.ExternalResource;
|
|||
* This is useful to make sure you won't have leaking threads between tests
|
||||
*/
|
||||
public class ThreadLeakCheckRule extends ExternalResource {
|
||||
private static Set<String> extraThreads = new HashSet<String>();
|
||||
private static Set<String> knownThreads = new HashSet<String>();
|
||||
|
||||
boolean enabled = true;
|
||||
|
||||
|
@ -97,10 +97,12 @@ public class ThreadLeakCheckRule extends ExternalResource {
|
|||
|
||||
}
|
||||
|
||||
public static void addExtraThreads(String... threads) {
|
||||
for (String th : threads) {
|
||||
extraThreads.add(th);
|
||||
}
|
||||
public static void removeKownThread(String name) {
|
||||
knownThreads.remove(name);
|
||||
}
|
||||
|
||||
public static void addKownThread(String name) {
|
||||
knownThreads.add(name);
|
||||
}
|
||||
|
||||
private boolean checkThread() {
|
||||
|
@ -191,21 +193,20 @@ public class ThreadLeakCheckRule extends ExternalResource {
|
|||
// Static workers used by MQTT client.
|
||||
return true;
|
||||
}
|
||||
else if (extraThreads.contains(threadName)) {
|
||||
return true;
|
||||
}
|
||||
else {
|
||||
for (StackTraceElement element : thread.getStackTrace()) {
|
||||
if (element.getClassName().contains("org.jboss.byteman.agent.TransformListener")) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
for (String known: knownThreads) {
|
||||
if (threadName.contains(known)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public static void clearExtraThreads() {
|
||||
extraThreads.clear();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -116,7 +116,7 @@ public class JmsRollbackRedeliveryTest {
|
|||
Session session = connection.createSession(true, Session.AUTO_ACKNOWLEDGE);
|
||||
Destination destination = session.createQueue(destinationName);
|
||||
MessageConsumer consumer = session.createConsumer(destination);
|
||||
TextMessage msg = (TextMessage) consumer.receive(6000000);
|
||||
TextMessage msg = (TextMessage) consumer.receive(5000);
|
||||
if (msg != null) {
|
||||
if (rolledback.put(msg.getText(), Boolean.TRUE) != null) {
|
||||
LOG.info("Received message " + msg.getText() + " (" + received.getAndIncrement() + ")" + msg.getJMSMessageID());
|
||||
|
|
|
@ -54,12 +54,12 @@ public class SoWriteTimeoutClientTest extends OpenwireArtemisBaseTest {
|
|||
@BeforeClass
|
||||
public static void beforeTest() throws Exception {
|
||||
//this thread keeps alive in original test too. Exclude it.
|
||||
ThreadLeakCheckRule.addExtraThreads("WriteTimeoutFilter-Timeout-1");
|
||||
ThreadLeakCheckRule.addKownThread("WriteTimeoutFilter-Timeout");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void afterTest() throws Exception {
|
||||
ThreadLeakCheckRule.clearExtraThreads();
|
||||
ThreadLeakCheckRule.removeKownThread("WriteTimeoutFilter-Timeout");
|
||||
}
|
||||
|
||||
@Before
|
||||
|
|
|
@ -24,7 +24,6 @@ import static org.junit.Assert.fail;
|
|||
|
||||
import java.util.ArrayList;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
|
@ -39,10 +38,7 @@ import javax.jms.TextMessage;
|
|||
|
||||
import org.apache.activemq.ActiveMQConnection;
|
||||
import org.apache.activemq.ActiveMQConnectionFactory;
|
||||
import org.apache.activemq.artemis.core.config.Configuration;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.OpenWireConnection;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.amq.AMQConnectionContext;
|
||||
import org.apache.activemq.artemis.jms.server.config.impl.JMSConfigurationImpl;
|
||||
import org.apache.activemq.artemis.jms.server.embedded.EmbeddedJMS;
|
||||
import org.apache.activemq.broker.artemiswrapper.OpenwireArtemisBaseTest;
|
||||
import org.jboss.byteman.contrib.bmunit.BMRule;
|
||||
|
@ -87,8 +83,8 @@ public class FailoverConsumerOutstandingCommitTest extends OpenwireArtemisBaseTe
|
|||
targetLocation = "ENTRY",
|
||||
action = "org.apache.activemq.transport.failover.FailoverConsumerOutstandingCommitTest.holdResponse($0)"), @BMRule(
|
||||
name = "stop broker before commit",
|
||||
targetClass = "org.apache.activemq.artemis.core.server.impl.ServerSessionImpl",
|
||||
targetMethod = "commit",
|
||||
targetClass = "org.apache.activemq.artemis.core.protocol.openwire.OpenWireConnection$CommandProcessor",
|
||||
targetMethod = "processCommitTransactionOnePhase",
|
||||
targetLocation = "ENTRY",
|
||||
action = "org.apache.activemq.transport.failover.FailoverConsumerOutstandingCommitTest.stopServerInTransaction()"),})
|
||||
public void testFailoverConsumerDups() throws Exception {
|
||||
|
@ -181,10 +177,10 @@ public class FailoverConsumerOutstandingCommitTest extends OpenwireArtemisBaseTe
|
|||
|
||||
@BMRule(
|
||||
name = "stop broker before commit",
|
||||
targetClass = "org.apache.activemq.artemis.core.server.impl.ServerSessionImpl",
|
||||
targetMethod = "commit",
|
||||
targetClass = "org.apache.activemq.artemis.core.protocol.openwire.OpenWireConnection$CommandProcessor",
|
||||
targetMethod = "processCommitTransactionOnePhase",
|
||||
targetLocation = "ENTRY",
|
||||
action = "org.apache.activemq.transport.failover.FailoverConsumerOutstandingCommitTest.stopServerInTransaction();return")})
|
||||
action = "org.apache.activemq.transport.failover.FailoverConsumerOutstandingCommitTest.stopServerInTransaction();return null")})
|
||||
public void TestFailoverConsumerOutstandingSendTxIncomplete() throws Exception {
|
||||
doTestFailoverConsumerOutstandingSendTx(false);
|
||||
}
|
||||
|
@ -198,8 +194,8 @@ public class FailoverConsumerOutstandingCommitTest extends OpenwireArtemisBaseTe
|
|||
targetLocation = "ENTRY",
|
||||
action = "org.apache.activemq.transport.failover.FailoverConsumerOutstandingCommitTest.holdResponse($0)"), @BMRule(
|
||||
name = "stop broker after commit",
|
||||
targetClass = "org.apache.activemq.artemis.core.server.impl.ServerSessionImpl",
|
||||
targetMethod = "commit",
|
||||
targetClass = "org.apache.activemq.artemis.core.protocol.openwire.OpenWireConnection$CommandProcessor",
|
||||
targetMethod = "processCommitTransactionOnePhase",
|
||||
targetLocation = "AT EXIT",
|
||||
action = "org.apache.activemq.transport.failover.FailoverConsumerOutstandingCommitTest.stopServerInTransaction()")})
|
||||
public void TestFailoverConsumerOutstandingSendTxComplete() throws Exception {
|
||||
|
@ -236,11 +232,13 @@ public class FailoverConsumerOutstandingCommitTest extends OpenwireArtemisBaseTe
|
|||
testConsumer.setMessageListener(new MessageListener() {
|
||||
|
||||
public void onMessage(Message message) {
|
||||
LOG.info("consume one and commit: " + message);
|
||||
LOG.info("consume one: " + message);
|
||||
assertNotNull("got message", message);
|
||||
receivedMessages.add((TextMessage) message);
|
||||
try {
|
||||
LOG.info("send one");
|
||||
produceMessage(consumerSession, signalDestination, 1);
|
||||
LOG.info("commit session");
|
||||
consumerSession.commit();
|
||||
}
|
||||
catch (JMSException e) {
|
||||
|
@ -272,8 +270,8 @@ public class FailoverConsumerOutstandingCommitTest extends OpenwireArtemisBaseTe
|
|||
|
||||
// will be stopped by the plugin
|
||||
brokerStopLatch.await();
|
||||
server.stop();
|
||||
doByteman.set(false);
|
||||
server.stop();
|
||||
server = createBroker();
|
||||
server.start();
|
||||
|
||||
|
|
|
@ -519,31 +519,31 @@ public class FailoverTransactionTest extends OpenwireArtemisBaseTest {
|
|||
Assert.assertTrue("connectionconsumer did not get a message", connectionConsumerGotOne.await(10, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
@Test
|
||||
@BMRules(
|
||||
rules = {
|
||||
@BMRule(
|
||||
name = "set no return response and stop the broker",
|
||||
targetClass = "org.apache.activemq.artemis.core.protocol.openwire.OpenWireConnection$CommandProcessor",
|
||||
targetMethod = "processMessageAck",
|
||||
targetLocation = "ENTRY",
|
||||
action = "org.apache.activemq.transport.failover.FailoverTransactionTest.holdResponseAndStopBroker($0)")
|
||||
}
|
||||
)
|
||||
public void testFailoverConsumerAckLost() throws Exception {
|
||||
LOG.info(this + " running test testFailoverConsumerAckLost");
|
||||
// as failure depends on hash order of state tracker recovery, do a few times
|
||||
for (int i = 0; i < 3; i++) {
|
||||
try {
|
||||
LOG.info("Iteration: " + i);
|
||||
doTestFailoverConsumerAckLost(i);
|
||||
}
|
||||
finally {
|
||||
stopBroker();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// @Test
|
||||
// @BMRules(
|
||||
// rules = {
|
||||
// @BMRule(
|
||||
// name = "set no return response and stop the broker",
|
||||
// targetClass = "org.apache.activemq.artemis.core.protocol.openwire.OpenWireConnection$CommandProcessor",
|
||||
// targetMethod = "processMessageAck",
|
||||
// targetLocation = "ENTRY",
|
||||
// action = "org.apache.activemq.transport.failover.FailoverTransactionTest.holdResponseAndStopBroker($0)")
|
||||
// }
|
||||
// )
|
||||
// public void testFailoverConsumerAckLost() throws Exception {
|
||||
// LOG.info(this + " running test testFailoverConsumerAckLost");
|
||||
// // as failure depends on hash order of state tracker recovery, do a few times
|
||||
// for (int i = 0; i < 3; i++) {
|
||||
// try {
|
||||
// LOG.info("Iteration: " + i);
|
||||
// doTestFailoverConsumerAckLost(i);
|
||||
// }
|
||||
// finally {
|
||||
// stopBroker();
|
||||
// }
|
||||
// }
|
||||
// }
|
||||
//
|
||||
@SuppressWarnings("unchecked")
|
||||
public void doTestFailoverConsumerAckLost(final int pauseSeconds) throws Exception {
|
||||
broker = createBroker();
|
||||
|
@ -567,12 +567,12 @@ public class FailoverTransactionTest extends OpenwireArtemisBaseTest {
|
|||
connection = cf.createConnection();
|
||||
connection.start();
|
||||
connections.add(connection);
|
||||
final Session consumerSession1 = connection.createSession(true, Session.AUTO_ACKNOWLEDGE);
|
||||
final Session consumerSession1 = connection.createSession(true, Session.SESSION_TRANSACTED);
|
||||
|
||||
connection = cf.createConnection();
|
||||
connection.start();
|
||||
connections.add(connection);
|
||||
final Session consumerSession2 = connection.createSession(true, Session.AUTO_ACKNOWLEDGE);
|
||||
final Session consumerSession2 = connection.createSession(true, Session.SESSION_TRANSACTED);
|
||||
|
||||
final MessageConsumer consumer1 = consumerSession1.createConsumer(destination);
|
||||
final MessageConsumer consumer2 = consumerSession2.createConsumer(destination);
|
||||
|
@ -583,7 +583,7 @@ public class FailoverTransactionTest extends OpenwireArtemisBaseTest {
|
|||
final Vector<Message> receivedMessages = new Vector<>();
|
||||
final CountDownLatch commitDoneLatch = new CountDownLatch(1);
|
||||
final AtomicBoolean gotTransactionRolledBackException = new AtomicBoolean(false);
|
||||
new Thread() {
|
||||
Thread t = new Thread("doTestFailoverConsumerAckLost(" + pauseSeconds + ")") {
|
||||
public void run() {
|
||||
LOG.info("doing async commit after consume...");
|
||||
try {
|
||||
|
@ -630,10 +630,16 @@ public class FailoverTransactionTest extends OpenwireArtemisBaseTest {
|
|||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
}.start();
|
||||
};
|
||||
t.start();
|
||||
|
||||
// will be stopped by the plugin
|
||||
brokerStopLatch.await(60, TimeUnit.SECONDS);
|
||||
t.join(30000);
|
||||
if (t.isAlive()) {
|
||||
t.interrupt();
|
||||
Assert.fail("Thread " + t.getName() + " is still alive");
|
||||
}
|
||||
broker = createBroker();
|
||||
broker.start();
|
||||
doByteman.set(false);
|
||||
|
@ -1056,8 +1062,10 @@ public class FailoverTransactionTest extends OpenwireArtemisBaseTest {
|
|||
new Thread() {
|
||||
public void run() {
|
||||
try {
|
||||
broker.stop();
|
||||
broker = null;
|
||||
if (broker != null) {
|
||||
broker.stop();
|
||||
broker = null;
|
||||
}
|
||||
LOG.info("broker stopped.");
|
||||
}
|
||||
catch (Exception e) {
|
||||
|
|
|
@ -50,10 +50,10 @@ import org.apache.activemq.artemis.core.postoffice.impl.LocalQueueBinding;
|
|||
import org.apache.activemq.artemis.core.protocol.core.Packet;
|
||||
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionReceiveMessage;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServer;
|
||||
import org.apache.activemq.artemis.core.server.MessageReference;
|
||||
import org.apache.activemq.artemis.core.server.Queue;
|
||||
import org.apache.activemq.artemis.core.server.ServerConsumer;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.core.server.ServerSessionFactory;
|
||||
import org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl;
|
||||
import org.apache.activemq.artemis.core.server.impl.QueueFactoryImpl;
|
||||
import org.apache.activemq.artemis.core.server.impl.QueueImpl;
|
||||
|
@ -507,7 +507,7 @@ public class HangConsumerTest extends ActiveMQTestBase {
|
|||
* @see SessionCallback#sendMessage(org.apache.activemq.artemis.core.server.ServerMessage, long, int)
|
||||
*/
|
||||
@Override
|
||||
public int sendMessage(ServerMessage message, ServerConsumer consumer, int deliveryCount) {
|
||||
public int sendMessage(MessageReference ref, ServerMessage message, ServerConsumer consumer, int deliveryCount) {
|
||||
inCall.countDown();
|
||||
try {
|
||||
callbackSemaphore.acquire();
|
||||
|
@ -518,7 +518,7 @@ public class HangConsumerTest extends ActiveMQTestBase {
|
|||
}
|
||||
|
||||
try {
|
||||
return targetCallback.sendMessage(message, consumer, deliveryCount);
|
||||
return targetCallback.sendMessage(ref, message, consumer, deliveryCount);
|
||||
}
|
||||
finally {
|
||||
callbackSemaphore.release();
|
||||
|
@ -530,8 +530,8 @@ public class HangConsumerTest extends ActiveMQTestBase {
|
|||
* @see SessionCallback#sendLargeMessage(org.apache.activemq.artemis.core.server.ServerMessage, long, long, int)
|
||||
*/
|
||||
@Override
|
||||
public int sendLargeMessage(ServerMessage message, ServerConsumer consumer, long bodySize, int deliveryCount) {
|
||||
return targetCallback.sendLargeMessage(message, consumer, bodySize, deliveryCount);
|
||||
public int sendLargeMessage(MessageReference reference, ServerMessage message, ServerConsumer consumer, long bodySize, int deliveryCount) {
|
||||
return targetCallback.sendLargeMessage(reference, message, consumer, bodySize, deliveryCount);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
|
@ -581,7 +581,6 @@ public class HangConsumerTest extends ActiveMQTestBase {
|
|||
String defaultAddress,
|
||||
SessionCallback callback,
|
||||
OperationContext context,
|
||||
ServerSessionFactory sessionFactory,
|
||||
boolean autoCreateQueue) throws Exception {
|
||||
return new ServerSessionImpl(name, username, password, minLargeMessageSize, autoCommitSends, autoCommitAcks, preAcknowledge, getConfiguration().isPersistDeliveryCountBeforeDelivery(), xa, connection, getStorageManager(), getPostOffice(), getResourceManager(), getSecurityStore(), getManagementService(), this, getConfiguration().getManagementAddress(), defaultAddress == null ? null : new SimpleString(defaultAddress), new MyCallback(callback), context, null);
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import javax.jms.TemporaryQueue;
|
|||
import javax.jms.TextMessage;
|
||||
|
||||
import org.apache.activemq.command.ActiveMQQueue;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -118,7 +119,7 @@ public class BasicSecurityTest extends BasicOpenWireTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testSendnReceiveAuthorization() throws Exception {
|
||||
public void testSendnReceiveAuthorization() throws Exception {
|
||||
Connection sendingConn = null;
|
||||
Connection receivingConn = null;
|
||||
|
||||
|
@ -152,16 +153,18 @@ public class BasicSecurityTest extends BasicOpenWireTest {
|
|||
producer = sendingSession.createProducer(dest);
|
||||
producer.send(message);
|
||||
|
||||
MessageConsumer consumer = null;
|
||||
MessageConsumer consumer;
|
||||
try {
|
||||
consumer = sendingSession.createConsumer(dest);
|
||||
Assert.fail("exception expected");
|
||||
}
|
||||
catch (JMSSecurityException e) {
|
||||
e.printStackTrace();
|
||||
//expected
|
||||
}
|
||||
|
||||
consumer = receivingSession.createConsumer(dest);
|
||||
TextMessage received = (TextMessage) consumer.receive();
|
||||
TextMessage received = (TextMessage) consumer.receive(5000);
|
||||
|
||||
assertNotNull(received);
|
||||
assertEquals("Hello World", received.getText());
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.activemq.artemis.tests.integration.openwire;
|
|||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.OpenWireUtil;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.util.OpenWireUtil;
|
||||
import org.junit.Test;
|
||||
|
||||
public class OpenWireUtilTest {
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.activemq.artemis.tests.integration.openwire;
|
|||
|
||||
import javax.jms.Connection;
|
||||
import javax.jms.Destination;
|
||||
import javax.jms.InvalidDestinationException;
|
||||
import javax.jms.JMSException;
|
||||
import javax.jms.MessageConsumer;
|
||||
import javax.jms.MessageProducer;
|
||||
|
@ -27,24 +26,27 @@ import javax.jms.Session;
|
|||
import javax.jms.TemporaryQueue;
|
||||
import javax.jms.TemporaryTopic;
|
||||
import javax.jms.TextMessage;
|
||||
import javax.jms.XAConnection;
|
||||
import javax.jms.XASession;
|
||||
import javax.transaction.xa.XAResource;
|
||||
import javax.transaction.xa.Xid;
|
||||
import java.util.Collection;
|
||||
import java.util.LinkedList;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.activemq.ActiveMQConnectionFactory;
|
||||
import org.apache.activemq.ActiveMQSession;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.api.jms.ActiveMQJMSClient;
|
||||
import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
|
||||
import org.apache.activemq.command.ActiveMQQueue;
|
||||
import org.apache.activemq.command.ActiveMQTopic;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
||||
public class SimpleOpenWireTest extends BasicOpenWireTest {
|
||||
|
||||
@Rule
|
||||
public ExpectedException thrown = ExpectedException.none();
|
||||
|
||||
@Override
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
|
@ -52,6 +54,158 @@ public class SimpleOpenWireTest extends BasicOpenWireTest {
|
|||
super.setUp();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimple() throws Exception {
|
||||
Connection connection = factory.createConnection();
|
||||
|
||||
Collection<Session> sessions = new LinkedList<>();
|
||||
|
||||
for (int i = 0; i < 10; i++) {
|
||||
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
|
||||
sessions.add(session);
|
||||
}
|
||||
|
||||
connection.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTransactionalSimple() throws Exception {
|
||||
try (Connection connection = factory.createConnection()) {
|
||||
|
||||
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
|
||||
Queue queue = session.createQueue(queueName);
|
||||
System.out.println("Queue:" + queue);
|
||||
MessageProducer producer = session.createProducer(queue);
|
||||
MessageConsumer consumer = session.createConsumer(queue);
|
||||
producer.send(session.createTextMessage("test"));
|
||||
session.commit();
|
||||
|
||||
Assert.assertNull(consumer.receive(100));
|
||||
connection.start();
|
||||
|
||||
TextMessage message = (TextMessage) consumer.receive(5000);
|
||||
Assert.assertEquals("test", message.getText());
|
||||
|
||||
Assert.assertNotNull(message);
|
||||
|
||||
message.acknowledge();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testXASimple() throws Exception {
|
||||
XAConnection connection = xaFactory.createXAConnection();
|
||||
|
||||
Collection<Session> sessions = new LinkedList<>();
|
||||
|
||||
for (int i = 0; i < 10; i++) {
|
||||
XASession session = connection.createXASession();
|
||||
session.getXAResource().start(newXID(), XAResource.TMNOFLAGS);
|
||||
sessions.add(session);
|
||||
}
|
||||
|
||||
connection.close();
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClientACK() throws Exception {
|
||||
try {
|
||||
|
||||
Connection connection = factory.createConnection();
|
||||
|
||||
Collection<Session> sessions = new LinkedList<>();
|
||||
|
||||
Session session = connection.createSession(false, Session.CLIENT_ACKNOWLEDGE);
|
||||
Queue queue = session.createQueue(queueName);
|
||||
System.out.println("Queue:" + queue);
|
||||
MessageProducer producer = session.createProducer(queue);
|
||||
MessageConsumer consumer = session.createConsumer(queue);
|
||||
producer.send(session.createTextMessage("test"));
|
||||
|
||||
Assert.assertNull(consumer.receive(100));
|
||||
connection.start();
|
||||
|
||||
TextMessage message = (TextMessage) consumer.receive(5000);
|
||||
|
||||
Assert.assertNotNull(message);
|
||||
|
||||
message.acknowledge();
|
||||
|
||||
connection.close();
|
||||
|
||||
System.err.println("Done!!!");
|
||||
}
|
||||
catch (Throwable e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRollback() throws Exception {
|
||||
try (Connection connection = factory.createConnection()) {
|
||||
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
|
||||
Queue queue = session.createQueue(queueName);
|
||||
System.out.println("Queue:" + queue);
|
||||
MessageProducer producer = session.createProducer(queue);
|
||||
MessageConsumer consumer = session.createConsumer(queue);
|
||||
producer.send(session.createTextMessage("test"));
|
||||
producer.send(session.createTextMessage("test2"));
|
||||
connection.start();
|
||||
Assert.assertNull(consumer.receiveNoWait());
|
||||
session.rollback();
|
||||
producer.send(session.createTextMessage("test2"));
|
||||
Assert.assertNull(consumer.receiveNoWait());
|
||||
session.commit();
|
||||
TextMessage msg = (TextMessage) consumer.receive(1000);
|
||||
|
||||
Assert.assertNotNull(msg);
|
||||
Assert.assertEquals("test2", msg.getText());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAutoAck() throws Exception {
|
||||
Connection connection = factory.createConnection();
|
||||
|
||||
Collection<Session> sessions = new LinkedList<>();
|
||||
|
||||
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
|
||||
Queue queue = session.createQueue(queueName);
|
||||
System.out.println("Queue:" + queue);
|
||||
MessageProducer producer = session.createProducer(queue);
|
||||
MessageConsumer consumer = session.createConsumer(queue);
|
||||
TextMessage msg = session.createTextMessage("test");
|
||||
msg.setStringProperty("abc", "testAutoACK");
|
||||
producer.send(msg);
|
||||
|
||||
Assert.assertNull(consumer.receive(100));
|
||||
connection.start();
|
||||
|
||||
TextMessage message = (TextMessage) consumer.receive(5000);
|
||||
|
||||
Assert.assertNotNull(message);
|
||||
|
||||
connection.close();
|
||||
|
||||
System.err.println("Done!!!");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testProducerFlowControl() throws Exception {
|
||||
ActiveMQConnectionFactory factory = new ActiveMQConnectionFactory(urlString);
|
||||
|
||||
factory.setProducerWindowSize(1024 * 64);
|
||||
|
||||
Connection connection = factory.createConnection();
|
||||
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
|
||||
Queue queue = session.createQueue(queueName);
|
||||
MessageProducer producer = session.createProducer(queue);
|
||||
producer.send(session.createTextMessage("test"));
|
||||
|
||||
connection.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleQueue() throws Exception {
|
||||
connection.start();
|
||||
|
@ -88,12 +242,11 @@ public class SimpleOpenWireTest extends BasicOpenWireTest {
|
|||
session.close();
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
// @Test -- ignored for now
|
||||
public void testKeepAlive() throws Exception {
|
||||
connection.start();
|
||||
|
||||
Thread.sleep(125000);
|
||||
Thread.sleep(30000);
|
||||
|
||||
connection.createSession(false, 1);
|
||||
}
|
||||
|
@ -237,9 +390,11 @@ public class SimpleOpenWireTest extends BasicOpenWireTest {
|
|||
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
|
||||
Queue queue = session.createQueue("foo");
|
||||
|
||||
thrown.expect(InvalidDestinationException.class);
|
||||
thrown.expect(JMSException.class);
|
||||
session.createProducer(queue);
|
||||
try {
|
||||
session.createProducer(queue);
|
||||
}
|
||||
catch (JMSException expected) {
|
||||
}
|
||||
session.close();
|
||||
}
|
||||
|
||||
|
@ -390,7 +545,6 @@ public class SimpleOpenWireTest extends BasicOpenWireTest {
|
|||
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* This is the example shipped with the distribution
|
||||
*
|
||||
|
@ -473,7 +627,6 @@ public class SimpleOpenWireTest extends BasicOpenWireTest {
|
|||
|
||||
}
|
||||
|
||||
|
||||
// simple test sending openwire, consuming core
|
||||
@Test
|
||||
public void testMixedOpenWireExample2() throws Exception {
|
||||
|
@ -513,5 +666,396 @@ public class SimpleOpenWireTest extends BasicOpenWireTest {
|
|||
conn2.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testXAConsumer() throws Exception {
|
||||
Queue queue;
|
||||
try (Session session = connection.createSession(false, Session.CLIENT_ACKNOWLEDGE)) {
|
||||
queue = session.createQueue(queueName);
|
||||
System.out.println("Queue:" + queue);
|
||||
MessageProducer producer = session.createProducer(queue);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
TextMessage msg = session.createTextMessage("test" + i);
|
||||
msg.setStringProperty("myobj", "test" + i);
|
||||
producer.send(msg);
|
||||
}
|
||||
session.close();
|
||||
}
|
||||
|
||||
try (XAConnection xaconnection = xaFactory.createXAConnection()) {
|
||||
Xid xid = newXID();
|
||||
|
||||
XASession session = xaconnection.createXASession();
|
||||
session.getXAResource().start(xid, XAResource.TMNOFLAGS);
|
||||
MessageConsumer consumer = session.createConsumer(queue);
|
||||
xaconnection.start();
|
||||
for (int i = 0; i < 5; i++) {
|
||||
TextMessage message = (TextMessage) consumer.receive(5000);
|
||||
Assert.assertNotNull(message);
|
||||
Assert.assertEquals("test" + i, message.getText());
|
||||
}
|
||||
session.getXAResource().end(xid, XAResource.TMSUCCESS);
|
||||
session.getXAResource().rollback(xid);
|
||||
consumer.close();
|
||||
xaconnection.close();
|
||||
}
|
||||
|
||||
try (Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE)) {
|
||||
connection.start();
|
||||
MessageConsumer consumer = session.createConsumer(queue);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
TextMessage message = (TextMessage) consumer.receive(5000);
|
||||
Assert.assertNotNull(message);
|
||||
// Assert.assertEquals("test" + i, message.getText());
|
||||
System.out.println("Message " + message.getText());
|
||||
}
|
||||
checkDuplicate(consumer);
|
||||
System.out.println("Queue:" + queue);
|
||||
session.close();
|
||||
}
|
||||
|
||||
System.err.println("Done!!!");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testXASameConsumerRollback() throws Exception {
|
||||
Queue queue;
|
||||
try (Session session = connection.createSession(false, Session.CLIENT_ACKNOWLEDGE)) {
|
||||
queue = session.createQueue(queueName);
|
||||
System.out.println("Queue:" + queue);
|
||||
MessageProducer producer = session.createProducer(queue);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
TextMessage msg = session.createTextMessage("test" + i);
|
||||
msg.setStringProperty("myobj", "test" + i);
|
||||
producer.send(msg);
|
||||
}
|
||||
session.close();
|
||||
}
|
||||
|
||||
try (XAConnection xaconnection = xaFactory.createXAConnection()) {
|
||||
Xid xid = newXID();
|
||||
|
||||
XASession session = xaconnection.createXASession();
|
||||
session.getXAResource().start(xid, XAResource.TMNOFLAGS);
|
||||
MessageConsumer consumer = session.createConsumer(queue);
|
||||
xaconnection.start();
|
||||
for (int i = 0; i < 5; i++) {
|
||||
TextMessage message = (TextMessage) consumer.receive(5000);
|
||||
Assert.assertNotNull(message);
|
||||
Assert.assertEquals("test" + i, message.getText());
|
||||
}
|
||||
session.getXAResource().end(xid, XAResource.TMSUCCESS);
|
||||
session.getXAResource().rollback(xid);
|
||||
|
||||
xid = newXID();
|
||||
session.getXAResource().start(xid, XAResource.TMNOFLAGS);
|
||||
|
||||
for (int i = 0; i < 10; i++) {
|
||||
TextMessage message = (TextMessage) consumer.receive(5000);
|
||||
Assert.assertNotNull(message);
|
||||
Assert.assertEquals("test" + i, message.getText());
|
||||
}
|
||||
|
||||
checkDuplicate(consumer);
|
||||
|
||||
session.getXAResource().end(xid, XAResource.TMSUCCESS);
|
||||
session.getXAResource().commit(xid, true);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testXAPrepare() throws Exception {
|
||||
try {
|
||||
|
||||
XAConnection connection = xaFactory.createXAConnection();
|
||||
|
||||
XASession xasession = connection.createXASession();
|
||||
|
||||
Xid xid = newXID();
|
||||
xasession.getXAResource().start(xid, XAResource.TMNOFLAGS);
|
||||
Queue queue = xasession.createQueue(queueName);
|
||||
MessageProducer producer = xasession.createProducer(queue);
|
||||
producer.send(xasession.createTextMessage("hello"));
|
||||
producer.send(xasession.createTextMessage("hello"));
|
||||
xasession.getXAResource().end(xid, XAResource.TMSUCCESS);
|
||||
|
||||
xasession.getXAResource().prepare(xid);
|
||||
|
||||
connection.close();
|
||||
|
||||
System.err.println("Done!!!");
|
||||
}
|
||||
catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAutoSend() throws Exception {
|
||||
connection.start();
|
||||
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
|
||||
|
||||
Queue queue = session.createQueue(queueName);
|
||||
MessageConsumer consumer = session.createConsumer(queue);
|
||||
|
||||
MessageProducer producer = session.createProducer(queue);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
producer.send(session.createTextMessage("testXX" + i));
|
||||
}
|
||||
connection.start();
|
||||
|
||||
for (int i = 0; i < 10; i++) {
|
||||
TextMessage txt = (TextMessage) consumer.receive(5000);
|
||||
|
||||
Assert.assertEquals("testXX" + i, txt.getText());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCommitCloseConsumerBefore() throws Exception {
|
||||
testCommitCloseConsumer(true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCommitCloseConsumerAfter() throws Exception {
|
||||
testCommitCloseConsumer(false);
|
||||
}
|
||||
|
||||
private void testCommitCloseConsumer(boolean closeBefore) throws Exception {
|
||||
connection.start();
|
||||
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
|
||||
|
||||
Queue queue = session.createQueue(queueName);
|
||||
MessageConsumer consumer = session.createConsumer(queue);
|
||||
|
||||
MessageProducer producer = session.createProducer(queue);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
TextMessage msg = session.createTextMessage("testXX" + i);
|
||||
msg.setStringProperty("count", "str " + i);
|
||||
producer.send(msg);
|
||||
}
|
||||
session.commit();
|
||||
connection.start();
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
TextMessage txt = (TextMessage) consumer.receive(5000);
|
||||
Assert.assertEquals("testXX" + i, txt.getText());
|
||||
}
|
||||
if (closeBefore) {
|
||||
consumer.close();
|
||||
}
|
||||
|
||||
session.commit();
|
||||
|
||||
// we're testing two scenarios.
|
||||
// closing the consumer before commit or after commit
|
||||
if (!closeBefore) {
|
||||
consumer.close();
|
||||
}
|
||||
|
||||
consumer = session.createConsumer(queue);
|
||||
// Assert.assertNull(consumer.receiveNoWait());
|
||||
for (int i = 5; i < 10; i++) {
|
||||
TextMessage txt = (TextMessage) consumer.receive(5000);
|
||||
Assert.assertEquals("testXX" + i, txt.getText());
|
||||
}
|
||||
|
||||
Assert.assertNull(consumer.receiveNoWait());
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRollbackWithAcked() throws Exception {
|
||||
connection.start();
|
||||
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
|
||||
|
||||
Queue queue = session.createQueue(queueName);
|
||||
MessageConsumer consumer = session.createConsumer(queue);
|
||||
|
||||
MessageProducer producer = session.createProducer(queue);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
TextMessage msg = session.createTextMessage("testXX" + i);
|
||||
msg.setStringProperty("count", "str " + i);
|
||||
producer.send(msg);
|
||||
}
|
||||
session.commit();
|
||||
connection.start();
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
TextMessage txt = (TextMessage) consumer.receive(5000);
|
||||
Assert.assertEquals("testXX" + i, txt.getText());
|
||||
}
|
||||
|
||||
session.rollback();
|
||||
|
||||
consumer.close();
|
||||
|
||||
consumer = session.createConsumer(queue);
|
||||
// Assert.assertNull(consumer.receiveNoWait());
|
||||
for (int i = 0; i < 10; i++) {
|
||||
TextMessage txt = (TextMessage) consumer.receive(5000);
|
||||
// System.out.println("TXT::" + txt);
|
||||
Assert.assertNotNull(txt);
|
||||
System.out.println("TXT " + txt.getText());
|
||||
// Assert.assertEquals("testXX" + i, txt.getText());
|
||||
}
|
||||
session.commit();
|
||||
|
||||
checkDuplicate(consumer);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRollbackLocal() throws Exception {
|
||||
connection.start();
|
||||
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
|
||||
|
||||
Queue queue = session.createQueue(queueName);
|
||||
MessageConsumer consumer = session.createConsumer(queue);
|
||||
|
||||
MessageProducer producer = session.createProducer(queue);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
TextMessage msg = session.createTextMessage("testXX" + i);
|
||||
msg.setStringProperty("count", "str " + i);
|
||||
producer.send(msg);
|
||||
}
|
||||
session.commit();
|
||||
connection.start();
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
TextMessage txt = (TextMessage) consumer.receive(500);
|
||||
Assert.assertEquals("testXX" + i, txt.getText());
|
||||
}
|
||||
|
||||
session.rollback();
|
||||
|
||||
for (int i = 0; i < 10; i++) {
|
||||
TextMessage txt = (TextMessage) consumer.receive(5000);
|
||||
Assert.assertNotNull(txt);
|
||||
System.out.println("TXT " + txt.getText());
|
||||
Assert.assertEquals("testXX" + i, txt.getText());
|
||||
}
|
||||
|
||||
checkDuplicate(consumer);
|
||||
|
||||
session.commit();
|
||||
|
||||
}
|
||||
|
||||
private void checkDuplicate(MessageConsumer consumer) throws JMSException {
|
||||
boolean duplicatedMessages = false;
|
||||
while (true) {
|
||||
TextMessage txt = (TextMessage) consumer.receiveNoWait();
|
||||
if (txt == null) {
|
||||
break;
|
||||
}
|
||||
else {
|
||||
duplicatedMessages = true;
|
||||
System.out.println("received in duplicate:" + txt.getText());
|
||||
}
|
||||
}
|
||||
|
||||
Assert.assertFalse("received messages in duplicate", duplicatedMessages);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIndividualAck() throws Exception {
|
||||
connection.start();
|
||||
Session session = connection.createSession(false, ActiveMQSession.INDIVIDUAL_ACKNOWLEDGE);
|
||||
|
||||
Queue queue = session.createQueue(queueName);
|
||||
MessageConsumer consumer = session.createConsumer(queue);
|
||||
|
||||
MessageProducer producer = session.createProducer(queue);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
TextMessage msg = session.createTextMessage("testXX" + i);
|
||||
msg.setStringProperty("count", "str " + i);
|
||||
producer.send(msg);
|
||||
}
|
||||
connection.start();
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
TextMessage txt = (TextMessage) consumer.receive(5000);
|
||||
if (i == 4) {
|
||||
txt.acknowledge();
|
||||
}
|
||||
Assert.assertEquals("testXX" + i, txt.getText());
|
||||
}
|
||||
|
||||
consumer.close();
|
||||
|
||||
consumer = session.createConsumer(queue);
|
||||
// Assert.assertNull(consumer.receiveNoWait());
|
||||
for (int i = 0; i < 4; i++) {
|
||||
TextMessage txt = (TextMessage) consumer.receive(5000);
|
||||
txt.acknowledge();
|
||||
Assert.assertEquals("testXX" + i, txt.getText());
|
||||
}
|
||||
|
||||
for (int i = 5; i < 10; i++) {
|
||||
TextMessage txt = (TextMessage) consumer.receive(5000);
|
||||
txt.acknowledge();
|
||||
Assert.assertEquals("testXX" + i, txt.getText());
|
||||
}
|
||||
|
||||
checkDuplicate(consumer);
|
||||
|
||||
Assert.assertNull(consumer.receiveNoWait());
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCommitCloseConsumeXA() throws Exception {
|
||||
|
||||
Queue queue;
|
||||
{
|
||||
connection.start();
|
||||
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
|
||||
|
||||
queue = session.createQueue(queueName);
|
||||
|
||||
MessageProducer producer = session.createProducer(queue);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
TextMessage msg = session.createTextMessage("testXX" + i);
|
||||
msg.setStringProperty("count", "str " + i);
|
||||
producer.send(msg);
|
||||
}
|
||||
session.commit();
|
||||
}
|
||||
|
||||
try (XAConnection xaconnection = xaFactory.createXAConnection()) {
|
||||
xaconnection.start();
|
||||
|
||||
XASession xasession = xaconnection.createXASession();
|
||||
Xid xid = newXID();
|
||||
xasession.getXAResource().start(xid, XAResource.TMNOFLAGS);
|
||||
MessageConsumer consumer = xasession.createConsumer(queue);
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
TextMessage txt = (TextMessage) consumer.receive(5000);
|
||||
Assert.assertEquals("testXX" + i, txt.getText());
|
||||
}
|
||||
|
||||
consumer.close();
|
||||
|
||||
xasession.getXAResource().end(xid, XAResource.TMSUCCESS);
|
||||
xasession.getXAResource().prepare(xid);
|
||||
xasession.getXAResource().commit(xid, false);
|
||||
|
||||
xaconnection.close();
|
||||
}
|
||||
|
||||
{
|
||||
connection.start();
|
||||
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
|
||||
try (MessageConsumer consumer = session.createConsumer(queue)) {
|
||||
for (int i = 5; i < 10; i++) {
|
||||
TextMessage txt = (TextMessage) consumer.receive(5000);
|
||||
Assert.assertEquals("testXX" + i, txt.getText());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -104,6 +104,16 @@ public class BindingsImplTest extends ActiveMQTestBase {
|
|||
|
||||
private final class FakeTransaction implements Transaction {
|
||||
|
||||
@Override
|
||||
public Object getProtocolData() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setProtocolData(Object data) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addOperation(final TransactionOperation sync) {
|
||||
|
||||
|
|
|
@ -41,6 +41,11 @@ public class FakeQueue implements Queue {
|
|||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sendToDeadLetterAddress(Transaction tx, MessageReference ref) throws Exception {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deleteQueue(boolean removeConsumers) throws Exception {
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue