You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jena.apache.org by an...@apache.org on 2011/07/27 13:52:14 UTC

svn commit: r1151436 - in /incubator/jena/Experimental/TxTDB/trunk: src-dev/tx/ src/main/java/com/hp/hpl/jena/tdb/transaction/ src/test/java/com/hp/hpl/jena/tdb/transaction/

Author: andy
Date: Wed Jul 27 11:52:12 2011
New Revision: 1151436

URL: http://svn.apache.org/viewvc?rev=1151436&view=rev
Log:
Journal entries record start/end position. Tidy journal I/O.

Modified:
    incubator/jena/Experimental/TxTDB/trunk/src-dev/tx/TxMain.java
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/BlockMgrJournal.java
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Journal.java
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/JournalControl.java
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/JournalEntry.java
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/NodeTableTrans.java
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/ObjectFileTrans.java
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transaction.java
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/TransactionManager.java
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transactional.java
    incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/transaction/AbstractTestNodeTableTrans.java
    incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/transaction/AbstractTestObjectFileTrans.java

Modified: incubator/jena/Experimental/TxTDB/trunk/src-dev/tx/TxMain.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src-dev/tx/TxMain.java?rev=1151436&r1=1151435&r2=1151436&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src-dev/tx/TxMain.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src-dev/tx/TxMain.java Wed Jul 27 11:52:12 2011
@@ -6,18 +6,13 @@
 
 package tx;
 
-import java.io.File ;
-
 import org.openjena.atlas.lib.Bytes ;
 import org.openjena.atlas.lib.FileOps ;
 import org.openjena.atlas.lib.Sink ;
-import org.openjena.atlas.lib.SinkNull ;
 import org.openjena.atlas.lib.SinkWrapper ;
 import org.openjena.atlas.logging.Log ;
 import org.openjena.riot.RiotReader ;
 import org.openjena.riot.RiotWriter ;
-import org.openjena.riot.WebReader ;
-import org.openjena.riot.lang.SinkToGraph ;
 import org.openjena.riot.lang.SinkTriplesToGraph ;
 
 import com.hp.hpl.jena.graph.Graph ;
@@ -42,11 +37,9 @@ import com.hp.hpl.jena.tdb.base.record.R
 import com.hp.hpl.jena.tdb.base.record.RecordFactory ;
 import com.hp.hpl.jena.tdb.setup.DatasetBuilderStd ;
 import com.hp.hpl.jena.tdb.store.DatasetGraphTDB ;
-import com.hp.hpl.jena.tdb.sys.Names ;
 import com.hp.hpl.jena.update.UpdateAction ;
 import com.hp.hpl.jena.update.UpdateFactory ;
 import com.hp.hpl.jena.update.UpdateRequest ;
-import com.hp.hpl.jena.util.FileManager ;
 
 public class TxMain
 {

Modified: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/BlockMgrJournal.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/BlockMgrJournal.java?rev=1151436&r1=1151435&r2=1151436&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/BlockMgrJournal.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/BlockMgrJournal.java Wed Jul 27 11:52:12 2011
@@ -66,7 +66,7 @@ public class BlockMgrJournal implements 
     }
     
     @Override
-    public void clearup(Transaction txn)
+    public void commitClearup(Transaction txn)
     {
         // Persistent state is in the system journal.
         reset(txn) ;

Modified: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Journal.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Journal.java?rev=1151436&r1=1151435&r2=1151436&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Journal.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Journal.java Wed Jul 27 11:52:12 2011
@@ -85,9 +85,16 @@ class Journal implements Iterable<Journa
         position = 0 ;
     }
     
+    synchronized
     public long writeJournal(JournalEntry entry)
     {
-        return  _write(entry.getType(), entry.getFileRef(), entry.getByteBuffer(), entry.getBlock()) ;
+        long posn = _write(entry.getType(), entry.getFileRef(), entry.getByteBuffer(), entry.getBlock()) ;
+        if ( entry.getPosition() < 0 )
+        {
+            entry.setPosition(posn) ;
+            entry.setEndPosition(position) ;
+        }
+        return posn ;
     }
     
     synchronized
@@ -167,10 +174,21 @@ class Journal implements Iterable<Journa
     synchronized
     public JournalEntry readJournal(long id)
     {
+        return _readJournal(id) ;
+    }
+    
+    JournalEntry _readJournal(long id)
+    {
         long x = channel.position() ;
-        channel.position(id) ;
+        if ( x != id ) 
+            channel.position(id) ;
         JournalEntry entry = _read() ;
-        channel.position(x) ;
+        long x2 = channel.position() ;
+        entry.setPosition(id) ;
+        entry.setEndPosition(x2) ;
+
+        if ( x != id )
+            channel.position(x) ;
         return entry ;
     }
     
