You are viewing a plain text version of this content. The canonical link for it is here.
Posted to scm@geronimo.apache.org by dj...@apache.org on 2010/02/19 21:36:28 UTC

svn commit: r911974 - in /geronimo/components/txmanager/trunk: geronimo-connector/src/main/java/org/apache/geronimo/connector/ geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/ geronimo-transaction/src/main/java/org/apache/geroni...

Author: djencks
Date: Fri Feb 19 20:36:27 2010
New Revision: 911974

URL: http://svn.apache.org/viewvc?rev=911974&view=rev
Log:
GERONIMO-5152 refactor what is supplied for recovery attempts to provide structure for retry.  Also a bunch of genericization

Added:
    geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ActivationSpecNamedXAResourceFactory.java   (with props)
    geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/OutboundNamedXAResourceFactory.java   (with props)
    geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/NamedXAResourceFactory.java   (with props)
Modified:
    geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ActivationSpecWrapper.java
    geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ResourceAdapterWrapper.java
    geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/AbstractConnectionManager.java
    geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/GenericConnectionManager.java
    geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/log/HOWLLog.java
    geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/GeronimoTransactionManager.java
    geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/RecoverableTransactionManager.java
    geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/Recovery.java
    geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/RecoveryImpl.java
    geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/TransactionLog.java
    geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/TransactionManagerImpl.java
    geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/XidImporter.java
    geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/MockLog.java
    geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/MockResource.java
    geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/MockResourceManager.java
    geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/TransactionManagerImplTest.java
    geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/XATransactionTester.java

