Added duplicate detection to the TransactionBroker - so can cope with rollbacks etc.

git-svn-id: https://svn.apache.org/repos/asf/activemq/trunk@552738 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Davies 2007-07-03 08:31:10 +00:00
parent c7469c433c
commit 25a252f348
16 changed files with 253 additions and 184 deletions

View File

@ -250,4 +250,8 @@ public interface Broker extends Region, Service {
* @return the URI that can be used to connect to the local Broker
*/
public URI getVmConnectorURI();
public void brokerServiceStarted();
BrokerService getBrokerService();
}

View File

@ -243,4 +243,11 @@ public class BrokerFilter implements Broker {
return next.getVmConnectorURI();
}
public void brokerServiceStarted(){
next.brokerServiceStarted();
}
public BrokerService getBrokerService(){
return next.getBrokerService();
}
}

View File

@ -153,6 +153,7 @@ public class BrokerService implements Service {
private int persistenceThreadPriority = Thread.MAX_PRIORITY;
private boolean useLocalHostBrokerName = false;
private CountDownLatch stoppedLatch = new CountDownLatch(1);
private boolean supportFailOver = false;
static{
String localHostName = "localhost";
@ -364,7 +365,7 @@ public class BrokerService implements Service {
/**
* @return true if this Broker is a slave to a Master
*/
public boolean isSlave(){
public synchronized boolean isSlave(){
return masterConnector != null && masterConnector.isSlave();
}
@ -436,6 +437,7 @@ public class BrokerService implements Service {
brokerId = broker.getBrokerId();
log.info("ActiveMQ JMS Message Broker (" + getBrokerName()+", "+brokerId+") started");
getBroker().brokerServiceStarted();
}
catch (Exception e) {
log.error("Failed to start ActiveMQ JMS Message Broker. Reason: " + e, e);
@ -486,7 +488,6 @@ public class BrokerService implements Service {
VMTransportFactory.stopped(getBrokerName());
stopped.set(true);
stoppedLatch.countDown();
log.info("ActiveMQ JMS Message Broker ("+getBrokerName()+", "+brokerId+") stopped");
stopper.throwFirstException();
}
@ -1105,6 +1106,20 @@ public class BrokerService implements Service {
}
}
/**
* @return the supportFailOver
*/
public boolean isSupportFailOver(){
return this.supportFailOver;
}
/**
* @param supportFailOver the supportFailOver to set
*/
public void setSupportFailOver(boolean supportFailOver){
this.supportFailOver=supportFailOver;
}
// Implementation methods
// -------------------------------------------------------------------------
/**
@ -1649,6 +1664,7 @@ public class BrokerService implements Service {
}
if (service instanceof MasterConnector) {
masterConnector = (MasterConnector) service;
supportFailOver=true;
}
}
@ -1677,6 +1693,4 @@ public class BrokerService implements Service {
}
}
}

View File

@ -242,4 +242,10 @@ public class EmptyBroker implements Broker {
return null;
}
public void brokerServiceStarted(){
}
public BrokerService getBrokerService(){
return null;
}
}

View File

@ -241,4 +241,11 @@ public class ErrorBroker implements Broker {
throw new BrokerStoppedException(this.message);
}
public void brokerServiceStarted(){
throw new BrokerStoppedException(this.message);
}
public BrokerService getBrokerService(){
throw new BrokerStoppedException(this.message);
}
}

View File

@ -255,4 +255,12 @@ public class MutableBrokerFilter implements Broker {
return getNext().getVmConnectorURI();
}
public void brokerServiceStarted(){
getNext().brokerServiceStarted();
}
public BrokerService getBrokerService(){
return getNext().getBrokerService();
}
}

View File

@ -19,6 +19,7 @@ package org.apache.activemq.broker;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.activemq.ActiveMQMessageAudit;
import org.apache.activemq.command.ConnectionInfo;
import org.apache.activemq.command.LocalTransactionId;
import org.apache.activemq.command.Message;
@ -28,6 +29,7 @@ import org.apache.activemq.command.XATransactionId;
import org.apache.activemq.store.TransactionRecoveryListener;
import org.apache.activemq.store.TransactionStore;
import org.apache.activemq.transaction.LocalTransaction;
import org.apache.activemq.transaction.Synchronization;
import org.apache.activemq.transaction.Transaction;
import org.apache.activemq.transaction.XATransaction;
import org.apache.activemq.util.IOExceptionSupport;
@ -36,6 +38,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import javax.jms.JMSException;
import javax.transaction.xa.XAException;
import java.util.ArrayList;
@ -55,6 +58,7 @@ public class TransactionBroker extends BrokerFilter {
// The prepared XA transactions.
private TransactionStore transactionStore;
private Map xaTransactions = new LinkedHashMap();
ActiveMQMessageAudit audit;
public TransactionBroker(Broker next, TransactionStore transactionStore) {
super(next);
@ -189,21 +193,42 @@ public class TransactionBroker extends BrokerFilter {
}
}
public void send(ProducerBrokerExchange producerExchange, Message message) throws Exception {
public void send(ProducerBrokerExchange producerExchange,final Message message) throws Exception{
// This method may be invoked recursively.
// Track original tx so that it can be restored.
final ConnectionContext context=producerExchange.getConnectionContext();
Transaction originalTx=context.getTransaction();
Transaction transaction=null;
Synchronization sync=null;
if(message.getTransactionId()!=null){
transaction=getTransaction(context,message.getTransactionId(),false);
if(transaction!=null){
sync=new Synchronization(){
public void afterRollback(){
if(audit!=null){
audit.rollbackMessageReference(message);
}
}
};
transaction.addSynchronization(sync);
}
}
if(audit==null||!audit.isDuplicateMessageReference(message)){
context.setTransaction(transaction);
try{
next.send(producerExchange,message);
}finally{
context.setTransaction(originalTx);
}
}else{
if(sync!=null&&transaction!=null){
transaction.removeSynchronization(sync);
}
if(log.isDebugEnabled()){
log.debug("IGNORING duplicate message "+message);
}
}
}
public void removeConnection(ConnectionContext context, ConnectionInfo info, Throwable error) throws Exception {
@ -249,4 +274,11 @@ public class TransactionBroker extends BrokerFilter {
}
}
public synchronized void brokerServiceStarted(){
super.brokerServiceStarted();
if(getBrokerService().isSupportFailOver()&&audit==null){
audit=new ActiveMQMessageAudit();
}
}
}

View File

@ -479,24 +479,7 @@ public class TransportConnection implements Service,Connection,Task,CommandVisit
public Response processMessage(Message messageSend) throws Exception{
ProducerId producerId=messageSend.getProducerId();
ProducerBrokerExchange producerExchange=getProducerBrokerExchange(producerId);
ProducerState producerState = null;
if(messageSend.getMessageId().getProducerId().equals(messageSend.getProducerId())){
producerState=producerExchange.getProducerState();
}
if(producerState!=null){
long seq=messageSend.getMessageId().getProducerSequenceId();
if(seq>producerState.getLastSequenceId()){
producerState.setLastSequenceId(seq);
broker.send(producerExchange,messageSend);
}else {
if (log.isDebugEnabled()) {
log.debug("Discarding duplicate: " + messageSend);
}
}
}else{
// producer not local to this broker
broker.send(producerExchange,messageSend);
}
return null;
}

View File

@ -132,6 +132,7 @@ public class TransportConnector implements Connector {
this.broker = broker;
brokerInfo.setBrokerId(broker.getBrokerId());
brokerInfo.setPeerBrokerInfos(broker.getPeerBrokerInfos());
brokerInfo.setFaultTolerantConfiguration(broker.isFaultTolerantConfiguration());
}
public void setBrokerName(String brokerName) {

View File

@ -288,9 +288,11 @@ public class MasterBroker extends InsertableMutableBrokerFilter{
mdn.setConsumerId(messageDispatch.getConsumerId());
mdn.setDeliverySequenceId(messageDispatch.getDeliverySequenceId());
mdn.setDestination(messageDispatch.getDestination());
if(messageDispatch.getMessage()!=null)
mdn.setMessageId(messageDispatch.getMessage().getMessageId());
if(messageDispatch.getMessage()!=null){
Message msg=messageDispatch.getMessage();
mdn.setMessageId(msg.getMessageId());
sendAsyncToSlave(mdn);
}
super.processDispatch(messageDispatch);
}

View File

@ -64,7 +64,6 @@ public class MasterConnector implements Service, BrokerServiceAware {
private Transport localBroker;
private Transport remoteBroker;
private TransportConnector connector;
private AtomicBoolean masterActive = new AtomicBoolean(false);
private AtomicBoolean started=new AtomicBoolean(false);
private final IdGenerator idGenerator=new IdGenerator();
private String userName;
@ -72,6 +71,7 @@ public class MasterConnector implements Service, BrokerServiceAware {
private ConnectionInfo connectionInfo;
private SessionInfo sessionInfo;
private ProducerInfo producerInfo;
final AtomicBoolean masterActive = new AtomicBoolean();
public MasterConnector(){
}
@ -107,8 +107,8 @@ public class MasterConnector implements Service, BrokerServiceAware {
localBroker=TransportFactory.connect(localURI);
remoteBroker=TransportFactory.connect(remoteURI);
log.info("Starting a network connection between "+localBroker+" and "+remoteBroker+" has been established.");
localBroker.setTransportListener(new DefaultTransportListener(){
public void onCommand(Object command){
}
@ -118,8 +118,8 @@ public class MasterConnector implements Service, BrokerServiceAware {
}
}
});
remoteBroker.setTransportListener(new DefaultTransportListener(){
public void onCommand(Object o){
Command command=(Command)o;
if(started.get()){
@ -133,23 +133,21 @@ public class MasterConnector implements Service, BrokerServiceAware {
}
}
});
masterActive.set(true);
Thread thead=new Thread(){
public void run(){
try{
localBroker.start();
remoteBroker.start();
startBridge();
}
catch (Exception e) {
}catch(Exception e){
masterActive.set(false);
log.error("Failed to start network bridge: "+e,e);
}
}
};
thead.start();
}
protected void startBridge() throws Exception{
@ -163,28 +161,22 @@ public class MasterConnector implements Service, BrokerServiceAware {
connectionInfo.copy(remoteInfo);
remoteInfo.setBrokerMasterConnector(true);
remoteBroker.oneway(connectionInfo);
sessionInfo=new SessionInfo(connectionInfo,1);
localBroker.oneway(sessionInfo);
remoteBroker.oneway(sessionInfo);
producerInfo=new ProducerInfo(sessionInfo,1);
producerInfo.setResponseRequired(false);
remoteBroker.oneway(producerInfo);
BrokerInfo brokerInfo=null;
if(connector!=null){
brokerInfo=connector.getBrokerInfo();
}
else {
}else{
brokerInfo=new BrokerInfo();
}
brokerInfo.setBrokerName(broker.getBrokerName());
brokerInfo.setPeerBrokerInfos(broker.getBroker().getPeerBrokerInfos());
brokerInfo.setSlaveBroker(true);
remoteBroker.oneway(brokerInfo);
log.info("Slave connection between "+localBroker+" and "+remoteBroker+" has been established.");
}
@ -192,7 +184,6 @@ public class MasterConnector implements Service, BrokerServiceAware {
if(!started.compareAndSet(true,false)){
return;
}
masterActive.set(false);
try{
// if (connectionInfo!=null){
@ -202,11 +193,9 @@ public class MasterConnector implements Service, BrokerServiceAware {
// remoteBroker.setTransportListener(null);
remoteBroker.oneway(new ShutdownInfo());
localBroker.oneway(new ShutdownInfo());
}
catch (IOException e) {
}catch(IOException e){
log.debug("Caught exception stopping",e);
}
finally {
}finally{
ServiceStopper ss=new ServiceStopper();
ss.stop(localBroker);
ss.stop(remoteBroker);
@ -215,7 +204,9 @@ public class MasterConnector implements Service, BrokerServiceAware {
}
protected void serviceRemoteException(IOException error){
log.error("Network connection between " + localBroker + " and " + remoteBroker + " shutdown: " + error.getMessage(), error);
log
.error("Network connection between "+localBroker+" and "+remoteBroker+" shutdown: "+error.getMessage(),
error);
shutDown();
}
@ -228,9 +219,7 @@ public class MasterConnector implements Service, BrokerServiceAware {
if(command.getDataStructureType()==CommandTypes.SHUTDOWN_INFO){
log.warn("The Master has shutdown");
shutDown();
}
else {
}else{
boolean responseRequired=command.isResponseRequired();
int commandId=command.getCommandId();
localBroker.oneway(command);
@ -240,8 +229,7 @@ public class MasterConnector implements Service, BrokerServiceAware {
remoteBroker.oneway(response);
}
}
}
catch (IOException e) {
}catch(IOException e){
serviceRemoteException(e);
}
}
@ -316,5 +304,4 @@ public class MasterConnector implements Service, BrokerServiceAware {
broker.masterFailed();
ServiceSupport.dispose(this);
}
}

View File

@ -17,7 +17,6 @@ package org.apache.activemq.broker.region;
import java.io.IOException;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.concurrent.atomic.AtomicBoolean;
import javax.jms.InvalidSelectorException;
import javax.jms.JMSException;
import org.apache.activemq.broker.Broker;
@ -55,7 +54,7 @@ abstract public class PrefetchSubscription extends AbstractSubscription{
protected long enqueueCounter;
protected long dispatchCounter;
protected long dequeueCounter;
private AtomicBoolean dispatching=new AtomicBoolean();
public PrefetchSubscription(Broker broker,ConnectionContext context,ConsumerInfo info,PendingMessageCursor cursor)
throws InvalidSelectorException{
@ -207,7 +206,9 @@ abstract public class PrefetchSubscription extends AbstractSubscription{
}
// this only happens after a reconnect - get an ack which is not valid
if(!callDispatchMatched){
log.info("Could not correlate acknowledgment with dispatched message: "+ack);
if (log.isDebugEnabled()) {
log.debug("Could not correlate acknowledgment with dispatched message: "+ack);
}
}
}else if(ack.isDeliveredAck()){
// Message was delivered but not acknowledged: update pre-fetch counters.
@ -376,8 +377,7 @@ abstract public class PrefetchSubscription extends AbstractSubscription{
}
protected synchronized void dispatchMatched() throws IOException{
if(!broker.isSlaveBroker()&&dispatching.compareAndSet(false,true)){
try{
if(!broker.isSlaveBroker()){
try{
int numberToDispatch=countBeforeFull();
if(numberToDispatch>0){
@ -403,9 +403,6 @@ abstract public class PrefetchSubscription extends AbstractSubscription{
}finally{
pending.release();
}
}finally{
dispatching.set(false);
}
}
}

View File

@ -28,6 +28,7 @@ import java.util.Set;
import javax.jms.InvalidClientIDException;
import javax.jms.JMSException;
import org.apache.activemq.ActiveMQMessageAudit;
import org.apache.activemq.broker.Broker;
import org.apache.activemq.broker.BrokerService;
import org.apache.activemq.broker.Connection;
@ -35,6 +36,7 @@ import org.apache.activemq.broker.ConnectionContext;
import org.apache.activemq.broker.ConsumerBrokerExchange;
import org.apache.activemq.broker.DestinationAlreadyExistsException;
import org.apache.activemq.broker.ProducerBrokerExchange;
import org.apache.activemq.broker.TransactionBroker;
import org.apache.activemq.broker.region.policy.PendingDurableSubscriberMessageStoragePolicy;
import org.apache.activemq.broker.region.policy.PolicyMap;
import org.apache.activemq.broker.region.policy.VMPendingDurableSubscriberMessageStoragePolicy;
@ -63,6 +65,8 @@ import org.apache.activemq.thread.TaskRunnerFactory;
import org.apache.activemq.util.IdGenerator;
import org.apache.activemq.util.LongSequenceGenerator;
import org.apache.activemq.util.ServiceStopper;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CopyOnWriteArrayList;
@ -74,7 +78,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
* @version $Revision$
*/
public class RegionBroker implements Broker {
private static final Log log = LogFactory.getLog(RegionBroker.class);
private static final IdGenerator brokerIdGenerator = new IdGenerator();
private final Region queueRegion;
@ -101,6 +105,7 @@ public class RegionBroker implements Broker {
protected final ConcurrentHashMap connectionStates = new ConcurrentHashMap();
public RegionBroker(BrokerService brokerService,TaskRunnerFactory taskRunnerFactory, UsageManager memoryManager, DestinationFactory destinationFactory, DestinationInterceptor destinationInterceptor) throws IOException {
this.brokerService = brokerService;
if (destinationFactory == null) {
@ -613,4 +618,13 @@ public class RegionBroker implements Broker {
public URI getVmConnectorURI(){
return brokerService.getVmConnectorURI();
}
public void brokerServiceStarted(){
}
public BrokerService getBrokerService(){
return brokerService;
}
}

View File

@ -33,10 +33,5 @@ public class ProducerState {
public ProducerInfo getInfo() {
return info;
}
public void setLastSequenceId(long lastSequenceId) {
this.lastSequenceId = lastSequenceId;
}
public long getLastSequenceId() {
return lastSequenceId;
}
}

View File

@ -56,6 +56,10 @@ public abstract class Transaction {
}
}
public void removeSynchronization(Synchronization r) {
synchronizations.remove(r);
}
public void prePrepare() throws Exception {
// Is it ok to call prepare now given the state of the

View File

@ -236,4 +236,12 @@ public class StubBroker implements Broker {
public URI getVmConnectorURI(){
return null;
}
public void brokerServiceStarted(){
}
public BrokerService getBrokerService(){
return null;
}
}