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/12/23 23:01:50 UTC

svn commit: r1222867 - in /incubator/jena/Jena2/TDB/trunk/src: main/java/com/hp/hpl/jena/tdb/base/objectfile/ main/java/com/hp/hpl/jena/tdb/nodetable/ main/java/com/hp/hpl/jena/tdb/transaction/ test/java/com/hp/hpl/jena/tdb/transaction/

Author: andy
Date: Fri Dec 23 22:01:49 2011
New Revision: 1222867

URL: http://svn.apache.org/viewvc?rev=1222867&view=rev
Log:
JENA-163

Added:
    incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/transaction/T_TransSystem.java
      - copied, changed from r1220914, incubator/jena/Scratch/AFS/Jena-Dev/trunk/src/dev/Jena163_TDBDifferentIds.java
Modified:
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFile.java
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileLogger.java
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileMem.java
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileStorage.java
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileWrapper.java
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/nodetable/NodeTableNative.java
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/NodeTableTrans.java
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/ObjectFileTrans.java
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transaction.java
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/TransactionManager.java

Modified: incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFile.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFile.java?rev=1222867&r1=1222866&r2=1222867&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFile.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFile.java Fri Dec 23 22:01:49 2011
@@ -54,6 +54,9 @@ public interface ObjectFile extends Sync
     /** Length, in units used by read/write for ids */
     public long length() ;
     