@@ -198,7 +216,7 @@ class Journal implements Iterable<Journa
         FileRef fileRef = FileRef.get(ref) ;
         ByteBuffer bb = ByteBuffer.allocate(len) ;
         Block block = null ;
-        channel.read(bb) ;
+        lenRead = channel.read(bb) ;
         bb.rewind() ;
         if ( type == Block )
         {
@@ -214,37 +232,41 @@ class Journal implements Iterable<Journa
     private class IteratorEntries extends IteratorSlotted<JournalEntry>
     {
         JournalEntry slot = null ;
-        boolean finished = false ;
         final long endPoint ;
+        long iterPosn ;
 
-        // MUST abstract out his Iterator pattern and leave
-        // MoveToNext / Hasnext
-        // See QueryIteratorBase
-        public IteratorEntries() 
+        public IteratorEntries(long startPosition) 
         {
+            iterPosn = startPosition ;
             endPoint = channel.size() ;
         }
         
         @Override
         protected JournalEntry moveToNext()
         {
-            return _read() ;
+            synchronized(Journal.this)
+            {
+                if ( iterPosn >= endPoint )
+                    return null ;
+                JournalEntry e = _readJournal(iterPosn) ;
+                iterPosn = e.getEndPosition() ;
+                return e ;
+            }
         }
 
         @Override
-        protected boolean hasMore()
-        {
-            return channel.position() < endPoint  ;
-        }
+        protected boolean hasMore()     { return iterPosn < endPoint  ; }
     }
     
+    public Iterator<JournalEntry> entries()         { return new IteratorEntries(0) ; }
+    
     synchronized
-    public Iterator<JournalEntry> entries()
+    public Iterator<JournalEntry> entries(long startPosition)
     {
-        position(0) ;
-        return new IteratorEntries() ;
+        return new IteratorEntries(startPosition) ;
     }
     
+    /** Convenience method to iterate over the whole journal */ 
     @Override
     public Iterator<JournalEntry> iterator() { return entries() ; }
 

Modified: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/JournalControl.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/JournalControl.java?rev=1151436&r1=1151435&r2=1151436&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/JournalControl.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/JournalControl.java Wed Jul 27 11:52:12 2011
@@ -60,6 +60,83 @@ public class JournalControl
         recoverSystemJournal(dsg) ;
     }
     
+    // New recovery - scan to commit, enact, scan, ....
+    
+//    /** Recovery from the system journal.
+//     *  Find if there is a commit record; if so, reply the journal to that point.
+//     *  Try to see if there is another commit record ...
+//     */
+//    private static void recoverSystemJournal(DatasetGraphTDB dsg)
+//    {
+//        Location loc = dsg.getLocation() ;
+//        String journalFilename = loc.absolute(Names.journalFile) ;
+//        File f = new File(journalFilename) ;
+//        //if ( FileOps.exists(journalFilename)
+//        if ( f.exists() && f.isFile() && f.length() > 0 )
+//        {
+//            Journal jrnl = Journal.create(loc) ;
+//            
+//            recoverSegment(jrnl, 0) ;
+//            
+//            // Scan for commit.
+//            
+//            JournalEntry eCommit = null ;
+//            
+//            for ( JournalEntry e : jrnl )
+//            {
+//                if ( e.getType() == JournalEntryType.Commit )
+//                {
+//                    eCommit = e ;
+//                    break ;
+//                }
+//
+//            }
+//            if ( eCommit != null )
+//            {
+//                syslog.info("Recovering committed transaction") ;
+//                // The NodeTable Journal has already been done!
+//                JournalControl.replay(jrnl, dsg) ;
+//            }
+//            // We have replayed the journals - clean up.
+//            jrnl.truncate(0) ;
+//            jrnl.close();
+//            dsg.sync() ;    // JournalControl.replay
+//        }
+//        
+//        if ( f.exists() )
+//            FileOps.delete(journalFilename) ;
+//    }
+//    
+//    private static long recoverSegment(Journal jrnl, long startPosn)
+//    {
+//        Iterator<JournalEntry> iter = jrnl.entries(startPosn) ;
+//        // Phase one.  Scan for a commit.
+//        JournalEntry eCommit = null ;
+//        for ( ; iter.hasNext() ; )
+//        {
+//            JournalEntry e = iter.next() ;
+//            if ( e.getType() == JournalEntryType.Commit )
+//            {
+//                eCommit = e ;
+//                break ;
+//            }
+//        }
+//        Iter.close(iter) ;
+//        // Phase two : act.
+//        if ( eCommit == null )
+//            return -1 ;
+//        iter = jrnl.entries(startPosn) ;
+//        for ( ; iter.hasNext() ; )
+//        {
+//            JournalEntry e = iter.next() ;
+//            if ( e.getType() == JournalEntryType.Commit )
+//            {
+//                eCommit = e ;
+//                break ;
+//            }
+//        }
+//    }
+    
     /** Recovery from the system journal.
      *  Find is there is a commit record; if so, reply the journal.
      */

