Fix for NPE in duplex network connection

git-svn-id: https://svn.apache.org/repos/asf/activemq/trunk@583595 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Davies 2007-10-10 19:53:07 +00:00
parent 5ac6f3a890
commit a9119eca19
7 changed files with 459 additions and 120 deletions

View File

@ -0,0 +1,125 @@
/**
* 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.broker;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.activemq.command.ConnectionId;
import org.apache.activemq.command.ConsumerId;
import org.apache.activemq.command.ProducerId;
import org.apache.activemq.command.SessionId;
/**
* @version $Revision: 1.8 $
*/
public class MapTransportConnectionStateRegister implements TransportConnectionStateRegister{
private Map <ConnectionId,TransportConnectionState>connectionStates = new ConcurrentHashMap<ConnectionId,TransportConnectionState>();
public TransportConnectionState registerConnectionState(ConnectionId connectionId,
TransportConnectionState state) {
TransportConnectionState rc = connectionStates.put(connectionId, state);
return rc;
}
public TransportConnectionState unregisterConnectionState(ConnectionId connectionId) {
TransportConnectionState rc = connectionStates.remove(connectionId);
return rc;
}
public List<TransportConnectionState> listConnectionStates() {
List<TransportConnectionState> rc = new ArrayList<TransportConnectionState>();
rc.addAll(connectionStates.values());
return rc;
}
public TransportConnectionState lookupConnectionState(String connectionId) {
return connectionStates.get(new ConnectionId(connectionId));
}
public TransportConnectionState lookupConnectionState(ConsumerId id) {
TransportConnectionState cs = lookupConnectionState(id.getConnectionId());
if (cs == null) {
throw new IllegalStateException(
"Cannot lookup a consumer from a connection that had not been registered: "
+ id.getParentId().getParentId());
}
return cs;
}
public TransportConnectionState lookupConnectionState(ProducerId id) {
TransportConnectionState cs = lookupConnectionState(id.getConnectionId());
if (cs == null) {
throw new IllegalStateException(
"Cannot lookup a producer from a connection that had not been registered: "
+ id.getParentId().getParentId());
}
return cs;
}
public TransportConnectionState lookupConnectionState(SessionId id) {
TransportConnectionState cs = lookupConnectionState(id.getConnectionId());
if (cs == null) {
throw new IllegalStateException(
"Cannot lookup a session from a connection that had not been registered: "
+ id.getParentId());
}
return cs;
}
public TransportConnectionState lookupConnectionState(ConnectionId connectionId) {
TransportConnectionState cs = connectionStates.get(connectionId);
if (cs == null) {
throw new IllegalStateException("Cannot lookup a connection that had not been registered: "
+ connectionId);
}
return cs;
}
public boolean doesHandleMultipleConnectionStates() {
return true;
}
public boolean isEmpty() {
return connectionStates.isEmpty();
}
public void clear() {
connectionStates.clear();
}
public void intialize(TransportConnectionStateRegister other) {
connectionStates.clear();
connectionStates.putAll(other.mapStates());
}
public Map<ConnectionId, TransportConnectionState> mapStates() {
HashMap<ConnectionId, TransportConnectionState> map = new HashMap<ConnectionId, TransportConnectionState>(connectionStates);
return map;
}
}

View File

