Repository: activemq
Updated Branches:
  refs/heads/master fc2553574 -> 450a5226e


https://issues.apache.org/jira/browse/AMQ-5961

Prevent deadlock by moving calls to afterCommit
and afterRollback out of synchronized blocks.

https://issues.apache.org/jira/browse/AMQ-5961
Lock list but keep global hashmap out for deadlock protection.

https://issues.apache.org/jira/browse/AMQ-5961
Lock safely in isInXATransaction.

https://issues.apache.org/jira/browse/AMQ-5961
Don't synchronize on list as we have an exclusive reference.


Project: http://git-wip-us.apache.org/repos/asf/activemq/repo
Commit: http://git-wip-us.apache.org/repos/asf/activemq/commit/9a78bc68
Tree: http://git-wip-us.apache.org/repos/asf/activemq/tree/9a78bc68
Diff: http://git-wip-us.apache.org/repos/asf/activemq/diff/9a78bc68

Branch: refs/heads/master
Commit: 9a78bc689f06dde61ba38be1ebc840a53ca02b68
Parents: fc25535
Author: erik-wramner <e...@wramner.name>
Authored: Thu Sep 10 18:44:36 2015 +0200
Committer: Christopher L. Shannon (cshannon) <christopher.l.shan...@gmail.com>
Committed: Thu Oct 1 13:31:00 2015 +0000

----------------------------------------------------------------------
 .../org/apache/activemq/TransactionContext.java | 102 +++++++++++--------
 1 file changed, 59 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/activemq/blob/9a78bc68/activemq-client/src/main/java/org/apache/activemq/TransactionContext.java
