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/06/14 12:12:17 UTC

svn commit: r1135458 - in /incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction: DatasetBuilderTxn.java DatasetGraphTxnTDB.java Journal.java Transaction.java TransactionManager.java

Author: andy
Date: Tue Jun 14 10:12:17 2011
New Revision: 1135458

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

Added:
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/DatasetGraphTxnTDB.java   (with props)
Modified:
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/DatasetBuilderTxn.java
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Journal.java
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transaction.java
    incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/TransactionManager.java

Modified: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/DatasetBuilderTxn.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/DatasetBuilderTxn.java?rev=1135458&r1=1135457&r2=1135458&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/DatasetBuilderTxn.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/DatasetBuilderTxn.java Tue Jun 14 10:12:17 2011
@@ -4,7 +4,9 @@
  * [See end of file]
  */
 
-package com.hp.hpl.jena.tdb.transaction;
+package com.hp.hpl.jena.tdb.transaction ;
+
+import java.util.Properties ;
 
 import setup.BlockMgrBuilder ;
 import setup.DatasetBuilderStd ;
@@ -13,52 +15,128 @@ import setup.NodeTableBuilder ;
 import setup.ObjectFileBuilder ;
 import setup.RangeIndexBuilder ;
 import setup.TupleIndexBuilder ;