@ -0,0 +1,155 @@
/**
* 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.broker;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.activemq.command.ConnectionId;
import org.apache.activemq.command.ConsumerId;
import org.apache.activemq.command.ProducerId;
import org.apache.activemq.command.SessionId;
/**
* @version $Revision: 1.8 $
*/
public class SingleTransportConnectionStateRegister implements TransportConnectionStateRegister{
private TransportConnectionState connectionState;
private ConnectionId connectionId;
public TransportConnectionState registerConnectionState(ConnectionId connectionId,
TransportConnectionState state) {
TransportConnectionState rc = connectionState;
connectionState = state;
this.connectionId = connectionId;
return rc;
}
public synchronized TransportConnectionState unregisterConnectionState(ConnectionId connectionId) {
TransportConnectionState rc = null;
if (connectionId != null && connectionState != null && this.connectionId!=null){
if (this.connectionId.equals(connectionId)){
rc = connectionState;
connectionState = null;
connectionId = null;
}
}
return rc;
}
public synchronized List<TransportConnectionState> listConnectionStates() {
List<TransportConnectionState> rc = new ArrayList<TransportConnectionState>();
if (connectionState != null) {
rc.add(connectionState);
}
return rc;
}
public synchronized TransportConnectionState lookupConnectionState(String connectionId) {
TransportConnectionState cs = connectionState;
if (cs == null) {
throw new IllegalStateException(
"Cannot lookup a connectionId for a connection that had not been registered: "
+ connectionId);
}
return cs;
}
public synchronized TransportConnectionState lookupConnectionState(ConsumerId id) {
TransportConnectionState cs = connectionState;
if (cs == null) {
throw new IllegalStateException(
"Cannot lookup a consumer from a connection that had not been registered: "
+ id.getParentId().getParentId());
}
return cs;
}
public synchronized TransportConnectionState lookupConnectionState(ProducerId id) {
TransportConnectionState cs = connectionState;
if (cs == null) {
throw new IllegalStateException(
"Cannot lookup a producer from a connection that had not been registered: "
+ id.getParentId().getParentId());
}
return cs;
}
public synchronized TransportConnectionState lookupConnectionState(SessionId id) {
TransportConnectionState cs = connectionState;
if (cs == null) {
throw new IllegalStateException(
"Cannot lookup a session from a connection that had not been registered: "
+ id.getParentId());
}
return cs;
}
public synchronized TransportConnectionState lookupConnectionState(ConnectionId connectionId) {
TransportConnectionState cs = connectionState;
if (cs == null) {
throw new IllegalStateException("Cannot lookup a connection that had not been registered: "
+ connectionId);
}
return cs;
}
public synchronized boolean doesHandleMultipleConnectionStates() {
return false;
}
public synchronized boolean isEmpty() {
return connectionState == null;
}
public void intialize(TransportConnectionStateRegister other) {
if (other.isEmpty()){
clear();
}else{
Map map = other.mapStates();
Iterator i = map.entrySet().iterator();
Map.Entry<ConnectionId, TransportConnectionState> entry = (Entry<ConnectionId, TransportConnectionState>) i.next();
connectionId = entry.getKey();
connectionState =entry.getValue();
}
}
public Map<ConnectionId, TransportConnectionState> mapStates() {
Map<ConnectionId, TransportConnectionState> map = new HashMap<ConnectionId, TransportConnectionState>();
if (!isEmpty()) {
map.put(connectionId, connectionState);
}
return map;
}
public void clear() {
connectionState=null;
connectionId=null;
}
}

View File