Modified: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/JournalEntry.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/JournalEntry.java?rev=1151436&r1=1151435&r2=1151436&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/JournalEntry.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/JournalEntry.java Wed Jul 27 11:52:12 2011
@@ -18,10 +18,11 @@ public class JournalEntry
     static public final JournalEntry Abort = new JournalEntry(JournalEntryType.Abort) ;
     static public final JournalEntry CheckPoint = new JournalEntry(JournalEntryType.Checkpoint) ;
     
-    private final JournalEntryType type ;   // One or other must be null - or both.
-    private final ByteBuffer byteBuffer ;
+    private long  position = -1 ;           // Location in the Journal (if known).
+    private long  endPosition = -1 ;        // End location in the Journal
+    private final JournalEntryType type ;
+    private final ByteBuffer byteBuffer ;   // One or other must be null - or both.
     private final Block block ;
-    private final int id = -98 ;
     private final FileRef fileRef ;
     
     private JournalEntry(JournalEntryType type)
@@ -48,8 +49,13 @@ public class JournalEntry
         this.block = block ;
         this.fileRef = fileRef ;
     }
-    
 
+    void setPosition(long posn)             { position = posn ; }
+    void setEndPosition(long endPosn)       { endPosition = endPosn ; }
+
+    public long getPosition()               { return position ; }
+    long getEndPosition()                   { return endPosition ; }
+    
     public JournalEntryType getType()       { return type ; }
     public ByteBuffer getByteBuffer()       { return byteBuffer ; }
     public Block getBlock()                 { return block ; }
@@ -79,6 +85,7 @@ public class JournalEntry
         sbuff.append("  "+type) ;
         return sbuff.toString() ;
     }
+
 }
 
 /*

Modified: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/NodeTableTrans.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/NodeTableTrans.java?rev=1151436&r1=1151435&r2=1151436&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/NodeTableTrans.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/NodeTableTrans.java Wed Jul 27 11:52:12 2011
@@ -166,18 +166,18 @@ public class NodeTableTrans implements N
     }
 
     @Override
-    public void abort(Transaction txn)
-    {
-        journal.reposition(0) ;
-    }
-    
-    @Override
-    public void clearup(Transaction txn)
+    public void commitClearup(Transaction txn)
     {
         journal.truncate(0);
         passthrough = true ;
     }
 
+    @Override
+    public void abort(Transaction txn)
+    {
+        journal.reposition(0) ;
+    }
+    
 //    private void clearUp()
 //    {
 //        passthrough = true ;

Modified: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/ObjectFileTrans.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/ObjectFileTrans.java?rev=1151436&r1=1151435&r2=1151436&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/ObjectFileTrans.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/ObjectFileTrans.java Wed Jul 27 11:52:12 2011
@@ -76,7 +76,7 @@ public class ObjectFileTrans implements 
     }
     
     @Override
-    public void clearup(Transaction txn)
+    public void commitClearup(Transaction txn)
     {
         other.truncate(0) ;
         passthrough = true ;

Modified: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transaction.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transaction.java?rev=1151436&r1=1151435&r2=1151436&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transaction.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transaction.java Wed Jul 27 11:52:12 2011
@@ -57,6 +57,14 @@ public class Transaction
         state = TxnState.ACTIVE ;
     }
 
+    /* Commit is a 4 step process
+     * 1/ commitPrepare - call all the components to tell them we are going to commit.
+     * 2/ Actually commit - write the commit point to the journal
+     * 3/ commitEnact -- make the changes to the original data
+     * 4/ commitClearup -- release resources
+     * The transaction manager is the place which knows all the components in a transaction. 
+     */
+    
     synchronized
     public void commit()
     {
@@ -73,6 +81,7 @@ public class Transaction
         }
 
         state = TxnState.COMMITED ;
+        // The transaction manager does the enact and clearup calls 
         txnMgr.notifyCommit(this) ;
     }
     