+import tx.transaction.DatasetGraphTxView ;
+
+import com.hp.hpl.jena.tdb.base.block.BlockMgr ;
+import com.hp.hpl.jena.tdb.base.file.BufferChannel ;
+import com.hp.hpl.jena.tdb.base.file.BufferChannelFile ;
+import com.hp.hpl.jena.tdb.base.file.BufferChannelMem ;
+import com.hp.hpl.jena.tdb.base.file.FileFactory ;
+import com.hp.hpl.jena.tdb.base.file.FileSet ;
+import com.hp.hpl.jena.tdb.base.file.Location ;
+import com.hp.hpl.jena.tdb.base.objectfile.ObjectFile ;
+import com.hp.hpl.jena.tdb.store.DatasetGraphTDB ;
 
 public class DatasetBuilderTxn extends DatasetBuilderStd
 {
-    
+    public DatasetBuilderTxn() { setStd() ; }
     
     @Override
     protected void setStd()
     {
-        ObjectFileBuilder objectFileBuilder     = new ObjectFileBuilderStd() ;
-        BlockMgrBuilder blockMgrBuilder         = new BlockMgrBuilderStd() ;
+        ObjectFileBuilder objectFileBuilder = new ObjectFileBuilderTx() ;
+        BlockMgrBuilder blockMgrBuilder = new BlockMgrBuilderTx() ;
 
         // These are the usual.
-        IndexBuilder indexBuilder               = new IndexBuilderStd(blockMgrBuilder, blockMgrBuilder) ;
-        RangeIndexBuilder rangeIndexBuilder     = new RangeIndexBuilderStd(blockMgrBuilder, blockMgrBuilder) ;
+        IndexBuilder indexBuilder = new IndexBuilderStd(blockMgrBuilder, blockMgrBuilder) ;
+        RangeIndexBuilder rangeIndexBuilder = new RangeIndexBuilderStd(blockMgrBuilder, blockMgrBuilder) ;
+
+        NodeTableBuilder nodeTableBuilder = new NodeTableBuilderStd(indexBuilder, objectFileBuilder) ;
+        TupleIndexBuilder tupleIndexBuilder = new TupleIndexBuilderStd(rangeIndexBuilder) ;
+        set(nodeTableBuilder, tupleIndexBuilder, indexBuilder, rangeIndexBuilder, blockMgrBuilder, objectFileBuilder) ;
+    }
+
+    @Override
+    public DatasetGraphTDB build(Location location, Properties config)
+    {
+        this.location = location ;
+        this.txn = txnMgr.createTransaction() ;
+        
+        BufferChannel chan ;
+        if (location.isMem()) chan = new BufferChannelMem("journal") ;
+        else
+            chan = new BufferChannelFile(location.getPath(journalFilename, journalExt)) ;
+
+        journal = new Journal(chan) ;
+        txn.add(journal) ;
+
+        DatasetGraphTDB dsg = super.build(location, config) ;
+        new DatasetGraphTxnTDB(dsg, txn) ;
+    }
 
-        NodeTableBuilder nodeTableBuilder       = new NodeTableBuilderStd(indexBuilder, objectFileBuilder) ;
-        TupleIndexBuilder tupleIndexBuilder     = new TupleIndexBuilderStd(rangeIndexBuilder) ;
-        set(nodeTableBuilder, tupleIndexBuilder, 
-            indexBuilder, rangeIndexBuilder, 
-            blockMgrBuilder, objectFileBuilder) ;
+    public static final String journalExt = "jrnl" ;
+    public static final String journalFilename = "journal" ;
+   
+    // Maybe part of DatasetGraphTDB
+    private static TransactionManager txnMgr = new TransactionManager() ;
+    private Journal  journal ;
+    private Transaction txn ;
+    private Location location ;
+
+    class ObjectFileBuilderTx implements ObjectFileBuilder
+    {
+        ObjectFileBuilder base = new ObjectFileBuilderStd() ;
+        @Override
+        public ObjectFile buildObjectFile(FileSet fileSet, String ext)
+        {
+            ObjectFile backing ;
+            ObjectFile main = base.buildObjectFile(fileSet, ext) ;
+            
+            if ( location.isMem() )
+                backing = FileFactory.createObjectFileMem() ;
+            else
+                backing = FileFactory.createObjectFileDisk(fileSet.filename(journalExt)) ;
+
+            ObjectFileTrans objFileTrans = new ObjectFileTrans(txn, main, backing) ;
+            txn.add(objFileTrans) ;
+            return objFileTrans ;
+        }
+    }
+
+    class BlockMgrBuilderTx implements BlockMgrBuilder
+    {
+        BlockMgrBuilder base = new BlockMgrBuilderStd() ;
+        @Override
+        public BlockMgr buildBlockMgr(FileSet fileSet, String ext, int blockSize)
+        {
+            BlockMgr baseMgr = base.buildBlockMgr(fileSet, ext, blockSize) ;
+            BlockMgrJournal blkMg = new BlockMgrJournal(txn, null, baseMgr, journal) ;
+            // Add to transaction.
+            return blkMg ;
+        }
     }
+
+    /** Add transactions to an existing datatset */
+    public static DatasetGraphTDB enhance(DatasetGraphTDB dsg)
+    {
+
+        return dsg ;
+
+    }
+
 }
 
 /*
- * (c) Copyright 2011 Epimorphics Ltd.
- * All rights reserved.
- *
+ * (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.
+ * 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

Added: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/DatasetGraphTxnTDB.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/DatasetGraphTxnTDB.java?rev=1135458&view=auto
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/DatasetGraphTxnTDB.java (added)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/DatasetGraphTxnTDB.java Tue Jun 14 10:12:17 2011
@@ -0,0 +1,55 @@
+/*
+ * (c) Copyright 2011 Epimorphics Ltd.
+ * All rights reserved.
+ * [See end of file]
+ */
+
+package com.hp.hpl.jena.tdb.transaction;
+
+import com.hp.hpl.jena.sparql.core.DatasetGraphWrapper ;
+import com.hp.hpl.jena.tdb.store.DatasetGraphTDB ;
+
+public class DatasetGraphTxnTDB extends DatasetGraphWrapper
+{
+    private final Transaction transaction ;
+
+    public DatasetGraphTxnTDB(DatasetGraphTDB dsg, Transaction txn)
+    {
+        super(dsg) ;
+        this.transaction = txn ;
+    }
+
+    public void commit() { transaction.commit() ; }
+    public void abort() { transaction.abort() ; }
+    
+    @Override
+    public String toString()
+    { return "Txn:"+getWrapped().toString() ; }
+}
+
+/*
+ * (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/transaction/DatasetGraphTxnTDB.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Journal.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Journal.java?rev=1135458&r1=1135457&r2=1135458&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Journal.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Journal.java Tue Jun 14 10:12:17 2011
@@ -11,6 +11,9 @@ import static com.hp.hpl.jena.tdb.sys.Sy
 import java.nio.ByteBuffer ;
 import java.util.Iterator ;
 
+import org.openjena.atlas.lib.Closeable ;
+import org.openjena.atlas.lib.Sync ;
+
 import tx.IteratorSlotted ;
 
 import com.hp.hpl.jena.tdb.base.file.BufferChannel ;
@@ -20,7 +23,7 @@ import com.hp.hpl.jena.tdb.base.file.Buf
  *  if the journal grows to the point where it needs to free up cache. 
  */
 public final