+    /** Any objects in this file? */
+    public boolean isEmpty() ;
+
     /** Reset the "append" point; may only be moved earlier.
      * The new position must correspond to a position returned by
      * {@link #write(ByteBuffer)} or an id in a {@link Block Block} from {@link #completeWrite(Block)}

Modified: incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileLogger.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileLogger.java?rev=1222867&r1=1222866&r2=1222867&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileLogger.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileLogger.java Fri Dec 23 22:01:49 2011
@@ -114,10 +114,15 @@ public class ObjectFileLogger implements
     @Override
     public long length()
     {
-        info("") ;
         return other.length() ;
     }
     
+    @Override
+    public boolean isEmpty()
+    {
+        return other.isEmpty() ;
+    }
+
     private void info(String string)
     {
         if ( label != null )

Modified: incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileMem.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileMem.java?rev=1222867&r1=1222866&r2=1222867&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileMem.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileMem.java Fri Dec 23 22:01:49 2011
@@ -61,7 +61,16 @@ public class ObjectFileMem implements Ob
             throw new IllegalStateException("Closed") ;
         return buffers.size() ;
     }
+    
+    @Override
+    public boolean isEmpty()
+    {
+        if ( closed )
+            throw new IllegalStateException("Closed") ;
+        return buffers.isEmpty() ;
+    }
 
+    
     @Override
     public ByteBuffer read(long id)
     {

Modified: incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileStorage.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileStorage.java?rev=1222867&r1=1222866&r2=1222867&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileStorage.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileStorage.java Fri Dec 23 22:01:49 2011
@@ -336,6 +336,14 @@ public class ObjectFileStorage implement
         if ( writeBuffer == null ) return filesize ; 
         return filesize+writeBuffer.position() ;
     }
+    
+    @Override
+    public boolean isEmpty()
+    {
+        if ( writeBuffer == null ) return filesize == 0  ;
+        return writeBuffer.position() == 0 &&  filesize == 0 ; 
+    }
+
 
     @Override
     public void close()                 { flushOutputBuffer() ; file.close() ; }

Modified: incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileWrapper.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileWrapper.java?rev=1222867&r1=1222866&r2=1222867&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileWrapper.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileWrapper.java Fri Dec 23 22:01:49 2011
@@ -68,4 +68,7 @@ public class ObjectFileWrapper implement
 
     @Override
     public long length()                            { return other.length() ; }
+    
+    @Override
+    public boolean isEmpty()                        { return other.isEmpty() ; }
 }

Modified: incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/nodetable/NodeTableNative.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/nodetable/NodeTableNative.java?rev=1222867&r1=1222866&r2=1222867&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/nodetable/NodeTableNative.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/nodetable/NodeTableNative.java Fri Dec 23 22:01:49 2011
@@ -262,6 +262,6 @@ public class NodeTableNative implements 
     @Override
     public boolean isEmpty()
     {
-        return false ;
+        return getObjects().isEmpty() ;
     }
 }

Modified: incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/NodeTableTrans.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/NodeTableTrans.java?rev=1222867&r1=1222866&r2=1222867&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/NodeTableTrans.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/NodeTableTrans.java Fri Dec 23 22:01:49 2011
@@ -18,6 +18,8 @@
 
 package com.hp.hpl.jena.tdb.transaction;
 
+import static org.openjena.atlas.logging.Log.warn ;
+
 import java.nio.ByteBuffer ;
 import java.util.Iterator ;
 
@@ -43,7 +45,7 @@ public class NodeTableTrans implements N
     private static Logger log = LoggerFactory.getLogger(NodeTableTrans.class) ;
     // TODO flag to note is any work is needed on commit.
     private final NodeTable base ;
-    private long offset ;
+    private long allocOffset ;
     
     private NodeTable nodeTableJournal = null ;
     private static int CacheSize = 10000 ;      // [TxTDB:TODO] Make configurable 
@@ -57,19 +59,13 @@ public class NodeTableTrans implements N
     private final String label ;
     private final Transaction txn ;     // Can be null (during recovery).
     
-    // *** TEMPORARY!
-    public static boolean FIXUP = false ;  
-    
     public NodeTableTrans(Transaction txn, String label, NodeTable sub, Index nodeIndex, ObjectFile objFile)
     {
         this.txn = txn ;
         this.base = sub ;
         this.nodeIndex = nodeIndex ;
         this.journalObjFile = objFile ;
-        
         this.label = label ; 
-        // Show the way tables are wired up
-        //debug("NTT[%s #%s] %s", label, Integer.toHexString(hashCode()), sub) ;
     }
 
     public void setPassthrough(boolean v)   { passthrough = v ; }
@@ -105,7 +101,7 @@ public class NodeTableTrans implements N
     {
         if ( passthrough ) return base.getNodeForNodeId(id) ;
         long x = id.getId() ;
-        if ( x < offset )
+        if ( x < allocOffset )
             return base.getNodeForNodeId(id) ;
         id = mapToJournal(id) ;
         Node node = nodeTableJournal.getNodeForNodeId(id) ;
@@ -119,7 +115,7 @@ public class NodeTableTrans implements N
            throw new TDBTransactionException("Not in an active transaction") ;
         if ( NodeId.isInline(id) )
             return id ; 
-        return NodeId.create(id.getId()-offset) ;
+        return NodeId.create(id.getId()-allocOffset) ;
     }
     
     /** Convert from a id in other to an external id  */ 
@@ -129,7 +125,7 @@ public class NodeTableTrans implements N
             throw new TDBTransactionException("Not in an active transaction") ;
         if ( NodeId.isInline(id) )
             return id ; 
-        return NodeId.create(id.getId()+offset) ; 
+        return NodeId.create(id.getId()+allocOffset) ; 
     }
     
     private NodeId allocate(Node node)
