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/05/06 06:00:51 UTC

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

djencks     2004/05/05 21:00:51

  Modified:    modules/transaction project.xml
               modules/transaction/src/java/org/apache/geronimo/transaction
                        TransactionManagerProxy.java
               modules/transaction/src/java/org/apache/geronimo/transaction/log
                        UnrecoverableLog.java
               modules/transaction/src/java/org/apache/geronimo/transaction/manager
                        TransactionImpl.java TransactionLog.java
                        TransactionManagerImpl.java XidFactory.java
                        XidImpl.java XidImporter.java
               modules/transaction/src/test/org/apache/geronimo/transaction/manager
                        XATransactionTester.java XidImporterTest.java
  Added:       modules/transaction/src/java/org/apache/geronimo/transaction/log
                        XidImpl2.java
               modules/transaction/src/java/org/apache/geronimo/transaction/manager
                        LogException.java Recovery.java XidFactoryImpl.java
               modules/transaction/src/test/org/apache/geronimo/transaction/log
                        AbstractLogTest.java XidSpeedTest.java
  Log:
  Preliminary work towards actual transaction logging and recovery
  
  Revision  Changes    Path
  1.14      +19 -4     incubator-geronimo/modules/transaction/project.xml
  
  Index: project.xml
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/transaction/project.xml,v
  retrieving revision 1.13
  retrieving revision 1.14
  diff -u -r1.13 -r1.14
  --- project.xml	1 May 2004 06:40:20 -0000	1.13
  +++ project.xml	6 May 2004 04:00:51 -0000	1.14
  @@ -6,16 +6,16 @@
       Licensed 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.
   -->
  -  
  +
   
   <!-- $Revision$ $Date$ -->
   
  @@ -75,6 +75,21 @@
               <version>1.0.3</version>
               <url>http://jakarta.apache.org/commons/logging/</url>
           </dependency>
  +
  +        <!--this is an experimental version compiled by David Jencks-->
  +        <!--dependency>
  +             <groupId>howl</groupId>
  +            <artifactId>howl-logger</artifactId>
  +            <version>SNAPSHOT</version>
  +            <url>http://www.objectweb.org</url>
  +        </dependency-->
  +
  +        <!--dependency>
  +            <groupId>pyrasun</groupId>
  +            <artifactId>PyrasunLog</artifactId>
  +            <version>0.2_Alpha</version>
  +            <url>http://www.krisnmike.com/mike</url>
  +        </dependency-->
   
           <dependency>
               <groupId>tranql</groupId>
  
  
  
  1.8       +27 -5     incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/TransactionManagerProxy.java
  
  Index: TransactionManagerProxy.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/TransactionManagerProxy.java,v
  retrieving revision 1.7
  retrieving revision 1.8
  diff -u -r1.7 -r1.8
  --- TransactionManagerProxy.java	12 Mar 2004 17:55:34 -0000	1.7
  +++ TransactionManagerProxy.java	6 May 2004 04:00:51 -0000	1.8
  @@ -34,6 +34,7 @@
   import javax.transaction.TransactionManager;
   import javax.transaction.xa.XAException;
   import javax.transaction.xa.Xid;
  +import javax.transaction.xa.XAResource;
   
   import org.apache.geronimo.gbean.GBeanInfo;
   import org.apache.geronimo.gbean.GBeanInfoFactory;
  @@ -42,6 +43,7 @@
   import org.apache.geronimo.gbean.GAttributeInfo;
   import org.apache.geronimo.transaction.manager.TransactionManagerImpl;
   import org.apache.geronimo.transaction.manager.XidImporter;
  +import org.apache.geronimo.transaction.manager.Recovery;
   
   /**
    * A wrapper for a TransactionManager that wraps all Transactions in a TransactionProxy
  @@ -60,6 +62,10 @@
       private final ThreadLocal threadTx = new ThreadLocal();
       private final Map importedTransactions = new HashMap();
       private Set activeTransactions = new HashSet();
  +    private boolean recoveryState = NOT_IN_RECOVERY;
  +    private static final boolean NOT_IN_RECOVERY = false;
  +    private static final boolean IN_RECOVERY = true;
  +    private Recovery recovery;
   
       /**
        * Constructor taking the TransactionManager to wrap.
  @@ -75,7 +81,7 @@
        */
       public TransactionManagerProxy() {
           this.delegate = new TransactionManagerImpl();
  -        this.importer = (XidImporter)delegate;
  +        this.importer = (XidImporter) delegate;
       }
   
       public void setTransactionTimeout(int timeout) throws SystemException {
  @@ -201,8 +207,24 @@
       /**
        * @see javax.resource.spi.XATerminator#recover(int)
        */
  -    public Xid[] recover(int arg0) throws XAException {
  -        throw new XAException("Not implemented.");
  +    public Xid[] recover(int flag) throws XAException {
  +        if (recoveryState == NOT_IN_RECOVERY) {
  +            if ((flag & XAResource.TMSTARTRSCAN) == 0) {
  +                throw new XAException(XAException.XAER_PROTO);
  +            }
  +            recoveryState = IN_RECOVERY;
  +        }
  +        if ((flag & XAResource.TMENDRSCAN) != 0) {
  +            recoveryState = NOT_IN_RECOVERY;
  +        }
  +        //we always return all xids in first call.
  +        //calling "startrscan" repeatedly starts at beginning of list again.
  +        if ((flag & XAResource.TMSTARTRSCAN) != 0) {
  +            importedTransactions.putAll(recovery.getExternalXids());
  +            return (Xid[]) importedTransactions.keySet().toArray(new Xid[importedTransactions.size()]);
  +        } else {
  +            return new Xid[0];
  +        }
       }
   
       /**
  @@ -229,7 +251,7 @@
               try {
                   tx = new TransactionProxy(importer.importXid(xid));
               } catch (SystemException e) {
  -                throw (XAException)new XAException("Could not import xid").initCause(e);
  +                throw (XAException) new XAException("Could not import xid").initCause(e);
               }
               importedTransactions.put(xid, tx);
           }
  
  
  
  1.4       +25 -5     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.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- UnrecoverableLog.java	10 Mar 2004 09:59:37 -0000	1.3
  +++ UnrecoverableLog.java	6 May 2004 04:00:51 -0000	1.4
  @@ -18,9 +18,13 @@
   package org.apache.geronimo.transaction.log;
   
   import java.io.IOException;
  +import java.util.List;
  +import java.util.ArrayList;
  +
   import javax.transaction.xa.Xid;
   
   import org.apache.geronimo.transaction.manager.TransactionLog;
  +import org.apache.geronimo.transaction.manager.LogException;
   
   /**
    * A log sink that doesn't actually do anything.
  @@ -30,15 +34,31 @@
    * @version $Revision$ $Date$
    */
   public class UnrecoverableLog implements TransactionLog {
  -    public void begin(Xid xid) throws IOException {
  +    public void begin(Xid xid) throws LogException {
  +    }
  +
  +    public void prepare(Xid xid) throws LogException {
  +    }
  +
  +    public void commit(Xid xid) throws LogException {
  +    }
  +
  +    public void rollback(Xid xid) throws LogException {
  +    }
  +
  +    public List recover() throws LogException {
  +        return new ArrayList();
       }
   
  -    public void prepare(Xid xid) throws IOException {
  +    public String getXMLStats() {
  +        return null;
       }
   
  -    public void commit(Xid xid) throws IOException {
  +    public int getAverageForceTime() {
  +        return 0;
       }
   
  -    public void rollback(Xid xid) throws IOException {
  +    public int getAverageBytesPerForce() {
  +        return 0;
       }
   }
  
  
  
  1.1                  incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/log/XidImpl2.java
  
  Index: XidImpl2.java
  ===================================================================
  /**
   *
   * Copyright 2003-2004 The Apache Software Foundation
   *
   *  Licensed 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.log;
  
  import java.io.Serializable;
  import java.util.Arrays;
  
  import javax.transaction.xa.Xid;
  
  import pyrasun.binlog.LogEntryKey;
  
  /**
   * Unique id for a transaction.  This implementation is backed by a single byte buffer
   * so can do less copying than one backed by several byte buffers for the different components.
   *
   * @version $Revision: 1.1 $ $Date: 2004/05/06 04:00:51 $
   */
  public class XidImpl2 implements Xid, Serializable {
      private static int HEADER_SIZE = 4;
      private static int ACTION_POS = 0;
      private static int GLOBALID_SIZE_POS = 1;
      private static int BRANCHID_SIZE_POS = 2;
      //3 unused
      private static int FORMAT_ID = 0x4765526f;  // Gero
      private static int FORMAT_SIZE = 4;
  
      private static byte[] FORMAT_ID_BYTES = "Gero".getBytes();
  
      private final byte[] buffer = new byte[HEADER_SIZE + FORMAT_SIZE + Xid.MAXGTRIDSIZE + Xid.MAXBQUALSIZE];
      private int hash;
      private Object key;
  
      /**
       * Constructor taking a global id (for the main transaction)
       * @param globalId the global transaction id
       */
      public XidImpl2(byte[] globalId) {
          System.arraycopy(FORMAT_ID_BYTES, 0, buffer, HEADER_SIZE, FORMAT_SIZE);
          buffer[GLOBALID_SIZE_POS] = (byte)globalId.length;
          System.arraycopy(globalId, 0, buffer, HEADER_SIZE + FORMAT_SIZE, Xid.MAXGTRIDSIZE);
  
          //this.hash = hash(buffer);
      }
  
      /**
       * Constructor for a branch id
       * @param global the xid of the global transaction this branch belongs to
       * @param branch the branch id
       */
      public XidImpl2(Xid global, byte[] branch) {
          if (global instanceof XidImpl2) {
              System.arraycopy(((XidImpl2)global).buffer, 0, buffer, 0, HEADER_SIZE +FORMAT_SIZE + Xid.MAXGTRIDSIZE);
          } else {
              System.arraycopy(FORMAT_ID_BYTES, 0, buffer, HEADER_SIZE, FORMAT_SIZE);
              byte[] globalId = global.getGlobalTransactionId();
              System.arraycopy(globalId, 0, buffer, HEADER_SIZE + FORMAT_SIZE, globalId.length);
          }
          buffer[BRANCHID_SIZE_POS] = (byte)branch.length;
          System.arraycopy(branch, 0, buffer, HEADER_SIZE + FORMAT_SIZE + Xid.MAXGTRIDSIZE, Xid.MAXBQUALSIZE);
          //hash = hash(buffer);
      }
  
      private int hash(byte[] id) {
          int hash = 0;
          for (int i = 0; i < id.length; i++) {
              hash = (hash * 37) + id[i];
          }
          return hash;
      }
  
      public int getFormatId() {
          return FORMAT_ID;
      }
  
      public byte[] getGlobalTransactionId() {
          byte[] globalId = new byte[buffer[GLOBALID_SIZE_POS]];
          System.arraycopy(buffer, HEADER_SIZE + FORMAT_SIZE, globalId, 0, buffer[GLOBALID_SIZE_POS]);
          return globalId;
      }
  
      public byte[] getBranchQualifier() {
          byte[] branchId = new byte[buffer[BRANCHID_SIZE_POS]];
          System.arraycopy(buffer, HEADER_SIZE + FORMAT_SIZE + Xid.MAXGTRIDSIZE, branchId, 0, buffer[BRANCHID_SIZE_POS]);
          return branchId;
      }
  
      public boolean equals(Object obj) {
          if (obj instanceof XidImpl2 == false) {
              return false;
          }
          XidImpl2 other = (XidImpl2) obj;
          return Arrays.equals(buffer, other.buffer);
      }
  
      public int hashCode() {
          if (hash == 0) {
              hash = hash(buffer);
          }
          return hash;
      }
  
      public String toString() {
          StringBuffer s = new StringBuffer("[formatId=Gero,");
          s.append("globalId=");
          for (int i = FORMAT_SIZE; i < FORMAT_SIZE + Xid.MAXGTRIDSIZE; i++) {
              s.append(Integer.toHexString(buffer[i]));
          }
          s.append(",branchId=");
          for (int i = FORMAT_SIZE + Xid.MAXGTRIDSIZE; i < buffer.length; i++) {
              s.append(Integer.toHexString(buffer[i]));
          }
          s.append("]");
          return s.toString();
      }
  
      byte[] getBuffer(byte action) {
          buffer[ACTION_POS] = action;
          return buffer;
      }
  
      public void setKey(Object key) {
          this.key = key;
      }
  
      public Object getKey() {
          return key;
      }
  }
  
  
  
  1.5       +20 -9     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.4
  retrieving revision 1.5
  diff -u -r1.4 -r1.5
  --- TransactionImpl.java	10 Mar 2004 09:59:37 -0000	1.4
  +++ TransactionImpl.java	6 May 2004 04:00:51 -0000	1.5
  @@ -17,7 +17,6 @@
   
   package org.apache.geronimo.transaction.manager;
   
  -import java.io.IOException;
   import java.util.ArrayList;
   import java.util.HashMap;
   import java.util.Iterator;
  @@ -66,7 +65,7 @@
           this.xid = xid;
           try {
               txnLog.begin(xid);
  -        } catch (IOException e) {
  +        } catch (LogException e) {
               status = Status.STATUS_MARKED_ROLLBACK;
               SystemException ex = new SystemException("Error logging begin; transaction marked for roll back)");
               ex.initCause(e);
  @@ -75,6 +74,14 @@
           status = Status.STATUS_ACTIVE;
       }
   
  +    //reconstruct a tx for an external tx found in recovery
  +    public TransactionImpl(Xid xid, TransactionLog txLog) {
  +        this.xidFactory = null;
  +        this.txnLog = txLog;
  +        this.xid = xid;
  +        status = Status.STATUS_PREPARED;
  +    }
  +
       public synchronized int getStatus() throws SystemException {
           return status;
       }
  @@ -148,9 +155,7 @@
   
               Xid branchId = xidFactory.createBranch(xid, resourceManagers.size() + 1);
               xaRes.start(branchId, XAResource.TMNOFLAGS);
  -            ResourceManager manager = new ResourceManager(xaRes, branchId);
  -            resourceManagers.add(manager);
  -            xaResources.put(xaRes, manager);
  +            addBranchXid(xaRes,  branchId);
               return true;
           } catch (XAException e) {
               log.warn("Unable to enlist XAResource " + xaRes, e);
  @@ -321,7 +326,7 @@
       private boolean internalPrepare() throws SystemException {
           try {
               txnLog.prepare(xid);
  -        } catch (IOException e) {
  +        } catch (LogException e) {
               try {
                   rollbackResources(resourceManagers);
               } catch (Exception se) {
  @@ -366,7 +371,7 @@
               } else {
                   txnLog.rollback(xid);
               }
  -        } catch (IOException e) {
  +        } catch (LogException e) {
               try {
                   rollbackResources(resourceManagers);
               } catch (Exception se) {
  @@ -397,7 +402,7 @@
           try {
               try {
                   txnLog.rollback(xid);
  -            } catch (IOException e) {
  +            } catch (LogException e) {
                   try {
                       rollbackResources(rms);
                   } catch (Exception se) {
  @@ -559,6 +564,12 @@
           } else {
               return false;
           }
  +    }
  +
  +    public void addBranchXid(XAResource xaRes, Xid branchId) {
  +        ResourceManager manager = new ResourceManager(xaRes, branchId);
  +        resourceManagers.add(manager);
  +        xaResources.put(xaRes, manager);
       }
   
   
  
  
  
  1.4       +17 -5     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.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- TransactionLog.java	10 Mar 2004 09:59:37 -0000	1.3
  +++ TransactionLog.java	6 May 2004 04:00:51 -0000	1.4
  @@ -18,19 +18,31 @@
   package org.apache.geronimo.transaction.manager;
   
   import java.io.IOException;
  +import java.util.List;
  +
   import javax.transaction.xa.Xid;
   
  +
   /**
    * Interface used to notify a logging subsystem of transaction events.
    *
    * @version $Revision$ $Date$
    */
   public interface TransactionLog {
  -    public void begin(Xid xid) throws IOException;
   
  -    public void prepare(Xid xid) throws IOException;
  +    void begin(Xid xid) throws LogException;
  +
  +    void prepare(Xid xid) throws LogException;
  +
  +    void commit(Xid xid) throws LogException;
  +
  +    void rollback(Xid xid) throws LogException;
  +
  +    List recover() throws LogException;
  +
  +    String getXMLStats();
   
  -    public void commit(Xid xid) throws IOException;
  +    int getAverageForceTime();
   
  -    public void rollback(Xid xid) throws IOException;
  +    int getAverageBytesPerForce();
   }
  
  
  
  1.5       +5 -2      incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/TransactionManagerImpl.java
  
  Index: TransactionManagerImpl.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/TransactionManagerImpl.java,v
  retrieving revision 1.4
  retrieving revision 1.5
  diff -u -r1.4 -r1.5
  --- TransactionManagerImpl.java	10 Mar 2004 09:59:37 -0000	1.4
  +++ TransactionManagerImpl.java	6 May 2004 04:00:51 -0000	1.5
  @@ -17,6 +17,8 @@
   
   package org.apache.geronimo.transaction.manager;
   
  +import java.util.Map;
  +
   import javax.transaction.HeuristicMixedException;
   import javax.transaction.HeuristicRollbackException;
   import javax.transaction.InvalidTransactionException;
  @@ -38,7 +40,7 @@
    */
   public class TransactionManagerImpl implements TransactionManager, XidImporter {
       private final TransactionLog txnLog;
  -    private final XidFactory xidFactory = new XidFactory();
  +    private final XidFactory xidFactory = new XidFactoryImpl();
       private volatile int timeout;
       private final ThreadLocal threadTx = new ThreadLocal();
   
  @@ -177,4 +179,5 @@
   
       public void setTransactionTimeout(long milliseconds) {
       }
  +
   }
  
  
  
  1.5       +10 -69    incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/XidFactory.java
  
  Index: XidFactory.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/XidFactory.java,v
  retrieving revision 1.4
  retrieving revision 1.5
  diff -u -r1.4 -r1.5
  --- XidFactory.java	10 Mar 2004 09:59:37 -0000	1.4
  +++ XidFactory.java	6 May 2004 04:00:51 -0000	1.5
  @@ -1,78 +1,19 @@
  -/**
  - *
  - * Copyright 2003-2004 The Apache Software Foundation
  - *
  - *  Licensed 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 java.net.InetAddress;
  -import java.net.UnknownHostException;
   import javax.transaction.xa.Xid;
   
   /**
  - * Factory for transaction ids.
  - * The Xid is constructed of three parts:
  - * <ol><li>8 byte count (LSB first)</li>
  - * <li>4 byte system id</li>
  - * <li>4 or 16 byte IP address of host</li>
  - * <ol>
  + *
  + *
    * @version $Revision$ $Date$
  - * todo Should have a way of setting baseId
  - */
  -public class XidFactory {
  -    byte[] baseId = new byte[Xid.MAXGTRIDSIZE];
  -    long count = 1;
  + *
  + * */
  +public interface XidFactory {
  +    Xid createXid();
   
  -    public XidFactory() {
  -        byte[] hostid;
  -        try {
  -            hostid = InetAddress.getLocalHost().getAddress();
  -        } catch (UnknownHostException e) {
  -            hostid = new byte[]{127, 0, 0, 1};
  -        }
  -        int uid = System.identityHashCode(this);
  -        baseId[8] = (byte) uid;
  -        baseId[9] = (byte) (uid >>> 8);
  -        baseId[10] = (byte) (uid >>> 16);
  -        baseId[11] = (byte) (uid >>> 24);
  -        System.arraycopy(hostid, 0, baseId, 12, hostid.length);
  -    }
  +    Xid createBranch(Xid globalId, int branch);
   
  -    public Xid createXid() {
  -        byte[] globalId = (byte[]) baseId.clone();
  -        long id;
  -        synchronized (this) {
  -            id = count++;
  -        }
  -        globalId[0] = (byte) id;
  -        globalId[1] = (byte) (id >>> 8);
  -        globalId[2] = (byte) (id >>> 16);
  -        globalId[3] = (byte) (id >>> 24);
  -        globalId[4] = (byte) (id >>> 32);
  -        globalId[5] = (byte) (id >>> 40);
  -        globalId[6] = (byte) (id >>> 48);
  -        globalId[7] = (byte) (id >>> 56);
  -        return new XidImpl(globalId);
  -    }
  +    boolean matchesGlobalId(byte[] globalTransactionId);
   
  -    public Xid createBranch(Xid globalId, int branch) {
  -        byte[] branchId = (byte[]) baseId.clone();
  -        branchId[0] = (byte) branch;
  -        branchId[1] = (byte) (branch >>> 8);
  -        branchId[2] = (byte) (branch >>> 16);
  -        branchId[3] = (byte) (branch >>> 24);
  -        return new XidImpl(globalId, branchId);
  -    }
  +    boolean matchesBranchId(byte[] branchQualifier);
   }
  
  
  
  1.5       +23 -9     incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/XidImpl.java
  
  Index: XidImpl.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/XidImpl.java,v
  retrieving revision 1.4
  retrieving revision 1.5
  diff -u -r1.4 -r1.5
  --- XidImpl.java	10 Mar 2004 09:59:37 -0000	1.4
  +++ XidImpl.java	6 May 2004 04:00:51 -0000	1.5
  @@ -28,17 +28,19 @@
    */
   public class XidImpl implements Xid, Serializable {
       private static int FORMAT_ID = 0x4765526f;  // Gero
  +    private final int formatId;
       private final byte[] globalId;
       private final byte[] branchId;
  -    private final int hash;
  +    private int hash;   //apparently never used by our code, so don't compute it.
   
       /**
        * Constructor taking a global id (for the main transaction)
        * @param globalId the global transaction id
        */
       public XidImpl(byte[] globalId) {
  +        this.formatId = FORMAT_ID;
           this.globalId = globalId;
  -        this.hash = hash(0, globalId);
  +        //this.hash = hash(0, globalId);
           branchId = new byte[Xid.MAXBQUALSIZE];
       }
   
  @@ -48,16 +50,23 @@
        * @param branch the branch id
        */
       public XidImpl(Xid global, byte[] branch) {
  -        int hash;
  +        this.formatId = FORMAT_ID;
  +        //int hash;
           if (global instanceof XidImpl) {
               globalId = ((XidImpl) global).globalId;
  -            hash = ((XidImpl) global).hash;
  +            //hash = ((XidImpl) global).hash;
           } else {
               globalId = global.getGlobalTransactionId();
  -            hash = hash(0, globalId);
  +            //hash = hash(0, globalId);
           }
           branchId = branch;
  -        this.hash = hash(hash, branchId);
  +        //this.hash = hash(hash, branchId);
  +    }
  +
  +    public XidImpl(int formatId, byte[] globalId, byte[] branchId) {
  +        this.formatId = formatId;
  +        this.globalId = globalId;
  +        this.branchId = branchId;
       }
   
       private int hash(int hash, byte[] id) {
  @@ -68,7 +77,7 @@
       }
   
       public int getFormatId() {
  -        return FORMAT_ID;
  +        return formatId;
       }
   
       public byte[] getGlobalTransactionId() {
  @@ -84,10 +93,15 @@
               return false;
           }
           XidImpl other = (XidImpl) obj;
  -        return Arrays.equals(globalId, other.globalId) && Arrays.equals(branchId, other.branchId);
  +        return formatId == other.formatId
  +                && Arrays.equals(globalId, other.globalId)
  +                && Arrays.equals(branchId, other.branchId);
       }
   
       public int hashCode() {
  +        if (hash == 0) {
  +            hash = hash(hash(0, globalId), branchId);
  +        }
           return hash;
       }
   
  
  
  
  1.3       +3 -1      incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/XidImporter.java
  
  Index: XidImporter.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/XidImporter.java,v
  retrieving revision 1.2
  retrieving revision 1.3
  diff -u -r1.2 -r1.3
  --- XidImporter.java	25 Feb 2004 09:58:19 -0000	1.2
  +++ XidImporter.java	6 May 2004 04:00:51 -0000	1.3
  @@ -17,6 +17,8 @@
   
   package org.apache.geronimo.transaction.manager;
   
  +import java.util.Map;
  +
   import javax.transaction.xa.Xid;
   import javax.transaction.xa.XAException;
   import javax.transaction.Transaction;
  
  
  
  1.1                  incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/LogException.java
  
  Index: LogException.java
  ===================================================================
  /**
   *
   * Copyright 2004 The Apache Software Foundation
   *
   *  Licensed 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;
  
  /**
   *
   *
   * @version $Revision: 1.1 $ $Date: 2004/05/06 04:00:51 $
   *
   * */
  public class LogException extends Exception {
  
         public LogException() {
  	super();
      }
  
      public LogException(String message) {
  	super(message);
      }
  
      public LogException(String message, Throwable cause) {
          super(message, cause);
      }
  
      public LogException(Throwable cause) {
          super(cause);
      }
  }
  
  
  
  1.1                  incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/Recovery.java
  
  Index: Recovery.java
  ===================================================================
  /**
   *
   * Copyright 2004 The Apache Software Foundation
   *
   *  Licensed 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 java.util.HashMap;
  import java.util.HashSet;
  import java.util.Iterator;
  import java.util.List;
  import java.util.Map;
  import java.util.Set;
  
  import javax.transaction.xa.XAException;
  import javax.transaction.xa.XAResource;
  import javax.transaction.xa.Xid;
  
  /**
   *
   *
   * @version $Revision: 1.1 $ $Date: 2004/05/06 04:00:51 $
   *
   * */
  public class Recovery {
  
      private final List xaResources;
      private final TransactionLog txLog;
      private XidFactory xidFactory;
  
      private Map externalXids;
  
      public Recovery(List xaResources, TransactionLog txLog, XidFactory xidFactory) {
          this.xaResources = xaResources;
          this.txLog = txLog;
          this.xidFactory = xidFactory;
      }
  
      public synchronized void recover() throws XAException {
          Set ourXids = new HashSet();
          externalXids = new HashMap();
          Map externalGlobalIdMap = new HashMap();
          List preparedXids = null;
          try {
              preparedXids = txLog.recover();
          } catch (LogException e) {
              throw (XAException)new XAException(XAException.XAER_RMERR).initCause(e);
          }
          for (Iterator iterator = preparedXids.iterator(); iterator.hasNext();) {
              Xid xid = (Xid) iterator.next();
              if (xidFactory.matchesGlobalId(xid.getGlobalTransactionId())) {
                  ourXids.add(xid);
              } else {
                  TransactionImpl externalTx = new TransactionImpl(xid, txLog);
                  externalXids.put(xid, externalTx);
                  externalGlobalIdMap.put(xid.getGlobalTransactionId(), externalTx);
              }
          }
  
          for (Iterator iterator = xaResources.iterator(); iterator.hasNext();) {
              XAResource xaResource = (XAResource) iterator.next();
              Xid[] prepared = xaResource.recover(XAResource.TMSTARTRSCAN + XAResource.TMENDRSCAN);
              for (int i = 0; i < prepared.length; i++) {
                  Xid xid = prepared[i];
                  if (ourXids.contains(xid)) {
                      xaResource.commit(xid, false);
                  } else if (xidFactory.matchesGlobalId(xid.getGlobalTransactionId())) {
                      //ours, but prepare not logged
                      xaResource.rollback(xid);
                  } else if (xidFactory.matchesBranchId(xid.getBranchQualifier())) {
                      //our branch, but we did not start this tx.
                      TransactionImpl externalTx = (TransactionImpl)externalGlobalIdMap.get(xid.getGlobalTransactionId());
                      if (externalTx == null) {
                          //we did not prepare this branch, rollback.
                          xaResource.rollback(xid);
                      } else {
                          //we prepared this branch, must wait for commit/rollback command.
                          externalTx.addBranchXid(xaResource, xid);
                      }
                  }
                  //else we had nothing to do with this xid.
              }
          }
  
  
      }
  
      public Map getExternalXids() {
          return new HashMap(externalXids);
      }
  }
  
  
  
  1.1                  incubator-geronimo/modules/transaction/src/java/org/apache/geronimo/transaction/manager/XidFactoryImpl.java
  
  Index: XidFactoryImpl.java
  ===================================================================
  /**
   *
   * Copyright 2003-2004 The Apache Software Foundation
   *
   *  Licensed 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 java.net.InetAddress;
  import java.net.UnknownHostException;
  import javax.transaction.xa.Xid;
  
  import org.apache.geronimo.transaction.log.XidImpl2;
  import org.apache.geronimo.gbean.GBeanInfo;
  import org.apache.geronimo.gbean.GBeanInfoFactory;
  
  /**
   * Factory for transaction ids.
   * The Xid is constructed of three parts:
   * <ol><li>8 byte count (LSB first)</li>
   * <li>4 byte system id</li>
   * <li>4 or 16 byte IP address of host</li>
   * <ol>
   * @version $Revision: 1.1 $ $Date: 2004/05/06 04:00:51 $
   * todo Should have a way of setting baseId
   */
  public class XidFactoryImpl implements XidFactory {
      private final byte[] baseId = new byte[Xid.MAXGTRIDSIZE];
      private long count = 1;
  
      public XidFactoryImpl(byte[] tmId) {
         System.arraycopy(tmId, 0, baseId, 8, tmId.length);
      }
  
      public XidFactoryImpl() {
          byte[] hostid;
          try {
              hostid = InetAddress.getLocalHost().getAddress();
          } catch (UnknownHostException e) {
              hostid = new byte[]{127, 0, 0, 1};
          }
          int uid = System.identityHashCode(this);
          baseId[8] = (byte) uid;
          baseId[9] = (byte) (uid >>> 8);
          baseId[10] = (byte) (uid >>> 16);
          baseId[11] = (byte) (uid >>> 24);
          System.arraycopy(hostid, 0, baseId, 12, hostid.length);
      }
  
      public Xid createXid() {
          byte[] globalId = (byte[]) baseId.clone();
          long id;
          synchronized (this) {
              id = count++;
          }
          globalId[0] = (byte) id;
          globalId[1] = (byte) (id >>> 8);
          globalId[2] = (byte) (id >>> 16);
          globalId[3] = (byte) (id >>> 24);
          globalId[4] = (byte) (id >>> 32);
          globalId[5] = (byte) (id >>> 40);
          globalId[6] = (byte) (id >>> 48);
          globalId[7] = (byte) (id >>> 56);
          return new XidImpl2(globalId);
      }
  
      public Xid createBranch(Xid globalId, int branch) {
          byte[] branchId = (byte[]) baseId.clone();
          branchId[0] = (byte) branch;
          branchId[1] = (byte) (branch >>> 8);
          branchId[2] = (byte) (branch >>> 16);
          branchId[3] = (byte) (branch >>> 24);
          return new XidImpl2(globalId, branchId);
      }
  
      public boolean matchesGlobalId(byte[] globalTransactionId) {
          if (globalTransactionId.length != Xid.MAXGTRIDSIZE) {
              return false;
          }
          for (int i = 8; i < globalTransactionId.length; i++) {
              if (globalTransactionId[i] != baseId[i]) {
                  return false;
              }
          }
          return true;
      }
  
      public boolean matchesBranchId(byte[] branchQualifier) {
          if (branchQualifier.length != Xid.MAXBQUALSIZE) {
              return false;
          }
          for (int i = 8; i < branchQualifier.length; i++) {
              if (branchQualifier[i] != baseId[i]) {
                  return false;
              }
          }
          return true;
      }
  
      public static final GBeanInfo GBEAN_INFO;
  
      static {
          GBeanInfoFactory infoFactory = new GBeanInfoFactory(XidFactoryImpl.class.getName());
          infoFactory.addInterface(XidFactory.class);
          GBEAN_INFO = infoFactory.getBeanInfo();
      }
  
      public static GBeanInfo getGBeanInfo() {
          return GBEAN_INFO;
      }
  }
  
  
  
  1.1                  incubator-geronimo/modules/transaction/src/test/org/apache/geronimo/transaction/log/AbstractLogTest.java
  
  Index: AbstractLogTest.java
  ===================================================================
  /**
   *
   * Copyright 2004 The Apache Software Foundation
   *
   *  Licensed 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.log;
  
  import java.io.File;
  import java.io.Writer;
  import java.io.FileWriter;
  import java.io.IOException;
  
  import javax.transaction.xa.Xid;
  
  import junit.framework.TestCase;
  import org.apache.geronimo.transaction.manager.TransactionLog;
  
  /**
   *
   *
   * @version $Revision: 1.1 $ $Date: 2004/05/06 04:00:51 $
   *
   * */
  public abstract class AbstractLogTest extends TestCase {
      private Object startBarrier = new Object();
      private Object stopBarrier = new Object();
      private int startedThreads = 0;
      private int stoppedThreads = 0;
      long totalDuration = 0;
      private Xid xid;
      final Object mutex = new Object();
      long totalXidCount = 0;
      private Writer resultsXML;
      private Writer resultsCSV;
  
      public void testTransactionLog() throws Exception {
          File resultFileXML = new File(getResultFileName() + ".xml");
          resultsXML = new FileWriter(resultFileXML);
          resultsXML.write("<log-test>\n");
          File resultFileCSV = new File(getResultFileName() + ".csv");
          resultsCSV = new FileWriter(resultFileCSV);
          resultsCSV.write("workerCount,xidCount,TotalXids,missingXids,DurationMilliseconds,XidsPerSecond,AverageForceTime,AverageBytesPerForce,AverageLatency\n");
          int xidCount = Integer.getInteger("xa.log.test.xid.count", 50).intValue();
          int minWorkerCount = Integer.getInteger("xa.log.test.worker.count.min", 20).intValue();
          int maxWorkerCount = Integer.getInteger("xa.log.test.worker.count.max", 40).intValue();
          int workerCountStep = Integer.getInteger("xa.log.test.worker.count.step", 20).intValue();
          int repCount = Integer.getInteger("xa.log.test.repetition.count", 1).intValue();
          long maxTime = Long.getLong("xa.log.test.max.time.seconds", 30).longValue() * 1000;
          int overtime = 0;
          try {
              for (int workers = minWorkerCount; workers <= maxWorkerCount; workers += workerCountStep) {
                  for (int reps = 0; reps < repCount; reps++) {
                      if (testTransactionLog(workers, xidCount) > maxTime) {
                          overtime++;
                          if (overtime > 1) {
                              return;
                          }
                      }
                      resultsCSV.flush();
                      resultsXML.flush();
                  }
              }
          } finally {
              resultsXML.write("</log-test>\n");
              resultsXML.flush();
              resultsXML.close();
              resultsCSV.flush();
              resultsCSV.close();
          }
      }
  
      protected abstract String getResultFileName();
  
      public long testTransactionLog(int workers, int xidCount) throws Exception {
          TransactionLog transactionLog = createTransactionLog();
  
          xid = new XidImpl2(new byte[Xid.MAXGTRIDSIZE]);
  
          long startTime = journalTest(transactionLog, workers, xidCount);
  
          long stopTime = System.currentTimeMillis();
  
          printSpeedReport(transactionLog, startTime, stopTime, workers, xidCount);
          closeTransactionLog(transactionLog);
          return stopTime - startTime;
      }
  
      protected abstract void closeTransactionLog(TransactionLog transactionLog) throws Exception ;
  
      protected abstract TransactionLog createTransactionLog() throws Exception;
  
      private long journalTest(final TransactionLog logger, final int workers, final int xidCount)
              throws Exception {
          totalXidCount = 0;
          startedThreads = 0;
          stoppedThreads = 0;
          totalDuration = 0;
          for (int i = 0; i < workers; i++) {
              new Thread() {
                  public void run() {
                      long localXidCount = 0;
                      boolean exception = false;
                      long localDuration = 0;
                      try {
                          synchronized (startBarrier) {
                              ++startedThreads;
                              startBarrier.notifyAll();
                              while (startedThreads < (workers + 1)) startBarrier.wait();
                          }
                          long localStartTime = System.currentTimeMillis();
  
                          for (int i = 0; i < xidCount; i++) {
                              // journalize COMMITTING record
                              logger.prepare(xid);
                              //localXidCount++;
  
                              // journalize FORGET record
                              logger.commit(xid);
                              localXidCount++;
                          }
                          localDuration = System.currentTimeMillis() - localStartTime;
                      } catch (Exception e) {
                          System.err.println(Thread.currentThread().getName());
                          e.printStackTrace(System.err);
                          exception = true;
                      } finally {
                          synchronized (mutex) {
                              totalXidCount += localXidCount;
                              totalDuration += localDuration;
                          }
                          synchronized (stopBarrier) {
                              ++stoppedThreads;
                              stopBarrier.notifyAll();
                          }
                      }
  
                  }
              }
                      .start();
          }
  
          // Wait for all the workers to be ready..
          long startTime = 0;
          synchronized (startBarrier) {
              while (startedThreads < workers) startBarrier.wait();
              ++startedThreads;
              startBarrier.notifyAll();
              startTime = System.currentTimeMillis();
          }
  
          // Wait for all the workers to finish.
          synchronized (stopBarrier) {
              while (stoppedThreads < workers) stopBarrier.wait();
          }
  
          return startTime;
  
      }
  
      void printSpeedReport(TransactionLog logger, long startTime, long stopTime, int workers, int xidCount) throws IOException {
          long mc = ((long) xidCount) * workers;
          long duration = (stopTime - startTime);
          long xidsPerSecond = (totalXidCount * 1000 / (duration));
          int averageForceTime = logger.getAverageForceTime();
          int averageBytesPerForce = logger.getAverageBytesPerForce();
          long averageLatency = totalDuration/totalXidCount;
          resultsXML.write("<run><workers>" + workers + "</workers><xids-per-thread>" + xidCount + "</xids-per-thread><expected-total-xids>" + mc + "</expected-total-xids><missing-xids>" + (mc - totalXidCount) + "</missing-xids><totalDuration-milliseconds>" + duration + "</totalDuration-milliseconds><xids-per-second>" + xidsPerSecond + "</xids-per-second></run>\n");
          resultsXML.write(logger.getXMLStats() + "\n");
          resultsCSV.write("" + workers + "," + xidCount + "," + mc + "," + (mc - totalXidCount) + "," + duration + "," + xidsPerSecond + "," + averageForceTime + "," + averageBytesPerForce  + "," + averageLatency + "\n");
  
      }
  }
  
  
  
  1.1                  incubator-geronimo/modules/transaction/src/test/org/apache/geronimo/transaction/log/XidSpeedTest.java
  
  Index: XidSpeedTest.java
  ===================================================================
  /**
   *
   * Copyright 2004 The Apache Software Foundation
   *
   *  Licensed 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.log;
  
  import javax.transaction.xa.Xid;
  
  import junit.framework.TestCase;
  import org.apache.geronimo.transaction.manager.XidImpl;
  
  /**
   *
   *
   * @version $Revision: 1.1 $ $Date: 2004/05/06 04:00:51 $
   *
   * */
  public class XidSpeedTest extends TestCase {
  
      public void testBufferTransferSpeed() throws Exception {
          HOWLLog log = new HOWLLog();
          Xid xid = new XidImpl(new byte[Xid.MAXGTRIDSIZE]);
          long start = System.currentTimeMillis();
          for (int i = 0; i < 1000000; i++) {
              log.getBytes(xid, HOWLLog.PREPARE);
          }
          long end = System.currentTimeMillis();
          System.err.println("millis for 1M getBytes impl1: " + (end - start));
      }
  
      public void testBufferTransferSpeed2() throws Exception {
          XidImpl2 xid = new XidImpl2(new byte[Xid.MAXGTRIDSIZE]);
          long start = System.currentTimeMillis();
          for (int i = 0; i < 1000000; i++) {
              xid.getBuffer(HOWLLog.PREPARE);
          }
          long end = System.currentTimeMillis();
          System.err.println("millis for 1M getBytes impl2: " + (end - start));
  
      }
  
  
      public void testXidImplCreationSpeed() throws Exception {
          Xid xid = null;
          long start = System.currentTimeMillis();
          for (int i = 0; i < 1000000; i++) {
              xid = new XidImpl(new byte[Xid.MAXGTRIDSIZE]);
          }
          long end = System.currentTimeMillis();
          System.err.println("millis for 1M create global xid impl1: " + (end - start));
  
          start = System.currentTimeMillis();
          for (int i = 0; i < 1000000; i++) {
              xid = new XidImpl(xid, new byte[Xid.MAXGTRIDSIZE]);
          }
          end = System.currentTimeMillis();
          System.err.println("millis for 1M create branch xid impl1: " + (end - start));
  
      }
  
      public void testXidImpl2CreationSpeed() throws Exception {
          Xid xid = null;
          long start = System.currentTimeMillis();
          for (int i = 0; i < 1000000; i++) {
              xid = new XidImpl2(new byte[Xid.MAXGTRIDSIZE]);
          }
          long end = System.currentTimeMillis();
          System.err.println("millis for 1M create global xid impl2: " + (end - start));
  
          start = System.currentTimeMillis();
          for (int i = 0; i < 1000000; i++) {
              xid = new XidImpl2(xid, new byte[Xid.MAXGTRIDSIZE]);
          }
          end = System.currentTimeMillis();
          System.err.println("millis for 1M create branch xid impl2: " + (end - start));
  
      }
  }
  
  
  
  1.4       +23 -7     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.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- XATransactionTester.java	10 Mar 2004 09:59:37 -0000	1.3
  +++ XATransactionTester.java	6 May 2004 04:00:51 -0000	1.4
  @@ -19,13 +19,13 @@
   
   import java.sql.Connection;
   import java.sql.Statement;
  -import java.io.IOException;
  +import java.util.List;
  +
   import javax.sql.XAConnection;
   import javax.sql.XADataSource;
   import javax.transaction.TransactionManager;
   import javax.transaction.xa.XAResource;
   import javax.transaction.xa.Xid;
  -import javax.transaction.xa.XAException;
   
   /**
    *
  @@ -100,17 +100,33 @@
   
       private class DummyLog implements TransactionLog {
   
  -        public void begin(Xid xid) throws IOException {
  +        public void begin(Xid xid) throws LogException {
               XATransactionTester.this.xid = xid;
           }
   
  -        public void prepare(Xid xid) throws IOException {
  +        public void prepare(Xid xid) throws LogException {
  +        }
  +
  +        public void commit(Xid xid) throws LogException {
  +        }
  +
  +        public void rollback(Xid xid) throws LogException {
  +        }
  +
  +        public List recover() throws LogException {
  +            return null;
  +        }
  +
  +        public String getXMLStats() {
  +            return null;
           }
   
  -        public void commit(Xid xid) throws IOException {
  +        public int getAverageForceTime() {
  +            return 0;
           }
   
  -        public void rollback(Xid xid) throws IOException {
  +        public int getAverageBytesPerForce() {
  +            return 0;
           }
       }
   }
  
  
  
  1.2       +2 -2      incubator-geronimo/modules/transaction/src/test/org/apache/geronimo/transaction/manager/XidImporterTest.java
  
  Index: XidImporterTest.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/transaction/src/test/org/apache/geronimo/transaction/manager/XidImporterTest.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- XidImporterTest.java	25 Feb 2004 18:05:51 -0000	1.1
  +++ XidImporterTest.java	6 May 2004 04:00:51 -0000	1.2
  @@ -41,7 +41,7 @@
       MockResource r2_2 = new MockResource(rm2);
   
       XidImporter tm = new TransactionManagerImpl();
  -    XidFactory xidFactory = new XidFactory();
  +    XidFactory xidFactory = new XidFactoryImpl();
   
       public void testImportXid() throws Exception {
           Xid externalXid = xidFactory.createXid();