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

svn commit: r1125503 - in /incubator/jena: Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/ Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base...

Author: andy
Date: Fri May 20 18:51:50 2011
New Revision: 1125503

URL: http://svn.apache.org/viewvc?rev=1125503&view=rev
Log: (empty)

Added:
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileStorage.java   (with props)
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/Storage.java   (with props)
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/StorageFile.java   (with props)
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/StorageMem.java   (with props)
    incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestFileAccessFixedSize.java   (contents, props changed)
      - copied, changed from r1125428, incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractFileAccessFixedSizeTest.java
    incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestFileAccessVarSize.java   (contents, props changed)
      - copied, changed from r1125428, incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractFileAccessVarSizeTest.java
    incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessDirect.java   (with props)
    incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessMapped.java   (with props)
    incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/
    incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/AbstractTestStorage.java   (with props)
    incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TS_Storage.java   (with props)
    incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TestStorageFile.java   (with props)
    incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TestStorageMem.java   (with props)
Removed:
    incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractFileAccessFixedSizeTest.java
    incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractFileAccessVarSizeTest.java
Modified:
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileAccess.java
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileAccessBase.java
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileAccessByteArray.java
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileDiskDirect.java
    incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/TC_Base.java
    incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TS_File.java
    incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessByteArray.java
    incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessMem.java
    incubator/jena/Jena2/Fuseki/trunk/src/main/java/org/openjena/fuseki/servlets/SPARQL_Query.java