@@ -153,7 +149,7 @@ public class NodeTableTrans implements N
     @Override
     public void begin(Transaction txn)
     {
-        debug("%s begin", txn.getLabel()) ;
+        //debug("%s begin", txn.getLabel()) ;
         
         if ( this.txn.getTxnId() != txn.getTxnId() )
             throw new TDBException(String.format("Different transactions: %s %s", this.txn.getLabel(), txn.getLabel())) ;
@@ -161,35 +157,19 @@ public class NodeTableTrans implements N
             throw new TDBException("Already active") ;
         passthrough = false ;
         
-        offset = base.allocOffset().getId() ;
-
+        allocOffset = base.allocOffset().getId() ;
+        // base node table empty e.g. first use.
         journalObjFileStartOffset = journalObjFile.length() ;
         if ( journalObjFileStartOffset != 0 )
         {
-            System.out.flush() ;
-            System.err.printf("\n%s journalStartOffset not zero: %d/0x%02X\n",txn.getLabel(), journalObjFileStartOffset, journalObjFileStartOffset) ;
-            
+            warn(log, "%s journalStartOffset not zero: %d/0x%02X",txn.getLabel(), journalObjFileStartOffset, journalObjFileStartOffset) ;
             // repeat for debugging.
             journalObjFile.length() ;
-            
-            if ( FIXUP )
-            {
-                // TEMP : if you see this code active in SVN, set it to false immediately.
-                // The question is how come the journal position was non-zero in the first place. 
-                System.err.println("journalStartOffset reset to zero") ;
-                journalObjFileStartOffset = 0 ;
-                journalObjFile.truncate(0) ;
-                //journalObjFile.sync() ;
-            }
         }
-        offset += journalObjFileStartOffset ;
-        
-        //debug("begin: %s %s", txn.getLabel(), label) ;
-        //debug("begin: base=%s  offset=0x%X journalOffset=0x%X", base, offset, journalOffset) ;
+        allocOffset += journalObjFileStartOffset ;
         
         this.nodeTableJournal = new NodeTableNative(nodeIndex, journalObjFile) ;
         this.nodeTableJournal = NodeTableCache.create(nodeTableJournal, CacheSize, CacheSize, 100) ;
-
         // This class knows about non-mappable inline values.   mapToJournal(NodeId)/mapFromJournal. 
         this.nodeTableJournal = NodeTableInline.create(nodeTableJournal) ;
     }
@@ -199,17 +179,10 @@ public class NodeTableTrans implements N
     /** Copy from the journal file to the real file */
     /*package*/ void append()
     {
-        //debug(">> append: %s",label) ;
-        // Assumes all() is in order from low to high.
-        
-        if ( APPEND_LOG ) 
-            System.out.printf(">> append: %s %s %d/0x%04X\n",label, base.allocOffset(), journalObjFile.length(), journalObjFile.length()) ;
-        
         Iterator<Pair<NodeId, Node>> iter = nodeTableJournal.all() ;
         Pair<NodeId, Node> firstPair = null ;
         Pair<NodeId, Node> lastPair = null ;
         
-        
         for ( ; iter.hasNext() ; )
         {
             Pair<NodeId, Node> x = iter.next() ;
@@ -224,32 +197,19 @@ public class NodeTableTrans implements N
             // This does the write.
             NodeId nodeId2 = base.getAllocateNodeId(node) ;
             if ( ! nodeId2.equals(mapFromJournal(nodeId)) )
-            {
-                String msg = String.format("Different ids for %s: allocated: expected %s, got %s", node, mapFromJournal(nodeId), nodeId2) ;
-                System.err.println() ;
-                System.err.println() ;
-                System.err.println(msg) ;
-                dump() ;   
-                System.err.println() ;
-                throw new TDBException(msg) ;
-            }
-        }
-        
-        if ( APPEND_LOG )
-        {
-            System.out.printf("+ First: %s -> %s\n", firstPair.car(), mapFromJournal(firstPair.car())) ;
-            System.out.printf("+ Last: %s -> %s\n", lastPair.car(), mapFromJournal(lastPair.car())) ;
-            System.out.printf("+ New base: %s\n", base.allocOffset()) ;
-            Node n1 = firstPair.cdr() ;
-            Node n2 = lastPair.cdr() ;
-            
-            if ( base.getNodeIdForNode(n1) == null )
-                throw new TDBException("1") ;
-            if ( base.getNodeIdForNode(n2) == null )
-                throw new TDBException("2") ;
+                inconsistent(node, nodeId, nodeId2) ;
         }
-        
-        //debug("<< append: %s",label) ;
+    }
+    
+    private void inconsistent(Node node , NodeId nodeId , NodeId nodeId2 )
+    {
+        String msg = String.format("Different ids for %s: allocated: expected %s, got %s", node, mapFromJournal(nodeId), nodeId2) ;
+        System.err.println() ;
+        System.err.println() ;
+        System.err.println(msg) ;
+        dump() ;   
+        System.err.println() ;
+        throw new TDBException(msg) ;
     }
     
     private void dump()
