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 2017/08/13 13:36:46 UTC

[2/4] jena git commit: JENA-1379: Build NodeTables from ObjectFiles and BlockMgrs.

JENA-1379: Build NodeTables from ObjectFiles and BlockMgrs.

Remove NodeTableBuilder
Remove NodeTableTrans


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/3d406fe1
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/3d406fe1
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/3d406fe1

Branch: refs/heads/master
Commit: 3d406fe16973bea5af6f51bde10f583546f3f6c3
Parents: 035ebba
Author: Andy Seaborne <an...@apache.org>
Authored: Fri Aug 4 17:14:33 2017 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Sat Aug 5 16:53:19 2017 +0100

----------------------------------------------------------------------
 .../org/apache/jena/tdb/setup/BuilderStdDB.java |  32 --
 .../apache/jena/tdb/setup/DatasetBuilder.java   |  33 --
 .../jena/tdb/setup/DatasetBuilderStd.java       | 178 ++++----
 .../apache/jena/tdb/setup/NodeTableBuilder.java |  26 --
 .../apache/jena/tdb/store/StorageConfig.java    |   6 +-
 .../jena/tdb/transaction/DatasetBuilderTxn.java | 134 ++----
 .../jena/tdb/transaction/JournalControl.java    |  54 +--
 .../jena/tdb/transaction/NodeTableTrans.java    | 413 -------------------
 .../jena/tdb/transaction/ObjectFileTrans.java   | 232 ++---------
 .../tdb/transaction/ObjectFileTransComplex.java | 253 ++++++++++++
 .../jena/tdb/transaction/Transaction.java       |  17 +-
 .../jena/tdb/extra/T_QuadsObjectIsNull.java     | 110 -----
 .../transaction/AbstractTestNodeTableTrans.java | 235 -----------
 .../AbstractTestObjectFileTrans.java            |  10 +-
 .../AbstractTestObjectFileTransComplex.java     | 158 +++++++
 .../jena/tdb/transaction/TS_TransactionTDB.java |   7 +-
 .../tdb/transaction/TestNodeTableTransDisk.java |  51 ---
 .../tdb/transaction/TestNodeTableTransMem.java  |  38 --
 .../TestObjectFileTransComplexMem.java          |  36 ++
 .../TestObjectFileTransComplexStorage.java      |  53 +++
 20 files changed, 702 insertions(+), 1374 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/3d406fe1/jena-tdb/src/main/java/org/apache/jena/tdb/setup/BuilderStdDB.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/setup/BuilderStdDB.java b/jena-tdb/src/main/java/org/apache/jena/tdb/setup/BuilderStdDB.java
index 2484b84..d7e5993 100644
--- a/jena-tdb/src/main/java/org/apache/jena/tdb/setup/BuilderStdDB.java
+++ b/jena-tdb/src/main/java/org/apache/jena/tdb/setup/BuilderStdDB.java
@@ -22,18 +22,11 @@ import org.apache.jena.tdb.base.file.FileFactory ;
 import org.apache.jena.tdb.base.file.FileSet ;
 import org.apache.jena.tdb.base.objectfile.ObjectFile ;
 import org.apache.jena.tdb.base.record.RecordFactory ;
-import org.apache.jena.tdb.index.Index ;
-import org.apache.jena.tdb.index.IndexBuilder ;
 import org.apache.jena.tdb.index.RangeIndex ;
 import org.apache.jena.tdb.index.RangeIndexBuilder ;
 import org.apache.jena.tdb.lib.ColumnMap ;
-import org.apache.jena.tdb.store.nodetable.NodeTable ;
-import org.apache.jena.tdb.store.nodetable.NodeTableCache ;
-import org.apache.jena.tdb.store.nodetable.NodeTableInline ;
-import org.apache.jena.tdb.store.nodetable.NodeTableNative ;
 import org.apache.jena.tdb.store.tupletable.TupleIndex ;
 import org.apache.jena.tdb.store.tupletable.TupleIndexRecord ;
-import org.apache.jena.tdb.sys.Names ;
 import org.apache.jena.tdb.sys.SystemTDB ;
 
 public class BuilderStdDB {
@@ -56,31 +49,6 @@ public class BuilderStdDB {
         }
     }
 