Modified: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileAccess.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileAccess.java?rev=1125503&r1=1125502&r2=1125503&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileAccess.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileAccess.java Fri May 20 18:51:50 2011
@@ -12,7 +12,7 @@ import org.openjena.atlas.lib.Sync ;
 import com.hp.hpl.jena.tdb.base.block.Block ;
 
 /** Interface to concrete storage.
- *  This is wrapped in a BlockMgrAccess to add the in-memory tracking of read and write blocks,
+ *  This is wrapped in a BlockMgrAccess to provide a higher level abstraction.
  */
 public interface FileAccess extends Sync, Closeable
 {

Modified: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileAccessBase.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileAccessBase.java?rev=1125503&r1=1125502&r2=1125503&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileAccessBase.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileAccessBase.java Fri May 20 18:51:50 2011
@@ -21,7 +21,7 @@ import com.hp.hpl.jena.tdb.base.block.Bl
 import com.hp.hpl.jena.tdb.base.block.BlockException ;
 import com.hp.hpl.jena.tdb.sys.SystemTDB ;
 
-/** Support for a disk file backed FielAccess */
+/** Support for a disk file backed FileAccess */
 public abstract class FileAccessBase implements FileAccess 
 {
     final protected int blockSize ;

Modified: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileAccessByteArray.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileAccessByteArray.java?rev=1125503&r1=1125502&r2=1125503&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileAccessByteArray.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/file/FileAccessByteArray.java Fri May 20 18:51:50 2011
@@ -18,7 +18,7 @@ public class FileAccessByteArray impleme
 {
     private ByteBuffer bytes ;
     private long length ;           // Bytes in use: 0 to length-1 
-    private long alloc ;           // Bytes in use: 0 to length-1
+    private long alloc ;            // Bytes allocated
     
     public FileAccessByteArray()
     {
@@ -32,7 +32,7 @@ public class FileAccessByteArray impleme
     {
         long addr = alloc ;
         ByteBuffer bb = ByteBuffer.allocate(size) ;
-        alloc += size + SizeOfInt ;
+        alloc += (size + SizeOfInt) ;
         return new Block((int)addr, bb) ; 
     }
 

Modified: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileDiskDirect.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileDiskDirect.java?rev=1125503&r1=1125502&r2=1125503&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileDiskDirect.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileDiskDirect.java Fri May 20 18:51:50 2011
@@ -20,8 +20,11 @@ import com.hp.hpl.jena.tdb.base.block.Bl
 import com.hp.hpl.jena.tdb.base.file.FileBase ;
 import com.hp.hpl.jena.tdb.base.file.FileException ;
 
-/** Variable length ByteBuffer file on disk.  Read by id ; write is append-only */  
+/** ObjectFile backed by a rgular disk file
+ *  Buffering of writes to disk.  
+ */
 
+@Deprecated
 public class ObjectFileDiskDirect implements ObjectFile 
 {
     /* 
@@ -35,7 +38,7 @@ public class ObjectFileDiskDirect implem
      * Writing is buffered.
      */
     
-    // Replaces with a FileAccess wrapper?
+    // Replaces with a Storage wrapper?
     
     // One disk file size.
     protected long filesize ;

Added: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileStorage.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileStorage.java?rev=1125503&view=auto
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileStorage.java (added)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileStorage.java Fri May 20 18:51:50 2011
@@ -0,0 +1,300 @@
+/*
+ * (c) Copyright 2008, 2009 Hewlett-Packard Development Company, LP
+ * All rights reserved.
+ * [See end of file]
+ */
+
+package com.hp.hpl.jena.tdb.base.objectfile;
+
+import static com.hp.hpl.jena.tdb.sys.SystemTDB.ObjectFileWriteCacheSize ;
+import static com.hp.hpl.jena.tdb.sys.SystemTDB.SizeOfInt ;
+
+import java.nio.ByteBuffer ;
+import java.util.Iterator ;
+
+import org.openjena.atlas.lib.Bytes ;
+import org.openjena.atlas.lib.Pair ;
+
+import com.hp.hpl.jena.tdb.base.file.FileException ;
+import com.hp.hpl.jena.tdb.base.storage.Storage ;
+
+/** Variable length ByteBuffer file on disk. 
+ *  Buffering for delayed writes.
+ */  
+
+public class ObjectFileStorage implements ObjectFile 
+{
+    /* 
+     * No synchronization - assumes that the caller has some appropriate lock
+     * because the combination of file and cache operations needs to be thread safe.
+     * 
+     * The position of the channel is assumed to be the end of the file always.
+     * Read operations are done with absolute channel calls, 
+     * which do not reset the position.
+     * 
+     * Writing is buffered.
+     */
+    
+    // One disk file size.
+    
+    // This adds to the length of the file  
+    private ByteBuffer lengthBuffer = ByteBuffer.allocate(SizeOfInt) ;
+    
+    // Delayed write buffer.
+    private ByteBuffer output = ByteBuffer.allocate(ObjectFileWriteCacheSize) ;
+    private int bufferSize ;
+    
+    private final Storage file ;                // Access to storage
+    private long filesize ;                     // Size of on-disk. 
+    
+    // Two-step write - alloc, write
+    private boolean inAllocWrite = true ;
+    private ByteBuffer allocByteBuffer = null ;
+    private long allocLocation = -1 ;
+
+    public ObjectFileStorage(Storage file, int bufferSize)
+    {
+        this.file = file ;
+        this.bufferSize = bufferSize ;
+        filesize = file.length() ;
+    }
+    
+    @Override
+    public long write(ByteBuffer bb)
+    {
+        
+        int len = bb.limit() - bb.position() ;
+        
+        if ( output.limit()+len > output.capacity() )
+            // No room - flush.
+            flushOutputBuffer() ;
+        // Is there room now?
+        // XXX
+        System.err.println("Use the delayed write buffer") ;
+        
+        lengthBuffer.clear() ;
+        lengthBuffer.putInt(0, len) ;
+        
+        long location = filesize ;
+        int x = file.write(bb, location) ;
+        long loc2 = location+SizeOfInt ;
+        x += file.write(bb, loc2) ;
+        filesize = filesize+x ;
+        return location ;
+    }
+    
+    @Override
+    public ByteBuffer allocWrite(int maxBytes)
+    {
+        // Include space for length.
+        int spaceRequired = maxBytes + SizeOfInt ;
+        // Find space.
+        if ( spaceRequired > output.remaining() )
+            flushOutputBuffer() ;
+        
+        if ( spaceRequired > output.remaining() )
+        {
+            // Too big.
+            inAllocWrite = true ;
+            allocByteBuffer = ByteBuffer.allocate(spaceRequired) ;
+            allocLocation = -1 ;
+            return allocByteBuffer ;  
+        }
+        
+        // Will fit.
+        inAllocWrite = true ;
+        int start = output.position() ;
+        // id (but don't tell the caller yet).
+        allocLocation = filesize+start ;
+        
+        // Slice it.
+        output.position(start + SizeOfInt) ;
+        output.limit(start+spaceRequired) ;
+        ByteBuffer bb = output.slice() ; 
+
+        allocByteBuffer = bb ;
+        return bb ;
+    }
+
+    @Override
+    public long completeWrite(ByteBuffer buffer)
+    {
+        if ( ! inAllocWrite )
+            throw new FileException("Not in the process of an allocated write operation pair") ;
+        if ( allocByteBuffer != buffer )
+            throw new FileException("Wrong byte buffer in an allocated write operation pair") ;
+
+        if ( allocLocation == -1 )
+            // It was too big to use the buffering.
+            return write(buffer) ;
+        
+        int actualLength = buffer.limit()-buffer.position() ;
+        // Insert object length
+        int idx = (int)(allocLocation-filesize) ;
+        output.putInt(idx, actualLength) ;
+        // And bytes to idx+actualLength+4 are used
+        inAllocWrite = false;
+        allocByteBuffer = null ;
+        int newLen = idx+actualLength+4 ;
+        output.position(newLen);
+        output.limit(output.capacity()) ;
+        return allocLocation ;
+    }
+
+    private void flushOutputBuffer()
+    {
+        long location = filesize ;
+        output.flip();
+        int x = file.write(output) ;
+        filesize += x ;
+        output.clear() ;
+    }
+
+
+    @Override
+    public ByteBuffer read(long loc)
+    {
+        if ( loc < 0 )
+            throw new IllegalArgumentException("ObjectFile.read: Bad read: "+loc) ;
+        
+        // Maybe it's in the in the write buffer
+        if ( loc >= filesize )
+        {
+            if ( loc > filesize+output.capacity() )
+                throw new IllegalArgumentException("ObjectFile.read: Bad read: "+loc) ;
+            
+            int x = output.position() ;
+            int y = output.limit() ;
+            
+            int offset = (int)(loc-filesize) ;
+            int len = output.getInt(offset) ;
+            int posn = offset + SizeOfInt ;
+            // Slice the data bytes,
+            output.position(posn) ;
+            output.limit(posn+len) ;
+            ByteBuffer bb = output.slice() ;
+            output.limit(y) ;
+            output.position(x) ;
+            return bb ; 
+        }
+        
+        // No - it's in the underlying file storage.
+        lengthBuffer.position(0) ;
+        int x = file.read(lengthBuffer, loc) ;
+        if ( x != 4 )
+            throw new FileException("ObjectFile.read: Failed to read the length : got "+x+" bytes") ;
+        int len = lengthBuffer.getInt(0) ;
+        ByteBuffer bb = ByteBuffer.allocate(len) ;
+        x = file.read(bb, loc+SizeOfInt) ;
+        bb.flip() ;
+        if ( x != len )
+            throw new FileException("ObjectFile.read: Failed to read the object ("+len+" bytes) : got "+x+" bytes") ;
+        return bb ;
+    }
+    
+    @Override
+    public long length()
+    {
+        return filesize ;
+    }
+
+    @Override
+    public void close()                 { flushOutputBuffer() ; file.close() ; }
+
+    @Override
+    public void sync()                  { flushOutputBuffer() ; file.sync() ; }
+
+    @Override
+    public Iterator<Pair<Long, ByteBuffer>> all()
+    {
+        file.position(0) ; 
+        ObjectIterator iter = new ObjectIterator(0, filesize) ;
+        return iter ;
+    }
+    
+    private class ObjectIterator implements Iterator<Pair<Long, ByteBuffer>>
+    {
+        final private long start ;
+        final private long finish ;
+        private long current ;
+
+        public ObjectIterator(long start, long finish)
+        {
+            this.start = start ;
+            this.finish = finish ;
+            this.current = start ;
+        }
+        
+        @Override
+        public boolean hasNext()
+        {
+            return ( current < finish ) ;
+        }
+
+        @Override
+        public Pair<Long, ByteBuffer> next()
+        {
+            long x = current ;
+            ByteBuffer bb = read(current) ;
+            current = current + bb.limit() + 4 ; 
+            return new Pair<Long, ByteBuffer>(x, bb) ;
+        }
+
+        @Override
+        public void remove()
+        { throw new UnsupportedOperationException() ; }
+    }
+    
+    // ---- Dump
+    public void dump() { dump(handler) ; }
+
+    public interface DumpHandler { void handle(long fileIdx, String str) ; }  
+    
+    public void dump(DumpHandler handler)
+    {
+        file.position(0) ; 
+        long fileIdx = 0 ;
+        while ( fileIdx < filesize )
+        {
+            ByteBuffer bb = read(fileIdx) ;
+            String str = Bytes.fromByteBuffer(bb) ;
+            handler.handle(fileIdx, str) ;
+            fileIdx = fileIdx + bb.limit() + 4 ;
+        }
+    }
+    
+    static DumpHandler handler = new DumpHandler() {
+        @Override
+        public void handle(long fileIdx, String str)
+        {
+            System.out.printf("0x%08X : %s\n", fileIdx, str) ;
+        }
+    } ;
+}
+
+/*
+ * (c) Copyright 2008, 2009 Hewlett-Packard Development Company, LP
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the above copyright
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ * 3. The name of the author may not be used to endorse or promote products
+ *    derived from this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
+ * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+ * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+ * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
\ No newline at end of file

Propchange: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/objectfile/ObjectFileStorage.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/Storage.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/Storage.java?rev=1125503&view=auto
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/Storage.java (added)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/Storage.java Fri May 20 18:51:50 2011
@@ -0,0 +1,80 @@
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ * [See end of file]
+ */
+
+package com.hp.hpl.jena.tdb.base.storage;
+
+import java.nio.ByteBuffer ;
+
+import org.openjena.atlas.lib.Closeable ;
+import org.openjena.atlas.lib.Sync ;
+
+import com.hp.hpl.jena.tdb.base.file.FileAccess ;
+
+/** Interface to storage : a simplifed version of FileChannel
+ *  @see FileAccess
+ */
+public interface Storage extends Sync, Closeable
+{
+    // This is a simple, low level "file = array of bytes" interface"
+    // This interface does not support slicing - so it's not suitable for memory mapped I/O
+    // but it is suitable for compression.
+    
+    /** return the position */
+    public long position() ;
+    
+    /** set the position */
+    public void position(long pos) ;
+
+    /** Read into a ByteBuffer. Returns the number of bytes read.
+     */
+    public int read(ByteBuffer buffer) ;
+    
+    /** Read into a ByteBuffer, starting at position loc. Return the number of bytes read.
+     * loc must be within the file.
+     */
+    public int read(ByteBuffer buffer, long loc) ;
+
+    /** Write from ByteBuffer, starting at position loc.  
+     * Return the number of bytes written
+     */
+    public int write(ByteBuffer buffer) ;
+    
+    /** Write from ByteBuffer, starting at position loc.  
+     * Return the number of bytes written.
+     * loc must be within 0 to length - writing at length is append */
+    public int write(ByteBuffer buffer, long loc) ;
+    
+    /** Length of storage, in bytes.*/
+    public long length() ;
+
+}
+
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the above copyright
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ * 3. The name of the author may not be used to endorse or promote products
+ *    derived from this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
+ * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+ * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+ * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
\ No newline at end of file

Propchange: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/Storage.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/StorageFile.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/StorageFile.java?rev=1125503&view=auto
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/StorageFile.java (added)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/StorageFile.java Fri May 20 18:51:50 2011
@@ -0,0 +1,117 @@
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ * [See end of file]
+ */
+
+package com.hp.hpl.jena.tdb.base.storage;
+
+import java.io.IOException ;
+import java.nio.ByteBuffer ;
+
+import org.openjena.atlas.io.IO ;
+
+import com.hp.hpl.jena.tdb.base.file.FileBase ;
+
+public class StorageFile implements Storage
+{
+    private FileBase file ;
+
+    public StorageFile(String filename)
+    {
+        file = new FileBase(filename) ;
+    }
+    
+    @Override
+    public long 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) ; } 
+        catch (IOException e) { IO.exception(e) ; }
+    }
+
+    @Override
+    public int read(ByteBuffer buffer)
+    {
+        try { return file.channel.read(buffer) ; } 
+        catch (IOException e) { IO.exception(e) ; return -1 ; }
+    }
+    
+    
+    @Override
+    public int read(ByteBuffer buffer, long 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) ; } 
+        catch (IOException e) { IO.exception(e) ; return -1 ; }
+    }
+
+    @Override
+    public int write(ByteBuffer buffer, long loc)
+    {
+        try { return file.channel.write(buffer, loc) ; } 
+        catch (IOException e) { IO.exception(e) ; return -1 ; }
+    }
+
+    @Override
+    public long length()
+    {
+        try { return file.channel.size() ; }
+        catch (IOException e) { IO.exception(e) ; return -1 ; }
+    }
+
+    @Override
+    public void sync()
+    { 
+        try { file.channel.force(true) ; }
+        catch (IOException e) { IO.exception(e) ; }
+    }
+
+    @Override
+    public void close()
+    {
+        try { file.channel.close() ; }
+        catch (IOException e) { IO.exception(e) ; }
+    }
+
+    
+}
+
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the above copyright
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ * 3. The name of the author may not be used to endorse or promote products
+ *    derived from this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
+ * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+ * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+ * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
\ No newline at end of file