-class Journal implements Iterable<JournalEntry>
+class Journal implements Iterable<JournalEntry>, Sync, Closeable
 {
     // Version 1 : issue might be excessive copying
     // [TxTDB:TODO] Caching
@@ -54,28 +57,22 @@ class Journal implements Iterable<Journa
     public long writeJournal(JournalEntryType type, ByteBuffer buffer)
     {
         long posn = position ;
+        int len ;
         // [TxDEV:TODO] CRC
-
-        // ?? buffer.position(0) ;
-        int len = buffer.remaining() ; 
+        if ( buffer == null )
+            len = 0 ;
+        else
+            len = buffer.remaining() ; 
         
         header.clear() ;
         header.putInt(type.id) ;
         header.putInt(len) ;
         header.flip() ;
-        
-//        switch (type)
-//        {
-//            case Block:
-//            case Object:
-//            case Commit:
-//            case Checkpoint:
-//            default:
-//        }
-        
+
         channel.write(header) ;
-        // Write bytes
-        channel.write(buffer) ;
+        if ( len > 0 )
+            // Write bytes
+            channel.write(buffer) ;
         
         position += len+Overhead ;
         return posn ;
@@ -173,6 +170,12 @@ class Journal implements Iterable<Journa
     
     @Override
     public Iterator<JournalEntry> iterator() { return entries() ; }
+
+    @Override
+    public void sync()  { channel.sync() ; }
+
+    @Override
+    public void close() { channel.close() ; }
 }
 
 /*

Modified: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transaction.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transaction.java?rev=1135458&r1=1135457&r2=1135458&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transaction.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/Transaction.java Tue Jun 14 10:12:17 2011
@@ -17,6 +17,8 @@ public class Transaction
     private final long id ;
     private final TransactionManager txnMgr ;
     private final List<Iterator<?>> iterators ; 
+    private final List<Journal> journals = new ArrayList<Journal>() ;
+    private final List<ObjectFileTrans> objJournals = new ArrayList<ObjectFileTrans>() ;
 
     public Transaction(long id, TransactionManager txnMgr)
     {
@@ -25,7 +27,26 @@ public class Transaction
         this.iterators = new ArrayList<Iterator<?>>() ;
     }
 
-    public void commit()                            { txnMgr.commit(this) ; }
+    public void commit()
+    {
+        // Write commit entry.
+        if ( journals.size() > 0 )
+        {
+            Journal jrnl = journals.get(0) ;
+            JournalEntry entry = new JournalEntry(JournalEntryType.Commit, null) ;
+            jrnl.writeJournal(entry) ;
+        }
+        
+        
+        for ( ObjectFileTrans jrnl : objJournals )
+            jrnl.sync() ;
+    
+        for ( Journal jrnl : journals )
+            jrnl.sync() ;
+        
+        txnMgr.commit(this) ;
+    }
+    
     public void abort()                             { txnMgr.abort(this) ; }
     public long getTxnId()                          { return id ; }
     public TransactionManager getTxnMgr()           { return txnMgr ; }
@@ -33,6 +54,16 @@ public class Transaction
     public void addIterator(Iterator<?> iter)       { iterators.add(iter) ; }
     public void removeIterator(Iterator<?> iter)    { iterators.remove(iter) ; }
     public List<Iterator<?>> iterators()            { return Collections.unmodifiableList(iterators) ; }
+    
+    public void add(Journal journal)
+    {
+        journals.add(journal) ;
+    }
+
+    public void add(ObjectFileTrans objFileTrans)
+    {
+        objJournals.add(objFileTrans) ;
+    }
 }
 
 /*

Modified: incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/TransactionManager.java
URL: http://svn.apache.org/viewvc/incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/TransactionManager.java?rev=1135458&r1=1135457&r2=1135458&view=diff
==============================================================================
--- incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/TransactionManager.java (original)
+++ incubator/jena/Experimental/TxTDB/trunk/src/main/java/com/hp/hpl/jena/tdb/transaction/TransactionManager.java Tue Jun 14 10:12:17 2011
@@ -6,6 +6,14 @@
 
 package com.hp.hpl.jena.tdb.transaction;
 
+import setup.DatasetBuilder ;
+
+import com.hp.hpl.jena.sparql.core.DatasetGraph ;
+import com.hp.hpl.jena.tdb.TDBException ;
+import com.hp.hpl.jena.tdb.base.file.Location ;
+import com.hp.hpl.jena.tdb.store.DatasetGraphTDB ;
+
+
 
 public class TransactionManager
 {
@@ -21,19 +29,42 @@ public class TransactionManager
         return txn ;
     }
     
-    public Transaction begin() { return createTransaction() ; }
-    
+    public Transaction begin(DatasetGraph dsg)
+    {
+        // If already a transaction ... 
+        // Subs transactions are a new view - commit is only comit to parent transaction.  
+        if ( dsg instanceof DatasetGraphTxnTDB )
+        {
+            throw new TDBException("Already in transactional DatasetGraph") ;
+            // Either:
+            //   error -> implies nested
+            //   create new transaction 
+        }
+        
+        if ( ! ( dsg instanceof DatasetGraphTDB ) )
+            throw new TDBException("Not a TDB-backed dataset") ;
+
+        DatasetGraphTDB dsgtdb = (DatasetGraphTDB)dsg ;
+        // For now, always build a parallel dataset - later, associate with the DatasetGraphTDB
+        Location location = dsgtdb.getLocation() ;
+        
+        
+        
+        DatasetBuilder x = new DatasetBuilderTxn(baseBlockMgrBuilder, nodeTable) ;
+        DatasetGraph dsg2 = x.build(Location.mem(), null) ;
+
+        Transaction_X txn = createTransaction() ;
+        return new DatasetGraphTxView(txn, dsg2) ;
+    }
+
     public void commit(Transaction transaction)
     {
-        System.err.println("Commit") ;
+        transaction.commit() ;
     }
 
     public void abort(Transaction transaction)
     {    
-        System.err.println("Abort") ;
-        // Release allocated blocks back to the
-        //   Reset allocation id.
-        // Forget any free blocks as being free. 
+        transaction.abort() ;
     }
 }