@ -150,45 +150,7 @@ public class TransportConnection implements Service, Connection, Task, CommandVi
private AtomicInteger protocolVersion = new AtomicInteger(CommandTypes.PROTOCOL_VERSION); private AtomicInteger protocolVersion = new AtomicInteger(CommandTypes.PROTOCOL_VERSION);
private DemandForwardingBridge duplexBridge; private DemandForwardingBridge duplexBridge;
private final TaskRunnerFactory taskRunnerFactory; private final TaskRunnerFactory taskRunnerFactory;
private TransportConnectionState connectionState; private TransportConnectionStateRegister connectionStateRegister = new SingleTransportConnectionStateRegister();
static class TransportConnectionState extends org.apache.activemq.state.ConnectionState {
private ConnectionContext context;
private TransportConnection connection;
private final Object connectMutex = new Object();
private AtomicInteger referenceCounter = new AtomicInteger();
public TransportConnectionState(ConnectionInfo info, TransportConnection transportConnection) {
super(info);
connection = transportConnection;
}
public ConnectionContext getContext() {
return context;
}
public TransportConnection getConnection() {
return connection;
}
public void setContext(ConnectionContext context) {
this.context = context;
}
public void setConnection(TransportConnection connection) {
this.connection = connection;
}
public int incrementReference() {
return referenceCounter.incrementAndGet();
}
public int decrementReference() {
return referenceCounter.decrementAndGet();
}
}
/** /**
* @param connector * @param connector
@ -555,7 +517,7 @@ public class TransportConnection implements Service, Connection, Task, CommandVi
SessionState ss = cs.getSessionState(sessionId); SessionState ss = cs.getSessionState(sessionId);
if (ss == null) { if (ss == null) {
throw new IllegalStateException( throw new IllegalStateException(
"Cannot add a consumer to a session that had not been registered: " broker.getBrokerName() + " Cannot add a consumer to a session that had not been registered: "
+ sessionId); + sessionId);
} }
// Avoid replaying dup commands // Avoid replaying dup commands
@ -598,6 +560,7 @@ public class TransportConnection implements Service, Connection, Task, CommandVi
try { try {
cs.addSession(info); cs.addSession(info);
} catch (IllegalStateException e) { } catch (IllegalStateException e) {
e.printStackTrace();
broker.removeSession(cs.getContext(), info); broker.removeSession(cs.getContext(), info);
} }
} }
@ -659,7 +622,7 @@ public class TransportConnection implements Service, Connection, Task, CommandVi
// If there are 2 concurrent connections for the same connection id, // If there are 2 concurrent connections for the same connection id,
// then last one in wins, we need to sync here // then last one in wins, we need to sync here
// to figure out the winner. // to figure out the winner.
synchronized (state.connectMutex) { synchronized (state.getConnectionMutex()) {
if (state.getConnection() != this) { if (state.getConnection() != this) {
LOG.debug("Killing previous stale connection: " + state.getConnection().getRemoteAddress()); LOG.debug("Killing previous stale connection: " + state.getConnection().getRemoteAddress());
state.getConnection().stop(); state.getConnection().stop();
@ -1306,90 +1269,44 @@ public class TransportConnection implements Service, Connection, Task, CommandVi
return null; return null;
} }
// ///////////////////////////////////////////////////////////////// protected synchronized TransportConnectionState registerConnectionState(ConnectionId connectionId,TransportConnectionState state) {
// TransportConnectionState cs = null;
// The following methods handle the logical connection state. It is possible if (!connectionStateRegister.isEmpty() && !connectionStateRegister.doesHandleMultipleConnectionStates()){
// multiple logical connections multiplexed over a single physical //swap implementations
// connection. TransportConnectionStateRegister newRegister = new MapTransportConnectionStateRegister();
// But have not yet exploited the feature from the clients, so for newRegister.intialize(connectionStateRegister);
// performance connectionStateRegister = newRegister;
// reasons (to avoid a hash lookup) this class only keeps track of 1
// logical connection state.
//
// A sub class could override these methods to a full multiple logical
// connection
// support.
//
// /////////////////////////////////////////////////////////////////
protected TransportConnectionState registerConnectionState(ConnectionId connectionId,
TransportConnectionState state) {
TransportConnectionState rc = connectionState;
connectionState = state;
return rc;
}
protected TransportConnectionState unregisterConnectionState(ConnectionId connectionId) {
TransportConnectionState rc = connectionState;
connectionState = null;
return rc;
}
protected List<TransportConnectionState> listConnectionStates() {
List<TransportConnectionState> rc = new ArrayList<TransportConnectionState>();
if (connectionState != null) {
rc.add(connectionState);
}
return rc;
}
protected TransportConnectionState lookupConnectionState(String connectionId) {
TransportConnectionState cs = connectionState;
if (cs == null) {
throw new IllegalStateException(
"Cannot lookup a connectionId for a connection that had not been registered: "
+ connectionId);
} }
cs= connectionStateRegister.registerConnectionState(connectionId, state);
return cs; return cs;
} }
protected TransportConnectionState lookupConnectionState(ConsumerId id) { protected synchronized TransportConnectionState unregisterConnectionState(ConnectionId connectionId) {
TransportConnectionState cs = connectionState; return connectionStateRegister.unregisterConnectionState(connectionId);
if (cs == null) {
throw new IllegalStateException(
"Cannot lookup a consumer from a connection that had not been registered: "
+ id.getParentId().getParentId());
}
return cs;
} }
protected TransportConnectionState lookupConnectionState(ProducerId id) { protected synchronized List<TransportConnectionState> listConnectionStates() {
TransportConnectionState cs = connectionState; return connectionStateRegister.listConnectionStates();
if (cs == null) {
throw new IllegalStateException(
"Cannot lookup a producer from a connection that had not been registered: "
+ id.getParentId().getParentId());
}
return cs;
} }
protected TransportConnectionState lookupConnectionState(SessionId id) { protected synchronized TransportConnectionState lookupConnectionState(String connectionId) {
TransportConnectionState cs = connectionState; return connectionStateRegister.lookupConnectionState(connectionId);
if (cs == null) {
throw new IllegalStateException(
"Cannot lookup a session from a connection that had not been registered: "
+ id.getParentId());
}
return cs;
} }
protected TransportConnectionState lookupConnectionState(ConnectionId connectionId) { protected synchronized TransportConnectionState lookupConnectionState(ConsumerId id) {
TransportConnectionState cs = connectionState; return connectionStateRegister.lookupConnectionState(id);
if (cs == null) {
throw new IllegalStateException("Cannot lookup a connection that had not been registered: "
+ connectionId);
} }
return cs;
protected synchronized TransportConnectionState lookupConnectionState(ProducerId id) {
return connectionStateRegister.lookupConnectionState(id);
}
protected synchronized TransportConnectionState lookupConnectionState(SessionId id) {
return connectionStateRegister.lookupConnectionState(id);
}
protected synchronized TransportConnectionState lookupConnectionState(ConnectionId connectionId) {
return connectionStateRegister.lookupConnectionState(connectionId);
} }
} }

View File

@ -0,0 +1,74 @@
/**
* 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.broker;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.activemq.command.ConnectionInfo;
/**
* @version $Revision: 1.8 $
*/
public class TransportConnectionState extends org.apache.activemq.state.ConnectionState {
private ConnectionContext context;
private TransportConnection connection;
private AtomicInteger referenceCounter = new AtomicInteger();
private final Object connectionMutex = new Object();
public TransportConnectionState(ConnectionInfo info, TransportConnection transportConnection) {
super(info);
connection = transportConnection;
}
public ConnectionContext getContext() {
return context;
}
public TransportConnection getConnection() {
return connection;
}
public void setContext(ConnectionContext context) {
this.context = context;
}
public void setConnection(TransportConnection connection) {
this.connection = connection;
}
public int incrementReference() {
return referenceCounter.incrementAndGet();
}
public int decrementReference() {
return referenceCounter.decrementAndGet();
}
public AtomicInteger getReferenceCounter() {
return referenceCounter;
}
public void setReferenceCounter(AtomicInteger referenceCounter) {
this.referenceCounter = referenceCounter;
}
public Object getConnectionMutex() {
return connectionMutex;
}
}