Propchange: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/StorageFile.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/StorageMem.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/StorageMem.java?rev=1125503&view=auto
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/StorageMem.java (added)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/StorageMem.java Fri May 20 18:51:50 2011
@@ -0,0 +1,156 @@
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ * [See end of file]
+ */
+
+package com.hp.hpl.jena.tdb.base.storage;
+
+import java.nio.ByteBuffer ;
+
+import com.hp.hpl.jena.tdb.base.StorageException ;
+
+public class StorageMem implements Storage
+{
+    private ByteBuffer bytes ;      // Position is our file position.
+    private long length ;           // Bytes in use: 0 to length-1
+    private String name ;
+    private static int INIT_SIZE = 1024 ;
+    private static int INC_SIZE = 1024 ;
+    
+    public StorageMem(String name)
+    {
+        bytes = ByteBuffer.allocate(1024) ;
+        length = 0 ; 
+        this.name = name ;
+    }
+
+    @Override
+    public long position()
+    {
+        checkIfClosed() ;
+        return bytes.position() ;
+    }
+
+    @Override
+    public void position(long pos)
+    { 
+        checkIfClosed() ;
+        if ( pos < 0 || pos > bytes.capacity() )
+            throw new StorageException("Out of range: "+pos) ;
+        bytes.position((int)pos) ;
+    }
+
+    @Override
+    public int read(ByteBuffer buffer)
+    {
+        checkIfClosed() ;
+        int x = bytes.position();
+        
+        int len = buffer.limit()-buffer.position() ;
+        if ( len > bytes.remaining() )
+            len = bytes.remaining() ;
+        // Copy out, moving the position of the bytes of stroage. 
+        for (int i = 0; i < len; i++)
+            buffer.put(bytes.get());
+        return len ;
+    }
+    
+    @Override
+    public int read(ByteBuffer buffer, long loc)
+    {
+        checkIfClosed() ;
+        if ( loc < 0 || loc > length )
+            throw new StorageException("Out of range: "+loc) ;
+        int x = buffer.position() ;
+        bytes.position((int)loc) ;
+        int len = read(buffer) ;
+        bytes.position(x) ;
+        return len ;
+    }
+
+    @Override
+    public int write(ByteBuffer buffer)
+    {
+        checkIfClosed() ;
+        int len = buffer.limit()-buffer.position() ;
+        int posn = bytes.position() ;
+
+        if ( len > bytes.remaining() )
+        {
+            int inc = len-bytes.remaining() ;
+            inc += INC_SIZE ;
+            ByteBuffer bb2 = ByteBuffer.allocate(bytes.capacity()+inc) ;
+            bytes.clear() ;
+            // Copy contents.
+            bb2.put(bytes) ;
+            bytes.position(posn) ;
+        }
+        bytes.put(buffer) ;
+        length = Math.max(length, posn+len) ;
+        return len ;
+    }
+    
+    @Override
+    public int write(ByteBuffer buffer, long loc)
+    {
+        checkIfClosed() ;
+        if ( loc < 0 || loc > length )
+            throw new StorageException("Out of range: "+loc) ;
+        int x = bytes.position() ; 
+        bytes.position((int)loc) ;
+        int len = write(buffer) ;
+        bytes.position(x) ;
+        return len ;
+    }
+
+    @Override
+    public long length()
+    {
+        checkIfClosed() ;
+        return length ;
+    }
+    
+    @Override
+    public void sync()
+    { 
+        checkIfClosed() ;
+    }
+
+    @Override
+    public void close()
+    { checkIfClosed() ; bytes = null ; }
+    
+    private void checkIfClosed()
+    {
+        if ( bytes == null )
+            throw new StorageException("Closed: "+name) ;
+    }
+}
+
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the above copyright
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ * 3. The name of the author may not be used to endorse or promote products
+ *    derived from this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
+ * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+ * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+ * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
\ No newline at end of file

