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 * @return the URI that can be used to connect to the local Broker
*/ */
public URI getVmConnectorURI(); public URI getVmConnectorURI();
public void brokerServiceStarted();
BrokerService getBrokerService();
} }

View File

@ -243,4 +243,11 @@ public class BrokerFilter implements Broker {
return next.getVmConnectorURI(); 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 int persistenceThreadPriority = Thread.MAX_PRIORITY;
private boolean useLocalHostBrokerName = false; private boolean useLocalHostBrokerName = false;
private CountDownLatch stoppedLatch = new CountDownLatch(1); private CountDownLatch stoppedLatch = new CountDownLatch(1);
private boolean supportFailOver = false;
static{ static{
String localHostName = "localhost"; String localHostName = "localhost";
@ -364,7 +365,7 @@ public class BrokerService implements Service {
/** /**
* @return true if this Broker is a slave to a Master * @return true if this Broker is a slave to a Master
*/ */
public boolean isSlave(){ public synchronized boolean isSlave(){
return masterConnector != null && masterConnector.isSlave(); return masterConnector != null && masterConnector.isSlave();
} }
@ -436,6 +437,7 @@ public class BrokerService implements Service {
brokerId = broker.getBrokerId(); brokerId = broker.getBrokerId();
log.info("ActiveMQ JMS Message Broker (" + getBrokerName()+", "+brokerId+") started"); log.info("ActiveMQ JMS Message Broker (" + getBrokerName()+", "+brokerId+") started");
getBroker().brokerServiceStarted();
} }
catch (Exception e) { catch (Exception e) {
log.error("Failed to start ActiveMQ JMS Message Broker. Reason: " + e, 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()); VMTransportFactory.stopped(getBrokerName());
stopped.set(true); stopped.set(true);
stoppedLatch.countDown(); stoppedLatch.countDown();
log.info("ActiveMQ JMS Message Broker ("+getBrokerName()+", "+brokerId+") stopped"); log.info("ActiveMQ JMS Message Broker ("+getBrokerName()+", "+brokerId+") stopped");
stopper.throwFirstException(); 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 // Implementation methods
// ------------------------------------------------------------------------- // -------------------------------------------------------------------------
/** /**
@ -1649,6 +1664,7 @@ public class BrokerService implements Service {
} }
if (service instanceof MasterConnector) { if (service instanceof MasterConnector) {
masterConnector = (MasterConnector) service; 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; 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); 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(); 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 java.util.concurrent.ConcurrentHashMap;
import org.apache.activemq.ActiveMQMessageAudit;
import org.apache.activemq.command.ConnectionInfo; import org.apache.activemq.command.ConnectionInfo;
import org.apache.activemq.command.LocalTransactionId; import org.apache.activemq.command.LocalTransactionId;
import org.apache.activemq.command.Message; 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.TransactionRecoveryListener;
import org.apache.activemq.store.TransactionStore; import org.apache.activemq.store.TransactionStore;
import org.apache.activemq.transaction.LocalTransaction; import org.apache.activemq.transaction.LocalTransaction;
import org.apache.activemq.transaction.Synchronization;
import org.apache.activemq.transaction.Transaction; import org.apache.activemq.transaction.Transaction;
import org.apache.activemq.transaction.XATransaction; import org.apache.activemq.transaction.XATransaction;
import org.apache.activemq.util.IOExceptionSupport; import org.apache.activemq.util.IOExceptionSupport;
@ -36,6 +38,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import javax.jms.JMSException; import javax.jms.JMSException;
import javax.transaction.xa.XAException; import javax.transaction.xa.XAException;
import java.util.ArrayList; import java.util.ArrayList;
@ -55,6 +58,7 @@ public class TransactionBroker extends BrokerFilter {
// The prepared XA transactions. // The prepared XA transactions.
private TransactionStore transactionStore; private TransactionStore transactionStore;
private Map xaTransactions = new LinkedHashMap(); private Map xaTransactions = new LinkedHashMap();
ActiveMQMessageAudit audit;
public TransactionBroker(Broker next, TransactionStore transactionStore) { public TransactionBroker(Broker next, TransactionStore transactionStore) {
super(next); 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. // This method may be invoked recursively.
// Track original tx so that it can be restored. // Track original tx so that it can be restored.
final ConnectionContext context=producerExchange.getConnectionContext(); final ConnectionContext context=producerExchange.getConnectionContext();
Transaction originalTx=context.getTransaction(); Transaction originalTx=context.getTransaction();
Transaction transaction=null; Transaction transaction=null;
Synchronization sync=null;
if(message.getTransactionId()!=null){ if(message.getTransactionId()!=null){
transaction=getTransaction(context,message.getTransactionId(),false); 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); context.setTransaction(transaction);
try{ try{
next.send(producerExchange,message); next.send(producerExchange,message);
}finally{ }finally{
context.setTransaction(originalTx); 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 { 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{ public Response processMessage(Message messageSend) throws Exception{
ProducerId producerId=messageSend.getProducerId(); ProducerId producerId=messageSend.getProducerId();
ProducerBrokerExchange producerExchange=getProducerBrokerExchange(producerId); 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); 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; return null;
} }

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -33,10 +33,5 @@ public class ProducerState {
public ProducerInfo getInfo() { public ProducerInfo getInfo() {
return info; 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 { public void prePrepare() throws Exception {
// Is it ok to call prepare now given the state of the // 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(){ public URI getVmConnectorURI(){
return null; return null;
} }
public void brokerServiceStarted(){
}
public BrokerService getBrokerService(){
return null;
}
} }