@@ -257,7 +217,7 @@ public class NodeTableTrans implements N
         System.err.println(">>>>>>>>>>") ;
         System.err.println("label = "+label) ;
         System.err.println("txn = "+txn) ;
-        System.err.println("offset = "+offset) ;
+        System.err.println("offset = "+allocOffset) ;
         System.err.println("journalStartOffset = "+journalObjFileStartOffset) ;
         System.err.println("journal = "+journalObjFile.getLabel()) ;
         if ( true )
@@ -299,7 +259,6 @@ public class NodeTableTrans implements N
     @Override
     public void commitPrepare(Transaction txn)
     {
-        debug("%s >> commitPrepare: %s", txn.getLabel(), label) ;
         // The node table is append-only so it can be written during prepare.
         // The index isn't written (via the transaction journal) until enact.
         if ( nodeTableJournal == null )
@@ -311,7 +270,6 @@ public class NodeTableTrans implements N
             long x = journalObjFile.length() ;
             throw new TDBTransactionException(txn.getLabel()+": journalObjFile not cleared ("+x+")") ;
         }
-        debug("%s << commitPrepare: %s", txn.getLabel(), label) ;
     }
     
     @Override
@@ -321,37 +279,45 @@ public class NodeTableTrans implements N
         // is append only.  Until here, pointers to the extra data aren't available
         // until the index is written.
         // The index is written via the transaction journal.
-        
-        //debug("commitEnact: %s", label) ;
         //writeJournal() ;
     }
 
     private void writeNodeJournal()
     {
-        if ( nodeTableJournal.isEmpty() )
+        //*****
+        //if ( false && nodeTableJournal.isEmpty() )
+        //if ( nodeIndex.isEmpty() )
+        if ( false && journalObjFile.isEmpty() )
+        {
+            journalObjFile.close() ;                                // Side effect is a buffer flush.
+            journalObjFile = null ;
+            base.sync() ;
+            allocOffset = -99 ; // base.allocOffset().getId() ; // Will be invalid as we may write through to the base table later.
+            passthrough = true ;
             return ;
+        }
+
+        long expected = base.allocOffset().getId() ;
+        long len = journalObjFile.length() ;
+        if ( expected != allocOffset )
+            System.err.println("************* UNEXPECTED [1]") ;
         
-        //debug("writeNodeJournal: (base alloc before) %s", base.allocOffset()) ;
-        append() ;      // Calls all() which does a buffer flish. 
-        //debug("writeNodeJournal: (base alloc after) %s",  base.allocOffset()) ;
-        //debug("writeNodeJournal: (nodeTableJournal) %s", nodeTableJournal.allocOffset()) ;
-        
+        long newbase = -1 ; 
+        append() ;      // Calls all() which does a buffer flish.
         // Reset (in case we use this again)
         nodeIndex.clear() ;
-        // Fixes nodeTableJournal
         journalObjFile.truncate(journalObjFileStartOffset) ;    // Side effect is a buffer flush.
         //journalObjFile.sync() ;
         journalObjFile.close() ;                                // Side effect is a buffer flush.
         journalObjFile = null ;
         base.sync() ;
-        offset = -99 ; // base.allocOffset().getId() ; // Will be invalid as we may write through to the base table later.
+        allocOffset = -99 ; // base.allocOffset().getId() ; // Will be invalid as we may write through to the base table later.
         passthrough = true ;
     }
 
     @Override
     public void commitClearup(Transaction txn)
     {
-        debug("%s ** commitClearup: %s",  txn.getLabel(), label) ;
         finish() ;
     }
 