Propchange: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/base/storage/StorageMem.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/TC_Base.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/TC_Base.java?rev=1125503&r1=1125502&r2=1125503&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/TC_Base.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/TC_Base.java Fri May 20 18:51:50 2011
@@ -1,5 +1,6 @@
 /*
  * (c) Copyright 2008, 2009 Hewlett-Packard Development Company, LP
+ * (c) Copyright 2011 Epimorphics ltd.
  * All rights reserved.
  * [See end of file]
  */
@@ -10,14 +11,16 @@ import com.hp.hpl.jena.tdb.base.block.TS
 import com.hp.hpl.jena.tdb.base.file.TS_File;
 import com.hp.hpl.jena.tdb.base.record.TS_Record;
 import com.hp.hpl.jena.tdb.base.recordfile.TS_RecordFile;
+import com.hp.hpl.jena.tdb.base.storage.TS_Storage ;
 
 import org.junit.runner.RunWith;
 import org.junit.runners.Suite;
 
 @RunWith(Suite.class)
 @Suite.SuiteClasses( {
-      TS_Block.class
+      TS_Storage.class
     , TS_File.class
+    , TS_Block.class
     , TS_Record.class
     , TS_RecordFile.class
 })
@@ -28,6 +31,7 @@ public class TC_Base
 
 /*
  * (c) Copyright 2008, 2009 Hewlett-Packard Development Company, LP
+ * (c) Copyright 2011 Epimorphics ltd.
  * All rights reserved.
  *
  * Redistribution and use in source and binary forms, with or without

Copied: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestFileAccessFixedSize.java (from r1125428, incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractFileAccessFixedSizeTest.java)
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestFileAccessFixedSize.java?p2=incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestFileAccessFixedSize.java&p1=incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractFileAccessFixedSizeTest.java&r1=1125428&r2=1125503&rev=1125503&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractFileAccessFixedSizeTest.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestFileAccessFixedSize.java Fri May 20 18:51:50 2011
@@ -13,13 +13,13 @@ import org.openjena.atlas.junit.BaseTest
 
 import com.hp.hpl.jena.tdb.base.block.Block ;
 
-public abstract class AbstractFileAccessFixedSizeTest extends BaseTest
+public abstract class AbstractTestFileAccessFixedSize extends BaseTest
 {
     // Fixed block tests.
     
     int blkSize ;
     
-    protected AbstractFileAccessFixedSizeTest(int blkSize)
+    protected AbstractTestFileAccessFixedSize(int blkSize)
     {
         this.blkSize = blkSize ;
     }
@@ -88,7 +88,17 @@ public abstract class AbstractFileAccess
         assertTrue(b8.getId() == b9.getId()) ;
     }
     
-    // Exceptions.
+    @Test(expected=FileException.class)
+    public void fileaccess_05()
+    {
+        FileAccess file = make() ;
+        Block b1 = data(file, 10) ;
+        Block b2 = data(file, 20) ;
+        file.write(b1) ;
+        
+        // Should not work. b2 not written.   
+        Block b2a = file.read(b2.getId()) ;
+    }    
 }
 
 /*

Propchange: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestFileAccessFixedSize.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Copied: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestFileAccessVarSize.java (from r1125428, incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractFileAccessVarSizeTest.java)
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestFileAccessVarSize.java?p2=incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestFileAccessVarSize.java&p1=incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractFileAccessVarSizeTest.java&r1=1125428&r2=1125503&rev=1125503&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractFileAccessVarSizeTest.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestFileAccessVarSize.java Fri May 20 18:51:50 2011
@@ -11,33 +11,29 @@ import org.junit.Test ;
 import com.hp.hpl.jena.tdb.base.block.Block ;
 
 
-public abstract class AbstractFileAccessVarSizeTest extends AbstractFileAccessFixedSizeTest
+public abstract class AbstractTestFileAccessVarSize extends AbstractTestFileAccessFixedSize
 {
-    AbstractFileAccessVarSizeTest()
+    protected AbstractTestFileAccessVarSize()
     {
         super(25) ;
     }
     
-    @Test public void fileaccess_50()
+    @Test
+    public void fileaccess_50()
     {
         FileAccess file = make() ;
         Block b1 = data(file, 10) ;
         Block b2 = data(file, 20) ;
         file.write(b1) ;
+        file.write(b2) ;
         
-        System.out.println("b1: "+b1) ;
-        System.out.println("b2: "+b2) ;
-        
-        // SHOULD NOT WORK.
-        Block b2a = file.read(b2.getId()) ;
         Block b1a = file.read(b1.getId()) ;
-        
+        Block b2a = file.read(b2.getId()) ;
+
         assertNotSame(b1a, b1) ;
         assertNotSame(b2a, b2) ;
         sameValue(b1, b1a) ;
         sameValue(b2, b2a) ;
-        
-        
     }        
 }
 

Propchange: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/AbstractTestFileAccessVarSize.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TS_File.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TS_File.java?rev=1125503&r1=1125502&r2=1125503&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TS_File.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TS_File.java Fri May 20 18:51:50 2011
@@ -1,5 +1,6 @@
 /*
  * (c) Copyright 2008, 2009 Hewlett-Packard Development Company, LP
+ * (c) Copyright 2011 Epimorphics Ltd.
  * All rights reserved.
  * [See end of file]
  */