Modified: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/TransactionManager.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/TransactionManager.java?rev=1151436&r1=1151435&r2=1151436&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/TransactionManager.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/TransactionManager.java Wed Jul 27 11:52:12 2011
@@ -124,6 +124,10 @@ public class TransactionManager
         return dsgTxn ;
     }
 
+    /* Signal a transaction has commited.  The journal has a commit record
+     * and a sync to disk. The code here manages the inter-transaction stage
+     *  of deciding how to play the changes back to the base data. 
+     */ 
     synchronized
     public void notifyCommit(Transaction transaction)
     {
@@ -162,9 +166,9 @@ public class TransactionManager
         {
             Transactional x = iter.next() ;
             x.commitEnact(transaction) ;
-            x.clearup(transaction) ;
+            x.commitClearup(transaction) ;
         }
-        // This cleans up as well.
+        // This cleans up the journal as well.
         JournalControl.replay(transaction) ;
     }
     

Modified: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transactional.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transactional.java?rev=1151436&r1=1151435&r2=1151436&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transactional.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transactional.java Wed Jul 27 11:52:12 2011
@@ -27,8 +27,8 @@ public interface Transactional
     /** Update the base data */ 
     public void commitEnact(Transaction txn) ;
     
-    /** All done - transaction committed and incorporated in the base dataset */
-    public void clearup(Transaction txn) ;
+    /** All done - transaction committed and incorporated in the base dataset - can now tidy up */
+    public void commitClearup(Transaction txn) ;
 }
 
 /*

Modified: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/transaction/AbstractTestNodeTableTrans.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/transaction/AbstractTestNodeTableTrans.java?rev=1151436&r1=1151435&r2=1151436&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/transaction/AbstractTestNodeTableTrans.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/transaction/AbstractTestNodeTableTrans.java Wed Jul 27 11:52:12 2011
@@ -79,7 +79,7 @@ public abstract class AbstractTestNodeTa
         // Check it is now in the base.
         assertEquals(node1, nt0.getNodeForNodeId(nodeId)) ;
         assertEquals(nodeId, nt0.getNodeIdForNode(node1)) ;
-        ntt.clearup(txn) ;
+        ntt.commitClearup(txn) ;
     }
 
     @Test public void nodetrans_03()
@@ -101,7 +101,7 @@ public abstract class AbstractTestNodeTa
         // Check it is not in the base.
         assertEquals(NodeId.NodeDoesNotExist, nt0.getNodeIdForNode(node1)) ;
         assertNull(nt0.getNodeForNodeId(nodeId)) ;
-        ntt.clearup(txn) ;
+        ntt.commitClearup(txn) ;
     }
     
     @Test public void nodetrans_04()
@@ -119,7 +119,7 @@ public abstract class AbstractTestNodeTa
         ntt.commitPrepare(txn) ;
         ntt.commitEnact(txn) ;
         assertEquals(nodeId, nt0.getNodeIdForNode(node2)) ;
-        ntt.clearup(txn) ;
+        ntt.commitClearup(txn) ;
     }
 
 }

Modified: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/transaction/AbstractTestObjectFileTrans.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/transaction/AbstractTestObjectFileTrans.java?rev=1151436&r1=1151435&r2=1151436&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/transaction/AbstractTestObjectFileTrans.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/transaction/AbstractTestObjectFileTrans.java Wed Jul 27 11:52:12 2011
@@ -95,7 +95,7 @@ public abstract class AbstractTestObject
         file.commitPrepare(txn) ;
         file.commitEnact(txn) ;
         contains(file1, "ABC") ;
-        file.clearup(txn) ;
+        file.commitClearup(txn) ;
     }
 
     @Test public void objFileTrans_03()
@@ -107,7 +107,7 @@ public abstract class AbstractTestObject
         file.commitPrepare(txn) ;
         file.commitEnact(txn) ;
         contains(file1, "ABC", "X") ;
-        file.clearup(txn) ;
+        file.commitClearup(txn) ;
     }
 
     @Test public void objFileTrans_04()
@@ -119,7 +119,7 @@ public abstract class AbstractTestObject
         file.commitPrepare(txn) ;
         file.commitEnact(txn) ;
         contains(file1, "ABC", "ABC", "ABCDEFGHIJKLMNOPQRSTUVWXYZ") ;
-        file.clearup(txn) ;
+        file.commitClearup(txn) ;
     }
 
     @Test public void objFileTrans_05()
@@ -130,7 +130,7 @@ public abstract class AbstractTestObject
         write(file, "ABCDEF") ;
         file.abort(txn) ;
         contains(file1, "ABC") ;
-        file.clearup(txn) ;
+        file.commitClearup(txn) ;
     }
 
     @Test public void objFileTrans_06()