View File

@ -0,0 +1,60 @@
/**
* 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.broker;
import java.util.List;
import java.util.Map;
import org.apache.activemq.command.ConnectionId;
import org.apache.activemq.command.ConsumerId;
import org.apache.activemq.command.ProducerId;
import org.apache.activemq.command.SessionId;
/**
* @version $Revision: 1.8 $
*/
public interface TransportConnectionStateRegister{
TransportConnectionState registerConnectionState(ConnectionId connectionId,
TransportConnectionState state);
TransportConnectionState unregisterConnectionState(ConnectionId connectionId);
List<TransportConnectionState> listConnectionStates();
Map<ConnectionId,TransportConnectionState>mapStates();
TransportConnectionState lookupConnectionState(String connectionId);
TransportConnectionState lookupConnectionState(ConsumerId id);
TransportConnectionState lookupConnectionState(ProducerId id);
TransportConnectionState lookupConnectionState(SessionId id);
TransportConnectionState lookupConnectionState(ConnectionId connectionId);
boolean isEmpty();
boolean doesHandleMultipleConnectionStates();
void intialize(TransportConnectionStateRegister other);
void clear();
}

View File

@ -100,6 +100,7 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge {
protected final ConcurrentHashMap<ConsumerId, DemandSubscription> subscriptionMapByRemoteId = new ConcurrentHashMap<ConsumerId, DemandSubscription>(); protected final ConcurrentHashMap<ConsumerId, DemandSubscription> subscriptionMapByRemoteId = new ConcurrentHashMap<ConsumerId, DemandSubscription>();
protected final BrokerId localBrokerPath[] = new BrokerId[] {null}; protected final BrokerId localBrokerPath[] = new BrokerId[] {null};
protected CountDownLatch startedLatch = new CountDownLatch(2); protected CountDownLatch startedLatch = new CountDownLatch(2);
protected CountDownLatch localStartedLatch = new CountDownLatch(1);
protected CountDownLatch remoteBrokerNameKnownLatch = new CountDownLatch(1); protected CountDownLatch remoteBrokerNameKnownLatch = new CountDownLatch(1);
protected final AtomicBoolean remoteInterupted = new AtomicBoolean(false); protected final AtomicBoolean remoteInterupted = new AtomicBoolean(false);
protected final AtomicBoolean lastConnectSucceeded = new AtomicBoolean(false); protected final AtomicBoolean lastConnectSucceeded = new AtomicBoolean(false);
@ -172,6 +173,7 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge {
localBridgeStarted.set(false); localBridgeStarted.set(false);
remoteBridgeStarted.set(false); remoteBridgeStarted.set(false);
startedLatch = new CountDownLatch(2); startedLatch = new CountDownLatch(2);
localStartedLatch = new CountDownLatch(1);
} }
} }
@ -261,6 +263,7 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge {
LOG.info("Network connection between " + localBroker + " and " + remoteBroker + "(" + remoteBrokerName + ") has been established."); LOG.info("Network connection between " + localBroker + " and " + remoteBroker + "(" + remoteBrokerName + ") has been established.");
startedLatch.countDown(); startedLatch.countDown();
localStartedLatch.countDown();
setupStaticDestinations(); setupStaticDestinations();
} }
} }
@ -339,6 +342,7 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge {
// stuck waiting for it to start up. // stuck waiting for it to start up.
startedLatch.countDown(); startedLatch.countDown();
startedLatch.countDown(); startedLatch.countDown();
localStartedLatch.countDown();
ss.throwFirstException(); ss.throwFirstException();
} }
} }
@ -406,6 +410,7 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge {
localBroker.oneway(command); localBroker.oneway(command);
break; break;
case ConsumerInfo.DATA_STRUCTURE_TYPE: case ConsumerInfo.DATA_STRUCTURE_TYPE:
localStartedLatch.await();
if (!addConsumerInfo((ConsumerInfo)command)) { if (!addConsumerInfo((ConsumerInfo)command)) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Ignoring ConsumerInfo: " + command); LOG.debug("Ignoring ConsumerInfo: " + command);
@ -430,6 +435,7 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge {
} }
} }
} catch (Throwable e) { } catch (Throwable e) {
e.printStackTrace();
serviceRemoteException(e); serviceRemoteException(e);
} }
} }
@ -554,7 +560,7 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge {
try { try {
if (command.isMessageDispatch()) { if (command.isMessageDispatch()) {
enqueueCounter.incrementAndGet(); enqueueCounter.incrementAndGet();
waitStarted(); //localStartedLatch.await();
final MessageDispatch md = (MessageDispatch)command; final MessageDispatch md = (MessageDispatch)command;
DemandSubscription sub = subscriptionMapByLocalId.get(md.getConsumerId()); DemandSubscription sub = subscriptionMapByLocalId.get(md.getConsumerId());
if (sub != null) { if (sub != null) {
@ -628,7 +634,8 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge {
LOG.warn("Unexpected local command: " + command); LOG.warn("Unexpected local command: " + command);
} }
} }
} catch (Exception e) { } catch (Throwable e) {
e.printStackTrace();
serviceLocalException(e); serviceLocalException(e);
} }
} }

