You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@activemq.apache.org by "Stirling Chow (JIRA)" <ji...@apache.org> on 2010/05/06 00:12:36 UTC

[jira] Created: (AMQ-2723) VM connection leaks during each attempt to create a network bridge to a non-existent broker.

VM connection leaks during each attempt to create a network bridge to a non-existent broker.
--------------------------------------------------------------------------------------------

                 Key: AMQ-2723
                 URL: https://issues.apache.org/activemq/browse/AMQ-2723
             Project: ActiveMQ
          Issue Type: Bug
          Components: Transport
    Affects Versions: 5.3.1
         Environment: ActiveMQ 5.3.1, Windows XP
            Reporter: Stirling Chow
         Attachments: ConnectionLeakTest.java

Symptom
========
We deployed ActiveMQ in a network of brokers using HTTP as the broker-to-broker transport and VM as the inter-broker transport.  Each broker uses a SimpleDiscoveryAgent with a list of HTTP URLs to potential peer brokers -- in many cases this list contains URLs for brokers that are inactive for long periods of time.  We performed a week-long test with three active brokers and 5 inactive brokers.  After one week, the active brokers began reporting OutOfMemory exceptions related to exhaused heap space (384MB max) and they stopped functioning.

The generated heap dump revealed 100K+ instances of DurableConduitBridge and related anonymous classes in DemandForwardingBridgeSupport.  Our expectation was that since there are only three active brokers, there should have been at most three instances of DurableConduitBridge.  It appeared that each attempt to create a bridge to a non-existent broker was resulting in leaking instances of DurableConduitBridge et al.

Unit Test
=======
A JUnit test is included with this ticket to demonstrate the issue.

Cause
=====
The leaking references to DirectConduitBridge et al. were due to the accumulation of VMTransport connections in TransportConnector#connections.  It seemed that each failed attempt to create a network bridge was resulting in an instance being added to TransportConnector#connections that was never being removed.  Here's the reason...

Each time a broker attempts to create a network bridge to another broker, a call is made to DiscoveryNetworkConnector::onServiceAdd(DiscoveryEvent) by SimpleDiscoveryAgent.  The broker initiating the connection creates a local and remote transport and then attempts to create a bridge between them:

   remoteTransport = TransportFactory.connect(connectUri);
...
   localTransport = createLocalTransport();
...
   try {
      bridge.start();
      ...
    } catch (Exception e) {
      ServiceSupport.dispose(localTransport);
      ServiceSupport.dispose(remoteTransport);
      ...
   }

If the remote broker does not exist (as is the case with our environment), bridge.start() throws an exception which triggers the disposal of the local and remote transports.

The localTransport is an instance of VMTransport, and its disposal will eventually call VMTransport#stop():

    public void stop() throws Exception {
...
                enqueueValve.turnOff();
                if (!disposed) {
                    started = false;
                    disposed = true;
...
                }
            } finally {
                stopping.set(false);
                enqueueValve.turnOn();
            }
...
            // let the peer know that we are disconnecting..
            try {
                oneway(DISCONNECT);
            } catch (Exception ignore) {
            }
        }
    }

