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/06 22:21:06 UTC

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

Author: andy
Date: Tue Dec  6 21:21:05 2011
New Revision: 1211161

URL: http://svn.apache.org/viewvc?rev=1211161&view=rev
Log:
Add file tracking.

Added:
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/ChannelManager.java
Modified:
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BlockAccessBase.java
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BlockAccessDirect.java
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BlockAccessMapped.java
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BufferChannelFile.java
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileBase.java
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/PlainFilePersistent.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/transaction/NodeTableTrans.java
    incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/TransactionManager.java
    incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestBlockAccessFixedSize.java
    incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestChannel.java
    incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestChannelFile.java
    incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestChannelMem.java

Modified: incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BlockAccessBase.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BlockAccessBase.java?rev=1211161&r1=1211160&r2=1211161&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BlockAccessBase.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BlockAccessBase.java Tue Dec  6 21:21:05 2011
@@ -43,7 +43,7 @@ public abstract class BlockAccessBase im
 
     public BlockAccessBase(String filename, int blockSize)
     {
-        file = new FileBase(filename) ;
+        file = FileBase.create(filename) ;
         this.blockSize = blockSize ;
         this.label = FileOps.basename(filename) ;
         long filesize = file.size() ;
@@ -125,7 +125,7 @@ public abstract class BlockAccessBase im
     }
     
     //@Override