Added: geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ActivationSpecNamedXAResourceFactory.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ActivationSpecNamedXAResourceFactory.java?rev=911974&view=auto
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ActivationSpecNamedXAResourceFactory.java (added)
+++ geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ActivationSpecNamedXAResourceFactory.java Fri Feb 19 20:36:27 2010
@@ -0,0 +1,66 @@
+/*
+ * 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.geronimo.connector;
+
+import javax.resource.ResourceException;
+import javax.resource.spi.ActivationSpec;
+import javax.resource.spi.ResourceAdapter;
+import javax.transaction.SystemException;
+import javax.transaction.xa.XAResource;
+import org.apache.geronimo.transaction.manager.NamedXAResource;
+import org.apache.geronimo.transaction.manager.NamedXAResourceFactory;
+import org.apache.geronimo.transaction.manager.WrapperNamedXAResource;
+
+/**
+ * @version $Rev$ $Date$
+ */
+public class ActivationSpecNamedXAResourceFactory implements NamedXAResourceFactory {
+
+    private final String name;
+    private final ActivationSpec activationSpec;
+    private final ResourceAdapter resourceAdapter;
+
+    public ActivationSpecNamedXAResourceFactory(String name, ActivationSpec activationSpec, ResourceAdapter resourceAdapter) {
+        this.name = name;
+        this.activationSpec = activationSpec;
+        this.resourceAdapter = resourceAdapter;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public NamedXAResource getNamedXAResource() throws SystemException {
+        try {
+            XAResource[] xaResources = resourceAdapter.getXAResources(new ActivationSpec[]{activationSpec});
+            if (xaResources == null || xaResources.length == 0) {
+                return null;
+            }
+            return new WrapperNamedXAResource(xaResources[0], name);
+        } catch (ResourceException e) {
+            throw (SystemException) new SystemException("Could not get XAResource for recovery for mdb: " + name).initCause(e);
+        }
+    }
+
+    public void returnNamedXAResource(NamedXAResource namedXAResource) {
+        // nothing to do AFAICT
+    }
+}

Propchange: geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ActivationSpecNamedXAResourceFactory.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ActivationSpecNamedXAResourceFactory.java
------------------------------------------------------------------------------
    svn:keywords = Date Revision

Propchange: geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ActivationSpecNamedXAResourceFactory.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ActivationSpecWrapper.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ActivationSpecWrapper.java?rev=911974&r1=911973&r2=911974&view=diff
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ActivationSpecWrapper.java (original)
+++ geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ActivationSpecWrapper.java Fri Feb 19 20:36:27 2010
@@ -101,6 +101,7 @@
     public void deactivate(final MessageEndpointFactory messageEndpointFactory) {
         ResourceAdapter resourceAdapter = activationSpec.getResourceAdapter();
         if (resourceAdapter != null) {
+            resourceAdapterWrapper.deregisterRecovery(containerId);
             resourceAdapterWrapper.endpointDeactivation(messageEndpointFactory, activationSpec);
         } else {
             //this should never happen, activation spec should have been registered with r.a.

Modified: geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ResourceAdapterWrapper.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ResourceAdapterWrapper.java?rev=911974&r1=911973&r2=911974&view=diff
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ResourceAdapterWrapper.java (original)
+++ geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/ResourceAdapterWrapper.java Fri Feb 19 20:36:27 2010
@@ -30,6 +30,7 @@
 import javax.transaction.xa.XAResource;
 
 import org.apache.geronimo.transaction.manager.NamedXAResource;
+import org.apache.geronimo.transaction.manager.NamedXAResourceFactory;
 import org.apache.geronimo.transaction.manager.RecoverableTransactionManager;
 import org.apache.geronimo.transaction.manager.WrapperNamedXAResource;
 
@@ -124,17 +125,11 @@
     }
 
     public void doRecovery(ActivationSpec activationSpec, String containerId) {
-        try {
-            XAResource[] xaResources = getXAResources(new ActivationSpec[]{activationSpec});
-            if (xaResources == null || xaResources.length == 0) {
-                return;
-            }
-            NamedXAResource xaResource = new WrapperNamedXAResource(xaResources[0], containerId);
-            transactionManager.recoverResourceManager(xaResource);
-        } catch (ResourceException e) {
-            transactionManager.recoveryError((SystemException) new SystemException("Could not get XAResource for recovery for mdb: " + containerId).initCause(e));
-        }
+        transactionManager.registerNamedXAResourceFactory(new ActivationSpecNamedXAResourceFactory(containerId, activationSpec, resourceAdapter));
+    }
 
+    public void deregisterRecovery(String containerId) {
+        transactionManager.unregisterNamedXAResourceFactory(containerId);
     }
 
     public void endpointDeactivation(final MessageEndpointFactory messageEndpointFactory, final ActivationSpec activationSpec) {

Modified: geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/AbstractConnectionManager.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/AbstractConnectionManager.java?rev=911974&r1=911973&r2=911974&view=diff
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/AbstractConnectionManager.java (original)
+++ geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/AbstractConnectionManager.java Fri Feb 19 20:36:27 2010
@@ -26,6 +26,7 @@
 
 import org.apache.geronimo.connector.outbound.connectionmanagerconfig.PoolingSupport;
 import org.apache.geronimo.transaction.manager.NamedXAResource;
+import org.apache.geronimo.transaction.manager.NamedXAResourceFactory;
 import org.apache.geronimo.transaction.manager.RecoverableTransactionManager;
 
 /**
@@ -34,16 +35,18 @@
 public abstract class AbstractConnectionManager implements ConnectionManagerContainer, ConnectionManager, LazyAssociatableConnectionManager, PoolingAttributes {
     protected final Interceptors interceptors;
     private final RecoverableTransactionManager transactionManager;
+    private final String name;
 
     //default constructor for use as endpoint
-    public AbstractConnectionManager() {
-        interceptors = null;
-        transactionManager = null;
-    }
+//    public AbstractConnectionManager() {
+//        interceptors = null;
+//        transactionManager = null;
+//    }
 
-    public AbstractConnectionManager(Interceptors interceptors, RecoverableTransactionManager transactionManager) {
+    public AbstractConnectionManager(Interceptors interceptors, RecoverableTransactionManager transactionManager, String name) {
         this.interceptors = interceptors;
         this.transactionManager = transactionManager;
+        this.name = name;
     }
 
     public Object createConnectionFactory(ManagedConnectionFactory mcf) throws ResourceException {
@@ -53,26 +56,12 @@
     protected ConnectionManager getConnectionManager() {
         return this;
     }
-    
+
     public void doRecovery(ManagedConnectionFactory managedConnectionFactory) {
-        try {
-            if (!getIsRecoverable()) {
-                return;
-            }
-            ManagedConnectionInfo mci = new ManagedConnectionInfo(managedConnectionFactory, null);
-
-            ConnectionInfo recoveryConnectionInfo = new ConnectionInfo(mci);
-            getRecoveryStack().getConnection(recoveryConnectionInfo);
-
-            // For pooled resources, we may now have a new MCI (not the one constructed above). Make sure we use the correct MCI
-            NamedXAResource xaResource = (NamedXAResource) recoveryConnectionInfo.getManagedConnectionInfo().getXAResource();
-            if (xaResource != null) {
-                transactionManager.recoverResourceManager(xaResource);
-                getRecoveryStack().returnConnection(recoveryConnectionInfo, ConnectionReturnAction.DESTROY);
-            }
-        } catch (ResourceException e) {
-            transactionManager.recoveryError((SystemException)new SystemException("Could not obtain recovery XAResource for managedConnectionFactory " + managedConnectionFactory).initCause(e));
+        if (!getIsRecoverable()) {
+            return;
         }
+        transactionManager.registerNamedXAResourceFactory(new OutboundNamedXAResourceFactory(name, getRecoveryStack(), managedConnectionFactory));
     }
 
     /**
@@ -196,10 +185,12 @@
     }
 
     public void doStop() throws Exception {
+        transactionManager.unregisterNamedXAResourceFactory(name);
         interceptors.getStack().destroy();
     }
 
     public void doFail() {
+        transactionManager.unregisterNamedXAResourceFactory(name);
         interceptors.getStack().destroy();
     }
 }

Modified: geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/GenericConnectionManager.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/GenericConnectionManager.java?rev=911974&r1=911973&r2=911974&view=diff
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/GenericConnectionManager.java (original)
+++ geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/GenericConnectionManager.java Fri Feb 19 20:36:27 2010
@@ -37,9 +37,9 @@
     protected static final Logger log = LoggerFactory.getLogger(AbstractSinglePoolConnectionInterceptor.class);
 
     //default constructor for use as endpoint
-    public GenericConnectionManager() {
-        super();
-    }
+//    public GenericConnectionManager() {
+//        super();
+//    }
 
     /**
      *
@@ -58,7 +58,7 @@
                                     RecoverableTransactionManager transactionManager,
                                     String name,
                                     ClassLoader classLoader) {
-        super(new InterceptorsImpl(transactionSupport, pooling, subjectSource, name, connectionTracker, transactionManager, classLoader), transactionManager);
+        super(new InterceptorsImpl(transactionSupport, pooling, subjectSource, name, connectionTracker, transactionManager, classLoader), transactionManager, name);
     }
 
     private static class InterceptorsImpl implements AbstractConnectionManager.Interceptors {

Added: geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/OutboundNamedXAResourceFactory.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/OutboundNamedXAResourceFactory.java?rev=911974&view=auto
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/OutboundNamedXAResourceFactory.java (added)
+++ geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/OutboundNamedXAResourceFactory.java Fri Feb 19 20:36:27 2010
@@ -0,0 +1,128 @@
+/*
+ * 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.geronimo.connector.outbound;
+
+import javax.resource.ResourceException;
+import javax.resource.spi.ManagedConnectionFactory;
+import javax.transaction.SystemException;
+import javax.transaction.xa.XAException;
+import javax.transaction.xa.XAResource;
+import javax.transaction.xa.Xid;
+import org.apache.geronimo.transaction.manager.NamedXAResource;
+import org.apache.geronimo.transaction.manager.NamedXAResourceFactory;
+
+/**
+ * @version $Rev$ $Date$
+ */
+public class OutboundNamedXAResourceFactory implements NamedXAResourceFactory {
+
+    private final String name;
+    private final ConnectionInterceptor recoveryStack;
+    private final ManagedConnectionFactory managedConnectionFactory;
+
+    public OutboundNamedXAResourceFactory(String name, ConnectionInterceptor recoveryStack, ManagedConnectionFactory managedConnectionFactory) {
+        this.name = name;
+        this.recoveryStack = recoveryStack;
+        this.managedConnectionFactory = managedConnectionFactory;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public NamedXAResource getNamedXAResource() throws SystemException {
+        try {
+            ManagedConnectionInfo mci = new ManagedConnectionInfo(managedConnectionFactory, null);
+
+            ConnectionInfo recoveryConnectionInfo = new ConnectionInfo(mci);
+            recoveryStack.getConnection(recoveryConnectionInfo);
+
+            // For pooled resources, we may now have a new MCI (not the one constructed above). Make sure we use the correct MCI
+            return new NamedXAResourceWithConnectioninfo((NamedXAResource) recoveryConnectionInfo.getManagedConnectionInfo().getXAResource(), recoveryConnectionInfo);
+        } catch (ResourceException e) {
+            throw (SystemException) new SystemException("Could not get XAResource for recovery for mcf: " + name).initCause(e);
+        }
+    }
+
+    public void returnNamedXAResource(NamedXAResource namedXAResource) {
+        NamedXAResourceWithConnectioninfo xares = (NamedXAResourceWithConnectioninfo) namedXAResource;
+        recoveryStack.returnConnection(xares.getConnectionInfo(), ConnectionReturnAction.DESTROY);
+    }
+
+    private static class NamedXAResourceWithConnectioninfo implements NamedXAResource {
+
+        private final NamedXAResource delegate;
+        private final ConnectionInfo connectionInfo;
+
+        private NamedXAResourceWithConnectioninfo(NamedXAResource delegate, ConnectionInfo connectionInfo) {
+            this.delegate = delegate;
+            this.connectionInfo = connectionInfo;
+        }
+
+        public ConnectionInfo getConnectionInfo() {
+            return connectionInfo;
+        }
+
+        public String getName() {
+            return delegate.getName();
+        }
+
+        public void commit(Xid xid, boolean b) throws XAException {
+            delegate.commit(xid, b);
+        }
+
+        public void end(Xid xid, int i) throws XAException {
+            delegate.end(xid, i);
+        }
+
+        public void forget(Xid xid) throws XAException {
+            delegate.forget(xid);
+        }
+
+        public int getTransactionTimeout() throws XAException {
+            return delegate.getTransactionTimeout();
+        }
+
+        public boolean isSameRM(XAResource xaResource) throws XAException {
+            return delegate.isSameRM(xaResource);
+        }
+
+        public int prepare(Xid xid) throws XAException {
+            return delegate.prepare(xid);
+        }
+
+        public Xid[] recover(int i) throws XAException {
+            return delegate.recover(i);
+        }
+
+        public void rollback(Xid xid) throws XAException {
+            delegate.rollback(xid);
+        }
+
+        public boolean setTransactionTimeout(int i) throws XAException {
+            return delegate.setTransactionTimeout(i);
+        }
+
+        public void start(Xid xid, int i) throws XAException {
+            delegate.start(xid, i);
+        }
+    }
+}

Propchange: geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/OutboundNamedXAResourceFactory.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/OutboundNamedXAResourceFactory.java
------------------------------------------------------------------------------
    svn:keywords = Date Revision

Propchange: geronimo/components/txmanager/trunk/geronimo-connector/src/main/java/org/apache/geronimo/connector/outbound/OutboundNamedXAResourceFactory.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/log/HOWLLog.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/log/HOWLLog.java?rev=911974&r1=911973&r2=911974&view=diff
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/log/HOWLLog.java (original)
+++ geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/log/HOWLLog.java Fri Feb 19 20:36:27 2010
@@ -67,7 +67,7 @@
     private final XALogger logger;
     private final Configuration configuration = new Configuration();
     private boolean started = false;
-    private HashMap recovered;
+    private HashMap<Xid, Recovery.XidBranchesPair> recovered;
 
     public HOWLLog(String bufferClassName,
                    int bufferSize,
@@ -219,7 +219,7 @@
         started = true;
         setLogFileDir(logFileDir);
         log.debug("Initiating transaction manager recovery");
-        recovered = new HashMap();
+        recovered = new HashMap<Xid, Recovery.XidBranchesPair>();
 
         logger.open(null);
 
@@ -241,15 +241,14 @@
     public void begin(Xid xid) throws LogException {
     }
 
-    public Object prepare(Xid xid, List branches) throws LogException {
+    public Object prepare(Xid xid, List<TransactionBranchInfo> branches) throws LogException {
         int branchCount = branches.size();
         byte[][] data = new byte[3 + 2 * branchCount][];
         data[0] = intToBytes(xid.getFormatId());
         data[1] = xid.getGlobalTransactionId();
         data[2] = xid.getBranchQualifier();
         int i = 3;
-        for (Iterator iterator = branches.iterator(); iterator.hasNext();) {
-            TransactionBranchInfo transactionBranchInfo = (TransactionBranchInfo) iterator.next();
+        for (TransactionBranchInfo transactionBranchInfo : branches) {
             data[i++] = transactionBranchInfo.getBranchXid().getBranchQualifier();
             data[i++] = transactionBranchInfo.getResourceName().getBytes();
         }
@@ -315,9 +314,9 @@
         }
     }
 
-    public Collection recover(XidFactory xidFactory) throws LogException {
+    public Collection<Recovery.XidBranchesPair> recover(XidFactory xidFactory) throws LogException {
         log.debug("Initiating transaction manager recovery");
-        Map recovered = new HashMap();
+        Map<Xid, Recovery.XidBranchesPair> recovered = new HashMap<Xid, Recovery.XidBranchesPair>();
         ReplayListener replayListener = new GeronimoReplayListener(xidFactory, recovered);
         logger.replayActiveTx(replayListener);
         log.debug("In doubt transactions recovered from log");
@@ -352,9 +351,9 @@
     private class GeronimoReplayListener implements ReplayListener {
 
         private final XidFactory xidFactory;
-        private final Map recoveredTx;
+        private final Map<Xid, Recovery.XidBranchesPair> recoveredTx;
 
-        public GeronimoReplayListener(XidFactory xidFactory, Map recoveredTx) {
+        public GeronimoReplayListener(XidFactory xidFactory, Map<Xid, Recovery.XidBranchesPair> recoveredTx) {
             this.xidFactory = xidFactory;
             this.recoveredTx = recoveredTx;
         }

Modified: geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/GeronimoTransactionManager.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/GeronimoTransactionManager.java?rev=911974&r1=911973&r2=911974&view=diff
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/GeronimoTransactionManager.java (original)
+++ geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/GeronimoTransactionManager.java Fri Feb 19 20:36:27 2010
@@ -30,6 +30,8 @@
 import javax.transaction.xa.Xid;
 
 /**
+ * Adds implementations of XATerminator and XAWork interfaces to basic TransactionManagerImpl
+ *
  * @version $Rev$ $Date$
  */
 public class GeronimoTransactionManager extends TransactionManagerImpl implements XATerminator, XAWork {
@@ -195,7 +197,7 @@
                 throw new XAException("No imported transaction for xid: " + xid);
             }
             if (importedTransaction != getTransaction()) {
-                throw new XAException("Imported transaction is not associated with the curren thread xid: " + xid);
+                throw new XAException("Imported transaction is not associated with the current thread xid: " + xid);
             }
             suspend();
         }

Added: geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/NamedXAResourceFactory.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/NamedXAResourceFactory.java?rev=911974&view=auto
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/NamedXAResourceFactory.java (added)
+++ geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/NamedXAResourceFactory.java Fri Feb 19 20:36:27 2010
@@ -0,0 +1,36 @@
+/*
+ * 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.geronimo.transaction.manager;
+
+import javax.transaction.SystemException;
+
+/**
+ * @version $Rev$ $Date$
+ */
+public interface NamedXAResourceFactory {
+
+    String getName();
+
+    NamedXAResource getNamedXAResource() throws SystemException;
+
+    void returnNamedXAResource(NamedXAResource namedXAResource);
+
+}

Propchange: geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/NamedXAResourceFactory.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/NamedXAResourceFactory.java
------------------------------------------------------------------------------
    svn:keywords = Date Revision

Propchange: geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/NamedXAResourceFactory.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/RecoverableTransactionManager.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/RecoverableTransactionManager.java?rev=911974&r1=911973&r2=911974&view=diff
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/RecoverableTransactionManager.java (original)
+++ geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/RecoverableTransactionManager.java Fri Feb 19 20:36:27 2010
@@ -28,5 +28,7 @@
 public interface RecoverableTransactionManager extends TransactionManager {
     void recoveryError(Exception e);
 
-    void recoverResourceManager(NamedXAResource xaResource);
+    void registerNamedXAResourceFactory(NamedXAResourceFactory namedXAResourceFactory);
+
+    void unregisterNamedXAResourceFactory(String namedXAResourceFactoryName);
 }

Modified: geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/Recovery.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/Recovery.java?rev=911974&r1=911973&r2=911974&view=diff
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/Recovery.java (original)
+++ geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/Recovery.java Fri Feb 19 20:36:27 2010
@@ -47,13 +47,13 @@
     //hard to implement.. needs ExternalTransaction to have a reference to externalXids.
 //    boolean remoteRecoveryComplete();
 
-    Map getExternalXids();
+    Map<Xid, TransactionImpl> getExternalXids();
 
     public static class XidBranchesPair {
         private final Xid xid;
 
         //set of TransactionBranchInfo
-        private final Set branches = new HashSet();
+        private final Set<TransactionBranchInfo> branches = new HashSet<TransactionBranchInfo>();
 
         private final Object mark;
 
@@ -66,7 +66,7 @@
             return xid;
         }
 
-        public Set getBranches() {
+        public Set<TransactionBranchInfo> getBranches() {
             return branches;
         }
 

Modified: geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/RecoveryImpl.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/RecoveryImpl.java?rev=911974&r1=911973&r2=911974&view=diff
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/RecoveryImpl.java (original)
+++ geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/RecoveryImpl.java Fri Feb 19 20:36:27 2010
@@ -51,12 +51,12 @@
     private final TransactionLog txLog;
     private final XidFactory xidFactory;
 
-    private final Map externalXids = new HashMap();
-    private final Map ourXids = new HashMap();
-    private final Map nameToOurTxMap = new HashMap();
-    private final Map externalGlobalIdMap = new HashMap();
+    private final Map<Xid, TransactionImpl> externalXids = new HashMap<Xid, TransactionImpl>();
+    private final Map<ByteArrayWrapper, XidBranchesPair> ourXids = new HashMap<ByteArrayWrapper, XidBranchesPair>();
+    private final Map<String, Set<XidBranchesPair>> nameToOurTxMap = new HashMap<String, Set<XidBranchesPair>>();
+    private final Map<byte[], TransactionImpl> externalGlobalIdMap = new HashMap<byte[], TransactionImpl>();
 
-    private final List recoveryErrors = new ArrayList();
+    private final List<Exception> recoveryErrors = new ArrayList<Exception>();
 
     public RecoveryImpl(final TransactionLog txLog, final XidFactory xidFactory) {
         this.txLog = txLog;
@@ -64,22 +64,21 @@
     }
 
     public synchronized void recoverLog() throws XAException {
-        Collection preparedXids = null;
+        Collection<XidBranchesPair> preparedXids;
         try {
             preparedXids = txLog.recover(xidFactory);
         } catch (LogException e) {
             throw (XAException) new XAException(XAException.XAER_RMERR).initCause(e);
         }
-        for (Iterator iterator = preparedXids.iterator(); iterator.hasNext();) {
-            XidBranchesPair xidBranchesPair = (Recovery.XidBranchesPair) iterator.next();
+        for (XidBranchesPair xidBranchesPair : preparedXids) {
             Xid xid = xidBranchesPair.getXid();
             if (xidFactory.matchesGlobalId(xid.getGlobalTransactionId())) {
                 ourXids.put(new ByteArrayWrapper(xid.getGlobalTransactionId()), xidBranchesPair);
-                for (Iterator branches = xidBranchesPair.getBranches().iterator(); branches.hasNext();) {
-                    String name = ((TransactionBranchInfo) branches.next()).getResourceName();
-                    Set transactionsForName = (Set)nameToOurTxMap.get(name);
+                for (TransactionBranchInfo transactionBranchInfo : xidBranchesPair.getBranches()) {
+                    String name = transactionBranchInfo.getResourceName();
+                    Set<XidBranchesPair> transactionsForName = nameToOurTxMap.get(name);
                     if (transactionsForName == null) {
-                        transactionsForName = new HashSet();
+                        transactionsForName = new HashSet<XidBranchesPair>();
                         nameToOurTxMap.put(name, transactionsForName);
                     }
                     transactionsForName.add(xidBranchesPair);
@@ -99,7 +98,7 @@
         for (int i = 0; prepared != null && i < prepared.length; i++) {
             Xid xid = prepared[i];
             ByteArrayWrapper globalIdWrapper = new ByteArrayWrapper(xid.getGlobalTransactionId());
-            XidBranchesPair xidNamesPair = (XidBranchesPair) ourXids.get(globalIdWrapper);
+            XidBranchesPair xidNamesPair = ourXids.get(globalIdWrapper);
             
             if (xidNamesPair != null) {
                 
@@ -125,7 +124,7 @@
                 }
             } else if (xidFactory.matchesBranchId(xid.getBranchQualifier())) {
                 //our branch, but we did not start this tx.
-                TransactionImpl externalTx = (TransactionImpl) externalGlobalIdMap.get(xid.getGlobalTransactionId());
+                TransactionImpl externalTx = externalGlobalIdMap.get(xid.getGlobalTransactionId());
                 if (externalTx == null) {
                     //we did not prepare this branch, rollback.
                     try {
@@ -141,18 +140,16 @@
             }
             //else we had nothing to do with this xid.
         }
-        Set transactionsForName = (Set)nameToOurTxMap.get(name);
+        Set<XidBranchesPair> transactionsForName = nameToOurTxMap.get(name);
         if (transactionsForName != null) {
-            for (Iterator transactions = transactionsForName.iterator(); transactions.hasNext();) {
-                XidBranchesPair xidBranchesPair = (XidBranchesPair) transactions.next();
+            for (XidBranchesPair xidBranchesPair : transactionsForName) {
                 removeNameFromTransaction(xidBranchesPair, name, false);
             }
         }
     }
 
     private boolean isNameInTransaction(XidBranchesPair xidBranchesPair, String name) {
-        for (Iterator branches = xidBranchesPair.getBranches().iterator(); branches.hasNext();) {
-            TransactionBranchInfo transactionBranchInfo = (TransactionBranchInfo) branches.next();
+        for (TransactionBranchInfo transactionBranchInfo : xidBranchesPair.getBranches()) {
             if (name.equals(transactionBranchInfo.getResourceName())) {
                 return true;
             }
@@ -187,7 +184,7 @@
         return !recoveryErrors.isEmpty();
     }
 
-    public synchronized List getRecoveryErrors() {
+    public synchronized List<Exception> getRecoveryErrors() {
         return Collections.unmodifiableList(recoveryErrors);
     }
 
@@ -203,8 +200,8 @@
 //    public boolean remoteRecoveryComplete() {
 //    }
 
-    public synchronized Map getExternalXids() {
-        return new HashMap(externalXids);
+    public synchronized Map<Xid, TransactionImpl> getExternalXids() {
+        return new HashMap<Xid, TransactionImpl>(externalXids);
     }
 
     private static class ByteArrayWrapper {
@@ -215,8 +212,8 @@
             assert bytes != null;
             this.bytes = bytes;
             int hash = 0;
-            for (int i = 0; i < bytes.length; i++) {
-                hash += 37 * bytes[i];
+            for (byte aByte : bytes) {
+                hash += 37 * aByte;
             }
             hashCode = hash;
         }
@@ -234,11 +231,13 @@
     }
 
     private static class ExternalTransaction extends TransactionImpl {
-        private Set resourceNames;
+        private final Set<String> resourceNames = new HashSet<String>();
 
-        public ExternalTransaction(Xid xid, TransactionLog txLog, Set resourceNames) {
+        public ExternalTransaction(Xid xid, TransactionLog txLog, Set<TransactionBranchInfo> resourceNames) {
             super(xid, txLog);
-            this.resourceNames = resourceNames;
+            for (TransactionBranchInfo info: resourceNames) {
+                this.resourceNames.add(info.getResourceName());
+            }
         }
 
         public boolean hasName(String name) {

Modified: geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/TransactionLog.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/TransactionLog.java?rev=911974&r1=911973&r2=911974&view=diff
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/TransactionLog.java (original)
+++ geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/TransactionLog.java Fri Feb 19 20:36:27 2010
@@ -36,9 +36,10 @@
      * log prepare for the global xid xid and the list of TransactionBranchInfo branches
      * @param xid global xid for the transactions
      * @param branches List of TransactionBranchInfo
-     * @throws LogException
+     * @return log mark to use in commit/rollback calls.
+     * @throws LogException on error
      */
-    Object prepare(Xid xid, List branches) throws LogException;
+    Object prepare(Xid xid, List<TransactionBranchInfo> branches) throws LogException;
 
     void commit(Xid xid, Object logMark) throws LogException;
 
@@ -48,11 +49,11 @@
      * Recovers the log, returning a map of (top level) xid to List of TransactionBranchInfo for the branches.
      * Uses the XidFactory to reconstruct the xids.
      *
-     * @param xidFactory
+     * @param xidFactory Xid factory
      * @return Map of recovered xid to List of TransactionBranchInfo representing the branches.
-     * @throws LogException
+     * @throws LogException on error
      */
-    Collection recover(XidFactory xidFactory) throws LogException;
+    Collection<Recovery.XidBranchesPair> recover(XidFactory xidFactory) throws LogException;
 
     String getXMLStats();
 

Modified: geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/TransactionManagerImpl.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/TransactionManagerImpl.java?rev=911974&r1=911973&r2=911974&view=diff
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/TransactionManagerImpl.java (original)
+++ geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/TransactionManagerImpl.java Fri Feb 19 20:36:27 2010
@@ -19,7 +19,6 @@
 
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
@@ -41,19 +40,21 @@
  */
 public class TransactionManagerImpl implements TransactionManager, UserTransaction, TransactionSynchronizationRegistry, XidImporter, MonitorableTransactionManager, RecoverableTransactionManager {
     private static final Logger log = LoggerFactory.getLogger(TransactionManagerImpl.class);
+    private static final Logger recoveryLog = LoggerFactory.getLogger("RecoveryController");
+
     protected static final int DEFAULT_TIMEOUT = 600;
     protected static final byte[] DEFAULT_TM_ID = new byte[] {71,84,77,73,68};
 
     final TransactionLog transactionLog;
     final XidFactory xidFactory;
     private final int defaultTransactionTimeoutMilliseconds;
-    private final ThreadLocal transactionTimeoutMilliseconds = new ThreadLocal();
-    private final ThreadLocal threadTx = new ThreadLocal();
-    private final ConcurrentHashMap associatedTransactions = new ConcurrentHashMap();
-    private static final Logger recoveryLog = LoggerFactory.getLogger("RecoveryController");
+    private final ThreadLocal<Long> transactionTimeoutMilliseconds = new ThreadLocal<Long>();
+    private final ThreadLocal<Transaction> threadTx = new ThreadLocal<Transaction>();
+    private final ConcurrentHashMap<Transaction, Thread> associatedTransactions = new ConcurrentHashMap<Transaction, Thread>();
     final Recovery recovery;
-    private final CopyOnWriteArrayList transactionAssociationListeners = new CopyOnWriteArrayList();
-    private List recoveryErrors = new ArrayList();
+    private final Map<String, NamedXAResourceFactory> namedXAResourceFactories = new ConcurrentHashMap<String, NamedXAResourceFactory>();
+    private final CopyOnWriteArrayList<TransactionManagerMonitor> transactionAssociationListeners = new CopyOnWriteArrayList<TransactionManagerMonitor>();
+    private List<Exception> recoveryErrors = new ArrayList<Exception>();
     // statistics
     private AtomicLong totalCommits = new AtomicLong(0);
     private AtomicLong totalRollBacks = new AtomicLong(0);
@@ -103,7 +104,7 @@
     }
 
     public Transaction getTransaction() {
-        return (Transaction) threadTx.get();
+        return threadTx.get();
     }
 
     private void associate(TransactionImpl tx) throws InvalidTransactionException {
@@ -137,7 +138,7 @@
         if (seconds == 0) {
             transactionTimeoutMilliseconds.set(null);
         } else {
-            transactionTimeoutMilliseconds.set(new Long(seconds * 1000));
+            transactionTimeoutMilliseconds.set((long) seconds * 1000);
         }
     }
 
@@ -226,7 +227,7 @@
 
     /**
      * jta 1.1 method so the jpa implementations can be told to flush their caches.
-     * @param synchronization
+     * @param synchronization interposed synchronization
      */
     public void registerInterposedSynchronization(Synchronization synchronization) {
         TransactionImpl tx = getActiveTransactionImpl();
@@ -272,8 +273,7 @@
         if (transactionTimeoutMilliseconds < 0) {
             throw new SystemException("transaction timeout must be positive or 0 to reset to default");
         }
-        TransactionImpl tx = new TransactionImpl(xid, xidFactory, transactionLog, getTransactionTimeoutMilliseconds(transactionTimeoutMilliseconds));
-        return tx;
+        return new TransactionImpl(xid, xidFactory, transactionLog, getTransactionTimeoutMilliseconds(transactionTimeoutMilliseconds));
     }
 
     public void commit(Transaction tx, boolean onePhase) throws XAException {
@@ -334,9 +334,9 @@
         if (transactionTimeoutMilliseconds != 0) {
             return transactionTimeoutMilliseconds;
         }
-        Long timeout = (Long) this.transactionTimeoutMilliseconds.get();
+        Long timeout = this.transactionTimeoutMilliseconds.get();
         if (timeout != null) {
-            return timeout.longValue();
+            return timeout;
         }
         return defaultTransactionTimeoutMilliseconds;
     }
@@ -347,16 +347,28 @@
         recoveryErrors.add(e);
     }
 
-    public void recoverResourceManager(NamedXAResource xaResource) {
+    public void registerNamedXAResourceFactory(NamedXAResourceFactory namedXAResourceFactory) {
+        namedXAResourceFactories.put(namedXAResourceFactory.getName(), namedXAResourceFactory);
         try {
-            recovery.recoverResourceManager(xaResource);
+            NamedXAResource namedXAResource = namedXAResourceFactory.getNamedXAResource();
+            try {
+                recovery.recoverResourceManager(namedXAResource);
+            } finally {
+                namedXAResourceFactory.returnNamedXAResource(namedXAResource);
+            }
         } catch (XAException e) {
             recoveryError(e);
+        } catch (SystemException e) {
+            recoveryError(e);
         }
     }
 
-    public Map getExternalXids() {
-        return new HashMap(recovery.getExternalXids());
+    public void unregisterNamedXAResourceFactory(String namedXAResourceFactoryName) {
+        namedXAResourceFactories.remove(namedXAResourceFactoryName);
+    }
+
+    public Map<Xid, TransactionImpl> getExternalXids() {
+        return new HashMap<Xid, TransactionImpl>(recovery.getExternalXids());
     }
 
     public void addTransactionAssociationListener(TransactionManagerMonitor listener) {
@@ -368,8 +380,7 @@
     }
 
     protected void fireThreadAssociated(Transaction tx) {
-        for (Iterator iterator = transactionAssociationListeners.iterator(); iterator.hasNext();) {
-            TransactionManagerMonitor listener = (TransactionManagerMonitor) iterator.next();
+        for (TransactionManagerMonitor listener : transactionAssociationListeners) {
             try {
                 listener.threadAssociated(tx);
             } catch (Exception e) {
@@ -379,8 +390,7 @@
     }
 
     protected void fireThreadUnassociated(Transaction tx) {
-        for (Iterator iterator = transactionAssociationListeners.iterator(); iterator.hasNext();) {
-            TransactionManagerMonitor listener = (TransactionManagerMonitor) iterator.next();
+        for (TransactionManagerMonitor listener : transactionAssociationListeners) {
             try {
                 listener.threadUnassociated(tx);
             } catch (Exception e) {
@@ -391,6 +401,7 @@
 
     /**
      * Returns the number of active transactions.
+     * @return the count of active transactions
      */
     public long getActiveCount() {
         return activeCount.longValue();
@@ -398,6 +409,7 @@
 
     /**
      * Return the number of total commits
+     * @return the number of commits since statistics were reset
      */
     public long getTotalCommits() {
         return totalCommits.longValue();
@@ -405,6 +417,7 @@
 
     /**
      * Returns the number of total rollbacks
+     * @return the number of rollbacks since statistics were reset
      */
     public long getTotalRollbacks() {
         return totalRollBacks.longValue();

Modified: geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/XidImporter.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/XidImporter.java?rev=911974&r1=911973&r2=911974&view=diff
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/XidImporter.java (original)
+++ geronimo/components/txmanager/trunk/geronimo-transaction/src/main/java/org/apache/geronimo/transaction/manager/XidImporter.java Fri Feb 19 20:36:27 2010
@@ -39,5 +39,5 @@
     int prepare(Transaction tx) throws XAException;
     void rollback(Transaction tx) throws XAException;
 
-    Map getExternalXids();
+    Map<Xid, TransactionImpl> getExternalXids();
 }

Modified: geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/MockLog.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/MockLog.java?rev=911974&r1=911973&r2=911974&view=diff
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/MockLog.java (original)
+++ geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/MockLog.java Fri Feb 19 20:36:27 2010
@@ -40,7 +40,7 @@
     public void begin(Xid xid) throws LogException {
     }
 
-    public Object prepare(Xid xid, List branches) throws LogException {
+    public Object prepare(Xid xid, List<TransactionBranchInfo> branches) throws LogException {
         Object mark = new Object();
         Recovery.XidBranchesPair xidBranchesPair = new Recovery.XidBranchesPair(xid, mark);
         xidBranchesPair.getBranches().addAll(branches);

Modified: geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/MockResource.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/MockResource.java?rev=911974&r1=911973&r2=911974&view=diff
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/MockResource.java (original)
+++ geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/MockResource.java Fri Feb 19 20:36:27 2010
@@ -20,6 +20,7 @@
 import java.util.Set;
 import java.util.HashSet;
 
+import javax.transaction.SystemException;
 import javax.transaction.xa.XAException;
 import javax.transaction.xa.XAResource;
 import javax.transaction.xa.Xid;
@@ -27,7 +28,7 @@
 /**
  * @version $Rev$ $Date$
  */
-public class MockResource implements NamedXAResource {
+public class MockResource implements NamedXAResource, NamedXAResourceFactory {
     private String xaResourceName = "mockResource";
     private Xid currentXid;
     private MockResourceManager manager;
@@ -153,4 +154,12 @@
         return xaResourceName;
     }
 
+    public NamedXAResource getNamedXAResource() throws SystemException {
+        return this;
+    }
+
+    public void returnNamedXAResource(NamedXAResource namedXAResource) {
+        if (this != namedXAResource) throw new RuntimeException("Wrong NamedXAResource returned: expected: " + this + " actual: " + namedXAResource);
+    }
+
 }

Modified: geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/MockResourceManager.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/MockResourceManager.java?rev=911974&r1=911973&r2=911974&view=diff
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/MockResourceManager.java (original)
+++ geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/MockResourceManager.java Fri Feb 19 20:36:27 2010
@@ -36,15 +36,12 @@
     private boolean willCommit;
     private Map xids = new HashMap();
 
-    private NamedXAResource resources;
-
     public MockResourceManager(boolean willCommit) {
         this.willCommit = willCommit;
     }
 
     public MockResource getResource(String xaResourceName) {
         MockResource mockResource =  new MockResource(this, xaResourceName);
-        resources = mockResource;
         return mockResource;
     }
 
@@ -71,8 +68,4 @@
         }
     }
 
-    public void doRecovery(RecoverableTransactionManager transactionManager) throws SystemException {
-        transactionManager.recoverResourceManager(resources);
-    }
-
 }

Modified: geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/TransactionManagerImplTest.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/TransactionManagerImplTest.java?rev=911974&r1=911973&r2=911974&view=diff
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/TransactionManagerImplTest.java (original)
+++ geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/TransactionManagerImplTest.java Fri Feb 19 20:36:27 2010
@@ -256,14 +256,21 @@
         tm.prepare(tx);
         //recover
         tm.recovery.recoverLog();
-        rm1.doRecovery(tm);
+        recover(r1_1);
+        recover(r1_2);
         assertTrue(r1_2.isCommitted());
         assertTrue(!r2_2.isCommitted());
-        rm2.doRecovery(tm);
+        recover(r2_1);
+        recover(r2_2);
         assertTrue(r2_2.isCommitted());
         assertTrue(tm.recovery.localRecoveryComplete());
     }
 
+    private void recover(MockResource mr) {
+        tm.registerNamedXAResourceFactory(mr);
+        tm.unregisterNamedXAResourceFactory(mr.getName());
+    }
+
     public void testImportedXidRecovery() throws Exception {
         //create a transaction from an external transaction manager.
         XidFactory xidFactory2 = new XidFactoryImpl("tm2".getBytes());
@@ -279,10 +286,12 @@
         tm.prepare(tx);
         //recover
         tm.recovery.recoverLog();
-        rm1.doRecovery(tm);
+        recover(r1_1);
+        recover(r1_2);
         assertTrue(!r1_2.isCommitted());
         assertTrue(!r2_2.isCommitted());
-        rm2.doRecovery(tm);
+        recover(r2_1);
+        recover(r2_2);
         assertTrue(!r2_2.isCommitted());
         //there are no transactions started here, so local recovery is complete
         assertTrue(tm.recovery.localRecoveryComplete());

Modified: geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/XATransactionTester.java
URL: http://svn.apache.org/viewvc/geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/XATransactionTester.java?rev=911974&r1=911973&r2=911974&view=diff
==============================================================================
--- geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/XATransactionTester.java (original)
+++ geronimo/components/txmanager/trunk/geronimo-transaction/src/test/java/org/apache/geronimo/transaction/manager/XATransactionTester.java Fri Feb 19 20:36:27 2010
@@ -103,7 +103,7 @@
             XATransactionTester.this.xid = xid;
         }
 
-        public Object prepare(Xid xid, List branches) throws LogException {
+        public Object prepare(Xid xid, List<TransactionBranchInfo> branches) throws LogException {
             return new Object();
         }
 
@@ -113,8 +113,8 @@
         public void rollback(Xid xid, Object logMark) throws LogException {
         }
 
-        public Collection recover(XidFactory xidFactory) throws LogException {
-            return new ArrayList();
+        public Collection<Recovery.XidBranchesPair> recover(XidFactory xidFactory) throws LogException {
+            return new ArrayList<Recovery.XidBranchesPair>();
         }
 
         public String getXMLStats() {