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 2004/07/22 05:39:01 UTC

cvs commit: incubator-geronimo/modules/transaction/src/test/org/apache/geronimo/transaction/manager MockLog.java XATransactionTester.java

djencks     2004/07/21 20:39:01

  Modified:    modules/connector/src/java/org/apache/geronimo/connector/outbound/transactionlog
                        JDBCLog.java
               modules/transaction/src/test/org/apache/geronimo/transaction/log
                        AbstractLogTest.java
               modules/transaction/src/java/org/apache/geronimo/transaction/log
                        HOWLLog.java UnrecoverableLog.java
               modules/transaction/src/java/org/apache/geronimo/transaction/manager
                        Recovery.java RecoveryImpl.java
                        TransactionImpl.java TransactionLog.java
               modules/transaction/src/test/org/apache/geronimo/transaction/manager
                        MockLog.java XATransactionTester.java
  Log:
  Let the transactionlog supply a key that we feed back on commit/rollback.  Used by HOWL to track which parts of the log may be discarded
  
  Revision  Changes    Path
  1.9       +45 -38    incubator-geronimo/modules/connector/src/java/org/apache/geronimo/connector/outbound/transactionlog/JDBCLog.java
  
  Index: JDBCLog.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/connector/src/java/org/apache/geronimo/connector/outbound/transactionlog/JDBCLog.java,v
  retrieving revision 1.8
  retrieving revision 1.9
  diff -u -r1.8 -r1.9
  --- JDBCLog.java	12 Jul 2004 06:07:49 -0000	1.8
  +++ JDBCLog.java	22 Jul 2004 03:39:01 -0000	1.9
  @@ -22,10 +22,9 @@
   import java.sql.ResultSet;
   import java.sql.SQLException;
   import java.util.ArrayList;
  -import java.util.HashMap;
  -import java.util.List;
  -import java.util.Map;
  +import java.util.Collection;
   import java.util.Iterator;
  +import java.util.List;
   
   import javax.sql.DataSource;
   import javax.transaction.xa.Xid;
  @@ -36,10 +35,11 @@
   import org.apache.geronimo.gbean.GBeanLifecycle;
   import org.apache.geronimo.gbean.WaitingException;
   import org.apache.geronimo.transaction.manager.LogException;
  -import org.apache.geronimo.transaction.manager.TransactionLog;
  -import org.apache.geronimo.transaction.manager.XidFactory;
  +import org.apache.geronimo.transaction.manager.Recovery;
   import org.apache.geronimo.transaction.manager.TransactionBranchInfo;
   import org.apache.geronimo.transaction.manager.TransactionBranchInfoImpl;
  +import org.apache.geronimo.transaction.manager.TransactionLog;
  +import org.apache.geronimo.transaction.manager.XidFactory;
   
   /**
    * "Last Resource optimization" for single servers wishing to have valid xa transactions with
  @@ -62,6 +62,12 @@
           this.managedConnectionFactoryWrapper = managedConnectionFactoryWrapper;
       }
   
  +    public JDBCLog(String systemId, DataSource dataSource) {
  +        this.systemId = systemId;
  +        this.managedConnectionFactoryWrapper = null;
  +        this.dataSource = dataSource;
  +    }
  +
       public void doStart() throws WaitingException, Exception {
           dataSource = (DataSource) managedConnectionFactoryWrapper.getProxy();
       }
  @@ -76,7 +82,7 @@
       public void begin(Xid xid) throws LogException {
       }
   
  -    public void prepare(Xid xid, List branches) throws LogException {
  +    public long prepare(Xid xid, List branches) throws LogException {
           int formatId = xid.getFormatId();
           byte[] globalTransactionId = xid.getGlobalTransactionId();
           byte[] branchQualifier = xid.getBranchQualifier();
  @@ -107,9 +113,10 @@
           } catch (SQLException e) {
               throw new LogException("Failure during prepare or commit", e);
           }
  +        return 0L;
       }
   
  -    public void commit(Xid xid) throws LogException {
  +    public void commit(Xid xid, long logMark) throws LogException {
           try {
               Connection connection = dataSource.getConnection();
               try {
  @@ -134,40 +141,45 @@
           }
       }
   
  -    public void rollback(Xid xid) throws LogException {
  +    public void rollback(Xid xid, long logMark) throws LogException {
           throw new LogException("JDBCLog does not support rollback of prepared transactions.  Use it only on servers that do not import transactions");
       }
   
  -    public Map recover(XidFactory xidFactory) throws LogException {
  +    public Collection recover(XidFactory xidFactory) throws LogException {
           try {
               Connection connection = dataSource.getConnection();
               try {
  -                Map xids = new HashMap();
  +                Collection recovered = new ArrayList();
                   PreparedStatement ps = connection.prepareStatement(RECOVER);
  -                ps.setString(0, systemId);
  -                ResultSet rs = ps.executeQuery();
  -                Xid lastXid = null;
  -                Xid currentXid = null;
  -                List branches = new ArrayList();
  -                while (rs.next()) {
  -                    int formatId = rs.getInt(0);
  -                    byte[] globalId = rs.getBytes(1);
  -                    byte[] globalBranchId = rs.getBytes(2);
  -                    byte[] branchBranchId = rs.getBytes(3);
  -                    String name = rs.getString(4);
  -                    currentXid = xidFactory.recover(formatId, globalId, globalBranchId);
  -                    Xid branchXid = xidFactory.recover(formatId, globalId, branchBranchId);
  -                    if (!currentXid.equals(lastXid) && lastXid != null) {
  -                        addRecoveredXid(xids, lastXid, branches);
  -                        branches.clear();
  -                        lastXid = currentXid;
  +                try {
  +                    ps.setString(0, systemId);
  +                    ResultSet rs = ps.executeQuery();
  +                    try {
  +                        Xid lastXid = null;
  +                        Xid currentXid = null;
  +                        Recovery.XidBranchesPair xidBranchesPair = null;
  +                        while (rs.next()) {
  +                            int formatId = rs.getInt(0);
  +                            byte[] globalId = rs.getBytes(1);
  +                            byte[] globalBranchId = rs.getBytes(2);
  +                            byte[] branchBranchId = rs.getBytes(3);
  +                            String name = rs.getString(4);
  +                            currentXid = xidFactory.recover(formatId, globalId, globalBranchId);
  +                            Xid branchXid = xidFactory.recover(formatId, globalId, branchBranchId);
  +                            if (!currentXid.equals(lastXid)) {
  +                                xidBranchesPair = new Recovery.XidBranchesPair(currentXid, 0L);
  +                                recovered.add(xidBranchesPair);
  +                                lastXid = currentXid;
  +                            }
  +                            xidBranchesPair.addBranch(new TransactionBranchInfoImpl(branchXid, name));
  +                        }
  +                        return recovered;
  +                    } finally {
  +                        rs.close();
                       }
  -                    branches.add(new TransactionBranchInfoImpl(branchXid, name));
  -                }
  -                if (currentXid != null) {
  -                    addRecoveredXid(xids, currentXid, branches);
  +                } finally {
  +                    ps.close();
                   }
  -                return xids;
               } finally {
                   connection.close();
               }
  @@ -175,11 +187,6 @@
               throw new LogException("Recovery failure", e);
           }
   
  -    }
  -
  -    private void addRecoveredXid(Map xids, Xid xid, List names) {
  -        String[] nameArray = (String[])names.toArray(new String[names.size()]);
  -        xids.put(xid, nameArray);
       }
   
       public String getXMLStats() {
  
  
  
  1.7       +3 -3      incubator-geronimo/modules/transaction/src/test/org/apache/geronimo/transaction/log/AbstractLogTest.java
  
  Index: AbstractLogTest.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/transaction/src/test/org/apache/geronimo/transaction/log/AbstractLogTest.java,v
  retrieving revision 1.6
  retrieving revision 1.7
  diff -u -r1.6 -r1.7
  --- AbstractLogTest.java	22 Jun 2004 18:41:00 -0000	1.6
  +++ AbstractLogTest.java	22 Jul 2004 03:39:01 -0000	1.7
  @@ -130,11 +130,11 @@
   
                           for (int i = 0; i < xidCount; i++) {
                               // journalize COMMITTING record
  -                            logger.prepare(xid, names);
  +                            long logMark = logger.prepare(xid, names);
                               //localXidCount++;
   
                               // journalize FORGET record
  -                            logger.commit(xid);
  +                            logger.commit(xid, logMark);
                               localXidCount++;
                           }
                           localDuration = System.currentTimeMillis() - localStartTime;
  
  
  
  1.6       +96 -60    incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/log/HOWLLog.java
  
  Index: HOWLLog.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/log/HOWLLog.java,v
  retrieving revision 1.5
  retrieving revision 1.6
  diff -u -r1.5 -r1.6
  --- HOWLLog.java	18 Jul 2004 21:51:51 -0000	1.5
  +++ HOWLLog.java	22 Jul 2004 03:39:01 -0000	1.6
  @@ -19,12 +19,13 @@
   
   import java.io.IOException;
   import java.nio.ByteBuffer;
  +import java.util.Collection;
   import java.util.HashMap;
  -import java.util.HashSet;
   import java.util.Iterator;
   import java.util.List;
   import java.util.Map;
  -import java.util.Set;
  +import java.util.SortedSet;
  +import java.util.TreeSet;
   
   import javax.transaction.xa.Xid;
   
  @@ -36,11 +37,13 @@
   import org.apache.geronimo.gbean.WaitingException;
   import org.apache.geronimo.system.serverinfo.ServerInfo;
   import org.apache.geronimo.transaction.manager.LogException;
  +import org.apache.geronimo.transaction.manager.Recovery;
   import org.apache.geronimo.transaction.manager.TransactionBranchInfo;
   import org.apache.geronimo.transaction.manager.TransactionBranchInfoImpl;
   import org.apache.geronimo.transaction.manager.TransactionLog;
   import org.apache.geronimo.transaction.manager.XidFactory;
   import org.objectweb.howl.log.Configuration;
  +import org.objectweb.howl.log.InvalidLogKeyException;
   import org.objectweb.howl.log.LogClosedException;
   import org.objectweb.howl.log.LogConfigurationException;
   import org.objectweb.howl.log.LogFileOverflowException;
  @@ -72,6 +75,8 @@
       private final Configuration configuration = new Configuration();
       private boolean started = false;
   
  +    private final SortedSet marks = new TreeSet();
  +
       public HOWLLog(
               String bufferClassName,
               int bufferSize,
  @@ -225,7 +230,7 @@
       public void begin(Xid xid) throws LogException {
       }
   
  -    public void prepare(Xid xid, List branches) throws LogException {
  +    public long prepare(Xid xid, List branches) throws LogException {
           int branchCount = branches.size();
           byte[][] data = new byte[4 + 2 * branchCount][];
           data[0] = new byte[]{PREPARE};
  @@ -239,7 +244,9 @@
               data[i++] = transactionBranchInfo.getResourceName().getBytes();
           }
           try {
  -            logger.put(data, true);
  +            long logMark = logger.put(data, true);
  +            addMark(logMark);
  +            return logMark;
           } catch (LogClosedException e) {
               throw (IllegalStateException) new IllegalStateException().initCause(e);
           } catch (LogRecordSizeException e) {
  @@ -253,7 +260,13 @@
           }
       }
   
  -    public void commit(Xid xid) throws LogException {
  +    private void addMark(long logMark) {
  +        synchronized (marks) {
  +            marks.add(new Long(logMark));
  +        }
  +    }
  +
  +    public void commit(Xid xid, long logMark) throws LogException {
           byte[][] data = new byte[4][];
           data[0] = new byte[]{COMMIT};
           data[1] = intToBytes(xid.getFormatId());
  @@ -261,6 +274,10 @@
           data[3] = xid.getBranchQualifier();
           try {
               logger.put(data, false);
  +            boolean doMark = removeMark(logMark);
  +            if (doMark) {
  +                logger.mark(logMark);
  +            }
           } catch (LogClosedException e) {
               throw (IllegalStateException) new IllegalStateException().initCause(e);
           } catch (LogRecordSizeException e) {
  @@ -271,10 +288,12 @@
               throw (IllegalStateException) new IllegalStateException().initCause(e);
           } catch (IOException e) {
               throw new LogException(e);
  +        } catch (InvalidLogKeyException e) {
  +            throw new LogException(e);
           }
       }
   
  -    public void rollback(Xid xid) throws LogException {
  +    public void rollback(Xid xid, long logMark) throws LogException {
           byte[][] data = new byte[4][];
           data[0] = new byte[]{ROLLBACK};
           data[1] = intToBytes(xid.getFormatId());
  @@ -282,6 +301,10 @@
           data[3] = xid.getBranchQualifier();
           try {
               logger.put(data, false);
  +            boolean doMark = removeMark(logMark);
  +            if (doMark) {
  +                logger.mark(logMark);
  +            }
           } catch (LogClosedException e) {
               throw (IllegalStateException) new IllegalStateException().initCause(e);
           } catch (LogRecordSizeException e) {
  @@ -292,20 +315,32 @@
               throw (IllegalStateException) new IllegalStateException().initCause(e);
           } catch (IOException e) {
               throw new LogException(e);
  +        } catch (InvalidLogKeyException e) {
  +            throw new LogException(e);
           }
       }
   
  -    public Map recover(XidFactory xidFactory) throws LogException {
  +    private boolean removeMark(long logMark) {
  +        boolean doMark = false;
  +        Long mark = new Long(logMark);
  +        synchronized (marks) {
  +            doMark = (mark.equals(marks.first()));
  +            marks.remove(mark);
  +        }
  +        return doMark;
  +    }
  +
  +    public Collection recover(XidFactory xidFactory) throws LogException {
           log.info("Initiating transaction manager recovery");
           Map recovered = new HashMap();
  -        ReplayListener replayListener = new GeronimoReplayListener(xidFactory, recovered, log);
  +        ReplayListener replayListener = new GeronimoReplayListener(xidFactory, recovered);
           try {
               logger.replay(replayListener);
           } catch (LogConfigurationException e) {
               throw new LogException(e);
           }
           log.info("In doubt transactions recovered from log");
  -        return recovered;
  +        return recovered.values();
       }
   
       public String getXMLStats() {
  @@ -329,70 +364,71 @@
           return buffer;
       }
   
  -    private static class GeronimoReplayListener implements ReplayListener {
  +    private class GeronimoReplayListener implements ReplayListener {
   
           private final XidFactory xidFactory;
           private final Map recoveredTx;
  -        private final Log log;
   
  -        public GeronimoReplayListener(XidFactory xidFactory, Map recoveredTx, Log log) {
  +        public GeronimoReplayListener(XidFactory xidFactory, Map recoveredTx) {
               this.xidFactory = xidFactory;
               this.recoveredTx = recoveredTx;
  -            this.log = log;
           }
   
           public void onRecord(LogRecord lr) {
               short recordType = lr.type;
  -            if (recordType != LogRecordType.USER) {
  -                if (recordType != LogRecordType.END_OF_LOG) {
  -                    log.warn("Received unexpected log record: " + lr);
  +            long logMark = lr.key;
  +            if (recordType == LogRecordType.USER) {
  +                ByteBuffer raw = lr.dataBuffer;
  +                if (raw.remaining() == 0) {
  +                    log.warn("Received empty log record of user type!");
  +                    return;
                   }
  -                return;
  -            }
  -            ByteBuffer raw = lr.dataBuffer;
  -            if (raw.remaining() == 0) {
  -                log.warn("Received empty log record of user type!");
  -                return;
  -            }
  -            //type (PREPARE etc)
  -            short size = raw.getShort();
  -            assert size == 1;
  -            byte type = raw.get();
  -            //format id integer
  -            size = raw.getShort();
  -            assert size == 4;
  -            int formatId = raw.getInt();
  -            //global id
  -            int globalIdLength = raw.getShort();
  -            byte[] globalId = new byte[globalIdLength];
  -            raw.get(globalId);
  -            //branch qualifier for master xid
  -            int branchIdLength = raw.getShort();
  -            byte[] branchId = new byte[branchIdLength];
  -            raw.get(branchId);
  -            Xid masterXid = xidFactory.recover(formatId, globalId, branchId);
  -            if (type == PREPARE) {
  -                Set branches = new HashSet();
  -                recoveredTx.put(masterXid, branches);
  +                //type (PREPARE etc)
  +                short size = raw.getShort();
  +                assert size == 1;
  +                byte type = raw.get();
  +                //format id integer
  +                size = raw.getShort();
  +                assert size == 4;
  +                int formatId = raw.getInt();
  +                //global id
  +                int globalIdLength = raw.getShort();
  +                byte[] globalId = new byte[globalIdLength];
  +                raw.get(globalId);
  +                //branch qualifier for master xid
  +                int branchIdLength = raw.getShort();
  +                byte[] branchId = new byte[branchIdLength];
  +                raw.get(branchId);
  +                Xid masterXid = xidFactory.recover(formatId, globalId, branchId);
  +                if (type == PREPARE) {
  +                    Recovery.XidBranchesPair xidBranchesPair = new Recovery.XidBranchesPair(masterXid, logMark);
  +                    recoveredTx.put(masterXid, xidBranchesPair);
  +                    addMark(logMark);
   //                log.info("recovered prepare record for master xid: " + masterXid);
  -                while (raw.hasRemaining()) {
  -                    int branchBranchIdLength = raw.getShort();
  -                    byte[] branchBranchId = new byte[branchBranchIdLength];
  -                    raw.get(branchBranchId);
  -                    Xid branchXid = xidFactory.recover(formatId, globalId, branchBranchId);
  -                    int nameLength = raw.getShort();
  -                    byte[] nameBytes = new byte[nameLength];
  -                    raw.get(nameBytes);
  -                    String name = new String(nameBytes);
  -                    TransactionBranchInfoImpl branchInfo = new TransactionBranchInfoImpl(branchXid, name);
  -                    branches.add(branchInfo);
  +                    while (raw.hasRemaining()) {
  +                        int branchBranchIdLength = raw.getShort();
  +                        byte[] branchBranchId = new byte[branchBranchIdLength];
  +                        raw.get(branchBranchId);
  +                        Xid branchXid = xidFactory.recover(formatId, globalId, branchBranchId);
  +                        int nameLength = raw.getShort();
  +                        byte[] nameBytes = new byte[nameLength];
  +                        raw.get(nameBytes);
  +                        String name = new String(nameBytes);
  +                        TransactionBranchInfoImpl branchInfo = new TransactionBranchInfoImpl(branchXid, name);
  +                        xidBranchesPair.addBranch(branchInfo);
   //                    log.info("recovered branch for resource manager, branchId " + name + ", " + branchXid);
  -                }
  -            } else if (type == COMMIT || type == ROLLBACK) {
  -                Object o = recoveredTx.remove(masterXid);
  +                    }
  +                } else if (type == COMMIT || type == ROLLBACK) {
  +                    recoveredTx.remove(masterXid);
  +                    removeMark(logMark);
   //                log.info("Recovered " + TYPE_NAMES[type] + " for xid: " + masterXid + " and branches: " + o);
  -            } else {
  -                log.error("Unknown recovery record received, type byte: " + type + ", buffer: " + raw);
  +                } else {
  +                    log.error("Unknown recovery record received, type byte: " + type + ", buffer: " + raw);
  +                }
  +            } else if (recordType == LogRecordType.MARKKEY) {
  +
  +            } else if (recordType != LogRecordType.END_OF_LOG) {
  +                log.warn("Received unexpected log record: " + lr);
               }
           }
   
  
  
  
  1.7       +9 -8      incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/log/UnrecoverableLog.java
  
  Index: UnrecoverableLog.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/log/UnrecoverableLog.java,v
  retrieving revision 1.6
  retrieving revision 1.7
  diff -u -r1.6 -r1.7
  --- UnrecoverableLog.java	11 Jun 2004 19:20:55 -0000	1.6
  +++ UnrecoverableLog.java	22 Jul 2004 03:39:01 -0000	1.7
  @@ -17,8 +17,8 @@
   
   package org.apache.geronimo.transaction.log;
   
  -import java.util.HashMap;
  -import java.util.Map;
  +import java.util.ArrayList;
  +import java.util.Collection;
   import java.util.List;
   
   import javax.transaction.xa.Xid;
  @@ -38,17 +38,18 @@
       public void begin(Xid xid) throws LogException {
       }
   
  -    public void prepare(Xid xid, List branches) throws LogException {
  +    public long prepare(Xid xid, List branches) throws LogException {
  +        return 0L;
       }
   
  -    public void commit(Xid xid) throws LogException {
  +    public void commit(Xid xid, long logMark) throws LogException {
       }
   
  -    public void rollback(Xid xid) throws LogException {
  +    public void rollback(Xid xid, long logMark) throws LogException {
       }
   
  -    public Map recover(XidFactory xidFactory) throws LogException {
  -        return new HashMap();
  +    public Collection recover(XidFactory xidFactory) throws LogException {
  +        return new ArrayList();
       }
   
       public String getXMLStats() {
  
  
  
  1.5       +32 -1     incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/Recovery.java
  
  Index: Recovery.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/Recovery.java,v
  retrieving revision 1.4
  retrieving revision 1.5
  diff -u -r1.4 -r1.5
  --- Recovery.java	11 Jun 2004 19:20:55 -0000	1.4
  +++ Recovery.java	22 Jul 2004 03:39:01 -0000	1.5
  @@ -25,6 +25,7 @@
   import java.util.ArrayList;
   import java.util.Collections;
   import java.util.Arrays;
  +import java.util.HashSet;
   
   import javax.transaction.SystemException;
   import javax.transaction.xa.XAException;
  @@ -58,5 +59,35 @@
   //    boolean remoteRecoveryComplete();
   
       Map getExternalXids();
  +
  +    public static class XidBranchesPair {
  +        private final Xid xid;
  +
  +        //set of TransactionBranchInfo
  +        private final Set branches = new HashSet();
  +
  +        private final long mark;
  +
  +        public XidBranchesPair(Xid xid, long mark) {
  +            this.xid = xid;
  +            this.mark = mark;
  +        }
  +
  +        public Xid getXid() {
  +            return xid;
  +        }
  +
  +        public Set getBranches() {
  +            return branches;
  +        }
  +
  +        public long getMark() {
  +            return mark;
  +        }
  +
  +        public void addBranch(TransactionBranchInfo branchInfo) {
  +            branches.add(branchInfo);
  +        }
  +    }
   
   }
  
  
  
  1.3       +13 -26    incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/RecoveryImpl.java
  
  Index: RecoveryImpl.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/RecoveryImpl.java,v
  retrieving revision 1.2
  retrieving revision 1.3
  diff -u -r1.2 -r1.3
  --- RecoveryImpl.java	19 Jun 2004 17:17:13 -0000	1.2
  +++ RecoveryImpl.java	22 Jul 2004 03:39:01 -0000	1.3
  @@ -26,6 +26,7 @@
   import java.util.Collections;
   import java.util.Arrays;
   import java.util.HashSet;
  +import java.util.Collection;
   
   import javax.transaction.SystemException;
   import javax.transaction.xa.XAException;
  @@ -60,20 +61,18 @@
       }
   
       public synchronized void recoverLog() throws XAException {
  -        Map preparedXids = null;
  +        Collection preparedXids = null;
           try {
               preparedXids = txLog.recover(xidFactory);
           } catch (LogException e) {
               throw (XAException) new XAException(XAException.XAER_RMERR).initCause(e);
           }
  -        for (Iterator iterator = preparedXids.entrySet().iterator(); iterator.hasNext();) {
  -            Map.Entry entry = (Map.Entry) iterator.next();
  -            Xid xid = (Xid) entry.getKey();
  +        for (Iterator iterator = preparedXids.iterator(); iterator.hasNext();) {
  +            XidBranchesPair xidBranchesPair = (Recovery.XidBranchesPair) iterator.next();
  +            Xid xid = xidBranchesPair.getXid();
               if (xidFactory.matchesGlobalId(xid.getGlobalTransactionId())) {
  -                Object o = entry.getValue();
  -                XidBranchesPair xidBranchesPair = new XidBranchesPair(xid, (Set) entry.getValue());
                   ourXids.put(new ByteArrayWrapper(xid.getGlobalTransactionId()), xidBranchesPair);
  -                for (Iterator branches = xidBranchesPair.branches.iterator(); branches.hasNext();) {
  +                for (Iterator branches = xidBranchesPair.getBranches().iterator(); branches.hasNext();) {
                       String name = ((TransactionBranchInfo) branches.next()).getResourceName();
                       Set transactionsForName = (Set)nameToOurTxMap.get(name);
                       if (transactionsForName == null) {
  @@ -82,8 +81,7 @@
                       transactionsForName.add(xidBranchesPair);
                   }
               } else {
  -                Object o = entry.getValue();
  -                TransactionImpl externalTx = new ExternalTransaction(xid, txLog, (Set) entry.getValue());
  +                TransactionImpl externalTx = new ExternalTransaction(xid, txLog, xidBranchesPair.getBranches());
                   externalXids.put(xid, externalTx);
                   externalGlobalIdMap.put(xid.getGlobalTransactionId(), externalTx);
               }
  @@ -143,7 +141,7 @@
   
       private void removeNameFromTransaction(XidBranchesPair xidBranchesPair, String name, boolean warn) {
           int removed = 0;
  -        for (Iterator branches = xidBranchesPair.branches.iterator(); branches.hasNext();) {
  +        for (Iterator branches = xidBranchesPair.getBranches().iterator(); branches.hasNext();) {
               TransactionBranchInfo transactionBranchInfo = (TransactionBranchInfo) branches.next();
               if (name.equals(transactionBranchInfo.getResourceName())) {
                   branches.remove();
  @@ -151,12 +149,12 @@
               }
           }
           if (warn && removed == 0) {
  -            log.error("XAResource named: " + name + " returned branch xid for xid: " + xidBranchesPair.xid + " but was not registered with that transaction!");
  +            log.error("XAResource named: " + name + " returned branch xid for xid: " + xidBranchesPair.getXid() + " but was not registered with that transaction!");
           }
  -        if (xidBranchesPair.branches.isEmpty()) {
  +        if (xidBranchesPair.getBranches().isEmpty()) {
               try {
  -                ourXids.remove(new ByteArrayWrapper(xidBranchesPair.xid.getGlobalTransactionId()));
  -                txLog.commit(xidBranchesPair.xid);
  +                ourXids.remove(new ByteArrayWrapper(xidBranchesPair.getXid().getGlobalTransactionId()));
  +                txLog.commit(xidBranchesPair.getXid(), xidBranchesPair.getMark());
               } catch (LogException e) {
                   recoveryErrors.add(e);
                   log.error(e);
  @@ -186,17 +184,6 @@
   
       public synchronized Map getExternalXids() {
           return new HashMap(externalXids);
  -    }
  -
  -    private static class XidBranchesPair {
  -        private final Xid xid;
  -        //set of TransactionBranchInfo
  -        private final Set branches;
  -
  -        public XidBranchesPair(Xid xid, Set branches) {
  -            this.xid = xid;
  -            this.branches = branches;
  -        }
       }
   
       private static class ByteArrayWrapper {
  
  
  
  1.8       +5 -4      incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/TransactionImpl.java
  
  Index: TransactionImpl.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/TransactionImpl.java,v
  retrieving revision 1.7
  retrieving revision 1.8
  diff -u -r1.7 -r1.8
  --- TransactionImpl.java	11 Jun 2004 19:20:55 -0000	1.7
  +++ TransactionImpl.java	22 Jul 2004 03:39:01 -0000	1.8
  @@ -54,6 +54,7 @@
       private List syncList = new ArrayList(5);
       private LinkedList resourceManagers = new LinkedList();
       private Map xaResources = new HashMap(3);
  +    private long logMark;
   
       TransactionImpl(XidFactory xidFactory, TransactionLog txnLog) throws SystemException {
           this(xidFactory.createXid(), xidFactory, txnLog);
  @@ -357,7 +358,7 @@
           // log our decision
           if (willCommit) {
               try {
  -                txnLog.prepare(xid, resourceManagers);
  +                logMark = txnLog.prepare(xid, resourceManagers);
               } catch (LogException e) {
                   try {
                       rollbackResources(resourceManagers);
  @@ -390,7 +391,7 @@
           try {
               rollbackResources(rms);
               try {
  -                txnLog.rollback(xid);
  +                txnLog.rollback(xid, logMark);
               } catch (LogException e) {
                   try {
                       rollbackResources(rms);
  @@ -511,7 +512,7 @@
               }
           }
           try {
  -            txnLog.commit(xid);
  +            txnLog.commit(xid, logMark);
           } catch (LogException e) {
               log.error("Unexpected exception logging commit completion for xid " + xid, e);
               throw (SystemException)new SystemException("Unexpected error logging commit completion for xid " + xid).initCause(e);
  
  
  
  1.8       +6 -6      incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/TransactionLog.java
  
  Index: TransactionLog.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/TransactionLog.java,v
  retrieving revision 1.7
  retrieving revision 1.8
  diff -u -r1.7 -r1.8
  --- TransactionLog.java	12 Jun 2004 18:40:52 -0000	1.7
  +++ TransactionLog.java	22 Jul 2004 03:39:01 -0000	1.8
  @@ -17,7 +17,7 @@
   
   package org.apache.geronimo.transaction.manager;
   
  -import java.util.Map;
  +import java.util.Collection;
   import java.util.List;
   
   import javax.transaction.xa.Xid;
  @@ -38,11 +38,11 @@
        * @param branches List of TransactionBranchInfo
        * @throws LogException
        */
  -    void prepare(Xid xid, List branches) throws LogException;
  +    long prepare(Xid xid, List branches) throws LogException;
   
  -    void commit(Xid xid) throws LogException;
  +    void commit(Xid xid, long logMark) throws LogException;
   
  -    void rollback(Xid xid) throws LogException;
  +    void rollback(Xid xid, long logMark) throws LogException;
   
       /**
        * Recovers the log, returning a map of (top level) xid to List of TransactionBranchInfo for the branches.
  @@ -52,7 +52,7 @@
        * @return Map of recovered xid to List of TransactionBranchInfo representing the branches.
        * @throws LogException
        */
  -    Map recover(XidFactory xidFactory) throws LogException;
  +    Collection recover(XidFactory xidFactory) throws LogException;
   
       String getXMLStats();
   
  
  
  
  1.3       +15 -18    incubator-geronimo/modules/transaction/src/test/org/apache/geronimo/transaction/manager/MockLog.java
  
  Index: MockLog.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/transaction/src/test/org/apache/geronimo/transaction/manager/MockLog.java,v
  retrieving revision 1.2
  retrieving revision 1.3
  diff -u -r1.2 -r1.3
  --- MockLog.java	11 Jun 2004 19:20:55 -0000	1.2
  +++ MockLog.java	22 Jul 2004 03:39:01 -0000	1.3
  @@ -17,13 +17,11 @@
   
   package org.apache.geronimo.transaction.manager;
   
  -import java.util.Map;
  -import java.util.HashMap;
   import java.util.ArrayList;
  +import java.util.Collection;
  +import java.util.HashMap;
   import java.util.List;
  -import java.util.Iterator;
  -import java.util.HashSet;
  -import java.util.Arrays;
  +import java.util.Map;
   
   import javax.transaction.xa.Xid;
   
  @@ -42,27 +40,26 @@
       public void begin(Xid xid) throws LogException {
       }
   
  -    public void prepare(Xid xid, List branches) throws LogException {
  -        prepared.put(xid, new HashSet(branches));
  +    public long prepare(Xid xid, List branches) throws LogException {
  +        Recovery.XidBranchesPair xidBranchesPair = new Recovery.XidBranchesPair(xid, 0L);
  +        xidBranchesPair.getBranches().addAll(branches);
  +        prepared.put(xid, xidBranchesPair);
  +        return 0L;
       }
   
  -    public void commit(Xid xid) throws LogException {
  +    public void commit(Xid xid, long logMark) throws LogException {
           committed.add(xid);
       }
   
  -    public void rollback(Xid xid) throws LogException {
  +    public void rollback(Xid xid, long logMark) throws LogException {
           rolledBack.add(xid);
       }
   
  -    public Map recover(XidFactory xidFactory) throws LogException {
  +    public Collection recover(XidFactory xidFactory) throws LogException {
           Map copy = new HashMap(prepared);
  -        for (Iterator iterator = committed.iterator(); iterator.hasNext();) {
  -            copy.remove(iterator.next());
  -        }
  -        for (Iterator iterator = rolledBack.iterator(); iterator.hasNext();) {
  -            copy.remove(iterator.next());
  -        }
  -        return copy;
  +        copy.keySet().removeAll(committed);
  +        copy.keySet().removeAll(rolledBack);
  +        return copy.values();
       }
   
       public String getXMLStats() {
  
  
  
  1.8       +10 -6     incubator-geronimo/modules/transaction/src/test/org/apache/geronimo/transaction/manager/XATransactionTester.java
  
  Index: XATransactionTester.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/transaction/src/test/org/apache/geronimo/transaction/manager/XATransactionTester.java,v
  retrieving revision 1.7
  retrieving revision 1.8
  diff -u -r1.7 -r1.8
  --- XATransactionTester.java	19 Jun 2004 17:17:13 -0000	1.7
  +++ XATransactionTester.java	22 Jul 2004 03:39:01 -0000	1.8
  @@ -21,6 +21,9 @@
   import java.sql.Statement;
   import java.util.Map;
   import java.util.List;
  +import java.util.Set;
  +import java.util.Collection;
  +import java.util.ArrayList;
   
   import javax.sql.XAConnection;
   import javax.sql.XADataSource;
  @@ -106,17 +109,18 @@
               XATransactionTester.this.xid = xid;
           }
   
  -        public void prepare(Xid xid, List branches) throws LogException {
  +        public long prepare(Xid xid, List branches) throws LogException {
  +            return 0L;
           }
   
  -        public void commit(Xid xid) throws LogException {
  +        public void commit(Xid xid, long logMark) throws LogException {
           }
   
  -        public void rollback(Xid xid) throws LogException {
  +        public void rollback(Xid xid, long logMark) throws LogException {
           }
   
  -        public Map recover(XidFactory xidFactory) throws LogException {
  -            return null;
  +        public Collection recover(XidFactory xidFactory) throws LogException {
  +            return new ArrayList();
           }
   
           public String getXMLStats() {