The DISCONNECT should get processed by the VMTransport#iterate() on the peer side:

    public boolean iterate() {
...
            if( command == DISCONNECT ) {
                tl.onException(new TransportDisposedIOException("Peer (" + peer.toString() + ") disposed."));
            } else {
...

tl is a reference to the TransportListener implemented by TransportConnection  and should result in a call to TransportConnection#doStop():

    protected void doStop() throws Exception, InterruptedException {
...
        connector.onStopped(this);

The call to connector.onStopped(this) is implemented by TransportConnector#onStopped(TransportConnection):

    public void onStopped(TransportConnection connection) {
        connections.remove(connection);
    }

This removes the connection represented by the local side of the bridge from the connections array.

*** HOWEVER *** in VMTransport#stop(), the disposed flag is set to true before the call to oneway(DISCONNECT); this causes the oneway(DISCONNECT) to fail  because of this code in VMTransport#oneway(Object):

    public void oneway(Object command) throws IOException {
        if (disposed) {
            throw new TransportDisposedIOException("Transport disposed.");
        }
...

In other words, the DISCONNECT never makes it to the peer and so is never processed by TransportConnection (as TransportListener).

Solution
=======
The solution is to send the DISCONNECT before setting the disposed flag to true.  However, care must be taken to prevent deadlock since VMTransport#stop() acquires a lock on enqueueValve and VMTransport#oneway(Object) acquires locks on the peer.enqueueValve --- if both peers try to stop concurrently, they may deadlock on the acquisition of VMTransport#enqueueValve.  To prevent this deadlock, it is necessary to send the DISCONNECT before the "local" enqueueValve is acquired --- this may mean sending the DISCONNECT unnecessarily (i.e., even if the client is already disposed), but this is not a problem since the resulting exception is ignored.



-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Resolved: (AMQ-2723) VM connection leaks during each attempt to create a network bridge to a non-existent broker.

Posted by "Dejan Bosanac (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/activemq/browse/AMQ-2723?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Dejan Bosanac resolved AMQ-2723.
--------------------------------

         Assignee: Dejan Bosanac
    Fix Version/s: 5.4.0
       Resolution: Duplicate

Thanks for verifying. Resolving the issue now.

> VM connection leaks during each attempt to create a network bridge to a non-existent broker.
> --------------------------------------------------------------------------------------------
>
>                 Key: AMQ-2723
>                 URL: https://issues.apache.org/activemq/browse/AMQ-2723
>             Project: ActiveMQ
>          Issue Type: Bug
>          Components: Transport
>    Affects Versions: 5.3.1
>         Environment: ActiveMQ 5.3.1, Windows XP
>            Reporter: Stirling Chow
>            Assignee: Dejan Bosanac
>             Fix For: 5.4.0
>
>         Attachments: ConnectionLeakTest.java, VMTransport.java, VMTransport.patch
>
>
> Symptom
> ========
> We deployed ActiveMQ in a network of brokers using HTTP as the broker-to-broker transport and VM as the inter-broker transport.  Each broker uses a SimpleDiscoveryAgent with a list of HTTP URLs to potential peer brokers -- in many cases this list contains URLs for brokers that are inactive for long periods of time.  We performed a week-long test with three active brokers and 5 inactive brokers.  After one week, the active brokers began reporting OutOfMemory exceptions related to exhaused heap space (384MB max) and they stopped functioning.
> The generated heap dump revealed 100K+ instances of DurableConduitBridge and related anonymous classes in DemandForwardingBridgeSupport.  Our expectation was that since there are only three active brokers, there should have been at most three instances of DurableConduitBridge.  It appeared that each attempt to create a bridge to a non-existent broker was resulting in leaking instances of DurableConduitBridge et al.
> Unit Test
> =======
> A JUnit test is included with this ticket to demonstrate the issue.
> Cause
> =====
> The leaking references to DirectConduitBridge et al. were due to the accumulation of VMTransport connections in TransportConnector#connections.  It seemed that each failed attempt to create a network bridge was resulting in an instance being added to TransportConnector#connections that was never being removed.  Here's the reason...
> Each time a broker attempts to create a network bridge to another broker, a call is made to DiscoveryNetworkConnector::onServiceAdd(DiscoveryEvent) by SimpleDiscoveryAgent.  The broker initiating the connection creates a local and remote transport and then attempts to create a bridge between them:
>    remoteTransport = TransportFactory.connect(connectUri);
> ...
>    localTransport = createLocalTransport();
> ...
>    try {
>       bridge.start();
>       ...
>     } catch (Exception e) {
>       ServiceSupport.dispose(localTransport);
>       ServiceSupport.dispose(remoteTransport);
>       ...
>    }
> If the remote broker does not exist (as is the case with our environment), bridge.start() throws an exception which triggers the disposal of the local and remote transports.
> The localTransport is an instance of VMTransport, and its disposal will eventually call VMTransport#stop():
>     public void stop() throws Exception {
> ...
>                 enqueueValve.turnOff();
>                 if (!disposed) {
>                     started = false;
>                     disposed = true;
> ...
>                 }
>             } finally {
>                 stopping.set(false);
>                 enqueueValve.turnOn();
>             }
> ...
>             // let the peer know that we are disconnecting..
>             try {
>                 oneway(DISCONNECT);
>             } catch (Exception ignore) {
>             }
>         }
>     }
> The DISCONNECT should get processed by the VMTransport#iterate() on the peer side:
>     public boolean iterate() {
> ...
>             if( command == DISCONNECT ) {
>                 tl.onException(new TransportDisposedIOException("Peer (" + peer.toString() + ") disposed."));
>             } else {
> ...
> tl is a reference to the TransportListener implemented by TransportConnection  and should result in a call to TransportConnection#doStop():
>     protected void doStop() throws Exception, InterruptedException {
> ...
>         connector.onStopped(this);
> The call to connector.onStopped(this) is implemented by TransportConnector#onStopped(TransportConnection):
>     public void onStopped(TransportConnection connection) {
>         connections.remove(connection);
>     }
> This removes the connection represented by the local side of the bridge from the connections array.
> *** HOWEVER *** in VMTransport#stop(), the disposed flag is set to true before the call to oneway(DISCONNECT); this causes the oneway(DISCONNECT) to fail  because of this code in VMTransport#oneway(Object):
>     public void oneway(Object command) throws IOException {
>         if (disposed) {
>             throw new TransportDisposedIOException("Transport disposed.");
>         }
> ...
> In other words, the DISCONNECT never makes it to the peer and so is never processed by TransportConnection (as TransportListener).
> Solution
> =======
> The solution is to send the DISCONNECT before setting the disposed flag to true.  However, care must be taken to prevent deadlock since VMTransport#stop() acquires a lock on enqueueValve and VMTransport#oneway(Object) acquires locks on the peer.enqueueValve --- if both peers try to stop concurrently, they may deadlock on the acquisition of VMTransport#enqueueValve.  To prevent this deadlock, it is necessary to send the DISCONNECT before the "local" enqueueValve is acquired --- this may mean sending the DISCONNECT unnecessarily (i.e., even if the client is already disposed), but this is not a problem since the resulting exception is ignored.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (AMQ-2723) VM connection leaks during each attempt to create a network bridge to a non-existent broker.

Posted by "Dejan Bosanac (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/activemq/browse/AMQ-2723?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=59214#action_59214 ] 

Dejan Bosanac commented on AMQ-2723:
------------------------------------

Hi,

this looks like a duplicate of https://issues.apache.org/activemq/browse/AMQ-2448

Can you test the latest SNAPSHOT and confirm if the bug is still there?

> VM connection leaks during each attempt to create a network bridge to a non-existent broker.
> --------------------------------------------------------------------------------------------
>
>                 Key: AMQ-2723
>                 URL: https://issues.apache.org/activemq/browse/AMQ-2723
>             Project: ActiveMQ
>          Issue Type: Bug
>          Components: Transport
>    Affects Versions: 5.3.1
>         Environment: ActiveMQ 5.3.1, Windows XP
>            Reporter: Stirling Chow
>         Attachments: ConnectionLeakTest.java, VMTransport.java, VMTransport.patch
>
>
> Symptom
> ========
> We deployed ActiveMQ in a network of brokers using HTTP as the broker-to-broker transport and VM as the inter-broker transport.  Each broker uses a SimpleDiscoveryAgent with a list of HTTP URLs to potential peer brokers -- in many cases this list contains URLs for brokers that are inactive for long periods of time.  We performed a week-long test with three active brokers and 5 inactive brokers.  After one week, the active brokers began reporting OutOfMemory exceptions related to exhaused heap space (384MB max) and they stopped functioning.
> The generated heap dump revealed 100K+ instances of DurableConduitBridge and related anonymous classes in DemandForwardingBridgeSupport.  Our expectation was that since there are only three active brokers, there should have been at most three instances of DurableConduitBridge.  It appeared that each attempt to create a bridge to a non-existent broker was resulting in leaking instances of DurableConduitBridge et al.
> Unit Test
> =======
> A JUnit test is included with this ticket to demonstrate the issue.
> Cause
> =====
> The leaking references to DirectConduitBridge et al. were due to the accumulation of VMTransport connections in TransportConnector#connections.  It seemed that each failed attempt to create a network bridge was resulting in an instance being added to TransportConnector#connections that was never being removed.  Here's the reason...
> Each time a broker attempts to create a network bridge to another broker, a call is made to DiscoveryNetworkConnector::onServiceAdd(DiscoveryEvent) by SimpleDiscoveryAgent.  The broker initiating the connection creates a local and remote transport and then attempts to create a bridge between them:
>    remoteTransport = TransportFactory.connect(connectUri);
> ...
>    localTransport = createLocalTransport();
> ...
>    try {
>       bridge.start();
>       ...
>     } catch (Exception e) {
>       ServiceSupport.dispose(localTransport);
>       ServiceSupport.dispose(remoteTransport);
>       ...
>    }
> If the remote broker does not exist (as is the case with our environment), bridge.start() throws an exception which triggers the disposal of the local and remote transports.
> The localTransport is an instance of VMTransport, and its disposal will eventually call VMTransport#stop():
>     public void stop() throws Exception {
> ...
>                 enqueueValve.turnOff();
>                 if (!disposed) {
>                     started = false;
>                     disposed = true;
> ...
>                 }
>             } finally {
>                 stopping.set(false);
>                 enqueueValve.turnOn();
>             }
> ...
>             // let the peer know that we are disconnecting..
>             try {
>                 oneway(DISCONNECT);
>             } catch (Exception ignore) {
>             }
>         }
>     }
> The DISCONNECT should get processed by the VMTransport#iterate() on the peer side:
>     public boolean iterate() {
> ...
>             if( command == DISCONNECT ) {
>                 tl.onException(new TransportDisposedIOException("Peer (" + peer.toString() + ") disposed."));
>             } else {
> ...
> tl is a reference to the TransportListener implemented by TransportConnection  and should result in a call to TransportConnection#doStop():
>     protected void doStop() throws Exception, InterruptedException {
> ...
>         connector.onStopped(this);
> The call to connector.onStopped(this) is implemented by TransportConnector#onStopped(TransportConnection):
>     public void onStopped(TransportConnection connection) {
>         connections.remove(connection);
>     }
> This removes the connection represented by the local side of the bridge from the connections array.
> *** HOWEVER *** in VMTransport#stop(), the disposed flag is set to true before the call to oneway(DISCONNECT); this causes the oneway(DISCONNECT) to fail  because of this code in VMTransport#oneway(Object):
>     public void oneway(Object command) throws IOException {
>         if (disposed) {
>             throw new TransportDisposedIOException("Transport disposed.");
>         }
> ...
> In other words, the DISCONNECT never makes it to the peer and so is never processed by TransportConnection (as TransportListener).
> Solution
> =======
> The solution is to send the DISCONNECT before setting the disposed flag to true.  However, care must be taken to prevent deadlock since VMTransport#stop() acquires a lock on enqueueValve and VMTransport#oneway(Object) acquires locks on the peer.enqueueValve --- if both peers try to stop concurrently, they may deadlock on the acquisition of VMTransport#enqueueValve.  To prevent this deadlock, it is necessary to send the DISCONNECT before the "local" enqueueValve is acquired --- this may mean sending the DISCONNECT unnecessarily (i.e., even if the client is already disposed), but this is not a problem since the resulting exception is ignored.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (AMQ-2723) VM connection leaks during each attempt to create a network bridge to a non-existent broker.

Posted by "Stirling Chow (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/activemq/browse/AMQ-2723?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stirling Chow updated AMQ-2723:
-------------------------------

    Attachment: VMTransport.patch

Patch to address issue.

> VM connection leaks during each attempt to create a network bridge to a non-existent broker.
> --------------------------------------------------------------------------------------------
>
>                 Key: AMQ-2723
>                 URL: https://issues.apache.org/activemq/browse/AMQ-2723
>             Project: ActiveMQ
>          Issue Type: Bug
>          Components: Transport
>    Affects Versions: 5.3.1
>         Environment: ActiveMQ 5.3.1, Windows XP
>            Reporter: Stirling Chow
>         Attachments: ConnectionLeakTest.java, VMTransport.patch
>
>
> Symptom
> ========
> We deployed ActiveMQ in a network of brokers using HTTP as the broker-to-broker transport and VM as the inter-broker transport.  Each broker uses a SimpleDiscoveryAgent with a list of HTTP URLs to potential peer brokers -- in many cases this list contains URLs for brokers that are inactive for long periods of time.  We performed a week-long test with three active brokers and 5 inactive brokers.  After one week, the active brokers began reporting OutOfMemory exceptions related to exhaused heap space (384MB max) and they stopped functioning.
> The generated heap dump revealed 100K+ instances of DurableConduitBridge and related anonymous classes in DemandForwardingBridgeSupport.  Our expectation was that since there are only three active brokers, there should have been at most three instances of DurableConduitBridge.  It appeared that each attempt to create a bridge to a non-existent broker was resulting in leaking instances of DurableConduitBridge et al.
> Unit Test
> =======
> A JUnit test is included with this ticket to demonstrate the issue.
> Cause
> =====
> The leaking references to DirectConduitBridge et al. were due to the accumulation of VMTransport connections in TransportConnector#connections.  It seemed that each failed attempt to create a network bridge was resulting in an instance being added to TransportConnector#connections that was never being removed.  Here's the reason...
> Each time a broker attempts to create a network bridge to another broker, a call is made to DiscoveryNetworkConnector::onServiceAdd(DiscoveryEvent) by SimpleDiscoveryAgent.  The broker initiating the connection creates a local and remote transport and then attempts to create a bridge between them:
>    remoteTransport = TransportFactory.connect(connectUri);
> ...
>    localTransport = createLocalTransport();
> ...
>    try {
>       bridge.start();
>       ...
>     } catch (Exception e) {
>       ServiceSupport.dispose(localTransport);
>       ServiceSupport.dispose(remoteTransport);
>       ...
>    }
> If the remote broker does not exist (as is the case with our environment), bridge.start() throws an exception which triggers the disposal of the local and remote transports.
> The localTransport is an instance of VMTransport, and its disposal will eventually call VMTransport#stop():
>     public void stop() throws Exception {
> ...
>                 enqueueValve.turnOff();
>                 if (!disposed) {
>                     started = false;
>                     disposed = true;
> ...
>                 }
>             } finally {
>                 stopping.set(false);
>                 enqueueValve.turnOn();
>             }
> ...
>             // let the peer know that we are disconnecting..
>             try {
>                 oneway(DISCONNECT);
>             } catch (Exception ignore) {
>             }
>         }
>     }
> The DISCONNECT should get processed by the VMTransport#iterate() on the peer side:
>     public boolean iterate() {
> ...
>             if( command == DISCONNECT ) {
>                 tl.onException(new TransportDisposedIOException("Peer (" + peer.toString() + ") disposed."));
>             } else {
> ...
> tl is a reference to the TransportListener implemented by TransportConnection  and should result in a call to TransportConnection#doStop():
>     protected void doStop() throws Exception, InterruptedException {
> ...
>         connector.onStopped(this);
> The call to connector.onStopped(this) is implemented by TransportConnector#onStopped(TransportConnection):
>     public void onStopped(TransportConnection connection) {
>         connections.remove(connection);
>     }
> This removes the connection represented by the local side of the bridge from the connections array.
> *** HOWEVER *** in VMTransport#stop(), the disposed flag is set to true before the call to oneway(DISCONNECT); this causes the oneway(DISCONNECT) to fail  because of this code in VMTransport#oneway(Object):
>     public void oneway(Object command) throws IOException {
>         if (disposed) {
>             throw new TransportDisposedIOException("Transport disposed.");
>         }
> ...
> In other words, the DISCONNECT never makes it to the peer and so is never processed by TransportConnection (as TransportListener).
> Solution
> =======
> The solution is to send the DISCONNECT before setting the disposed flag to true.  However, care must be taken to prevent deadlock since VMTransport#stop() acquires a lock on enqueueValve and VMTransport#oneway(Object) acquires locks on the peer.enqueueValve --- if both peers try to stop concurrently, they may deadlock on the acquisition of VMTransport#enqueueValve.  To prevent this deadlock, it is necessary to send the DISCONNECT before the "local" enqueueValve is acquired --- this may mean sending the DISCONNECT unnecessarily (i.e., even if the client is already disposed), but this is not a problem since the resulting exception is ignored.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (AMQ-2723) VM connection leaks during each attempt to create a network bridge to a non-existent broker.

Posted by "Stirling Chow (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/activemq/browse/AMQ-2723?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stirling Chow updated AMQ-2723:
-------------------------------

    Attachment: ConnectionLeakTest.java

Unit test demonstrating the leaking of VMTransport connections.

> VM connection leaks during each attempt to create a network bridge to a non-existent broker.
> --------------------------------------------------------------------------------------------
>
>                 Key: AMQ-2723
>                 URL: https://issues.apache.org/activemq/browse/AMQ-2723
>             Project: ActiveMQ
>          Issue Type: Bug
>          Components: Transport
>    Affects Versions: 5.3.1
>         Environment: ActiveMQ 5.3.1, Windows XP
>            Reporter: Stirling Chow
>         Attachments: ConnectionLeakTest.java
>
>
> Symptom
> ========
> We deployed ActiveMQ in a network of brokers using HTTP as the broker-to-broker transport and VM as the inter-broker transport.  Each broker uses a SimpleDiscoveryAgent with a list of HTTP URLs to potential peer brokers -- in many cases this list contains URLs for brokers that are inactive for long periods of time.  We performed a week-long test with three active brokers and 5 inactive brokers.  After one week, the active brokers began reporting OutOfMemory exceptions related to exhaused heap space (384MB max) and they stopped functioning.
> The generated heap dump revealed 100K+ instances of DurableConduitBridge and related anonymous classes in DemandForwardingBridgeSupport.  Our expectation was that since there are only three active brokers, there should have been at most three instances of DurableConduitBridge.  It appeared that each attempt to create a bridge to a non-existent broker was resulting in leaking instances of DurableConduitBridge et al.
> Unit Test
> =======
> A JUnit test is included with this ticket to demonstrate the issue.
> Cause
> =====
> The leaking references to DirectConduitBridge et al. were due to the accumulation of VMTransport connections in TransportConnector#connections.  It seemed that each failed attempt to create a network bridge was resulting in an instance being added to TransportConnector#connections that was never being removed.  Here's the reason...
> Each time a broker attempts to create a network bridge to another broker, a call is made to DiscoveryNetworkConnector::onServiceAdd(DiscoveryEvent) by SimpleDiscoveryAgent.  The broker initiating the connection creates a local and remote transport and then attempts to create a bridge between them:
>    remoteTransport = TransportFactory.connect(connectUri);
> ...
>    localTransport = createLocalTransport();
> ...
>    try {
>       bridge.start();
>       ...
>     } catch (Exception e) {
>       ServiceSupport.dispose(localTransport);
>       ServiceSupport.dispose(remoteTransport);
>       ...
>    }
> If the remote broker does not exist (as is the case with our environment), bridge.start() throws an exception which triggers the disposal of the local and remote transports.
> The localTransport is an instance of VMTransport, and its disposal will eventually call VMTransport#stop():
>     public void stop() throws Exception {
> ...
>                 enqueueValve.turnOff();
>                 if (!disposed) {
>                     started = false;
>                     disposed = true;
> ...
>                 }
>             } finally {
>                 stopping.set(false);
>                 enqueueValve.turnOn();
>             }
> ...
>             // let the peer know that we are disconnecting..
>             try {
>                 oneway(DISCONNECT);
>             } catch (Exception ignore) {
>             }
>         }
>     }
> The DISCONNECT should get processed by the VMTransport#iterate() on the peer side:
>     public boolean iterate() {
> ...
>             if( command == DISCONNECT ) {
>                 tl.onException(new TransportDisposedIOException("Peer (" + peer.toString() + ") disposed."));
>             } else {
> ...
> tl is a reference to the TransportListener implemented by TransportConnection  and should result in a call to TransportConnection#doStop():
>     protected void doStop() throws Exception, InterruptedException {
> ...
>         connector.onStopped(this);
> The call to connector.onStopped(this) is implemented by TransportConnector#onStopped(TransportConnection):
>     public void onStopped(TransportConnection connection) {
>         connections.remove(connection);
>     }
> This removes the connection represented by the local side of the bridge from the connections array.
> *** HOWEVER *** in VMTransport#stop(), the disposed flag is set to true before the call to oneway(DISCONNECT); this causes the oneway(DISCONNECT) to fail  because of this code in VMTransport#oneway(Object):
>     public void oneway(Object command) throws IOException {
>         if (disposed) {
>             throw new TransportDisposedIOException("Transport disposed.");
>         }
> ...
> In other words, the DISCONNECT never makes it to the peer and so is never processed by TransportConnection (as TransportListener).
> Solution
> =======
> The solution is to send the DISCONNECT before setting the disposed flag to true.  However, care must be taken to prevent deadlock since VMTransport#stop() acquires a lock on enqueueValve and VMTransport#oneway(Object) acquires locks on the peer.enqueueValve --- if both peers try to stop concurrently, they may deadlock on the acquisition of VMTransport#enqueueValve.  To prevent this deadlock, it is necessary to send the DISCONNECT before the "local" enqueueValve is acquired --- this may mean sending the DISCONNECT unnecessarily (i.e., even if the client is already disposed), but this is not a problem since the resulting exception is ignored.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (AMQ-2723) VM connection leaks during each attempt to create a network bridge to a non-existent broker.

Posted by "Stirling Chow (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/activemq/browse/AMQ-2723?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stirling Chow updated AMQ-2723:
-------------------------------

    Attachment: VMTransport.java

Patched VMTransport.java file (i.e., AMQ 5.3.1 version with VMTransport.patch applied).

> VM connection leaks during each attempt to create a network bridge to a non-existent broker.
> --------------------------------------------------------------------------------------------
>
>                 Key: AMQ-2723
>                 URL: https://issues.apache.org/activemq/browse/AMQ-2723
>             Project: ActiveMQ
>          Issue Type: Bug
>          Components: Transport
>    Affects Versions: 5.3.1
>         Environment: ActiveMQ 5.3.1, Windows XP
>            Reporter: Stirling Chow
>         Attachments: ConnectionLeakTest.java, VMTransport.java, VMTransport.patch
>
>
> Symptom
> ========
> We deployed ActiveMQ in a network of brokers using HTTP as the broker-to-broker transport and VM as the inter-broker transport.  Each broker uses a SimpleDiscoveryAgent with a list of HTTP URLs to potential peer brokers -- in many cases this list contains URLs for brokers that are inactive for long periods of time.  We performed a week-long test with three active brokers and 5 inactive brokers.  After one week, the active brokers began reporting OutOfMemory exceptions related to exhaused heap space (384MB max) and they stopped functioning.
> The generated heap dump revealed 100K+ instances of DurableConduitBridge and related anonymous classes in DemandForwardingBridgeSupport.  Our expectation was that since there are only three active brokers, there should have been at most three instances of DurableConduitBridge.  It appeared that each attempt to create a bridge to a non-existent broker was resulting in leaking instances of DurableConduitBridge et al.
> Unit Test
> =======
> A JUnit test is included with this ticket to demonstrate the issue.
> Cause
> =====
> The leaking references to DirectConduitBridge et al. were due to the accumulation of VMTransport connections in TransportConnector#connections.  It seemed that each failed attempt to create a network bridge was resulting in an instance being added to TransportConnector#connections that was never being removed.  Here's the reason...
> Each time a broker attempts to create a network bridge to another broker, a call is made to DiscoveryNetworkConnector::onServiceAdd(DiscoveryEvent) by SimpleDiscoveryAgent.  The broker initiating the connection creates a local and remote transport and then attempts to create a bridge between them:
>    remoteTransport = TransportFactory.connect(connectUri);
> ...
>    localTransport = createLocalTransport();
> ...
>    try {
>       bridge.start();
>       ...
>     } catch (Exception e) {
>       ServiceSupport.dispose(localTransport);
>       ServiceSupport.dispose(remoteTransport);
>       ...
>    }
> If the remote broker does not exist (as is the case with our environment), bridge.start() throws an exception which triggers the disposal of the local and remote transports.
> The localTransport is an instance of VMTransport, and its disposal will eventually call VMTransport#stop():
>     public void stop() throws Exception {
> ...
>                 enqueueValve.turnOff();
>                 if (!disposed) {
>                     started = false;
>                     disposed = true;
> ...
>                 }
>             } finally {
>                 stopping.set(false);
>                 enqueueValve.turnOn();
>             }
> ...
>             // let the peer know that we are disconnecting..
>             try {
>                 oneway(DISCONNECT);
>             } catch (Exception ignore) {
>             }
>         }
>     }
> The DISCONNECT should get processed by the VMTransport#iterate() on the peer side:
>     public boolean iterate() {
> ...
>             if( command == DISCONNECT ) {
>                 tl.onException(new TransportDisposedIOException("Peer (" + peer.toString() + ") disposed."));
>             } else {
> ...
> tl is a reference to the TransportListener implemented by TransportConnection  and should result in a call to TransportConnection#doStop():
>     protected void doStop() throws Exception, InterruptedException {
> ...
>         connector.onStopped(this);
> The call to connector.onStopped(this) is implemented by TransportConnector#onStopped(TransportConnection):
>     public void onStopped(TransportConnection connection) {
>         connections.remove(connection);
>     }
> This removes the connection represented by the local side of the bridge from the connections array.
> *** HOWEVER *** in VMTransport#stop(), the disposed flag is set to true before the call to oneway(DISCONNECT); this causes the oneway(DISCONNECT) to fail  because of this code in VMTransport#oneway(Object):
>     public void oneway(Object command) throws IOException {
>         if (disposed) {
>             throw new TransportDisposedIOException("Transport disposed.");
>         }
> ...
> In other words, the DISCONNECT never makes it to the peer and so is never processed by TransportConnection (as TransportListener).
> Solution
> =======
> The solution is to send the DISCONNECT before setting the disposed flag to true.  However, care must be taken to prevent deadlock since VMTransport#stop() acquires a lock on enqueueValve and VMTransport#oneway(Object) acquires locks on the peer.enqueueValve --- if both peers try to stop concurrently, they may deadlock on the acquisition of VMTransport#enqueueValve.  To prevent this deadlock, it is necessary to send the DISCONNECT before the "local" enqueueValve is acquired --- this may mean sending the DISCONNECT unnecessarily (i.e., even if the client is already disposed), but this is not a problem since the resulting exception is ignored.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (AMQ-2723) VM connection leaks during each attempt to create a network bridge to a non-existent broker.

Posted by "Stirling Chow (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/activemq/browse/AMQ-2723?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=59236#action_59236 ] 

Stirling Chow commented on AMQ-2723:
------------------------------------

Indeed, this issue has been fixed by r932403 on the trunk.

I had to modify the mock object in the unit test to accept twice the number of serviceFailed calls (inconsequential) and it's now passing on the May 5 instance of apache-activemq-5.4-SNAPSHOT.

I compared the change checked in to VMTransport.java with the patch and they're virtually the same, so all is good.

> VM connection leaks during each attempt to create a network bridge to a non-existent broker.
> --------------------------------------------------------------------------------------------
>
>                 Key: AMQ-2723
>                 URL: https://issues.apache.org/activemq/browse/AMQ-2723
>             Project: ActiveMQ
>          Issue Type: Bug
>          Components: Transport
>    Affects Versions: 5.3.1
>         Environment: ActiveMQ 5.3.1, Windows XP
>            Reporter: Stirling Chow
>         Attachments: ConnectionLeakTest.java, VMTransport.java, VMTransport.patch
>
>
> Symptom
> ========
> We deployed ActiveMQ in a network of brokers using HTTP as the broker-to-broker transport and VM as the inter-broker transport.  Each broker uses a SimpleDiscoveryAgent with a list of HTTP URLs to potential peer brokers -- in many cases this list contains URLs for brokers that are inactive for long periods of time.  We performed a week-long test with three active brokers and 5 inactive brokers.  After one week, the active brokers began reporting OutOfMemory exceptions related to exhaused heap space (384MB max) and they stopped functioning.
> The generated heap dump revealed 100K+ instances of DurableConduitBridge and related anonymous classes in DemandForwardingBridgeSupport.  Our expectation was that since there are only three active brokers, there should have been at most three instances of DurableConduitBridge.  It appeared that each attempt to create a bridge to a non-existent broker was resulting in leaking instances of DurableConduitBridge et al.
> Unit Test
> =======
> A JUnit test is included with this ticket to demonstrate the issue.
> Cause
> =====
> The leaking references to DirectConduitBridge et al. were due to the accumulation of VMTransport connections in TransportConnector#connections.  It seemed that each failed attempt to create a network bridge was resulting in an instance being added to TransportConnector#connections that was never being removed.  Here's the reason...
> Each time a broker attempts to create a network bridge to another broker, a call is made to DiscoveryNetworkConnector::onServiceAdd(DiscoveryEvent) by SimpleDiscoveryAgent.  The broker initiating the connection creates a local and remote transport and then attempts to create a bridge between them:
>    remoteTransport = TransportFactory.connect(connectUri);
> ...
>    localTransport = createLocalTransport();
> ...
>    try {
>       bridge.start();
>       ...
>     } catch (Exception e) {
>       ServiceSupport.dispose(localTransport);
>       ServiceSupport.dispose(remoteTransport);
>       ...
>    }
> If the remote broker does not exist (as is the case with our environment), bridge.start() throws an exception which triggers the disposal of the local and remote transports.
> The localTransport is an instance of VMTransport, and its disposal will eventually call VMTransport#stop():
>     public void stop() throws Exception {
> ...
>                 enqueueValve.turnOff();
>                 if (!disposed) {
>                     started = false;
>                     disposed = true;
> ...
>                 }
>             } finally {
>                 stopping.set(false);
>                 enqueueValve.turnOn();
>             }
> ...
>             // let the peer know that we are disconnecting..
>             try {
>                 oneway(DISCONNECT);
>             } catch (Exception ignore) {
>             }
>         }
>     }
> The DISCONNECT should get processed by the VMTransport#iterate() on the peer side:
>     public boolean iterate() {
> ...
>             if( command == DISCONNECT ) {
>                 tl.onException(new TransportDisposedIOException("Peer (" + peer.toString() + ") disposed."));
>             } else {
> ...
> tl is a reference to the TransportListener implemented by TransportConnection  and should result in a call to TransportConnection#doStop():
>     protected void doStop() throws Exception, InterruptedException {
> ...
>         connector.onStopped(this);
> The call to connector.onStopped(this) is implemented by TransportConnector#onStopped(TransportConnection):
>     public void onStopped(TransportConnection connection) {
>         connections.remove(connection);
>     }
> This removes the connection represented by the local side of the bridge from the connections array.
> *** HOWEVER *** in VMTransport#stop(), the disposed flag is set to true before the call to oneway(DISCONNECT); this causes the oneway(DISCONNECT) to fail  because of this code in VMTransport#oneway(Object):
>     public void oneway(Object command) throws IOException {
>         if (disposed) {
>             throw new TransportDisposedIOException("Transport disposed.");
>         }
> ...
> In other words, the DISCONNECT never makes it to the peer and so is never processed by TransportConnection (as TransportListener).
> Solution
> =======
> The solution is to send the DISCONNECT before setting the disposed flag to true.  However, care must be taken to prevent deadlock since VMTransport#stop() acquires a lock on enqueueValve and VMTransport#oneway(Object) acquires locks on the peer.enqueueValve --- if both peers try to stop concurrently, they may deadlock on the acquisition of VMTransport#enqueueValve.  To prevent this deadlock, it is necessary to send the DISCONNECT before the "local" enqueueValve is acquired --- this may mean sending the DISCONNECT unnecessarily (i.e., even if the client is already disposed), but this is not a problem since the resulting exception is ignored.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.