@@ -15,6 +16,8 @@ import org.junit.runners.Suite;
     , TestStringFile.class 
     , TestFileAccessMem.class
     , TestFileAccessByteArray.class
+    , TestFileAccessDirect.class
+    , TestFileAccessMapped.class
 })
 
 
@@ -23,6 +26,7 @@ public class TS_File
 
 /*
  * (c) Copyright 2008, 2009 Hewlett-Packard Development Company, LP
+ * (c) Copyright 2011 Epimorphics Ltd.
  * All rights reserved.
  *
  * Redistribution and use in source and binary forms, with or without

Modified: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessByteArray.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessByteArray.java?rev=1125503&r1=1125502&r2=1125503&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessByteArray.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessByteArray.java Fri May 20 18:51:50 2011
@@ -6,7 +6,7 @@
 
 package com.hp.hpl.jena.tdb.base.file;
 
-public class TestFileAccessByteArray extends AbstractFileAccessVarSizeTest
+public class TestFileAccessByteArray extends AbstractTestFileAccessVarSize
 {
 
     @Override

Added: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessDirect.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessDirect.java?rev=1125503&view=auto
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessDirect.java (added)
+++ incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessDirect.java Fri May 20 18:51:50 2011
@@ -0,0 +1,59 @@
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ * [See end of file]
+ */
+
+package com.hp.hpl.jena.tdb.base.file;
+
+import org.junit.AfterClass ;
+import org.openjena.atlas.lib.FileOps ;
+
+import com.hp.hpl.jena.tdb.ConfigTest ;
+
+public class TestFileAccessDirect extends AbstractTestFileAccessFixedSize
+{
+    static String filename = ConfigTest.getTestingDir()+"/test-file-access-direct" ;
+    
+    static final int BlockSize = 50 ;
+    public TestFileAccessDirect()
+    {
+        super(BlockSize) ;
+    }
+
+    @AfterClass public static void cleanup() { FileOps.deleteSilent(filename) ; } 
+    
+    @Override
+    protected FileAccess make()
+    {
+        FileOps.deleteSilent(filename) ;
+        return new FileAccessDirect(filename, BlockSize) ;
+    }
+}
+
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the above copyright
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ * 3. The name of the author may not be used to endorse or promote products
+ *    derived from this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
+ * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+ * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+ * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
\ No newline at end of file