-    final public boolean isClosed() { return file.channel == null ; }  
+    final public boolean isClosed() { return file.channel() == null ; }  
     
     protected final void checkIfClosed() 
     { 

Modified: incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BlockAccessDirect.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BlockAccessDirect.java?rev=1211161&r1=1211160&r2=1211161&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BlockAccessDirect.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BlockAccessDirect.java Tue Dec  6 21:21:05 2011
@@ -67,7 +67,7 @@ public class BlockAccessDirect extends B
     private void readByteBuffer(long id, ByteBuffer dst)
     {
         try {
-            int len = file.channel.read(dst, filePosition(id)) ;
+            int len = file.channel().read(dst, filePosition(id)) ;
             if ( len != blockSize )
                 throw new FileException(format("get: short read (%d, not %d)", len, blockSize)) ;   
         } catch (IOException ex)
@@ -90,7 +90,7 @@ public class BlockAccessDirect extends B
         bb.limit(bb.capacity()) ;   // It shouldn't have been changed.
         bb.rewind() ;
         try {
-            int len = file.channel.write(bb, filePosition(block.getId())) ;
+            int len = file.channel().write(bb, filePosition(block.getId())) ;
             if ( len != blockSize )
                 throw new FileException(format("write: short write (%d, not %d)", len, blockSize)) ;   
         } catch (IOException ex)

Modified: incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BlockAccessMapped.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BlockAccessMapped.java?rev=1211161&r1=1211160&r2=1211161&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BlockAccessMapped.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BlockAccessMapped.java Tue Dec  6 21:21:05 2011
@@ -217,7 +217,7 @@ public class BlockAccessMapped extends B
         if ( segBuffer == null )
         {
             try {
-                segBuffer = file.channel.map(MapMode.READ_WRITE, offset, SegmentSize) ;
+                segBuffer = file.channel().map(MapMode.READ_WRITE, offset, SegmentSize) ;
                 if ( getLog().isDebugEnabled() )
                     getLog().debug(format("Segment: %d", seg)) ;
                 segments[seg] = segBuffer ;

Modified: incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BufferChannelFile.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BufferChannelFile.java?rev=1211161&r1=1211160&r2=1211161&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BufferChannelFile.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/BufferChannelFile.java Tue Dec  6 21:21:05 2011
@@ -31,7 +31,7 @@ public class BufferChannelFile implement
 
     public BufferChannelFile(String filename)
     {
-        file = new FileBase(filename) ;
+        file = FileBase.create(filename) ;
     }
     
     @Override
@@ -43,14 +43,14 @@ public class BufferChannelFile implement
     @Override
     public long position()
     {
-        try { return file.channel.position() ; } 
+        try { return file.channel().position() ; } 
         catch (IOException e) { IO.exception(e) ; return -1 ; }
     }
 
     @Override
     public void position(long pos)
     {
-        try { file.channel.position(pos) ; } 
+        try { file.channel().position(pos) ; } 
         catch (IOException e) { IO.exception(e) ; }
     }
 
@@ -59,9 +59,9 @@ public class BufferChannelFile implement
     {
         try { 
             // http://bugs.sun.com/view_bug.do?bug_id=6191269
-            if ( length < file.channel.position() )
-                file.channel.position(length) ;
-            file.channel.truncate(length) ;
+            if ( length < file.channel().position() )
+                file.channel().position(length) ;
+            file.channel().truncate(length) ;
         }
         catch (IOException e) { IO.exception(e) ; }
     }
@@ -69,7 +69,7 @@ public class BufferChannelFile implement
     @Override
     public int read(ByteBuffer buffer)
     {
-        try { return file.channel.read(buffer) ; } 
+        try { return file.channel().read(buffer) ; } 
         catch (IOException e) { IO.exception(e) ; return -1 ; }
     }
     
@@ -77,28 +77,28 @@ public class BufferChannelFile implement
     @Override
     public int read(ByteBuffer buffer, long loc)
     {
-        try { return file.channel.read(buffer, loc) ; } 
+        try { return file.channel().read(buffer, loc) ; } 
         catch (IOException e) { IO.exception(e) ; return -1 ; }
     }
 
     @Override
     public int write(ByteBuffer buffer)
     {
-        try { return file.channel.write(buffer) ; } 
+        try { return file.channel().write(buffer) ; } 
         catch (IOException e) { IO.exception(e) ; return -1 ; }
     }
 
     @Override
     public int write(ByteBuffer buffer, long loc)
     {
-        try { return file.channel.write(buffer, loc) ; } 
+        try { return file.channel().write(buffer, loc) ; } 
         catch (IOException e) { IO.exception(e) ; return -1 ; }
     }
 
     @Override
     public long size()
     {
-        try { return file.channel.size() ; }
+        try { return file.channel().size() ; }
         catch (IOException e) { IO.exception(e) ; return -1 ; }
     }
 

Added: incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/ChannelManager.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/ChannelManager.java?rev=1211161&view=auto
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/ChannelManager.java (added)
+++ incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/ChannelManager.java Tue Dec  6 21:21:05 2011
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.hp.hpl.jena.tdb.base.file;
+
+import java.io.IOException ;
+import java.io.RandomAccessFile ;
+import java.nio.channels.Channel ;
+import java.nio.channels.FileChannel ;
+import java.util.HashMap ;
+import java.util.Map ;
+
+import org.openjena.atlas.io.IO ;
+
+public class ChannelManager
+{
+    // Because "FileManager" is already in use
+    // ChannelManager
+    
+    // Filebase ==> OpenFileRef
+    // ChannelRef
+    
+    public static FileChannel open(String filename)
+    {
+        return open(filename, "rw") ;
+    }
+    
+    public static FileChannel open(String filename, String mode)
+    {
+        return openref$(filename, "rw") ;
+    }
+    
+    static private Map<String, FileChannel> name2channel = new HashMap<String, FileChannel>() ;
+    static private Map<FileChannel, String> channel2name = new HashMap<FileChannel, String>() ;
+    
+    private static FileChannel openref$(String filename, String mode)
+    {
+        FileChannel chan = name2channel.get(filename) ;
+        if ( chan != null )
+        {
+            // Temp - for now, only journal files.
+            if ( filename.endsWith(".jrnl") )
+                // Scream - it's currently open.
+                throw new FileException("Already open: "+filename) ;
+        }
+        chan = open$(filename, mode) ;
+        name2channel.put(filename, chan) ;
+        channel2name.put(chan, filename) ;
+        return chan ;
+    }
+    
+    private static FileChannel open$(String filename, String mode)
+    {
+        try {
+            // "rwd" - Syncs only the file contents
+            // "rws" - Syncs the file contents and metadata
+            // "rw"  - OS write behind possible
+            // "r"   - read only
+            RandomAccessFile out = new RandomAccessFile(filename, mode) ;
+            FileChannel channel = out.getChannel() ;
+            return channel ;
+        } catch (IOException ex) { throw new FileException("Failed to open: "+filename+" (mode="+mode+")", ex) ; }
+    }
+    
+    public static void close(Channel chan)
+    {
+        IO.close(chan) ;
+        String name = channel2name.remove(chan) ;
+        name2channel.remove(name) ;
+    }
+}
+

Modified: incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileBase.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileBase.java?rev=1211161&r1=1211160&r2=1211161&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileBase.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileBase.java Tue Dec  6 21:21:05 2011
@@ -19,7 +19,6 @@
 package com.hp.hpl.jena.tdb.base.file;
 
 import java.io.IOException ;
-import java.io.RandomAccessFile ;
 import java.nio.channels.FileChannel ;
 
 import org.openjena.atlas.io.IO ;
@@ -28,26 +27,26 @@ import org.openjena.atlas.lib.Sync ;
 import org.slf4j.Logger ;
 import org.slf4j.LoggerFactory ;
 
-import com.hp.hpl.jena.tdb.base.block.BlockException ;
-
-public class FileBase implements Sync, Closeable
+public final class FileBase implements Sync, Closeable
 {
     static private Logger log = LoggerFactory.getLogger(FileBase.class) ; 
-    // Usually used as a mixin, which java does not support very well.
+    // A mixin, which java does not support very well.
     public final String filename ;
-    public FileChannel channel ;
-    public RandomAccessFile out ;
+    private FileChannel channel ;
     public static boolean DEBUG = false ;
     private final boolean DebugThis  ;
     private static long counter = 0 ;
     private final long id ;
 
-    public FileBase(String filename)
+    static FileBase create(String filename) { return new FileBase(filename) ; }
+    static FileBase create(String filename, String mode) { return new FileBase(filename, mode) ; }
+    
+    private /*public*/ FileBase(String filename)
     {
         this(filename, "rw") ;
     }
     
-    public FileBase(String filename, String mode)
+    private /*public*/ FileBase(String filename, String mode)
     {
         DebugThis = DEBUG && filename.contains("nodes.dat-jrnl") ;
         id  = (counter++) ;
@@ -55,15 +54,19 @@ public class FileBase implements Sync, C
         if ( DebugThis )
             log.debug("open: ["+id+"]"+filename) ;
         this.filename = filename ;
-        try {
-            // "rwd" - Syncs only the file contents
-            // "rws" - Syncs the file contents and metadata
-            // "rw" - cached?
-            out = new RandomAccessFile(filename, mode) ;
-            channel = out.getChannel() ;
-        } catch (IOException ex) { throw new BlockException("Failed to create FileBase", ex) ; }
+        channel = ChannelManager.open(filename, mode) ;
+//        try {
+//            // "rwd" - Syncs only the file contents
+//            // "rws" - Syncs the file contents and metadata
+//            // "rw"  - OS write behind possible
+//            // "r"   - read only
+//            RandomAccessFile out = new RandomAccessFile(filename, mode) ;
+//            channel = out.getChannel() ;
+//        } catch (IOException ex) { throw new BlockException("Failed to create FileBase", ex) ; }
     }
-
+    
+    public final FileChannel channel() { return channel ; }
+    
     public long size()
     {
         try {
@@ -71,19 +74,25 @@ public class FileBase implements Sync, C
         } catch (IOException ex)
         { IO.exception(ex) ; return -1L ; }
     }
+
+    public boolean isClosed()
+    {
+        return channel == null ;
+    }
+
     
     @Override
     public void close()
     {
         if ( DebugThis )
             log.debug("close: ["+id+"]: "+filename) ;
-        try {
-            channel.close() ;
-            channel = null ;
-            out = null ;
-        } catch (IOException ex)
-        { throw new FileException("FileBase.close", ex) ; }
-
+        ChannelManager.close(channel) ;
+        channel = null ;
+//        try {
+//            channel.close() ;
+//            channel = null ;
+//        } catch (IOException ex)
+//        { throw new FileException("FileBase.close", ex) ; }
     }
 
     @Override

Modified: incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/PlainFilePersistent.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/PlainFilePersistent.java?rev=1211161&r1=1211160&r2=1211161&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/PlainFilePersistent.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/PlainFilePersistent.java Tue Dec  6 21:21:05 2011
@@ -41,7 +41,7 @@ public class PlainFilePersistent extends
     
     PlainFilePersistent(String filename)
     {
-        file = new FileBase(filename) ;
+        file = FileBase.create(filename) ;
         //long filesize = file.out.length() ;
         //if ( channel.size() == 0 ) {}
         byteBuffer = allocateBuffer(filesize) ;
@@ -62,7 +62,7 @@ public class PlainFilePersistent extends
     @Override
     protected ByteBuffer allocateBuffer(long size)
     {
-        try { return file.channel.map(FileChannel.MapMode.READ_WRITE, 0, size) ; }
+        try { return file.channel().map(FileChannel.MapMode.READ_WRITE, 0, size) ; }
         catch (IOException ex)  { IO.exception(ex) ; return null ; }
     }
 }

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=1211161&r1=1211160&r2=1211161&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 Tue Dec  6 21:21:05 2011
@@ -285,12 +285,12 @@ public class ObjectFileStorage implement
         lengthBuffer.clear() ;
         int x = file.read(lengthBuffer, loc) ;
         if ( x != 4 )
-            throw new FileException("ObjectFile.read["+file.getLabel()+"]("+loc+")["+filesize+"]["+file.size()+"]: Failed to read the length : got "+x+" bytes") ;
+            throw new FileException("ObjectFile.read["+file.getLabel()+"]("+loc+")[filesize="+filesize+"]filesize="+file.size()+"]: Failed to read the length : got "+x+" bytes") ;
         int len = lengthBuffer.getInt(0) ;
         // Sanity check. 
         if ( len > filesize-(loc+SizeOfInt) )
         {
-            String msg = "ObjectFile.read["+file.getLabel()+"]("+loc+")["+filesize+"]["+file.size()+"]: Impossibly large object : "+len+" bytes" ;
+            String msg = "ObjectFile.read["+file.getLabel()+"]("+loc+")[filesize="+filesize+"][filesize="+file.size()+"]: Impossibly large object : "+len+" bytes > filesize-(loc+SizeOfInt)="+(filesize-(loc+SizeOfInt)) ;
             SystemTDB.errlog.error(msg) ;
             throw new FileException(msg) ;
         }

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=1211161&r1=1211160&r2=1211161&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 Tue Dec  6 21:21:05 2011
@@ -179,6 +179,7 @@ public class NodeTableTrans implements N
                 System.err.printf("journalStartOffset reset to zero") ;
                 journalObjFileStartOffset = 0 ;
                 journalObjFile.truncate(0) ;
+                journalObjFile.sync() ;
             }
         }
         offset += journalObjFileStartOffset ;
@@ -276,7 +277,7 @@ public class NodeTableTrans implements N
             throw new TDBTransactionException(txn.getLabel()+": Not in a transaction for a commit to happen") ;
         writeNodeJournal() ;
         
-        if ( journalObjFile.length() != 0 )
+        if ( journalObjFile != null && journalObjFile.length() != 0 )
         {
             long x = journalObjFile.length() ;
             throw new TDBTransactionException(txn.getLabel()+": journalObjFile not cleared ("+x+")") ;
@@ -310,7 +311,9 @@ public class NodeTableTrans implements N
         nodeIndex.clear() ;
         // Fixes nodeTableJournal
         journalObjFile.truncate(journalObjFileStartOffset) ;
-        journalObjFile.sync() ;
+        //journalObjFile.sync() ;
+        journalObjFile.close() ;
+        journalObjFile = null ;
         base.sync() ;
         offset = -99 ; // base.allocOffset().getId() ; // Will be invalid as we may write through to the base table later.
         passthrough = true ;

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=1211161&r1=1211160&r2=1211161&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 Tue Dec  6 21:21:05 2011
@@ -343,6 +343,9 @@ public class TransactionManager
             if ( DEBUG ) System.out.print('_') ;
         }
         Transaction txn = createTransaction(dsg, mode, label) ;
+        
+        log("begin$", txn) ;
+        
         DatasetGraphTxn dsgTxn = (DatasetGraphTxn)new DatasetBuilderTxn(this).build(txn, mode, dsg) ;
         txn.setActiveDataset(dsgTxn) ;
 
@@ -424,7 +427,7 @@ public class TransactionManager
         if ( log() )
             log("Start flush delayed commits", txn) ;
         
-        if ( DEBUG ) checkNodesDatJrnl(txn) ;
+        if ( DEBUG ) checkNodesDatJrnl("1", txn) ;
         
         if ( queue.size() == 0 && txn != null )
             // Nothing to do - journal should be empty. 
@@ -442,7 +445,7 @@ public class TransactionManager
                     continue ;
                 if ( log() )
                     log("Flush delayed commit of "+txn2.getLabel(), txn) ;
-                if ( DEBUG ) checkNodesDatJrnl(txn) ;
+                if ( DEBUG ) checkNodesDatJrnl("2", txn) ;
                 checkReplaySafe() ;
                 enactTransaction(txn2) ;
                 commitedAwaitingFlush.remove(txn2) ;
@@ -451,12 +454,12 @@ public class TransactionManager
         }
 
         checkReplaySafe() ;
-        if ( DEBUG ) checkNodesDatJrnl(txn) ;
+        if ( DEBUG ) checkNodesDatJrnl("3", txn) ;
 
         // Whole journal to base database
         JournalControl.replay(journal, baseDataset) ;
 
-        if ( DEBUG ) checkNodesDatJrnl(txn) ;
+        if ( DEBUG ) checkNodesDatJrnl("4", txn) ;
         
         checkReplaySafe() ;
         if ( log() )
@@ -466,18 +469,14 @@ public class TransactionManager
         
     }
 
-    private static void checkNodesDatJrnl(Transaction txn)
+    private static void checkNodesDatJrnl(String label, Transaction txn)
     {
         if (txn != null)
         {
-            String x = txn.getBaseDataset().getLocation().getPath("nodes.dat-jrnl") ;
+            String x = txn.getBaseDataset().getLocation().getPath(label+": nodes.dat-jrnl") ;
             long len = new File(x).length() ;
             if (len != 0)
-            {
-                System.out.flush() ;
-                System.err.println("Not zero 1") ;
-                System.err.println("Not zero 2") ;
-            }
+                log("nodes.dat-jrnl: not empty", txn) ;
         }   
     }
     
@@ -570,12 +569,12 @@ public class TransactionManager
         return journal ;
     }
 