----------------------------------------------------------------------
diff --git 
a/activemq-client/src/main/java/org/apache/activemq/TransactionContext.java 
b/activemq-client/src/main/java/org/apache/activemq/TransactionContext.java
index da05059..27cb49d 100755
--- a/activemq-client/src/main/java/org/apache/activemq/TransactionContext.java
+++ b/activemq-client/src/main/java/org/apache/activemq/TransactionContext.java
@@ -98,13 +98,11 @@ public class TransactionContext implements XAResource {
         if (transactionId != null && transactionId.isXATransaction()) {
             return true;
         } else {
-            if (!ENDED_XA_TRANSACTION_CONTEXTS.isEmpty()) {
-                synchronized(ENDED_XA_TRANSACTION_CONTEXTS) {
-                    for(List<TransactionContext> transactions : 
ENDED_XA_TRANSACTION_CONTEXTS.values()) {
-                        if (transactions.contains(this)) {
-                            return true;
-                        }
-                    }
+            synchronized(ENDED_XA_TRANSACTION_CONTEXTS) {
+                for(List<TransactionContext> transactions : 
ENDED_XA_TRANSACTION_CONTEXTS.values()) {
+                      if (transactions.contains(this)) {
+                          return true;
+                      }
                 }
             }
         }
@@ -470,9 +468,14 @@ public class TransactionContext implements XAResource {
             IntegerResponse response = 
(IntegerResponse)syncSendPacketWithInterruptionHandling(info);
             if (XAResource.XA_RDONLY == response.getResult()) {
                 // transaction stops now, may be syncs that need a callback
+                List<TransactionContext> l;
                 synchronized(ENDED_XA_TRANSACTION_CONTEXTS) {
-                    List<TransactionContext> l = 
ENDED_XA_TRANSACTION_CONTEXTS.remove(x);
-                    if (l != null && !l.isEmpty()) {
+                    l = ENDED_XA_TRANSACTION_CONTEXTS.remove(x);
+                }
+                // After commit may be expensive and can deadlock, do it 
outside global synch block
+                // No risk for concurrent updates as we own the list now
+                if (l != null) {
+                    if(! l.isEmpty()) {
                         LOG.debug("firing afterCommit callbacks on XA_RDONLY 
from prepare: {}", xid);
                         for (TransactionContext ctx : l) {
                             ctx.afterCommit();
@@ -484,16 +487,19 @@ public class TransactionContext implements XAResource {
 
         } catch (JMSException e) {
             LOG.warn("prepare of: " + x + " failed with: " + e, e);
+            List<TransactionContext> l;
             synchronized(ENDED_XA_TRANSACTION_CONTEXTS) {
-                List<TransactionContext> l = 
ENDED_XA_TRANSACTION_CONTEXTS.remove(x);
-                if (l != null && !l.isEmpty()) {
-                    for (TransactionContext ctx : l) {
-                        try {
-                            ctx.afterRollback();
-                        } catch (Throwable ignored) {
-                            LOG.debug("failed to firing afterRollback 
callbacks on prepare " +
-                                      "failure, txid: {}, context: {}", x, 
ctx, ignored);
-                        }
+                l = ENDED_XA_TRANSACTION_CONTEXTS.remove(x);
+            }
+            // After rollback may be expensive and can deadlock, do it outside 
global synch block
+            // No risk for concurrent updates as we own the list now
+            if (l != null) {
+                for (TransactionContext ctx : l) {
+                    try {
+                        ctx.afterRollback();
+                    } catch (Throwable ignored) {
+                        LOG.debug("failed to firing afterRollback callbacks on 
prepare " +
+                                  "failure, txid: {}, context: {}", x, ctx, 
ignored);
                     }
                 }
             }
@@ -530,13 +536,16 @@ public class TransactionContext implements XAResource {
             TransactionInfo info = new TransactionInfo(getConnectionId(), x, 
TransactionInfo.ROLLBACK);
             syncSendPacketWithInterruptionHandling(info);
 
+            List<TransactionContext> l;
             synchronized(ENDED_XA_TRANSACTION_CONTEXTS) {
-                List<TransactionContext> l = 
ENDED_XA_TRANSACTION_CONTEXTS.remove(x);
-                if (l != null && !l.isEmpty()) {
-                    for (TransactionContext ctx : l) {
-                        ctx.afterRollback();
-                    }
-                }
+                l = ENDED_XA_TRANSACTION_CONTEXTS.remove(x);
+            }
+            // After rollback may be expensive and can deadlock, do it outside 
global synch block
+            // No risk for concurrent updates as we own the list now
+            if (l != null) {
+                for (TransactionContext ctx : l) {
+                    ctx.afterRollback();
+                }                  
             }
         } catch (JMSException e) {
             throw toXAException(e);
@@ -574,15 +583,18 @@ public class TransactionContext implements XAResource {
 
             syncSendPacketWithInterruptionHandling(info);
 
+            List<TransactionContext> l;
             synchronized(ENDED_XA_TRANSACTION_CONTEXTS) {
-                List<TransactionContext> l = 
ENDED_XA_TRANSACTION_CONTEXTS.remove(x);
-                if (l != null && !l.isEmpty()) {
-                    for (TransactionContext ctx : l) {
-                        try {
-                            ctx.afterCommit();
-                        } catch (Exception ignored) {
-                            LOG.debug("ignoring exception from after 
completion on ended transaction: {}", ignored, ignored);
-                        }
+                l = ENDED_XA_TRANSACTION_CONTEXTS.remove(x);
+            }
+            // After commit may be expensive and can deadlock, do it outside 
global synch block
+            // No risk for concurrent updates as we own the list now
+            if (l != null) {
+                for (TransactionContext ctx : l) {
+                    try {
+                        ctx.afterCommit();
+                    } catch (Exception ignored) {
+                        LOG.debug("ignoring exception from after completion on 
ended transaction: {}", ignored, ignored);
                     }
                 }
             }
@@ -590,15 +602,18 @@ public class TransactionContext implements XAResource {
         } catch (JMSException e) {
             LOG.warn("commit of: " + x + " failed with: " + e, e);
             if (onePhase) {
+                List<TransactionContext> l;
                 synchronized(ENDED_XA_TRANSACTION_CONTEXTS) {
-                    List<TransactionContext> l = 
ENDED_XA_TRANSACTION_CONTEXTS.remove(x);
-                    if (l != null && !l.isEmpty()) {
-                        for (TransactionContext ctx : l) {
-                            try {
-                                ctx.afterRollback();
-                            } catch (Throwable ignored) {
-                                LOG.debug("failed to firing afterRollback 
callbacks commit failure, txid: {}, context: {}", x, ctx, ignored);
-                            }
+                    l = ENDED_XA_TRANSACTION_CONTEXTS.remove(x);
+                }
+                // After rollback may be expensive and can deadlock, do it 
outside global synch block
+                // No risk for concurrent updates as we own the list now
+                if (l != null) {
+                    for (TransactionContext ctx : l) {
+                        try {
+                            ctx.afterRollback();
+                        } catch (Throwable ignored) {
+                            LOG.debug("failed to firing afterRollback 
callbacks commit failure, txid: {}, context: {}", x, ctx, ignored);
                         }
                     }
                 }
@@ -735,13 +750,14 @@ public class TransactionContext implements XAResource {
 
                 // Add our self to the list of contexts that are interested in
                 // post commit/rollback events.
+                List<TransactionContext> l;
                 synchronized(ENDED_XA_TRANSACTION_CONTEXTS) {
-                    List<TransactionContext> l = 
ENDED_XA_TRANSACTION_CONTEXTS.get(transactionId);
+                    l = ENDED_XA_TRANSACTION_CONTEXTS.get(transactionId);
                     if (l == null) {
                         l = new ArrayList<TransactionContext>(3);
                         ENDED_XA_TRANSACTION_CONTEXTS.put(transactionId, l);
-                        l.add(this);
-                    } else if (!l.contains(this)) {
+                    }
+                    if (!l.contains(this)) {
                         l.add(this);
                     }
                 }

Reply via email to