View File

@ -90,18 +90,19 @@ public class SimpleNetworkTest extends TestCase {
} }
} }
public void xtestFiltering() throws Exception { public void testFiltering() throws Exception {
MessageConsumer includedConsumer = remoteSession.createConsumer(included); MessageConsumer includedConsumer = remoteSession.createConsumer(included);
MessageConsumer excludedConsumer = remoteSession.createConsumer(excluded); MessageConsumer excludedConsumer = remoteSession.createConsumer(excluded);
MessageProducer includedProducer = localSession.createProducer(included); MessageProducer includedProducer = localSession.createProducer(included);
MessageProducer excludedProducer = localSession.createProducer(excluded); MessageProducer excludedProducer = localSession.createProducer(excluded);
Thread.sleep(1000); // allow for consumer infos to perculate arround
Thread.sleep(2000);
Message test = localSession.createTextMessage("test"); Message test = localSession.createTextMessage("test");
includedProducer.send(test); includedProducer.send(test);
excludedProducer.send(test); excludedProducer.send(test);
assertNull(excludedConsumer.receive(500)); assertNull(excludedConsumer.receive(1000));
assertNotNull(includedConsumer.receive(500)); assertNotNull(includedConsumer.receive(1000));
} }
public void xtestConduitBridge() throws Exception { public void xtestConduitBridge() throws Exception {