Modified: incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/ObjectFileTrans.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/ObjectFileTrans.java?rev=1222867&r1=1222866&r2=1222867&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/ObjectFileTrans.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/ObjectFileTrans.java Fri Dec 23 22:01:49 2011
@@ -32,7 +32,7 @@ import com.hp.hpl.jena.tdb.base.objectfi
 
 public class ObjectFileTrans implements ObjectFile, TransactionLifecycle
 {
-    private final ObjectFile other ;
+    private final ObjectFile transObjects ;
     private long otherAllocOffset ;           // record where we start allocating
     private boolean passthrough = false ;
     private boolean inTransaction = false ;
@@ -42,9 +42,9 @@ public class ObjectFileTrans implements 
     
     public ObjectFileTrans(Transaction txn, ObjectFile base, ObjectFile other)
     {
-        // The other object file must use the same allocation policy.
+        // The "other" object file must use the same allocation policy.
         this.base = base ;
-        this.other = other ;
+        this.transObjects = other ;
         inTransaction = false ;
 
         //  [TxTDB:PATCH-UP] Begin is not being called.
@@ -59,7 +59,7 @@ public class ObjectFileTrans implements 
     {
         passthrough = false ;
         inTransaction = true ;
-        other.reposition(0) ;
+        transObjects.reposition(0) ;
         this.otherAllocOffset = base.length() ;
     }
     
@@ -68,7 +68,7 @@ public class ObjectFileTrans implements 
     {
         if ( ! inTransaction )
             throw new TDBTransactionException("Not in a transaction for a commit to happen") ; 
-        other.sync() ;
+        transObjects.sync() ;
     }
 
     @Override
@@ -78,19 +78,19 @@ public class ObjectFileTrans implements 
             throw new TDBTransactionException("Not in a transaction for a commit to happen") ; 
         append() ;
         base.sync() ;
-        other.reposition(0) ;
+        transObjects.reposition(0) ;
     }
 
     @Override
     public void abort(Transaction txn)
     {
-        other.reposition(0) ;
+        transObjects.reposition(0) ;
     }
     
     @Override
     public void commitClearup(Transaction txn)
     {
-        other.truncate(0) ;
+        transObjects.truncate(0) ;
         passthrough = true ;
     }
 
@@ -106,7 +106,7 @@ public class ObjectFileTrans implements 
         // Truncate/position the ObjectFile.
         base.reposition(otherAllocOffset) ;
         
-        Iterator<Pair<Long, ByteBuffer>> iter = other.all() ;
+        Iterator<Pair<Long, ByteBuffer>> iter = transObjects.all() ;
         for ( ; iter.hasNext() ; )
         {
             Pair<Long, ByteBuffer> p = iter.next() ;
@@ -127,13 +127,13 @@ public class ObjectFileTrans implements 
         if ( passthrough ) { base.reposition(id) ; return ; }
         if ( id > otherAllocOffset )
         {
-            other.reposition(mapToOther(id)) ;
+            transObjects.reposition(mapToOther(id)) ;
             return ;
         }
         
         Log.warn(this, "Unexpected: Attempt to reposition over base file") ;
         base.reposition(id) ;
-        other.reposition(0) ;
+        transObjects.reposition(0) ;
         otherAllocOffset = base.length() ;
     }
     
@@ -143,11 +143,11 @@ public class ObjectFileTrans implements 
         if ( passthrough ) { base.truncate(id) ; return ; }
         if ( id > otherAllocOffset )
         {
-            other.truncate(mapToOther(id)) ;
+            transObjects.truncate(mapToOther(id)) ;
             return ;
         }
         base.truncate(id) ;
-        other.truncate(0) ;
+        transObjects.truncate(0) ;
         otherAllocOffset = base.length() ;
     }
 
@@ -155,7 +155,7 @@ public class ObjectFileTrans implements 
     public Block allocWrite(int maxBytes)
     {
         if ( passthrough ) return base.allocWrite(maxBytes) ;
-        Block block = other.allocWrite(maxBytes) ;
+        Block block = transObjects.allocWrite(maxBytes) ;
         block = new Block(block.getId()+otherAllocOffset, block.getByteBuffer()) ;
         return block ;
     }
@@ -165,7 +165,7 @@ public class ObjectFileTrans implements 
     {
         if ( passthrough ) { base.completeWrite(block) ; return ; } 
         block = new Block(block.getId()-otherAllocOffset, block.getByteBuffer()) ;
-        other.completeWrite(block) ;
+        transObjects.completeWrite(block) ;
     }
 
     /** Convert from a id to the id in the "other" file */ 