Propchange: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessDirect.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessMapped.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessMapped.java?rev=1125503&view=auto
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessMapped.java (added)
+++ incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessMapped.java Fri May 20 18:51:50 2011
@@ -0,0 +1,59 @@
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ * [See end of file]
+ */
+
+package com.hp.hpl.jena.tdb.base.file;
+
+import org.junit.AfterClass ;
+import org.openjena.atlas.lib.FileOps ;
+
+import com.hp.hpl.jena.tdb.ConfigTest ;
+
+public class TestFileAccessMapped extends AbstractTestFileAccessFixedSize
+{
+    static String filename = ConfigTest.getTestingDir()+"/test-file-access-mapped" ;
+    
+    static final int BlockSize = 64 ;
+    public TestFileAccessMapped()
+    {
+        super(BlockSize) ;
+    }
+
+    @AfterClass public static void cleanup() { FileOps.deleteSilent(filename) ; } 
+    
+    @Override
+    protected FileAccess make()
+    {
+        FileOps.deleteSilent(filename) ;
+        return new FileAccessMapped(filename, BlockSize) ;
+    }
+}
+
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the above copyright
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ * 3. The name of the author may not be used to endorse or promote products
+ *    derived from this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
+ * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+ * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+ * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
\ No newline at end of file

Propchange: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessMapped.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessMem.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessMem.java?rev=1125503&r1=1125502&r2=1125503&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessMem.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/file/TestFileAccessMem.java Fri May 20 18:51:50 2011
@@ -6,7 +6,7 @@
 
 package com.hp.hpl.jena.tdb.base.file;
 