-    public static class NodeTableBuilderStd implements NodeTableBuilder
-    {
-        private final IndexBuilder indexBuilder ;
-        private final ObjectFileBuilder objectFileBuilder ;
-        
-        public NodeTableBuilderStd(IndexBuilder indexBuilder, ObjectFileBuilder objectFileBuilder) {
-            this.indexBuilder = indexBuilder ;
-            this.objectFileBuilder = objectFileBuilder ;
-        }
-    
-        @Override
-        public NodeTable buildNodeTable(FileSet fsIndex, FileSet fsObjectFile, StoreParams params) {
-            RecordFactory recordFactory = new RecordFactory(SystemTDB.LenNodeHash, SystemTDB.SizeOfNodeId) ;
-            Index idx = indexBuilder.buildIndex(fsIndex, recordFactory, params) ;
-            ObjectFile objectFile = objectFileBuilder.buildObjectFile(fsObjectFile, Names.extNodeData) ;
-            NodeTable nodeTable = new NodeTableNative(idx, objectFile) ;
-            nodeTable = NodeTableCache.create(nodeTable, 
-                                              params.getNode2NodeIdCacheSize(),
-                                              params.getNodeId2NodeCacheSize(),
-                                              params.getNodeMissCacheSize()) ;
-            nodeTable = NodeTableInline.create(nodeTable) ;
-            return nodeTable ;
-        }
-    }
-
     public static class ObjectFileBuilderStd implements ObjectFileBuilder
     {
         public ObjectFileBuilderStd() { }

http://git-wip-us.apache.org/repos/asf/jena/blob/3d406fe1/jena-tdb/src/main/java/org/apache/jena/tdb/setup/DatasetBuilder.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/setup/DatasetBuilder.java b/jena-tdb/src/main/java/org/apache/jena/tdb/setup/DatasetBuilder.java
deleted file mode 100644
index 75a8fe9..0000000
--- a/jena-tdb/src/main/java/org/apache/jena/tdb/setup/DatasetBuilder.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * 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 org.apache.jena.tdb.setup;
-
-import org.apache.jena.tdb.base.file.Location ;
-import org.apache.jena.tdb.store.DatasetGraphTDB ;
-
-/** A builder of datasets in a Location (usually, a directory on disk) */ 
-
-public interface DatasetBuilder
-{
-    /** 
-     * Build a new Java object for a dataset.
-     * Location may already contains files for the state, or may need initializing.  
-     */
-    public DatasetGraphTDB build(Location location, StoreParams params) ;
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/3d406fe1/jena-tdb/src/main/java/org/apache/jena/tdb/setup/DatasetBuilderStd.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/setup/DatasetBuilderStd.java b/jena-tdb/src/main/java/org/apache/jena/tdb/setup/DatasetBuilderStd.java
index d538c0a..da82781 100644
--- a/jena-tdb/src/main/java/org/apache/jena/tdb/setup/DatasetBuilderStd.java
+++ b/jena-tdb/src/main/java/org/apache/jena/tdb/setup/DatasetBuilderStd.java
@@ -35,17 +35,21 @@ import org.apache.jena.tdb.base.file.BufferChannel ;
 import org.apache.jena.tdb.base.file.FileSet ;
 import org.apache.jena.tdb.base.file.Location ;
 import org.apache.jena.tdb.base.objectfile.ObjectFile;
-import org.apache.jena.tdb.index.BuilderStdIndex ;
-import org.apache.jena.tdb.index.IndexBuilder ;
-import org.apache.jena.tdb.index.IndexParams ;
-import org.apache.jena.tdb.index.RangeIndexBuilder ;
+import org.apache.jena.tdb.base.record.RecordFactory ;
+import org.apache.jena.tdb.index.* ;
+import org.apache.jena.tdb.index.bplustree.BPlusTree ;
+import org.apache.jena.tdb.index.bplustree.BPlusTreeParams ;
 import org.apache.jena.tdb.lib.ColumnMap ;
 import org.apache.jena.tdb.solver.OpExecutorTDB1 ;
 import org.apache.jena.tdb.store.* ;
 import org.apache.jena.tdb.store.nodetable.NodeTable ;
+import org.apache.jena.tdb.store.nodetable.NodeTableCache ;
+import org.apache.jena.tdb.store.nodetable.NodeTableInline ;
+import org.apache.jena.tdb.store.nodetable.NodeTableNative ;
 import org.apache.jena.tdb.store.nodetupletable.NodeTupleTable ;
 import org.apache.jena.tdb.store.nodetupletable.NodeTupleTableConcrete ;
 import org.apache.jena.tdb.store.tupletable.TupleIndex ;
+import org.apache.jena.tdb.store.tupletable.TupleIndexRecord ;
 import org.apache.jena.tdb.sys.* ;
 import org.slf4j.Logger ;
 
@@ -54,15 +58,65 @@ import org.slf4j.Logger ;
  * BlockMgr/BufferChannel/NodeTable for use by the transaction builder.
  */
 
-public class DatasetBuilderStd implements DatasetBuilder {
+public class DatasetBuilderStd {
     private static final Logger log = TDB.logInfo ;
 
-    private NodeTableBuilder    nodeTableBuilder ;
-    private TupleIndexBuilder   tupleIndexBuilder ;
-    private Recorder            recorder = null ;   
+    private BlockMgrBuilder blockMgrBuilder = new BuilderStdIndex.BlockMgrBuilderStd();
+    private ObjectFileBuilder objectFileBuilder = new BuilderStdDB.ObjectFileBuilderStd(); 
+
+    private void setupRecord() {
+        if ( this.blockMgrBuilder instanceof BlockMgrBuilderRecorder)
+            throw new TDBException("Already recording (BlockMgrBuilder)");
+        if ( this.objectFileBuilder instanceof ObjectFileBuilderRecorder)
+            throw new TDBException("Already recording (ObjectFileBuilder)");
+        
+        this.blockMgrBuilder = new BlockMgrBuilderRecorder(blockMgrBuilder, recorder);
+        this.objectFileBuilder= new ObjectFileBuilderRecorder(objectFileBuilder, recorder);
+    }
+    
+    private RangeIndex buildRangeIndex(FileSet fileSet, RecordFactory recordFactory, IndexParams indexParams) {
+        int blkSize = indexParams.getBlockSize() ;
+        int order = BPlusTreeParams.calcOrder(blkSize, recordFactory.recordLength()) ;
+        RangeIndex rIndex = createBPTree(fileSet, order, blockMgrBuilder, blockMgrBuilder, recordFactory, indexParams) ;
+        return rIndex ;
+    }
+    
+    private Index buildIndex(FileSet fileSet, RecordFactory recordFactory, IndexParams indexParams) {
+        // Cheap.
+        return buildRangeIndex(fileSet, recordFactory, indexParams) ;
+    }
+    
+    /** Knowing all the parameters, create a B+Tree */
+    private RangeIndex createBPTree(FileSet fileset, int order, 
+                                    BlockMgrBuilder blockMgrBuilderNodes,
+                                    BlockMgrBuilder blockMgrBuilderRecords,
+                                    RecordFactory factory, IndexParams indexParams)
+    {
+        // ---- Checking
+        {
+            int blockSize = indexParams.getBlockSize() ;
+            if (blockSize < 0 )
+                throw new IllegalArgumentException("Negative blocksize: "+blockSize) ;
+            if (blockSize < 0 && order < 0) throw new IllegalArgumentException("Neither blocksize nor order specified") ;
+            if (blockSize >= 0 && order < 0) order = BPlusTreeParams.calcOrder(blockSize, factory.recordLength()) ;
+            if (blockSize >= 0 && order >= 0)
+            {
+                int order2 = BPlusTreeParams.calcOrder(blockSize, factory.recordLength()) ;
+                if (order != order2) 
+                    throw new IllegalArgumentException("Wrong order (" + order + "), calculated = " + order2) ;
+            }
+        }
+
+        BPlusTreeParams params = new BPlusTreeParams(order, factory) ;
+        
+        BlockMgr blkMgrNodes = blockMgrBuilderNodes.buildBlockMgr(fileset, Names.bptExtTree, indexParams) ;
+        BlockMgr blkMgrRecords = blockMgrBuilderRecords.buildBlockMgr(fileset, Names.bptExtRecords, indexParams) ;
+        return BPlusTree.create(params, blkMgrNodes, blkMgrRecords) ;
+    }
+    
+    private Recorder            recorder = new Recorder() ;  
     
     /**
-     * 
      * @param location
      * @return DatasetGraphTDB
      */
@@ -86,7 +140,6 @@ public class DatasetBuilderStd implements DatasetBuilder {
         boolean newArea = TDBInternal.isNewDatabaseArea(location) ;
         StoreParams params = Build.decideStoreParams(location, newArea, appParams, locParams, dftParams) ;
         DatasetBuilderStd x = new DatasetBuilderStd() ;
-        x.standardSetup() ;
         DatasetGraphTDB dsg = x.build(location, params) ;
         return dsg ;
     }
@@ -97,45 +150,29 @@ public class DatasetBuilderStd implements DatasetBuilder {
     }
 
     public static DatasetBuilderStd stdBuilder() {
-        DatasetBuilderStd x = new DatasetBuilderStd() ;
-        x.standardSetup() ;
-        return x ;
+        return new DatasetBuilderStd() ;
     }
 
-    protected DatasetBuilderStd() { }
-
-    // Used by DatasetBuilderTxn
-    public DatasetBuilderStd(BlockMgrBuilder blockMgrBuilder, NodeTableBuilder nodeTableBuilder) {
-        setupRecord(blockMgrBuilder, nodeTableBuilder) ;
+    protected DatasetBuilderStd() { 
+        this(new BuilderStdIndex.BlockMgrBuilderStd(),
+             new BuilderStdDB.ObjectFileBuilderStd());
     }
 
-    private void standardSetup() {
-        ObjectFileBuilder objectFileBuilder = new BuilderStdDB.ObjectFileBuilderStd() ;
-        BlockMgrBuilder blockMgrBuilder = new BuilderStdIndex.BlockMgrBuilderStd() ;
-        IndexBuilder indexBuilderNT = new BuilderStdIndex.IndexBuilderStd(blockMgrBuilder, blockMgrBuilder) ;
-        NodeTableBuilder nodeTableBuilder = new BuilderStdDB.NodeTableBuilderStd(indexBuilderNT, objectFileBuilder) ;
-        setupRecord(blockMgrBuilder, nodeTableBuilder) ;
-    }
-
-    protected void setupRecord(BlockMgrBuilder blockMgrBuilder, NodeTableBuilder nodeTableBuilder) {
-        recorder = new Recorder() ;
-        BlockMgrBuilder blockMgrBuilderRec = new BlockMgrBuilderRecorder(blockMgrBuilder, recorder) ;
-
-        IndexBuilder indexBuilder = new BuilderStdIndex.IndexBuilderStd(blockMgrBuilderRec, blockMgrBuilderRec) ;
-        RangeIndexBuilder rangeIndexBuilder = new BuilderStdIndex.RangeIndexBuilderStd(blockMgrBuilderRec, blockMgrBuilderRec) ;
-
-        this.nodeTableBuilder = nodeTableBuilder ;
-        nodeTableBuilder = new NodeTableBuilderRecorder(nodeTableBuilder, recorder) ;
-
-        TupleIndexBuilder tupleIndexBuilder = new BuilderStdDB.TupleIndexBuilderStd(rangeIndexBuilder) ;
-        set(nodeTableBuilder, tupleIndexBuilder) ;
-    }
-
-    protected void set(NodeTableBuilder nodeTableBuilder, TupleIndexBuilder tupleIndexBuilder) {
-        this.nodeTableBuilder = nodeTableBuilder ;
-        this.tupleIndexBuilder = tupleIndexBuilder ;
+    public DatasetBuilderStd(BlockMgrBuilder blockMgrBuilder, ObjectFileBuilder objectFileBuilder) {
+        this.blockMgrBuilder = blockMgrBuilder;
+        this.objectFileBuilder = objectFileBuilder; 
+        this.recorder = new Recorder() ;
+        // XXX YUK
+        setupRecord();
     }
     
+//    private void standardSetup() {
+//        ObjectFileBuilder objectFileBuilder = new BuilderStdDB.ObjectFileBuilderStd() ;
+//        BlockMgrBuilder blockMgrBuilder = new BuilderStdIndex.BlockMgrBuilderStd() ;
+//        IndexBuilder indexBuilderNT = new BuilderStdIndex.IndexBuilderStd(blockMgrBuilder, blockMgrBuilder) ;
+//        NodeTableBuilder nodeTableBuilder = new BuilderStdDB.NodeTableBuilderStd(indexBuilderNT, objectFileBuilder) ;
+//        setupRecord(blockMgrBuilder, nodeTableBuilder) ;
+//    }
 
     private static void checkLocation(Location location) { 
         if ( location.isMem() )
@@ -153,7 +190,6 @@ public class DatasetBuilderStd implements DatasetBuilder {
             error(log, "Directory not writeable: "+dirname) ;
     }
 
-    @Override
     public DatasetGraphTDB build(Location location, StoreParams params) {
         // Ensure that there is global synchronization
         synchronized (DatasetBuilderStd.class) {
@@ -184,7 +220,7 @@ public class DatasetBuilderStd implements DatasetBuilder {
         ReorderTransformation transform = (_transform == null) ? chooseReorderTransformation(location) : _transform ;
 
         StorageConfig storageConfig = new StorageConfig(location, params, writeable,
-                                                        recorder.blockMgrs, recorder.objectFiles, recorder.bufferChannels, recorder.nodeTables) ;
+                                                        recorder.blockMgrs, recorder.objectFiles, recorder.bufferChannels) ;
         
         recorder.finish() ;
         
@@ -287,24 +323,44 @@ public class DatasetBuilderStd implements DatasetBuilder {
         // Commonly, name == indexOrder.
         FileSet fs = new FileSet(location, name) ;
         ColumnMap colMap = new ColumnMap(primary, indexOrder) ;
-        return tupleIndexBuilder.buildTupleIndex(fs, colMap, indexOrder, params) ;
+        return /*tupleIndexBuilder.*/buildTupleIndex(fs, colMap, indexOrder, params) ;
+    }
+    
+    //-------------
+    private TupleIndex buildTupleIndex(FileSet fileSet, ColumnMap colMap, String name, StoreParams params) {
+        RecordFactory recordFactory = new RecordFactory(SystemTDB.SizeOfNodeId * colMap.length(), 0) ;
+        RangeIndex rIdx = /*rangeIndexBuilder.*/buildRangeIndex(fileSet, recordFactory, params) ;
+        TupleIndex tIdx = new TupleIndexRecord(colMap.length(), colMap, name, recordFactory, rIdx) ;
+        return tIdx ;
     }
 
     public NodeTable makeNodeTable(Location location, StoreParams params) {
-        FileSet fsNodeToId = new FileSet(location, params.getIndexNode2Id()) ;
-        FileSet fsId2Node = new FileSet(location, params.getIndexId2Node()) ;
-        NodeTable nt = nodeTableBuilder.buildNodeTable(fsNodeToId, fsId2Node, params) ;
-        return nt ;
+        return makeNodeTable$(location, params.getIndexNode2Id(), params.getIndexId2Node(), params);
     }
     
     /** Make a node table overriding the node->id and id->node table names */ 
     private NodeTable makeNodeTable$(Location location, String indexNode2Id, String indexId2Node, StoreParams params) {
         FileSet fsNodeToId = new FileSet(location, indexNode2Id) ;
         FileSet fsId2Node = new FileSet(location, indexId2Node) ;
-        NodeTable nt = nodeTableBuilder.buildNodeTable(fsNodeToId, fsId2Node, params) ;
+        NodeTable nt = /*nodeTableBuilder.*/buildNodeTable(fsNodeToId, fsId2Node, params) ;
         return nt ;
     }
     
+    //-------------
+    
+    private NodeTable buildNodeTable(FileSet fsIndex, FileSet fsObjectFile, StoreParams params) {
+        RecordFactory recordFactory = new RecordFactory(SystemTDB.LenNodeHash, SystemTDB.SizeOfNodeId) ;
+        Index idx = /*indexBuilder.*/buildIndex(fsIndex, recordFactory, params) ;
+        ObjectFile objectFile = objectFileBuilder.buildObjectFile(fsObjectFile, Names.extNodeData) ;
+        NodeTable nodeTable = new NodeTableNative(idx, objectFile) ;
+        nodeTable = NodeTableCache.create(nodeTable, 
+                                          params.getNode2NodeIdCacheSize(),
+                                          params.getNodeId2NodeCacheSize(),
+                                          params.getNodeMissCacheSize()) ;
+        nodeTable = NodeTableInline.create(nodeTable) ;
+        return nodeTable ;
+    }
+    
     protected NodeTable makeNodeTableNoCache(Location location, String indexNode2Id, String indexId2Node, StoreParams params) {
         StoreParamsBuilder spb = StoreParams.builder(params)
             .node2NodeIdCacheSize(-1)
@@ -386,26 +442,6 @@ public class DatasetBuilderStd implements DatasetBuilder {
         void record(FileRef fileRef, NodeTable nodeTable) ;
     }
 
-    static class NodeTableBuilderRecorder implements NodeTableBuilder {
-        private final NodeTableBuilder builder ;
-        private final RecordNodeTable  recorder ;
-
-        NodeTableBuilderRecorder(NodeTableBuilder ntb, RecordNodeTable recorder) {
-            this.builder = ntb ;
-            this.recorder = recorder ;
-        }
-
-        @Override
-        public NodeTable buildNodeTable(FileSet fsIndex, FileSet fsObjectFile, StoreParams params) {
-            NodeTable nt = builder.buildNodeTable(fsIndex, fsObjectFile, params) ;
-            // It just knows, right?
-            FileRef ref = FileRef.create(fsObjectFile.filename(Names.extNodeData)) ;
-            recorder.record(ref, nt) ;
-            return nt ;
-        }
-
-    }
-
     static class ObjectFileBuilderRecorder implements ObjectFileBuilder {
         private final ObjectFileBuilder builder ;
         private final RecordObjectFile  recorder ;

http://git-wip-us.apache.org/repos/asf/jena/blob/3d406fe1/jena-tdb/src/main/java/org/apache/jena/tdb/setup/NodeTableBuilder.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/setup/NodeTableBuilder.java b/jena-tdb/src/main/java/org/apache/jena/tdb/setup/NodeTableBuilder.java
deleted file mode 100644
index ebe90b3..0000000
--- a/jena-tdb/src/main/java/org/apache/jena/tdb/setup/NodeTableBuilder.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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 org.apache.jena.tdb.setup;
-
-import org.apache.jena.tdb.base.file.FileSet ;
-import org.apache.jena.tdb.store.nodetable.NodeTable ;
-
-public interface NodeTableBuilder {
-    NodeTable buildNodeTable(FileSet fsIndex, FileSet fsObjectFile, StoreParams params) ;
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/3d406fe1/jena-tdb/src/main/java/org/apache/jena/tdb/store/StorageConfig.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/store/StorageConfig.java b/jena-tdb/src/main/java/org/apache/jena/tdb/store/StorageConfig.java
index 1da389b..1ebbea2 100644
--- a/jena-tdb/src/main/java/org/apache/jena/tdb/store/StorageConfig.java
+++ b/jena-tdb/src/main/java/org/apache/jena/tdb/store/StorageConfig.java
@@ -25,7 +25,6 @@ import org.apache.jena.tdb.base.file.BufferChannel ;
 import org.apache.jena.tdb.base.file.Location ;
 import org.apache.jena.tdb.base.objectfile.ObjectFile;
 import org.apache.jena.tdb.setup.StoreParams ;
-import org.apache.jena.tdb.store.nodetable.NodeTable;
 import org.apache.jena.tdb.sys.FileRef ;
 
 public class StorageConfig
@@ -34,22 +33,19 @@ public class StorageConfig
     public final Map<FileRef, BlockMgr> blockMgrs ;
     public final Map<FileRef, ObjectFile> objectFiles;
     public final Map<FileRef, BufferChannel> bufferChannels;
-    public final Map<FileRef, NodeTable> nodeTables;
     public final Location location ;
     public final boolean writeable ;
 
     public StorageConfig(Location location, StoreParams params, boolean writeable, 
                          Map<FileRef, BlockMgr> blockMgrs, 
                          Map<FileRef, ObjectFile> objectFiles,
-                         Map<FileRef, BufferChannel> bufferChannels, 
-                         Map<FileRef, NodeTable> nodeTables)
+                         Map<FileRef, BufferChannel> bufferChannels)
     {
         this.location = location ;
         this.params = params ;
         this.blockMgrs = blockMgrs ;
         this.objectFiles = objectFiles ;
         this.bufferChannels = bufferChannels ;
-        this.nodeTables = nodeTables;
         this.writeable = writeable ;
     }
     

http://git-wip-us.apache.org/repos/asf/jena/blob/3d406fe1/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/DatasetBuilderTxn.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/DatasetBuilderTxn.java b/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/DatasetBuilderTxn.java
index e32bc2a..49d7264 100644
--- a/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/DatasetBuilderTxn.java
+++ b/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/DatasetBuilderTxn.java
@@ -23,30 +23,23 @@ import java.util.Map ;
 import org.apache.jena.query.ReadWrite ;
 import org.apache.jena.tdb.TDBException ;
 import org.apache.jena.tdb.base.block.BlockMgr ;
-import org.apache.jena.tdb.base.block.BlockMgrLogger ;
 import org.apache.jena.tdb.base.block.BlockMgrReadonly ;
-import org.apache.jena.tdb.base.file.FileFactory;
 import org.apache.jena.tdb.base.file.FileSet ;
 import org.apache.jena.tdb.base.objectfile.ObjectFile ;
-import org.apache.jena.tdb.base.objectfile.ObjectFileReadonly;
-import org.apache.jena.tdb.base.record.RecordFactory;
-import org.apache.jena.tdb.index.*;
-import org.apache.jena.tdb.setup.*;
+import org.apache.jena.tdb.base.objectfile.ObjectFileReadonly ;
+import org.apache.jena.tdb.index.IndexParams ;
+import org.apache.jena.tdb.setup.BlockMgrBuilder ;
+import org.apache.jena.tdb.setup.DatasetBuilderStd ;
+import org.apache.jena.tdb.setup.ObjectFileBuilder ;
 import org.apache.jena.tdb.store.DatasetGraphTDB ;
-import org.apache.jena.tdb.store.nodetable.*;
 import org.apache.jena.tdb.sys.FileRef ;
-import org.apache.jena.tdb.sys.Names;
-import org.apache.jena.tdb.sys.SystemTDB;
 
 public class DatasetBuilderTxn
 {
-    // Ideally, don't make a DatasetGraphTDB to pass to new DatasetGraphTxn as it rips it apart.
-    
     // Context for the build.
     private final TransactionManager txnMgr ;
     private final Map<FileRef, BlockMgr> blockMgrs ; 
     private final Map<FileRef, ObjectFile> objectFiles; 
-    private final Map<FileRef, NodeTable> nodeTables;
     private final DatasetGraphTDB dsg ;
     private Transaction txn;
 
@@ -54,7 +47,6 @@ public class DatasetBuilderTxn
         this.txnMgr = txnMgr ;
         this.blockMgrs = dsg.getConfig().blockMgrs ;
         this.objectFiles = dsg.getConfig().objectFiles ;
-        this.nodeTables = dsg.getConfig().nodeTables ;
         this.dsg = dsg ;
     }
     
@@ -77,72 +69,23 @@ public class DatasetBuilderTxn
 
     private DatasetGraphTDB buildReadonly() {
         BlockMgrBuilder blockMgrBuilder = new BlockMgrBuilderReadonly();
-        NodeTableBuilder nodeTableBuilder = new NodeTableBuilderReadonly();
-        DatasetBuilderStd x = new DatasetBuilderStd(blockMgrBuilder, nodeTableBuilder) ;
+        ObjectFileBuilder objectFileBuilder = new ObjectFileBuilderReadonly();
+        DatasetBuilderStd x = new DatasetBuilderStd(blockMgrBuilder, objectFileBuilder) ;
         DatasetGraphTDB dsg2 = x._build(dsg.getLocation(), dsg.getConfig().params, false, dsg.getReorderTransform()) ;
         return dsg2 ;
     }
 
     private DatasetGraphTDB buildWritable() {
         BlockMgrBuilder blockMgrBuilder = new BlockMgrBuilderTx() ;
-        NodeTableBuilder nodeTableBuilder = new NodeTableBuilderTx();
-        DatasetBuilderStd x = new DatasetBuilderStd(blockMgrBuilder, nodeTableBuilder) ;
+        ObjectFileBuilder objectFileBuilder = new ObjectFileBuilderTx();
+        DatasetBuilderStd x = new DatasetBuilderStd(blockMgrBuilder, objectFileBuilder);
         DatasetGraphTDB dsg2 = x._build(dsg.getLocation(), dsg.getConfig().params, true, dsg.getReorderTransform()) ;
         dsg2.getContext().putAll(dsg.getContext()) ;
         return dsg2 ;
     }
 
-    // ---- Add logging to a BlockMgr when built.
-    static BlockMgrBuilder logging(BlockMgrBuilder other) { return new BlockMgrBuilderLogger(other) ; }
-    
-    static class BlockMgrBuilderLogger implements BlockMgrBuilder {
-        public BlockMgrBuilder other ;
-
-        public BlockMgrBuilderLogger(BlockMgrBuilder other) {
-            this.other = other ;
-        }
-
-        @Override
-        public BlockMgr buildBlockMgr(FileSet fileSet, String ext, IndexParams params) {
-            BlockMgr blkMgr = other.buildBlockMgr(fileSet, ext, params) ;
-            blkMgr = new BlockMgrLogger(blkMgr.getLabel(), blkMgr, true) ;
-            return blkMgr ;
-        }
-    }
-
     // ---- Build transactional versions for update.
     
-    class NodeTableBuilderTx implements NodeTableBuilder
-    {
-        @Override
-        public NodeTable buildNodeTable(FileSet fsIndex, FileSet fsObjectFile, StoreParams params) {
-            FileRef ref = FileRef.create(fsObjectFile.filename(Names.extNodeData)) ;
-            NodeTable ntBase = nodeTables.get(ref) ;
-            if ( ntBase == null )
-                throw new TDBException("No NodeTable for " + ref) ;
-
-            RecordFactory recordFactory = new RecordFactory(SystemTDB.LenNodeHash, SystemTDB.SizeOfNodeId) ;
-            Index idx = new IndexMap(recordFactory) ;
-            String objFilename = fsObjectFile.filename(Names.extNodeData + "-" + Names.extJournal) ;
-            ObjectFile objectFile ;
-
-            if ( fsObjectFile.isMem() )
-                objectFile = FileFactory.createObjectFileMem(objFilename) ;
-            else
-                objectFile = FileFactory.createObjectFileDisk(objFilename) ;
-
-            // Allow for a modified base NodeTable. 
-            NodeTable ntBaseTrans = ntBase;
-            
-            NodeTableTrans ntt = new NodeTableTrans(txn, fsObjectFile.getBasename(), ntBaseTrans, idx, objectFile) ;
-            txn.addComponent(ntt) ;
-
-            // Add inline wrapper.
-            NodeTable nt = NodeTableInline.create(ntt) ;
-            return nt ;
-        }
-    }
-    
     class BlockMgrBuilderTx implements BlockMgrBuilder
     {
         @Override
@@ -150,33 +93,32 @@ public class DatasetBuilderTxn
             // Find from file ref.
             FileRef ref = FileRef.create(fileSet, ext) ;
             BlockMgr baseMgr = blockMgrs.get(ref) ;
-            if ( baseMgr == null )
-                throw new TDBException("No BlockMgr for " + ref) ;
+            if ( baseMgr == null ) {
+                //System.out.flush();
+                System.out.println("No BlockMgr for " + ref+" : "+blockMgrs.keySet());
+                //throw new TDBException("No BlockMgr for " + ref) ;
+            }
             BlockMgrJournal blkMgr = new BlockMgrJournal(txn, ref, baseMgr) ;
             txn.addComponent(blkMgr) ;
             return blkMgr ;
         }
     }
 
-    // Object files currently don't need journalling. Because they are append only, they
-    // are "self journalling" - we can append to them while used read-only elsewhere if
-    // the index to access them is transactional and does not point into the appended new
-    // bytes.
-//    class ObjectFileBuilderTx implements ObjectFileBuilder
-//    {
-//        @Override
-//        public ObjectFile buildObjectFile(FileSet fileSet, String ext) {
-//            FileRef ref = FileRef.create(fileSet, ext) ;
-//            ObjectFile baseObjFile = objectFiles.get(ref);
-//            if ( baseObjFile == null )
-//                throw new TDBException("No ObjectFile for " + ref) ;
-//            ObjectFileJournal objFile = new ObjectFileJournal(txn, ref, baseObjectFile); 
-//            return objFile;
-//        }
-//    }
-    
+    class ObjectFileBuilderTx implements ObjectFileBuilder
+    {
+        @Override
+        public ObjectFile buildObjectFile(FileSet fileSet, String ext) {
+            FileRef ref = FileRef.create(fileSet, ext) ;
+            ObjectFile base = objectFiles.get(ref) ;
+            // Just write to the (append only) ObjectFile and manage aborts.
+            ObjectFileTrans objFileTxn = new ObjectFileTrans(txn, base) ;
+            txn.addComponent(objFileTxn);
+            return objFileTxn;
+        }
+    }
+
     // ---- Build passthrough versions for readonly access
-    
+
     class BlockMgrBuilderReadonly implements BlockMgrBuilder
     {
         @Override
@@ -190,28 +132,14 @@ public class DatasetBuilderTxn
         }
     }
     
-    class NodeTableBuilderReadonly implements NodeTableBuilder
-    {
-        @Override
-        public NodeTable buildNodeTable(FileSet fsIndex, FileSet fsObjectFile, StoreParams params) {
-            FileRef ref = FileRef.create(fsObjectFile.filename(Names.extNodeData)) ;
-            NodeTable nt = nodeTables.get(ref) ;
-            nt = new NodeTableReadonly(nt) ;
-            return nt ;
-        }
-    }
- 
     class ObjectFileBuilderReadonly implements ObjectFileBuilder
     {
         @Override
         public ObjectFile buildObjectFile(FileSet fileSet, String ext) {
             FileRef ref = FileRef.create(fileSet, ext) ;
-            ObjectFile objFile = objectFiles.get(ref);
-            if ( objFile == null )
-                throw new TDBException("No ObjectFile for " + ref) ;
-            objFile = new ObjectFileReadonly(objFile) ;
-            return objFile;
+            ObjectFile file = objectFiles.get(ref) ;
+            return new ObjectFileReadonly(file);
         }
     }
- 
 }
+

http://git-wip-us.apache.org/repos/asf/jena/blob/3d406fe1/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/JournalControl.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/JournalControl.java b/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/JournalControl.java
index f384908..03a5d32 100644
--- a/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/JournalControl.java
+++ b/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/JournalControl.java
@@ -29,23 +29,16 @@ import java.util.Iterator ;
 
 import org.apache.jena.atlas.iterator.Iter ;
 import org.apache.jena.atlas.lib.FileOps ;
-import org.apache.jena.atlas.logging.FmtLog;
 import org.apache.jena.tdb.TDBException ;
 import org.apache.jena.tdb.base.block.Block ;
 import org.apache.jena.tdb.base.block.BlockMgr ;
 import org.apache.jena.tdb.base.file.BufferChannel ;
 import org.apache.jena.tdb.base.file.BufferChannelFile ;
-import org.apache.jena.tdb.base.file.FileFactory;
 import org.apache.jena.tdb.base.file.Location ;
-import org.apache.jena.tdb.base.objectfile.ObjectFile;
-import org.apache.jena.tdb.base.record.RecordFactory;
-import org.apache.jena.tdb.index.IndexMap;
 import org.apache.jena.tdb.store.DatasetGraphTDB ;
 import org.apache.jena.tdb.store.StorageConfig ;
-import org.apache.jena.tdb.store.nodetable.NodeTable;
 import org.apache.jena.tdb.sys.FileRef ;
 import org.apache.jena.tdb.sys.Names ;
-import org.apache.jena.tdb.sys.SystemTDB;
 import org.slf4j.Logger ;
 import org.slf4j.LoggerFactory ;
 
@@ -92,17 +85,13 @@ public class JournalControl
         if ( journal == null || journal.isEmpty() )
             return ;
         
-        
-        for ( FileRef fileRef : dsg.getConfig().nodeTables.keySet() )
+        for ( FileRef fileRef : dsg.getConfig().objectFiles.keySet() )
             recoverNodeDat(dsg, fileRef) ;
-        // Not used currently.
-//        for ( FileRef fileRef : dsg.getConfig().objectFiles.keySet() )
-//            recoverObjectFile(dsg, fileRef) ;
         recoverFromJournal(dsg.getConfig(), journal) ;
         
         journal.close() ;
         // Recovery complete.  Tidy up.  Node journal files have already been handled.
-        if ( journal.getFilename() != null )
+        if ( journal.getFilename() != null )  
         {
             if ( FileOps.exists(journal.getFilename()) )
                 FileOps.delete(journal.getFilename()) ;
@@ -201,35 +190,24 @@ public class JournalControl
         } finally { Iter.close(iter) ; }
     }
     
-    private static void recoverObjectFile(DatasetGraphTDB dsg, FileRef fileRef) {
-        FmtLog.warn(log, "Not recovered: recoverObjectFile[%s]", fileRef);
-    }
-
     /** Recover a node data file (".dat").
-     *  Node data files are append-only so recovering, then not using the data is safe.
-     *  Node data file is a precursor for full recovery that works from the master journal.
+     *  Node data files are append-only so recovering.
+     *  This code is only for ObjectFileTransComplex.
      */
     private static void recoverNodeDat(DatasetGraphTDB dsg, FileRef fileRef)
     {
-        // See DatasetBuilderTxn - same name generation code.
-        
-        RecordFactory recordFactory = new RecordFactory(SystemTDB.LenNodeHash, SystemTDB.SizeOfNodeId) ;
-        NodeTable baseNodeTable = dsg.getConfig().nodeTables.get(fileRef) ;
-        String objFilename = fileRef.getFilename()+"-"+Names.extJournal ;
-        objFilename = dsg.getLocation().absolute(objFilename) ;
-        File jrnlFile = new File(objFilename) ;
-        if ( jrnlFile.exists() && jrnlFile.length() > 0 )
-        {
-            syslog.info("Recovering node data: "+fileRef.getFilename()) ;
-            ObjectFile dataJrnl = FileFactory.createObjectFileDisk(objFilename) ;
-            NodeTableTrans ntt = new NodeTableTrans(null, objFilename, baseNodeTable, new IndexMap(recordFactory), dataJrnl) ;
-            ntt.append() ;
-            ntt.close() ;
-            dataJrnl.close() ;
-            baseNodeTable.sync() ;
-        }
-        if ( jrnlFile.exists() )
-            FileOps.delete(objFilename) ;
+//        // See DatasetBuilderTxn - same name generation code.
+//        String objFilename = fileRef.getFilename()+"-"+Names.extJournal ;
+//        objFilename = dsg.getLocation().absolute(objFilename) ;
+//        File jrnlFile = new File(objFilename) ;
+//        if ( jrnlFile.exists() && jrnlFile.length() > 0 )
+//        {
+//            syslog.info("Clearing node data: "+fileRef.getFilename()) ;
+//            ObjectFile dataJrnl = FileFactory.createObjectFileDisk(objFilename) ;
+//            dataJrnl.truncate(0);
+//        }
+//        if ( jrnlFile.exists() )
+//            FileOps.delete(objFilename) ;
     }
     
     public static void replay(Transaction transaction)

http://git-wip-us.apache.org/repos/asf/jena/blob/3d406fe1/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/NodeTableTrans.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/NodeTableTrans.java b/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/NodeTableTrans.java
deleted file mode 100644
index 98615e1..0000000
--- a/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/NodeTableTrans.java
+++ /dev/null
@@ -1,413 +0,0 @@
-/*
- * 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 org.apache.jena.tdb.transaction;
-
-import static org.apache.jena.atlas.logging.FmtLog.warn ;
-
-import java.nio.ByteBuffer ;
-import java.util.Iterator ;
-
-import org.apache.jena.atlas.iterator.Iter ;
-import org.apache.jena.atlas.lib.ByteBufferLib ;
-import org.apache.jena.atlas.lib.Pair ;
-import org.apache.jena.graph.Node ;
-import org.apache.jena.tdb.TDBException ;
-import org.apache.jena.tdb.base.objectfile.ObjectFile ;
-import org.apache.jena.tdb.base.record.Record ;
-import org.apache.jena.tdb.index.Index ;
-import org.apache.jena.tdb.store.NodeId ;
-import org.apache.jena.tdb.store.nodetable.NodeTable ;
-import org.apache.jena.tdb.store.nodetable.NodeTableCache ;
-import org.apache.jena.tdb.store.nodetable.NodeTableInline ;
-import org.apache.jena.tdb.store.nodetable.NodeTableNative ;
-import org.slf4j.Logger ;
-import org.slf4j.LoggerFactory ;
-
-public class NodeTableTrans implements NodeTable, TransactionLifecycle
-{
-    private static Logger log = LoggerFactory.getLogger(NodeTableTrans.class) ;
-    // TODO flag to note is any work is needed on commit.
-    private final NodeTable base ;
-    private long allocOffset ;
-    
-    private NodeTable nodeTableJournal = null ;
-    private static int CacheSize = 10000 ;      // Make configurable 
-    private boolean passthrough = false ;
-    
-    private Index nodeIndex ;
-    private ObjectFile journalObjFile ;
-    // Start of the journal file for this transaction.
-    // Always zero currently but allows for future  
-    private long journalObjFileStartOffset ; 
-    private final String label ;
-    private final Transaction txn ;     // Can be null (during recovery).
-    
-    public NodeTableTrans(Transaction txn, String label, NodeTable sub, Index nodeIndex, ObjectFile journalObjFile)
-    {
-        this.txn = txn ;
-        this.base = sub ;
-        this.nodeIndex = nodeIndex ;
-        // Workspace for 
-        this.journalObjFile = journalObjFile ;
-        // Clear bytes from an old run
-        // (a crash while writing means the old transaction did not commit
-        //  any bytes in the file are junk)
-        // This is coupled to the fact the prepare phase does the actually data writing. 
-        journalObjFile.truncate(0) ;
-        this.label = label ; 
-    }
-
-    public void setPassthrough(boolean v)   { passthrough = v ; }
-    public NodeTable getBaseNodeTable()     { return base ; }
-    public NodeTable getJournalTable()      { return nodeTableJournal ; }
-    public Transaction getTransaction()     { return txn ; }
-    
-    @Override
-    public NodeId getAllocateNodeId(Node node)
-    {
-        if ( passthrough ) return base.getAllocateNodeId(node) ;
-        NodeId nodeId = getNodeIdForNode(node) ;
-        if ( ! NodeId.isDoesNotExist(nodeId) )
-            return nodeId ;
-        // add to journal
-        nodeId = allocate(node) ;
-        return nodeId ;
-    }
-    
-    @Override
-    public NodeId getNodeIdForNode(Node node)
-    {
-        if ( node == Node.ANY )
-            return NodeId.NodeIdAny ;
-        if ( passthrough ) return base.getNodeIdForNode(node) ;
-        NodeId nodeId = nodeTableJournal.getNodeIdForNode(node) ;
-        if ( ! NodeId.isDoesNotExist(nodeId) )
-            return mapFromJournal(nodeId) ;
-        nodeId = base.getNodeIdForNode(node) ;
-        return nodeId ;
-    }
-
-    @Override
-    public Node getNodeForNodeId(NodeId id)
-    {
-        if ( NodeId.isAny(id) ) 
-            return Node.ANY ;
-        if ( passthrough ) return base.getNodeForNodeId(id) ;
-        long x = id.getId() ;
-        if ( x < allocOffset )
-            return base.getNodeForNodeId(id) ;
-        id = mapToJournal(id) ;
-        Node node = nodeTableJournal.getNodeForNodeId(id) ;
-        return node ;
-    }
-
-    @Override
-    public boolean containsNode(Node node) {
-        NodeId x = getNodeIdForNode(node) ;
-        return NodeId.isDoesNotExist(x) ;
-    }
-
-    @Override
-    public boolean containsNodeId(NodeId nodeId) {
-        Node x = getNodeForNodeId(nodeId) ;
-        return x == null ;
-    }
-
-
-
-    /** Convert from a id to the id in the "journal" file */ 
-    private NodeId mapToJournal(NodeId id)
-    { 
-        if ( passthrough )
-           throw new TDBTransactionException("Not in an active transaction") ;
-        if ( NodeId.isInline(id) )
-            return id ; 
-        return NodeId.create(id.getId()-allocOffset) ;
-    }
-    
-    /** Convert from a id in other to an external id  */ 
-    private NodeId mapFromJournal(NodeId id)
-    { 
-        if ( passthrough )
-            throw new TDBTransactionException("Not in an active transaction") ;
-        if ( NodeId.isInline(id) )
-            return id ; 
-        return NodeId.create(id.getId()+allocOffset) ; 
-    }
-    
-    private NodeId allocate(Node node)
-    {
-        NodeId nodeId = nodeTableJournal.getAllocateNodeId(node) ;
-        nodeId = mapFromJournal(nodeId) ;
-        return nodeId ;
-    }
-    
-    @Override
-    public NodeId allocOffset()
-    {
-        if ( passthrough ) return base.allocOffset() ;
-        // If we have done the append stage, this is invalid as the base may change under our feet
-        // Would need to track base operations.
-        NodeId x1 = nodeTableJournal.allocOffset() ;
-        NodeId x2 = mapFromJournal(x1) ;
-        return x2 ;
-    }
-
-    @Override
-    public void begin(Transaction txn)
-    {
-        //debug("%s begin", txn.getLabel()) ;
-        
-        if ( this.txn.getTxnId() != txn.getTxnId() )
-            throw new TDBException(String.format("Different transactions: %s %s", this.txn.getLabel(), txn.getLabel())) ;
-        if ( passthrough )
-            throw new TDBException("Already active") ;
-        passthrough = false ;
-        
-        allocOffset = base.allocOffset().getId() ;
-        // base node table empty e.g. first use.
-        journalObjFileStartOffset = journalObjFile.length() ;
-        // Because the data is written in prepare, the journal of object data is
-        // always empty at the start of a transaction.
-        if ( journalObjFileStartOffset != 0 )
-            warn(log, "%s journalStartOffset not zero: %d/0x%02X",txn.getLabel(), journalObjFileStartOffset, journalObjFileStartOffset) ;
-        allocOffset += journalObjFileStartOffset ;
-        
-        this.nodeTableJournal = new NodeTableNative(nodeIndex, journalObjFile) ;
-        this.nodeTableJournal = NodeTableCache.create(nodeTableJournal, CacheSize, CacheSize, 100) ;
-        // This class knows about non-mappable inline values.   mapToJournal(NodeId)/mapFromJournal. 
-        this.nodeTableJournal = NodeTableInline.create(nodeTableJournal) ;
-    }
-    
-    static public boolean APPEND_LOG = false ; 
-    
-    /** Copy from the journal file to the real file */
-    /*package*/ void append()
-    {
-        Iterator<Pair<NodeId, Node>> iter = nodeTableJournal.all() ;
-        Pair<NodeId, Node> firstPair = null ;
-        Pair<NodeId, Node> lastPair = null ;
-        
-        for ( ; iter.hasNext() ; )
-        {
-            Pair<NodeId, Node> x = iter.next() ;
-            
-            if ( firstPair == null )
-                firstPair = x ;
-            lastPair = x ;
-            
-            NodeId nodeId = x.getLeft() ;
-            Node node = x.getRight() ;
-            debug("  append: %s -> %s", x, mapFromJournal(nodeId)) ;
-            // This does the write.
-            NodeId nodeId2 = base.getAllocateNodeId(node) ;
-            if ( ! nodeId2.equals(mapFromJournal(nodeId)) )
-                inconsistent(node, nodeId, nodeId2) ;
-        }
-    }
-    
-    private void inconsistent(Node node , NodeId nodeId , NodeId nodeId2 )
-    {
-        String msg = String.format("Different ids for %s: allocated: expected %s, got %s", node, mapFromJournal(nodeId), nodeId2) ;
-        System.err.println() ;
-        System.err.println() ;
-        System.err.println(msg) ;
-        dump() ;   
-        System.err.println() ;
-        throw new TDBException(msg) ;
-    }
-    
-    // Debugging only
-    private void dump()
-    {
-        System.err.println(">>>>>>>>>>") ;
-        System.err.println("label = "+label) ;
-        System.err.println("txn = "+txn) ;
-        System.err.println("offset = "+allocOffset) ;
-        System.err.println("journalStartOffset = "+journalObjFileStartOffset) ;
-        System.err.println("journal = "+journalObjFile.getLabel()) ;
-        if ( true )
-            return ;
-        
-        System.err.println("nodeTableJournal >>>") ;
-        Iterator<Pair<NodeId, Node>> iter = nodeTableJournal.all() ;
-        for ( ; iter.hasNext() ; )
-        {
-            Pair<NodeId, Node> x = iter.next() ;
-            NodeId nodeId = x.getLeft() ;
-            Node node = x.getRight() ;
-            NodeId mapped = mapFromJournal(nodeId) ;
-            //debug("append: %s -> %s", x, mapFromJournal(nodeId)) ;
-            // This does the write.
-            NodeId nodeId2 = base.getAllocateNodeId(node) ;
-            System.err.println(x + "  mapped=" + mapped + " getAlloc="+nodeId2) ;
-        }
-        
-        System.err.println("journal >>>") ;
-        Iterator<Pair<Long, ByteBuffer>> iter1 = this.journalObjFile.all() ;
-        for ( ; iter1.hasNext() ; )
-        {
-            Pair<Long, ByteBuffer> p = iter1.next() ;
-            System.err.println(p.getLeft()+" : "+p.getRight()) ;
-            ByteBufferLib.print(System.err, p.getRight()) ;
-        }
-        
-        System.err.println("nodeIndex >>>") ;
-        Iterator<Record> iter2 = this.nodeIndex.iterator() ;
-        for ( ; iter2.hasNext() ; )
-        {
-            Record r = iter2.next() ;
-            System.err.println(r) ;
-        }
-        System.err.println("<<<<<<<<<<") ;
-    }
-    
-    @Override
-    public void commitPrepare(Transaction txn)
-    {
-        // The index "node2id", which is Node hash to NodeId, is done because it has a BlockMgr.
-        debug("commitPrepare") ;
-        
-        // The node table is append-only so it can be written during prepare.
-        // The index isn't written (via the transaction journal) until enact.
-        if ( nodeTableJournal == null )
-            throw new TDBTransactionException(txn.getLabel()+": Not in a transaction for a commit to happen") ;
-        writeNodeJournal() ;
-        
-        if ( journalObjFile != null && journalObjFile.length() != 0 )
-        {
-            long x = journalObjFile.length() ;
-            throw new TDBTransactionException(txn.getLabel()+": journalObjFile not cleared ("+x+")") ;
-        }
-    }
-    
-    @Override
-    public void commitEnact(Transaction txn)
-    {
-        debug("commitEnact") ;
-        // The work was done in commitPrepare, using the fact that node data file
-        // is append only.  Until here, pointers to the extra data aren't available
-        // until the index is written.
-        // The index is written via the transaction journal.
-        //writeJournal() ;
-    }
-
-    private void writeNodeJournal()
-    {
-        long expected = base.allocOffset().getId() ;
-        long len = journalObjFile.length() ;
-        if ( expected != allocOffset )
-            warn(log, "Inconsistency: base.allocOffset() = %d : allocOffset = %d", expected, allocOffset) ;
-        
-        long newbase = -1 ; 
-        // Copy to the base NodeTable. 
-        append() ;
-        // Reset (in case we use this again)
-        nodeIndex.clear() ;
-        journalObjFile.truncate(journalObjFileStartOffset) ;    // Side effect is a buffer flush.
-        //journalObjFile.sync() ;
-        journalObjFile.close() ;                                // Side effect is a buffer flush.
-        journalObjFile = null ;
-        base.sync() ;
-        allocOffset = -99 ; // base.allocOffset().getId() ; // Will be invalid as we may write through to the base table later.
-        passthrough = true ;
-    }
-
-    @Override
-    public void commitClearup(Transaction txn)
-    {
-        debug("commitClearup") ;
-        finish() ;
-    }
-
-    @Override
-    public void abort(Transaction txn)
-    {
-        debug("abort") ;
-        if ( nodeTableJournal == null )
-            throw new TDBTransactionException(txn.getLabel()+": Not in a transaction for a commit to happen") ;
-        // Ensure the cache does not flush.
-        nodeTableJournal = null ;
-        // then make sure the journal file is empty.
-        if ( journalObjFile != null )
-        {
-            journalObjFile.truncate(journalObjFileStartOffset) ;
-            journalObjFile.sync() ;
-        }
-        finish() ;
-    }
-    
-    private void finish()
-    {
-        close() ;
-        passthrough = true ;
-        nodeTableJournal = null ;
-        journalObjFile = null ;
-   }
-
-    @Override
-    public Iterator<Pair<NodeId, Node>> all()
-    {
-        // Better would be to convert the spill file format.
-        return Iter.concat(base.all(), nodeTableJournal.all()) ;
-    }
-
-    @Override
-    public boolean isEmpty()
-    {
-        return nodeTableJournal.isEmpty() && base.isEmpty() ;
-    }
-
-    @Override
-    public void sync()
-    {
-        if ( passthrough )
-            base.sync() ;
-    }
-
-    @Override
-    public void close()
-    {
-        if ( nodeIndex != null )
-            nodeIndex.close() ;
-        nodeIndex = null ;
-        // Closing the journal flushes it; i.e. disk IO. 
-        if ( journalObjFile != null )
-            journalObjFile.close() ;
-        journalObjFile = null ;
-    }
-
-    @Override
-    public String toString() { return "NodeTableTrans:"+label+"(#"+Integer.toHexString(super.hashCode())+")" ; }
-    
-    private void debug(String fmt, Object... args)
-    {
-        if ( log.isDebugEnabled() )
-        {
-            String x = String.format(fmt, args) ;
-            log.debug(label+": "+x) ;
-        }
-    }
-
-    // Return the base table, not the nodeTableJournal
-    @Override
-    public NodeTable wrapped() {
-        return base ;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/3d406fe1/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/ObjectFileTrans.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/ObjectFileTrans.java b/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/ObjectFileTrans.java
index 82059ff..d88ea1e 100644
--- a/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/ObjectFileTrans.java
+++ b/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/ObjectFileTrans.java
@@ -18,223 +18,47 @@
 
 package org.apache.jena.tdb.transaction;
 
-import java.nio.ByteBuffer ;
-import java.util.Iterator ;
-
-import org.apache.jena.atlas.iterator.Iter ;
-import org.apache.jena.atlas.lib.Pair ;
-import org.apache.jena.atlas.lib.StrUtils ;
-import org.apache.jena.atlas.logging.Log ;
-import org.apache.jena.tdb.base.block.Block ;
-import org.apache.jena.tdb.base.file.FileException ;
 import org.apache.jena.tdb.base.objectfile.ObjectFile ;
-
-public class ObjectFileTrans implements ObjectFile, TransactionLifecycle
-{
-    private final ObjectFile transObjects ;
-    private long otherAllocOffset ;           // record where we start allocating
-    private boolean passthrough = false ;
-    private boolean inTransaction = false ;
-    private final ObjectFile base ;
-    
-    // For recovery replay, we need to truncate "base" first. 
-    
-    public ObjectFileTrans(Transaction txn, ObjectFile base, ObjectFile other)
-    {
-        // The "other" object file must use the same allocation policy.
-        this.base = base ;
-        this.transObjects = other ;
-        inTransaction = false ;
-
-        //  [TxTDB:PATCH-UP] Begin is not being called.
-        this.otherAllocOffset = base.length() ;
-        //Log.info(this, getLabel()+": otherAllocOffset = "+otherAllocOffset) ;
-    }
-
-    // Begin read ==> passthrough.
-    
-    @Override
-    public void begin(Transaction txn)
-    {
-        passthrough = false ;
-        inTransaction = true ;
-        transObjects.reposition(0) ;
-        this.otherAllocOffset = base.length() ;
-    }
-    
-    @Override
-    public void commitPrepare(Transaction txn)
-    {
-        if ( ! inTransaction )
-            throw new TDBTransactionException("Not in a transaction for a commit to happen") ; 
-        transObjects.sync() ;
-    }
-
-    @Override
-    public void commitEnact(Transaction txn)
-    {
-        if ( ! inTransaction )
-            throw new TDBTransactionException("Not in a transaction for a commit to happen") ; 
-        append() ;
-        base.sync() ;
-        transObjects.reposition(0) ;
-    }
-
-    @Override
-    public void abort(Transaction txn)
-    {
-        transObjects.reposition(0) ;
-    }
-    
-    @Override
-    public void commitClearup(Transaction txn)
-    {
-        transObjects.truncate(0) ;
-        passthrough = true ;
-    }
-
-    /** Copy from the temporary file to the real file */
-    private void append()
-    {
-        // We could write directly to the real file if:
-        //    we record the truncate point needed for an abort
-        //    manage partial final writes
-        //    deny the existence of nodes after the transaction mark.
-        // Later - stay simple for now.
-        
-        // Truncate/position the ObjectFile.
-        base.reposition(otherAllocOffset) ;
-        
-        Iterator<Pair<Long, ByteBuffer>> iter = transObjects.all() ;
-        for ( ; iter.hasNext() ; )
-        {
-            Pair<Long, ByteBuffer> p = iter.next() ;
-            String s = StrUtils.fromUTF8bytes(p.getRight().array()) ;
-            
-            long x = base.write(p.getRight()) ;
-            
-            if ( p.getLeft()+otherAllocOffset != x )
-                throw new FileException("Expected id of "+(p.getLeft()+otherAllocOffset)+", got an id of "+x) ;
-        }
-    }
-    
-    //public void setPassthrough(boolean v) { passthrough = v ; }
-    
-    @Override
-    public void reposition(long id)
-    {
-        if ( passthrough ) { base.reposition(id) ; return ; }
-        if ( id > otherAllocOffset )
-        {
-            transObjects.reposition(mapToOther(id)) ;
-            return ;
-        }
-        
-        Log.warn(this, "Unexpected: Attempt to reposition over base file") ;
-        base.reposition(id) ;
-        transObjects.reposition(0) ;
-        otherAllocOffset = base.length() ;
-    }
-    
-    @Override
-    public void truncate(long id)
-    {
-        if ( passthrough ) { base.truncate(id) ; return ; }
-        if ( id > otherAllocOffset )
-        {
-            transObjects.truncate(mapToOther(id)) ;
-            return ;
-        }
-        base.truncate(id) ;
-        transObjects.truncate(0) ;
-        otherAllocOffset = base.length() ;
-    }
-
-    @Override
-    public Block allocWrite(int maxBytes)
-    {
-        if ( passthrough ) return base.allocWrite(maxBytes) ;
-        Block block = transObjects.allocWrite(maxBytes) ;
-        block = new Block(block.getId()+otherAllocOffset, block.getByteBuffer()) ;
-        return block ;
-    }
-
-    @Override
-    public void completeWrite(Block block)
-    {
-        if ( passthrough ) { base.completeWrite(block) ; return ; } 
-        block = new Block(block.getId()-otherAllocOffset, block.getByteBuffer()) ;
-        transObjects.completeWrite(block) ;
-    }
-    
-    @Override
-    public void abortWrite(Block block)
-    {
-        if ( passthrough ) { base.abortWrite(block) ; return ; } 
-        block = new Block(block.getId()-otherAllocOffset, block.getByteBuffer()) ;
-        transObjects.abortWrite(block) ;
-    }
-
-    /** Convert from a id to the id in the "other" file */ 
-    private long mapToOther(long x) { return x-otherAllocOffset ; }
-    /** Convert from a id in other to an external id  */ 
-    private long mapFromOther(long x) { return x+otherAllocOffset ; }
-    
-    @Override
-    public long write(ByteBuffer buffer)
-    {
-        if ( passthrough ) { return base.write(buffer) ; } 
-        // Write to auxillary
-        long x = transObjects.write(buffer) ;
-        return mapFromOther(x) ;
-    }
-
-    @Override
-    public ByteBuffer read(long id)
-    {
-        if ( passthrough ) { return base.read(id) ; } 
-        if ( id < otherAllocOffset )
-            return base.read(id) ;
-        long x = mapToOther(id) ; 
-        return transObjects.read(id-otherAllocOffset) ;
+import org.apache.jena.tdb.base.objectfile.ObjectFileWrapper ;
+
+/** Add transactionality control to an ObjectFile.
+ * ObjectFiles are "append only" so with a single rwiter environment, 
+ * we just need to manage a reset on abort.
+ * A crash in a transaction will accumulate some junk in the file.
+ * This is now a trandeoff of speed and space.
+ * 
+ * Speed : append to the original file directly and tolerate junk.
+ * 
+ * Space : use a journal file and write to main file on commit.
+ * 
+ * @see ObjectFileTransComplex  
+ */
+class ObjectFileTrans extends ObjectFileWrapper implements TransactionLifecycle {
+    ObjectFileTrans(Transaction txn /*unused*/, ObjectFile other) {
+        super(other);
     }
 
-    @Override
-    public long length()
-    {
-        if ( passthrough ) { return base.length() ; } 
-        return otherAllocOffset+transObjects.length() ;
-    }
+    private long start = 0;
     
     @Override
-    public boolean isEmpty()
-    {
-        if ( passthrough ) { return base.isEmpty() ; } 
-        return transObjects.isEmpty() && base.isEmpty() ;
+    public void begin(Transaction txn) {
+        start = other.length();
     }
 
     @Override
-    public Iterator<Pair<Long, ByteBuffer>> all()
-    {
-        if ( passthrough ) { return base.all() ; } 
-        return Iter.concat(base.all(), transObjects.all()) ;
+    public void abort(Transaction txn) {
+        other.truncate(start);
     }
 
     @Override
-    public void sync()
-    { 
-        if ( passthrough ) { base.sync() ; return ; } 
+    public void commitPrepare(Transaction txn) {
+        // Sync early - before the journal, with its index blocks, is committed.
+        other.sync();
     }
 
     @Override
-    public void close()
-    {
-        if ( passthrough ) { base.close() ; return ; }
-    }
+    public void commitEnact(Transaction txn) { }
 
     @Override
-    public String getLabel()
-    {
-        return "("+base.getLabel()+":"+transObjects.getLabel()+")" ;
-    }
-}
+    public void commitClearup(Transaction txn) {}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/3d406fe1/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/ObjectFileTransComplex.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/ObjectFileTransComplex.java b/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/ObjectFileTransComplex.java
new file mode 100644
index 0000000..59b7d83
--- /dev/null
+++ b/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/ObjectFileTransComplex.java
@@ -0,0 +1,253 @@
+/*
+ * 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 org.apache.jena.tdb.transaction;
+
+import java.nio.ByteBuffer ;
+import java.util.Iterator ;
+
+import org.apache.jena.atlas.iterator.Iter ;
+import org.apache.jena.atlas.lib.Pair ;
+import org.apache.jena.atlas.lib.StrUtils ;
+import org.apache.jena.atlas.logging.Log ;
+import org.apache.jena.tdb.base.block.Block ;
+import org.apache.jena.tdb.base.file.FileException ;
+import org.apache.jena.tdb.base.objectfile.ObjectFile ;
+
+/**
+ * This version works with two files - the main, base file and a per-transction journal
+ * file. It writes to the journal side.
+ * 
+ * To flush, it writes the journal to the base file.
+ * 
+ * It is relatively untested.
+ * 
+ * Compare to {@link ObjectFileTrans} uses the fact that object files are append only so
+ * writes directly and resets on abort.
+ * 
+ * @see ObjectFileTrans
+ */
+public class ObjectFileTransComplex implements ObjectFile, TransactionLifecycle
+{
+    private final ObjectFile transObjects ;
+    private long otherAllocOffset ;           // record where we start allocating
+    private boolean passthrough = false ;
+    private boolean inTransaction = false ;
+    private final ObjectFile base ;
+    
+    // For recovery replay, we need to truncate "base" first. 
+    
+    public ObjectFileTransComplex(Transaction txn, ObjectFile base, ObjectFile other)
+    {
+        // The "other" object file must use the same allocation policy.
+        this.base = base ;
+        this.transObjects = other ;
+        inTransaction = false ;
+
+        //  [TxTDB:PATCH-UP] Begin is not being called.
+        this.otherAllocOffset = base.length() ;
+        //Log.info(this, getLabel()+": otherAllocOffset = "+otherAllocOffset) ;
+    }
+
+    // Begin read ==> passthrough.
+    
+    @Override
+    public void begin(Transaction txn)
+    {
+        passthrough = false ;
+        inTransaction = true ;
+        transObjects.reposition(0) ;
+        this.otherAllocOffset = base.length() ;
+    }
+    
+    @Override
+    public void commitPrepare(Transaction txn)
+    {
+        if ( ! inTransaction )
+            throw new TDBTransactionException("Not in a transaction for a commit to happen") ; 
+        transObjects.sync() ;
+    }
+
+    @Override
+    public void commitEnact(Transaction txn)
+    {
+        if ( ! inTransaction )
+            throw new TDBTransactionException("Not in a transaction for a commit to happen") ; 
+        append() ;
+        base.sync() ;
+        transObjects.reposition(0) ;
+    }
+
+    @Override
+    public void abort(Transaction txn)
+    {
+        transObjects.reposition(0) ;
+    }
+    
+    @Override
+    public void commitClearup(Transaction txn)
+    {
+        transObjects.truncate(0) ;
+        passthrough = true ;
+    }
+
+    /** Copy from the temporary file to the real file */
+    private void append()
+    {
+        // We could write directly to the real file if:
+        //    we record the truncate point needed for an abort
+        //    manage partial final writes
+        //    deny the existence of nodes after the transaction mark.
+        // Later - stay simple for now.
+        
+        // Truncate/position the ObjectFile.
+        base.reposition(otherAllocOffset) ;
+        
+        Iterator<Pair<Long, ByteBuffer>> iter = transObjects.all() ;
+        for ( ; iter.hasNext() ; )
+        {
+            Pair<Long, ByteBuffer> p = iter.next() ;
+            String s = StrUtils.fromUTF8bytes(p.getRight().array()) ;
+            
+            long x = base.write(p.getRight()) ;
+            
+            if ( p.getLeft()+otherAllocOffset != x )
+                throw new FileException("Expected id of "+(p.getLeft()+otherAllocOffset)+", got an id of "+x) ;
+        }
+    }
+    
+    //public void setPassthrough(boolean v) { passthrough = v ; }
+    
+    @Override
+    public void reposition(long id)
+    {
+        if ( passthrough ) { base.reposition(id) ; return ; }
+        if ( id > otherAllocOffset )
+        {
+            transObjects.reposition(mapToOther(id)) ;
+            return ;
+        }
+        
+        Log.warn(this, "Unexpected: Attempt to reposition over base file") ;
+        base.reposition(id) ;
+        transObjects.reposition(0) ;
+        otherAllocOffset = base.length() ;
+    }
+    
+    @Override
+    public void truncate(long id)
+    {
+        if ( passthrough ) { base.truncate(id) ; return ; }
+        if ( id > otherAllocOffset )
+        {
+            transObjects.truncate(mapToOther(id)) ;
+            return ;
+        }
+        base.truncate(id) ;
+        transObjects.truncate(0) ;
+        otherAllocOffset = base.length() ;
+    }
+
+    @Override
+    public Block allocWrite(int maxBytes)
+    {
+        if ( passthrough ) return base.allocWrite(maxBytes) ;
+        Block block = transObjects.allocWrite(maxBytes) ;
+        block = new Block(block.getId()+otherAllocOffset, block.getByteBuffer()) ;
+        return block ;
+    }
+
+    @Override
+    public void completeWrite(Block block)
+    {
+        if ( passthrough ) { base.completeWrite(block) ; return ; } 
+        block = new Block(block.getId()-otherAllocOffset, block.getByteBuffer()) ;
+        transObjects.completeWrite(block) ;
+    }
+    
+    @Override
+    public void abortWrite(Block block)
+    {
+        if ( passthrough ) { base.abortWrite(block) ; return ; } 
+        block = new Block(block.getId()-otherAllocOffset, block.getByteBuffer()) ;
+        transObjects.abortWrite(block) ;
+    }
+
+    /** Convert from a id to the id in the "other" file */ 
+    private long mapToOther(long x) { return x-otherAllocOffset ; }
+    /** Convert from a id in other to an external id  */ 
+    private long mapFromOther(long x) { return x+otherAllocOffset ; }
+    
+    @Override
+    public long write(ByteBuffer buffer)
+    {
+        if ( passthrough ) { return base.write(buffer) ; } 
+        // Write to auxillary
+        long x = transObjects.write(buffer) ;
+        return mapFromOther(x) ;
+    }
+
+    @Override
+    public ByteBuffer read(long id)
+    {
+        if ( passthrough ) { return base.read(id) ; } 
+        if ( id < otherAllocOffset )
+            return base.read(id) ;
+        long x = mapToOther(id) ; 
+        return transObjects.read(id-otherAllocOffset) ;
+    }
+
+    @Override
+    public long length()
+    {
+        if ( passthrough ) { return base.length() ; } 
+        return otherAllocOffset+transObjects.length() ;
+    }
+    
+    @Override
+    public boolean isEmpty()
+    {
+        if ( passthrough ) { return base.isEmpty() ; } 
+        return transObjects.isEmpty() && base.isEmpty() ;
+    }
+
+    @Override
+    public Iterator<Pair<Long, ByteBuffer>> all()
+    {
+        if ( passthrough ) { return base.all() ; } 
+        return Iter.concat(base.all(), transObjects.all()) ;
+    }
+
+    @Override
+    public void sync()
+    { 
+        if ( passthrough ) { base.sync() ; return ; } 
+    }
+
+    @Override
+    public void close()
+    {
+        if ( passthrough ) { base.close() ; return ; }
+    }
+
+    @Override
+    public String getLabel()
+    {
+        return "("+base.getLabel()+":"+transObjects.getLabel()+")" ;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/3d406fe1/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/Transaction.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/Transaction.java b/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/Transaction.java
index 5bfd6f0..61859f8 100644
--- a/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/Transaction.java
+++ b/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/Transaction.java
@@ -38,7 +38,7 @@ public class Transaction
     private final Journal journal ;
     private final ReadWrite mode ;
     
-    private final List<NodeTableTrans> nodeTableTrans = new ArrayList<>() ;
+    private final List<ObjectFileTrans> objectFileTrans = new ArrayList<>() ;
     private final List<BlockMgrJournal> blkMgrs = new ArrayList<>() ;
     // The dataset this is a transaction over - may be a commited, pending dataset.
     private final DatasetGraphTDB   basedsg ;
@@ -171,9 +171,9 @@ public class Transaction
 
     private void prepare() {
         state = TxnState.PREPARING ;
-        for ( BlockMgrJournal x : blkMgrs )
+        for ( TransactionLifecycle x : objectFileTrans )
             x.commitPrepare(this) ;
-        for ( NodeTableTrans x : nodeTableTrans )
+        for ( TransactionLifecycle x : blkMgrs )
             x.commitPrepare(this) ;
     }
 
@@ -189,10 +189,9 @@ public class Transaction
                         throw new TDBTransactionException("Transaction has already committed or aborted") ;
                     try {
                         // Clearup.
-                        for ( BlockMgrJournal x : blkMgrs )
+                        for ( TransactionLifecycle x : objectFileTrans )
                             x.abort(this) ;
-
-                        for ( NodeTableTrans x : nodeTableTrans )
+                        for ( TransactionLifecycle x : blkMgrs )
                             x.abort(this) ;
                     }
                     catch (RuntimeException ex) {
@@ -314,13 +313,13 @@ public class Transaction
     /** Return the list of items registered for the transaction lifecycle */ 
     public List<TransactionLifecycle> lifecycleComponents() {
         List<TransactionLifecycle> x = new ArrayList<>() ;
-        x.addAll(nodeTableTrans) ;
+        x.addAll(objectFileTrans) ;
         x.addAll(blkMgrs) ;
         return x ;
     }
     
-    /*package*/ void addComponent(NodeTableTrans ntt) {
-        nodeTableTrans.add(ntt) ;
+    /*package*/ void addComponent(ObjectFileTrans oft) {
+        objectFileTrans.add(oft);
     }
 
     /*package*/ void addComponent(BlockMgrJournal blkMgr) {

http://git-wip-us.apache.org/repos/asf/jena/blob/3d406fe1/jena-tdb/src/test/java/org/apache/jena/tdb/extra/T_QuadsObjectIsNull.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/test/java/org/apache/jena/tdb/extra/T_QuadsObjectIsNull.java b/jena-tdb/src/test/java/org/apache/jena/tdb/extra/T_QuadsObjectIsNull.java
deleted file mode 100644
index ca701df..0000000
--- a/jena-tdb/src/test/java/org/apache/jena/tdb/extra/T_QuadsObjectIsNull.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * 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 org.apache.jena.tdb.extra ;
-
-import java.util.Iterator ;
-
-import org.apache.jena.atlas.lib.FileOps ;
-import org.apache.jena.atlas.logging.LogCtl ;
-import org.apache.jena.query.ReadWrite ;
-import org.apache.jena.sparql.core.Quad ;
-import org.apache.jena.sparql.sse.SSE ;
-import org.apache.jena.tdb.TDBFactory ;
-import org.apache.jena.tdb.base.file.Location ;
-import org.apache.jena.tdb.sys.SystemTDB ;
-import org.apache.jena.tdb.transaction.DatasetGraphTransaction ;
-import org.apache.jena.tdb.transaction.Journal ;
-import org.apache.jena.tdb.transaction.JournalControl ;
-import org.apache.jena.tdb.transaction.NodeTableTrans ;
-
-public class T_QuadsObjectIsNull {
-//	static {
-//		ARQ.getContext().set(SystemTDB.symFileMode, "direct");
-//		TDB.getContext().set(TDB.symUnionDefaultGraph, true);
-//	}
-
-	static String DIR = "DBX" ;
-	static Location location = Location.create(DIR) ;
-
-	public static void main(String[] args) {
-
-	    if ( false )
-	    {
-    	    LogCtl.enable(SystemTDB.syslog.getName()) ;
-    	    LogCtl.enable(Journal.class) ;
-    	    LogCtl.enable(JournalControl.class) ;
-    	    LogCtl.enable(NodeTableTrans.class) ;
-	    }
-	    if ( false )
-	    {
-	        String journal = "DBX/journal.jrnl" ;
-	        if ( FileOps.exists(journal))
-	            JournalControl.print(journal) ;
-	    } 
-	    
-	    if ( false ) {
-	        FileOps.ensureDir(DIR) ;
-	        FileOps.clearDirectory(DIR) ;
-	    }
-	    one() ;
-	}
-	
-	public static void write(DatasetGraphTransaction dsg, Quad quad)
-	{
-        dsg.begin(ReadWrite.WRITE) ;
-        dsg.add(quad) ;
-        if ( ! dsg.contains(quad) )
-            throw new RuntimeException("No quad: "+quad) ;
-        dsg.commit() ;
-        dsg.end() ;
-	}
-	
-    private static void dump(DatasetGraphTransaction dsg)
-    {
-        dsg.begin(ReadWrite.READ);
-        Iterator<Quad> iter = dsg.find() ;
-        for ( ; iter.hasNext() ; )
-        {
-            Quad q = iter.next() ;
-            System.out.println(q) ;
-        }
-        //RiotWriter.writeNQuads(System.out, dsg) ;
-        dsg.commit();
-        dsg.end();
-    }
-
-    public static void one()
-	{
-	    Quad q1 = SSE.parseQuad("(<g1> <s1> <p1> '1')") ;
-	    Quad q2 = SSE.parseQuad("(<g2> <s2> <p2> '2')") ;
-        Quad q3 = SSE.parseQuad("(<g3> <s3> <p3> '3')") ;
-
-        DatasetGraphTransaction dsg = (DatasetGraphTransaction)TDBFactory.createDatasetGraph(location);
-        System.out.println("Start") ;
-        dump(dsg) ;
-        
-        write(dsg, q1) ;
-        write(dsg, q2) ;
-        //write(dsg, q3) ;
-        System.out.println("Finish") ;
-        dump(dsg) ;
-	}
-
-	
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/3d406fe1/jena-tdb/src/test/java/org/apache/jena/tdb/transaction/AbstractTestNodeTableTrans.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/test/java/org/apache/jena/tdb/transaction/AbstractTestNodeTableTrans.java b/jena-tdb/src/test/java/org/apache/jena/tdb/transaction/AbstractTestNodeTableTrans.java
deleted file mode 100644
index c676c39..0000000
--- a/jena-tdb/src/test/java/org/apache/jena/tdb/transaction/AbstractTestNodeTableTrans.java
+++ /dev/null
@@ -1,235 +0,0 @@
-/*
- * 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 org.apache.jena.tdb.transaction;
-
-import org.apache.jena.atlas.junit.BaseTest ;
-import org.apache.jena.graph.Node ;
-import org.apache.jena.query.ReadWrite ;
-import org.apache.jena.sparql.util.NodeFactoryExtra ;
-import org.apache.jena.tdb.base.file.Location ;
-import org.apache.jena.tdb.base.objectfile.ObjectFile ;
-import org.apache.jena.tdb.base.record.RecordFactory ;
-import org.apache.jena.tdb.index.Index ;
-import org.apache.jena.tdb.index.IndexMap ;
-import org.apache.jena.tdb.setup.Build ;
-import org.apache.jena.tdb.store.NodeId ;
-import org.apache.jena.tdb.store.nodetable.NodeTable ;
-import org.apache.jena.tdb.sys.SystemTDB ;
-import org.apache.jena.tdb.transaction.NodeTableTrans ;
-import org.apache.jena.tdb.transaction.Transaction ;
-import org.junit.Test ;
-
-public abstract class AbstractTestNodeTableTrans extends BaseTest
-{
-    abstract protected ObjectFile createObjectFile() ;
-    abstract protected Location getLocation() ;
-    
-    private NodeTableTrans create(Transaction txn, Node...nodes)
-    {
-        NodeTable base = Build.makeNodeTable(getLocation()) ;
-        for ( Node n : nodes )
-            base.getAllocateNodeId(n) ;
-        return create(txn, base) ;
-    }
-    
-    private NodeTableTrans create(Transaction txn, NodeTable base)
-    {
-        RecordFactory recordFactory = new RecordFactory(SystemTDB.LenNodeHash, SystemTDB.SizeOfNodeId) ;
-        Index idx = new IndexMap(recordFactory) ;
-        ObjectFile objectFile = createObjectFile() ;
-        NodeTableTrans ntt = new NodeTableTrans(txn, "test", base, idx, objectFile) ;
-        return ntt ;
-    }
-    
-    protected static Node node1 = NodeFactoryExtra.parseNode("<x>") ;
-    protected static Node node2 = NodeFactoryExtra.parseNode("<y>") ;
-    protected static Node node3 = NodeFactoryExtra.parseNode("<z>") ;
-
-    static void contains(NodeTable nt, Node...nodes)
-    {
-        for ( Node n : nodes)
-        {
-            NodeId nodeId = nt.getNodeIdForNode(n) ;
-            assertFalse(NodeId.isDoesNotExist(nodeId)) ;
-        }
-    }
-    
-    Transaction createTxn(long id) 
-    {
-        return new Transaction(null, 99, ReadWrite.WRITE, id, null, null) ; 
-    }
-    
-    @Test public void nodetrans_01()
-    {
-        Transaction txn = createTxn(11) ; 
-        NodeTableTrans ntt = create(txn) ;
-        ntt.begin(txn) ;
-        ntt.abort(txn) ;
-    }
-
-    
-    @Test public void nodetrans_02()
-    {
-        Transaction txn = createTxn(11) ; 
-        NodeTableTrans ntt = create(txn) ;
-        NodeTable nt0 = ntt.getBaseNodeTable() ;
-        
-        ntt.begin(txn) ;
-        // Add a node
-        NodeId nodeId = ntt.getAllocateNodeId(node1) ;
-        // Check not in the base.
-        assertNull(nt0.getNodeForNodeId(nodeId)) ;
-        // Check is in the transaction node table.
-        assertEquals(NodeId.NodeDoesNotExist, nt0.getNodeIdForNode(node1)) ;
-        assertEquals(node1, ntt.getNodeForNodeId(nodeId)) ;
-        
-        ntt.commitPrepare(txn) ;
-        ntt.commitEnact(txn) ;
-        // Check it is now in the base.
-        assertEquals(node1, nt0.getNodeForNodeId(nodeId)) ;
-        assertEquals(nodeId, nt0.getNodeIdForNode(node1)) ;
-        ntt.commitClearup(txn) ;
-    }
-
-    @Test public void nodetrans_03()
-    {
-        Transaction txn = createTxn(11) ; 
-        NodeTableTrans ntt = create(txn) ;
-        NodeTable nt0 = ntt.getBaseNodeTable() ;
-         
-        ntt.begin(txn) ;
-        // Add a node
-        NodeId nodeId = ntt.getAllocateNodeId(node1) ;
-        // Check not in the base.
-        assertEquals(NodeId.NodeDoesNotExist, nt0.getNodeIdForNode(node1)) ;
-        assertNull(nt0.getNodeForNodeId(nodeId)) ;
-        // Check is in the transaction node table.
-        assertEquals(node1, ntt.getNodeForNodeId(nodeId)) ;
-        
-        ntt.abort(txn) ;
-        // Check it is not in the base.
-        assertEquals(NodeId.NodeDoesNotExist, nt0.getNodeIdForNode(node1)) ;
-        assertNull(nt0.getNodeForNodeId(nodeId)) ;
-        ntt.commitClearup(txn) ;
-    }
-    
-    @Test public void nodetrans_04()
-    {
-        Transaction txn = createTxn(11) ; 
-        NodeTableTrans ntt = create(txn, node1) ;
-        NodeTable nt0 = ntt.getBaseNodeTable() ;
-        ntt.begin(txn) ;
-        // Add a node
-        NodeId nodeId = ntt.getAllocateNodeId(node2) ;
-        // Not here
-        assertEquals(NodeId.NodeDoesNotExist, nt0.getNodeIdForNode(node2)) ;
-        // Is here
-        assertEquals(nodeId, ntt.getNodeIdForNode(node2)) ;
-        ntt.commitPrepare(txn) ;
-        ntt.commitEnact(txn) ;
-        assertEquals(nodeId, nt0.getNodeIdForNode(node2)) ;
-        ntt.commitClearup(txn) ;
-    }
-    
-    
-    @Test 
-    public void nodetrans_05()
-    {   
-        // 2 transactions - no blocking reader - create a second NodeTableTrans
-        Transaction txn1 = createTxn(11) ; 
-        NodeTableTrans ntt1 = create(txn1, node1) ;
-        NodeId nodeId1 = ntt1.getBaseNodeTable().getNodeIdForNode(node1) ;
-        
-        ntt1.begin(txn1) ;
-        NodeId nodeId2 = ntt1.getAllocateNodeId(node2) ;
-        ntt1.commitPrepare(txn1) ;
-        ntt1.commitEnact(txn1) ;
-        ntt1.commitClearup(txn1) ;
-        
-        Transaction txn2 = createTxn(12) ;
-        NodeTableTrans ntt2 = create(txn2, ntt1.getBaseNodeTable()) ;
-        ntt2.begin(txn2) ;
-        assertEquals(nodeId1, ntt2.getNodeIdForNode(node1)) ;
-        assertEquals(nodeId2, ntt2.getNodeIdForNode(node2)) ;
-        NodeId nodeId3 = ntt2.getAllocateNodeId(node3) ;
-        assertEquals(nodeId3, ntt2.getNodeIdForNode(node3)) ;
-        ntt2.commitPrepare(txn2) ;
-        ntt2.commitEnact(txn2) ;
-        ntt2.commitClearup(txn2) ;
-
-        assertEquals(nodeId1, ntt1.getBaseNodeTable().getNodeIdForNode(node1)) ;
-        assertEquals(nodeId2, ntt1.getBaseNodeTable().getNodeIdForNode(node2)) ;
-        assertEquals(nodeId3, ntt1.getBaseNodeTable().getNodeIdForNode(node3)) ;
-    }
-
-    @Test 
-    public void nodetrans_06()
-    {   
-        // 2 transactions - blocking reader - create a second NodeTableTrans
-        Transaction txn1 = createTxn(11) ; 
-        NodeTableTrans ntt1 = create(txn1, node1) ;
-        NodeId nodeId1 = ntt1.getBaseNodeTable().getNodeIdForNode(node1) ;
-        
-        ntt1.begin(txn1) ;
-        NodeId nodeId2 = ntt1.getAllocateNodeId(node2) ;
-        ntt1.commitPrepare(txn1) ;
-        
-        // READ - don't enact
-        Transaction txn2 = createTxn(12) ; 
-        NodeTableTrans ntt2 = create(txn2, ntt1.getBaseNodeTable()) ;
-        ntt2.begin(txn2) ;
-        assertEquals(nodeId1, ntt2.getNodeIdForNode(node1)) ;
-        assertEquals(nodeId2, ntt2.getNodeIdForNode(node2)) ;
-        
-        NodeId nodeId3 = ntt2.getAllocateNodeId(node3) ;
-        assertEquals(nodeId3, ntt2.getNodeIdForNode(node3)) ;
-        ntt2.commitPrepare(txn2) ;
-
-        
-        // READ ends.
-        
-        ntt1.commitEnact(txn1) ;
-        ntt1.commitClearup(txn1) ;
-        
-        ntt2.commitEnact(txn2) ;
-        ntt2.commitClearup(txn2) ;
-
-        assertEquals(nodeId1, ntt1.getBaseNodeTable().getNodeIdForNode(node1)) ;
-        assertEquals(nodeId2, ntt1.getBaseNodeTable().getNodeIdForNode(node2)) ;
-        assertEquals(nodeId3, ntt1.getBaseNodeTable().getNodeIdForNode(node3)) ;
-    }
-    
-    @Test 
-    public void nodetrans_07()
-    { 
-        Transaction txn = createTxn(13) ;
-        NodeTableTrans ntt = create(txn) ;
-
-        ntt.begin(txn) ;
-        assertEquals(NodeId.NodeIdAny, ntt.getNodeIdForNode(Node.ANY)) ;
-        assertEquals(Node.ANY, ntt.getNodeForNodeId(NodeId.NodeIdAny)) ;
-        
-        assertTrue(ntt.isEmpty()) ;
-        
-        ntt.commitPrepare(txn) ;
-        ntt.commitEnact(txn) ;
-        ntt.commitClearup(txn) ;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/3d406fe1/jena-tdb/src/test/java/org/apache/jena/tdb/transaction/AbstractTestObjectFileTrans.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/test/java/org/apache/jena/tdb/transaction/AbstractTestObjectFileTrans.java b/jena-tdb/src/test/java/org/apache/jena/tdb/transaction/AbstractTestObjectFileTrans.java
index 49dc7ee..dbbcf7d 100644
--- a/jena-tdb/src/test/java/org/apache/jena/tdb/transaction/AbstractTestObjectFileTrans.java
+++ b/jena-tdb/src/test/java/org/apache/jena/tdb/transaction/AbstractTestObjectFileTrans.java
@@ -37,7 +37,6 @@ public abstract class AbstractTestObjectFileTrans extends BaseTest
 {
     static long count = 0 ;
     ObjectFile file1 ;
-    ObjectFile file2 ;
     ObjectFileTrans file ;
     Transaction txn ;
     
@@ -51,7 +50,7 @@ public abstract class AbstractTestObjectFileTrans extends BaseTest
     {
         txn = new Transaction(null, 5, ReadWrite.WRITE, ++count, null, tm) ;
         file1 = createFile("base") ;
-        file2 = createFile("log") ;
+        //file2 = createFile("log") ;
     }
 
     @After
@@ -83,7 +82,10 @@ public abstract class AbstractTestObjectFileTrans extends BaseTest
     }
     
 
-    private void init() { file = new ObjectFileTrans(null, file1, file2) ; } 
+    private void init() { 
+        //file = new ObjectFileTransComplex(null, file1, file2) ;
+        file = new ObjectFileTrans(null, file1) ;
+    }
     
     static void fill(ObjectFile file, String... contents)
     {
@@ -103,7 +105,7 @@ public abstract class AbstractTestObjectFileTrans extends BaseTest
         init() ;
         
         file.begin(txn) ; 
-        contains(file2) ;
+        //contains(file2) ;
         file.commitPrepare(txn) ;
         file.commitEnact(txn) ;
         contains(file1, "ABC") ;