-    private boolean log()
+    private static boolean log()
     {
         return syslog.isDebugEnabled() || log.isDebugEnabled() ;
     }
     
-    private void log(String msg, Transaction txn)
+    private static void log(String msg, Transaction txn)
     {
         if ( ! log() )
             return ;
@@ -585,7 +584,7 @@ public class TransactionManager
             logger().debug(txn.getLabel()+": "+msg) ;
     }
 
-    private Logger logger()
+    private static Logger logger()
     {
         if ( syslog.isDebugEnabled() )
             return syslog ;

Modified: incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestBlockAccessFixedSize.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestBlockAccessFixedSize.java?rev=1211161&r1=1211160&r2=1211161&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestBlockAccessFixedSize.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestBlockAccessFixedSize.java Tue Dec  6 21:21:05 2011
@@ -19,6 +19,8 @@
 package com.hp.hpl.jena.tdb.base.file;
 
 import static com.hp.hpl.jena.tdb.base.BufferTestLib.sameValue ;
+import org.junit.After ;
+import org.junit.Before ;
 import org.junit.Test ;
 import org.openjena.atlas.junit.BaseTest ;
 
@@ -44,22 +46,23 @@ public abstract class AbstractTestBlockA
         return b ;
     }
 
+    private BlockAccess file ;
+    @Before public void before() { file = make() ; }
+    @After  public void after()  { file.close() ; }
+
     @Test public void fileaccess_01()
     {
-        BlockAccess file = make() ;
         assertTrue(file.isEmpty()) ;
     }
     
     @Test public void fileaccess_02()
     {
-        BlockAccess file = make() ;
         Block b = data(file, blkSize) ;
         file.write(b) ;
     }
 
     @Test public void fileaccess_03()
     {
-        BlockAccess file = make() ;
         Block b1 = data(file, blkSize) ;
         file.write(b1) ;
         long x = b1.getId() ;
@@ -73,7 +76,6 @@ public abstract class AbstractTestBlockA
     
     @Test public void fileaccess_04()
     {
-        BlockAccess file = make() ;
         Block b1 = data(file, blkSize) ;
         Block b2 = data(file, blkSize) ;
         file.write(b1) ;
@@ -89,7 +91,6 @@ public abstract class AbstractTestBlockA
     @Test(expected=FileException.class)
     public void fileaccess_05()
     {
-        BlockAccess file = make() ;
         Block b1 = data(file, 10) ;
         Block b2 = data(file, 20) ;
         file.write(b1) ;

Modified: incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestChannel.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestChannel.java?rev=1211161&r1=1211160&r2=1211161&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestChannel.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestChannel.java Tue Dec  6 21:21:05 2011
@@ -22,19 +22,20 @@ import java.nio.ByteBuffer ;
 
 import com.hp.hpl.jena.tdb.base.file.BufferChannel ;
 
+import org.junit.After ;
+import org.junit.Before ;
 import org.junit.Test ;
 import org.openjena.atlas.junit.BaseTest ;
 
 public abstract class AbstractTestChannel extends BaseTest
 {
-    protected abstract BufferChannel make() ;
-    static final int blkSize = 100 ;
+    protected abstract BufferChannel open() ;
     
-    @Test public void storage_01() 
-    {
-        BufferChannel store = make() ;
-        assertEquals(0, store.size()) ;
-    }
+    private BufferChannel store ;
+    @Before public void before() { store = open() ; }
+    @After  public void after()  { store.close() ; }
+    
+    static final int blkSize = 100 ;
     
     protected static ByteBuffer data(int len)
     {
@@ -47,7 +48,6 @@ public abstract class AbstractTestChanne
     
     protected static boolean same(ByteBuffer bb1, ByteBuffer bb2)
     {
-        
         if ( bb1.capacity() != bb2.capacity() ) return false ;
         bb1.clear() ;
         bb2.clear() ;
@@ -56,9 +56,13 @@ public abstract class AbstractTestChanne
         return true ;
     }
 
+    @Test public void storage_01() 
+    {
+        assertEquals(0, store.size()) ;
+    }
+    
     @Test public void storage_02()
     {
-        BufferChannel store = make() ;
         ByteBuffer b = data(blkSize) ;
         store.write(b) ;
         long x = store.size() ;
@@ -67,7 +71,6 @@ public abstract class AbstractTestChanne
 
     @Test public void storage_03()
     {
-        BufferChannel store = make() ;
         ByteBuffer b1 = data(blkSize) ;
         long posn = store.position() ; 
         store.write(b1) ;
@@ -79,7 +82,6 @@ public abstract class AbstractTestChanne
     
     @Test public void storage_04()
     {
-        BufferChannel store = make() ;
         ByteBuffer b1 = data(blkSize) ;
         ByteBuffer b2 = data(blkSize/2) ;
 
@@ -94,7 +96,6 @@ public abstract class AbstractTestChanne
     
     @Test public void storage_05()
     {
-        BufferChannel store = make() ;
         ByteBuffer b1 = data(blkSize) ;
         ByteBuffer b1a = ByteBuffer.allocate(blkSize) ;
         ByteBuffer b2 = data(blkSize/2) ;
@@ -110,7 +111,6 @@ public abstract class AbstractTestChanne
     
     @Test public void storage_06()
     {
-        BufferChannel store = make() ;
         ByteBuffer b1 = data(blkSize) ;
         store.write(b1) ;
         store.truncate(0) ;
@@ -122,7 +122,6 @@ public abstract class AbstractTestChanne
     
     @Test public void storage_07()
     {
-        BufferChannel store = make() ;
         ByteBuffer b1 = data(blkSize) ;
         store.write(b1) ;
         store.position(10) ;
@@ -134,7 +133,6 @@ public abstract class AbstractTestChanne
     
     @Test public void storage_08()
     {
-        BufferChannel store = make() ;
         ByteBuffer b1 = data(blkSize) ;
         ByteBuffer b2 = data(blkSize) ;
         store.write(b1) ;

Modified: incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestChannelFile.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestChannelFile.java?rev=1211161&r1=1211160&r2=1211161&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestChannelFile.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestChannelFile.java Tue Dec  6 21:21:05 2011
@@ -32,7 +32,7 @@ public class TestChannelFile extends Abs
     @AfterClass public static void cleanup() { FileOps.deleteSilent(filename) ; } 
     
     @Override
-    protected BufferChannel make()
+    protected BufferChannel open()
     {
         FileOps.deleteSilent(filename) ;
         return new BufferChannelFile(filename) ;

Modified: incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestChannelMem.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestChannelMem.java?rev=1211161&r1=1211160&r2=1211161&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestChannelMem.java (original)
+++ incubator/jena/Jena2/TDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestChannelMem.java Tue Dec  6 21:21:05 2011
@@ -27,7 +27,7 @@ public class TestChannelMem extends Abst
     static int counter = 0 ;
     
     @Override
-    protected BufferChannel make()
+    protected BufferChannel open()
     {
         return BufferChannelMem.create("Test-"+(counter++)) ;
     }