@@ -178,7 +178,7 @@ public class ObjectFileTrans implements 
     {
         if ( passthrough ) { return base.write(buffer) ; } 
         // Write to auxillary
-        long x = other.write(buffer) ;
+        long x = transObjects.write(buffer) ;
         return mapFromOther(x) ;
     }
 
@@ -189,21 +189,28 @@ public class ObjectFileTrans implements 
         if ( id < otherAllocOffset )
             return base.read(id) ;
         long x = mapToOther(id) ; 
-        return other.read(id-otherAllocOffset) ;
+        return transObjects.read(id-otherAllocOffset) ;
     }
 
     @Override
     public long length()
     {
         if ( passthrough ) { return base.length() ; } 
-        return otherAllocOffset+other.length() ;
+        return otherAllocOffset+transObjects.length() ;
+    }
+    
+    @Override
+    public boolean isEmpty()
+    {
+        if ( passthrough ) { return base.isEmpty() ; } 
+        return transObjects.isEmpty() && base.isEmpty() ;
     }
 
     @Override
     public Iterator<Pair<Long, ByteBuffer>> all()
     {
         if ( passthrough ) { return base.all() ; } 
-        return Iter.concat(base.all(), other.all()) ;
+        return Iter.concat(base.all(), transObjects.all()) ;
     }
 
     @Override
@@ -221,6 +228,6 @@ public class ObjectFileTrans implements 
     @Override
     public String getLabel()
     {
-        return "("+base.getLabel()+":"+other.getLabel()+")" ;
+        return "("+base.getLabel()+":"+transObjects.getLabel()+")" ;
     }
 }

Modified: incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transaction.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transaction.java?rev=1222867&r1=1222866&r2=1222867&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transaction.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transaction.java Fri Dec 23 22:01:49 2011
@@ -214,7 +214,8 @@ public class Transaction
     }
 
     public ReadWrite getMode()                      { return mode ; }
-    public TxnState getState()                      { return state ; }
+    public boolean   isRead()                       { return mode == ReadWrite.READ ; }
+    public TxnState  getState()                     { return state ; }
     
     public long getTxnId()                          { return id ; }
     public TransactionManager getTxnMgr()           { return txnMgr ; }

Modified: incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/TransactionManager.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/TransactionManager.java?rev=1222867&r1=1222866&r2=1222867&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/TransactionManager.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/TransactionManager.java Fri Dec 23 22:01:49 2011
@@ -149,6 +149,17 @@ public class TransactionManager
         @Override public void writerCommits(Transaction txn)        { log("commit", txn) ; }
         @Override public void writerAborts(Transaction txn)         { log("abort", txn) ; }
     }