-public class TestFileAccessMem extends AbstractFileAccessFixedSizeTest
+public class TestFileAccessMem extends AbstractTestFileAccessFixedSize
 {
     static final int BlockSize = 50 ;
     public TestFileAccessMem()

Added: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/AbstractTestStorage.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/AbstractTestStorage.java?rev=1125503&view=auto
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/AbstractTestStorage.java (added)
+++ incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/AbstractTestStorage.java Fri May 20 18:51:50 2011
@@ -0,0 +1,105 @@
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ * [See end of file]
+ */
+
+package com.hp.hpl.jena.tdb.base.storage;
+
+import java.nio.ByteBuffer ;
+
+import org.junit.Test ;
+import org.openjena.atlas.junit.BaseTest ;
+
+public abstract class AbstractTestStorage extends BaseTest
+{
+    protected abstract Storage make() ;
+    static final int blkSize = 100 ;
+    
+    @Test public void storage_01() 
+    {
+        Storage store = make() ;
+        assertEquals(0, store.length()) ;
+    }
+    
+    protected static ByteBuffer data(int len)
+    {
+        ByteBuffer b = ByteBuffer.allocate(len) ;
+        for (int i = 0 ; i < len ; i++ )
+            b.put((byte)(i&0xFF)) ;
+        b.clear() ;
+        return b ;
+    }
+    
+    protected static boolean same(ByteBuffer bb1, ByteBuffer bb2)
+    {
+        if ( bb1.capacity() != bb2.capacity() ) return false ;
+        
+        for ( int i = 0 ; i < bb1.capacity() ; i++ )
+            if ( bb1.get(i) != bb2.get(i) ) return false ;
+        return true ;
+    }
+
+    @Test public void storage_02()
+    {
+        Storage store = make() ;
+        ByteBuffer b = data(blkSize) ;
+        store.write(b) ;
+        long x = store.length() ;
+        assertEquals(blkSize, x) ;
+    }
+
+    @Test public void storage_03()
+    {
+        Storage store = make() ;
+        ByteBuffer b1 = data(blkSize) ;
+        long posn = store.position() ; 
+        store.write(b1) ;
+        ByteBuffer b9 = ByteBuffer.allocate(blkSize) ;
+        int r = store.read(b9, posn) ;
+        assertEquals(blkSize, r) ;
+        assertTrue(same(b1, b9)) ;
+    }
+    
+    @Test public void storage_04()
+    {
+        Storage store = make() ;
+        ByteBuffer b1 = data(blkSize) ;
+        ByteBuffer b2 = data(blkSize/2) ;
+
+        store.write(b2, 0) ;
+        store.write(b1, 0) ;
+        
+        assertEquals(blkSize, store.length()) ;
+        ByteBuffer b9 = ByteBuffer.allocate(5) ;
+        int z = store.read(b9) ;
+        assertEquals(5, z) ;
+    }
+}
+
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the above copyright
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ * 3. The name of the author may not be used to endorse or promote products
+ *    derived from this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
+ * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+ * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+ * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
\ No newline at end of file

Propchange: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/AbstractTestStorage.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TS_Storage.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TS_Storage.java?rev=1125503&view=auto
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TS_Storage.java (added)
+++ incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TS_Storage.java Fri May 20 18:51:50 2011
@@ -0,0 +1,41 @@
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ * [See end of file]
+ */
+
+package com.hp.hpl.jena.tdb.base.storage;
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+
+@RunWith(Suite.class)
+@Suite.SuiteClasses( { TestStorageMem.class , TestStorageFile.class } )
+
+public class TS_Storage {}
+
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the above copyright
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ * 3. The name of the author may not be used to endorse or promote products
+ *    derived from this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
+ * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+ * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+ * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
\ No newline at end of file

Propchange: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TS_Storage.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TestStorageFile.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TestStorageFile.java?rev=1125503&view=auto
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TestStorageFile.java (added)
+++ incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TestStorageFile.java Fri May 20 18:51:50 2011
@@ -0,0 +1,53 @@
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ * [See end of file]
+ */
+
+package com.hp.hpl.jena.tdb.base.storage;
+
+import org.junit.AfterClass ;
+import org.openjena.atlas.lib.FileOps ;
+
+import com.hp.hpl.jena.tdb.ConfigTest ;
+
+public class TestStorageFile extends AbstractTestStorage
+{
+    static String filename = ConfigTest.getTestingDir()+"/test-storage" ;
+
+    @AfterClass public static void cleanup() { FileOps.deleteSilent(filename) ; } 
+    
+    @Override
+    protected Storage make()
+    {
+        FileOps.deleteSilent(filename) ;
+        return new StorageFile(filename) ;
+    }
+}
+
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the above copyright
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ * 3. The name of the author may not be used to endorse or promote products
+ *    derived from this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
+ * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+ * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+ * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
\ No newline at end of file

Propchange: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TestStorageFile.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TestStorageMem.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TestStorageMem.java?rev=1125503&view=auto
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TestStorageMem.java (added)
+++ incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TestStorageMem.java Fri May 20 18:51:50 2011
@@ -0,0 +1,47 @@
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ * [See end of file]
+ */
+
+package com.hp.hpl.jena.tdb.base.storage;
+
+
+public class TestStorageMem extends AbstractTestStorage
+{
+    static int counter = 0 ;
+    
+    @Override
+    protected Storage make()
+    {
+        return new StorageMem("Test-"+(counter++)) ;
+    }
+
+}
+
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the above copyright
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ * 3. The name of the author may not be used to endorse or promote products
+ *    derived from this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
+ * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+ * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+ * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
\ No newline at end of file

Propchange: incubator/jena/Experimental/TxTDB/trunk/src/test/java/com/hp/hpl/jena/tdb/base/storage/TestStorageMem.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: incubator/jena/Jena2/Fuseki/trunk/src/main/java/org/openjena/fuseki/servlets/SPARQL_Query.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/Fuseki/trunk/src/main/java/org/openjena/fuseki/servlets/SPARQL_Query.java?rev=1125503&r1=1125502&r2=1125503&view=diff
==============================================================================
--- incubator/jena/Jena2/Fuseki/trunk/src/main/java/org/openjena/fuseki/servlets/SPARQL_Query.java (original)
+++ incubator/jena/Jena2/Fuseki/trunk/src/main/java/org/openjena/fuseki/servlets/SPARQL_Query.java Fri May 20 18:51:50 2011
@@ -166,8 +166,7 @@ public abstract class SPARQL_Query exten
                 mustHaveQueryParam = false ;
                 //error(HttpSC.UNSUPPORTED_MEDIA_TYPE_415, "Unofficial "+WebContent.contentTypeSPARQLQuery+" not supported") ;
             }
-            else if ( WebContent.contentTypeForm.equals(incoming) )
-                ; // OK
+            else if ( WebContent.contentTypeForm.equals(incoming) ) {}
             else
                 error(HttpSC.UNSUPPORTED_MEDIA_TYPE_415, "Unsupported: "+incoming) ;
         }