+
+    /** More detailed */
+    class TSM_LoggerDebug extends TSM_Base
+    {
+        @Override public void readerStarts(Transaction txn)         { logInternal("start",  txn) ; }
+        @Override public void readerFinishes(Transaction txn)       { logInternal("finish", txn) ; }
+        @Override public void writerStarts(Transaction txn)         { logInternal("begin",  txn) ; }
+        @Override public void writerCommits(Transaction txn)        { logInternal("commit", txn) ; }
+        @Override public void writerAborts(Transaction txn)         { logInternal("abort",  txn) ; }
+    }
+
     
     class TSM_Counters implements TSM
     {
@@ -252,6 +263,7 @@ public class TransactionManager
     
     private TSM[] actions = new TSM[] { 
         new TSM_Counters() ,           // Must be first.
+        //new TSM_LoggerDebug() ,
         new TSM_Logger() ,
         (recordHistory ? new TSM_Record() : null ) ,
         new TSM_WriteBackEndTxn()        // Write back policy. Must be last.
@@ -345,7 +357,7 @@ public class TransactionManager
         Transaction txn = createTransaction(dsg, mode, label) ;
         
         log("begin$", txn) ;
-        
+
         DatasetGraphTxn dsgTxn = (DatasetGraphTxn)new DatasetBuilderTxn(this).build(txn, mode, dsg) ;
         txn.setActiveDataset(dsgTxn) ;
 
@@ -583,6 +595,14 @@ public class TransactionManager
         else
             logger().debug(txn.getLabel()+": "+msg) ;
     }
+    
+    private void logInternal(String action, Transaction txn)
+    {
+        if ( ! log() )
+            return ;
+        String txnStr = ( txn == null ) ? "<null>" : txn.getLabel() ;
+        System.err.printf(format("%6s %s -- %s", action, txnStr, state())) ;
+    }
 
     private static Logger logger()
     {

Copied: incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/transaction/T_TransSystem.java (from r1220914, incubator/jena/Scratch/AFS/Jena-Dev/trunk/src/dev/Jena163_TDBDifferentIds.java)
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/transaction/T_TransSystem.java?p2=incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/transaction/T_TransSystem.java&p1=incubator/jena/Scratch/AFS/Jena-Dev/trunk/src/dev/Jena163_TDBDifferentIds.java&r1=1220914&r2=1222867&rev=1222867&view=diff
==============================================================================
--- incubator/jena/Scratch/AFS/Jena-Dev/trunk/src/dev/Jena163_TDBDifferentIds.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/transaction/T_TransSystem.java Fri Dec 23 22:01:49 2011
@@ -16,10 +16,11 @@
  * limitations under the License.
  */
 
-package dev ;
+package com.hp.hpl.jena.tdb.transaction ;
 
-import static java.lang.Math.* ;
 import static com.hp.hpl.jena.tdb.transaction.TransTestLib.count ;
+import static java.lang.Math.max ;
+import static java.lang.Math.min ;
 import static java.lang.String.format ;
 
 import java.util.Iterator ;
@@ -48,19 +49,16 @@ import com.hp.hpl.jena.tdb.DatasetGraphT
 import com.hp.hpl.jena.tdb.StoreConnection ;
 import com.hp.hpl.jena.tdb.TDBException ;
 import com.hp.hpl.jena.tdb.base.block.FileMode ;
-import com.hp.hpl.jena.tdb.base.file.FileBase ;
 import com.hp.hpl.jena.tdb.base.file.Location ;
-import com.hp.hpl.jena.tdb.base.objectfile.ObjectFileStorage ;
 import com.hp.hpl.jena.tdb.sys.SystemTDB ;
-import com.hp.hpl.jena.tdb.transaction.NodeTableTrans ;
 import com.hp.hpl.jena.tdb.transaction.SysTxnState ;
 import com.hp.hpl.jena.tdb.transaction.TransactionManager ;
 
 /** System testing of the transactions. */
-public class Jena163_TDBDifferentIds
+public class T_TransSystem
 {
     static { org.openjena.atlas.logging.Log.setLog4j() ; }
-    private static Logger log = LoggerFactory.getLogger(Jena163_TDBDifferentIds.class) ;
+    private static Logger log = LoggerFactory.getLogger(T_TransSystem.class) ;
 
     /* Notes:
      * MS Windows does not allow memory mapped files to be deleted during the run of a JVM.
@@ -69,8 +67,8 @@ public class Jena163_TDBDifferentIds
      */
     
     static boolean MEM                  = true ;
-    //static String location              = false ? "/mnt/ssd1/tmp/DB163" : ConfigTest.getTestingDirDB() ;     // Using an SSD here is very helpful
-    static String location              = ConfigTest.getTestingDirDB() ;     // Using an SSD here is very helpful
+    static String location              = true ? "/mnt/ssd1/tmp/DB163" : ConfigTest.getTestingDirDB() ;     // Using an SSD here is very helpful
+    //static String location              = ConfigTest.getTestingDirDB() ;     // Using an SSD here is very helpful
     static final Location LOC           = MEM ? Location.mem() : new Location(location) ;
     
     static { 
@@ -83,6 +81,20 @@ public class Jena163_TDBDifferentIds
         
         FileOps.ensureDir(location) ;
     }
+
+    private static boolean inlineProgress       = false ;   // Set true so that every transaction print a letter for what it does.
+    private static boolean silent               = false ;   // No progress output 
+    
+    static {
+        TransactionManager.DEBUG = inlineProgress ;     // This cause one character details to be printed. 
+
+        // Various flags (may not still exist)
+        //ObjectFileStorage.logging = true ;
+        // FileBase.DEBUG = inlineProgress ;
+        //NodeTableTrans.FIXUP = true ;
+        //NodeTableTrans.APPEND_LOG = true ;
+        // See also log4j.properties.
+    }
     
     static final int Iterations                 = MEM ? 10000 : 10000 ;
     // Output style.
@@ -92,16 +104,16 @@ public class Jena163_TDBDifferentIds
     // Jena-163 - good number choice?
     // 1/0/2  8/10  3/3/10  4
     
-    static final int numReaderTasks             = 1 ;   // 5
-    static final int numWriterTasksA            = 0 ;   // 3
-    static final int numWriterTasksC            = 2 ;   // 5
-
-    static final int readerSeqRepeats           = 8 ;   // 8
-    static final int readerMaxPause             = 10 ;  // 20
-
-    static final int writerAbortSeqRepeats      = 3 ;   // 4
-    static final int writerCommitSeqRepeats     = 3 ;   // 4
-    static final int writerMaxPause             = 10 ;  // 20
+    static final int numReaderTasks             = 5 ;   // 5
+    static final int numWriterTasksA            = 3 ;   // 3
+    static final int numWriterTasksC            = 5 ;   // 5
+
+    static final int readerSeqRepeats           = 4 ;   // 8
+    static final int readerMaxPause             = 20 ;  // 20
+
+    static final int writerAbortSeqRepeats      = 4 ;   // 4
+    static final int writerCommitSeqRepeats     = 4 ;   // 4
+    static final int writerMaxPause             = 20 ;  // 20
 
     static final int numThreadsInPool           = 4 ;           // If <= 0 then use an unbounded thread pool.   
     private static ExecutorService execService  = null ;
@@ -110,17 +122,6 @@ public class Jena163_TDBDifferentIds
     private static int numIterationsPerBlock    = 100 ;
     private static int colCount                 = 0 ;
     private static int colMax                   = 200 ;
-    private static boolean inlineProgress       = true ;   // Set true so that every transaction print a letter for what it does.
-    
-    static {
-        // Clearup !!!!!!!!!!!!!
-        TransactionManager.DEBUG = inlineProgress ; 
-        if ( false ) ObjectFileStorage.logging = true ;
-        if ( false ) FileBase.DEBUG = inlineProgress ;
-        NodeTableTrans.FIXUP = true ;
-        NodeTableTrans.APPEND_LOG = true ;
-        // See also log4j.properties.
-    }
     
     // Queue treads starting
     private static Semaphore startPoint ;
@@ -159,7 +160,7 @@ public class Jena163_TDBDifferentIds
             startTestIteration() ;         
             
             try {
-                new Jena163_TDBDifferentIds().manyReaderAndOneWriter() ;
+                new T_TransSystem().manyReaderAndOneWriter() ;
             } catch (TDBException ex)
             {
                 System.err.println() ;
@@ -427,7 +428,7 @@ public class Jena163_TDBDifferentIds
         return sConn ;
     }
     
-    public Jena163_TDBDifferentIds() {}
+    public T_TransSystem() {}
         
     //@Test
     public void manyRead()
@@ -620,7 +621,6 @@ public class Jena163_TDBDifferentIds
 
     private static void endTest()
     {
-        
         if ( colCount > 0 || iteration%numIterationsPerBlock != 0 )
         {
             println() ;
@@ -631,21 +631,27 @@ public class Jena163_TDBDifferentIds
 
     private static void print(String str)
     {
+        if ( silent ) return ;
         System.out.print(str) ;
     }
 
     private static void println(String string)
     {
-        print(string) ; println() ; 
+        if ( silent ) return ;
+        print(string) ;
+        println() ; 
     }
 
     private static void println()
     {
-        printf("\n") ; System.out.flush() ;
+        if ( silent ) return ;
+        printf("\n") ; 
+        System.out.flush() ;
     }
 
     private static void printf(String string, Object...args)
     {
+        if ( silent ) return ;
         System.out.printf(string, args) ;
     }