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 2014/09/06 16:24:24 UTC

svn commit: r1622864 - in /jena/trunk/jena-tdb/src: main/java/com/hp/hpl/jena/tdb/assembler/ main/java/com/hp/hpl/jena/tdb/base/block/ main/java/com/hp/hpl/jena/tdb/index/ main/java/com/hp/hpl/jena/tdb/index/factories/ main/java/com/hp/hpl/jena/tdb/set...

Author: andy
Date: Sat Sep  6 14:24:23 2014
New Revision: 1622864

URL: http://svn.apache.org/r1622864
Log:
Partial refactoring of DB building (to avoid svn issues)

Added:
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexParams.java   (with props)
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BuilderDB.java
      - copied, changed from r1622774, jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/Builder.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BuilderIndex.java   (with props)
Removed:
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/Builder.java
    jena/trunk/jena-tdb/src/test/java/tdb/
Modified:
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/assembler/IndexAssembler.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/base/block/BlockMgrCache.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/base/block/BlockMgrFactory.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexBuilder.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexFactory.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/RangeIndexFactory.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/factories/IndexFactoryBPlusTree.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/factories/IndexFactoryExtHash.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/factories/IndexFactoryExtHashMem.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/B.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BlockMgrBuilder.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/DatasetBuilderBasic.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/DatasetBuilderStd.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/IndexBuilder.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/NodeTableBuilder.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/RangeIndexBuilder.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/SystemParams.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/TupleIndexBuilder.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/store/DatasetPrefixesTDB.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/store/nodetable/NodeTableFactory.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/transaction/DatasetBuilderTxn.java
    jena/trunk/jena-tdb/src/test/java/com/hp/hpl/jena/tdb/store/tupletable/TestTupleIndexRecord.java
    jena/trunk/jena-tdb/src/test/java/com/hp/hpl/jena/tdb/store/tupletable/TestTupleIndexRecordDirect.java

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/assembler/IndexAssembler.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/assembler/IndexAssembler.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/assembler/IndexAssembler.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/assembler/IndexAssembler.java Sat Sep  6 14:24:23 2014
@@ -18,28 +18,30 @@
 
 package com.hp.hpl.jena.tdb.assembler;
 
-import static com.hp.hpl.jena.sparql.util.graph.GraphUtils.exactlyOneProperty;
-import static com.hp.hpl.jena.sparql.util.graph.GraphUtils.getAsStringValue;
-import static com.hp.hpl.jena.tdb.assembler.VocabTDB.pDescription;
-import static com.hp.hpl.jena.tdb.assembler.VocabTDB.pFile;
+import static com.hp.hpl.jena.sparql.util.graph.GraphUtils.exactlyOneProperty ;
+import static com.hp.hpl.jena.sparql.util.graph.GraphUtils.getAsStringValue ;
+import static com.hp.hpl.jena.tdb.assembler.VocabTDB.pDescription ;
+import static com.hp.hpl.jena.tdb.assembler.VocabTDB.pFile ;
 
 import java.util.Locale ;
 
 import org.apache.jena.atlas.lib.ColumnMap ;
 
-import com.hp.hpl.jena.rdf.model.Resource;
-import com.hp.hpl.jena.assembler.Assembler;
-import com.hp.hpl.jena.assembler.Mode;
-import com.hp.hpl.jena.assembler.assemblers.AssemblerBase;
-import com.hp.hpl.jena.tdb.TDBException;
-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.record.RecordFactory;
-import com.hp.hpl.jena.tdb.index.IndexBuilder;
-import com.hp.hpl.jena.tdb.index.RangeIndex;
+import com.hp.hpl.jena.assembler.Assembler ;
+import com.hp.hpl.jena.assembler.Mode ;
+import com.hp.hpl.jena.assembler.assemblers.AssemblerBase ;
+import com.hp.hpl.jena.rdf.model.Resource ;
+import com.hp.hpl.jena.tdb.TDBException ;
+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.record.RecordFactory ;
+import com.hp.hpl.jena.tdb.index.IndexParams ;
+import com.hp.hpl.jena.tdb.index.RangeIndex ;
+import com.hp.hpl.jena.tdb.setup.B ;
+import com.hp.hpl.jena.tdb.setup.SystemParams ;
 import com.hp.hpl.jena.tdb.store.tupletable.TupleIndex ;
 import com.hp.hpl.jena.tdb.store.tupletable.TupleIndexRecord ;
-import com.hp.hpl.jena.tdb.sys.Names;
+import com.hp.hpl.jena.tdb.sys.Names ;
 import com.hp.hpl.jena.tdb.sys.SystemTDB ;
 
 public class IndexAssembler extends AssemblerBase //implements Assembler
@@ -83,16 +85,8 @@ public class IndexAssembler extends Asse
         }
         // Problems with spotting the index technology.
         FileSet fileset = null ; //FileSet.fromFilename(filename) ;
-        
-        RangeIndex rIndex = IndexBuilder.createRangeIndex(fileset, rf) ;
+        IndexParams idxParams = SystemParams.getDftSystemParams() ;
+        RangeIndex rIndex = B.buildRangeIndex(fileset, rf, idxParams) ;
         return new TupleIndexRecord(desc.length(), new ColumnMap(primary, desc), desc, rf, rIndex) ;
     }
-
-//    public static RangeIndex rangeIndex(String filename, String name)
-//    {
-//     // Problems with spotting the index technology.
-//        FileSet fileset = IndexBuilder.filesetForIndex(new Location(filename), desc) ;
-//        return IndexBuilder.createRangeIndex(new Location(filename), name, FactoryGraphTDB.indexRecordTripleFactory) ;
-//    }
-
 }

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/base/block/BlockMgrCache.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/base/block/BlockMgrCache.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/base/block/BlockMgrCache.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/base/block/BlockMgrCache.java Sat Sep  6 14:24:23 2014
@@ -42,23 +42,21 @@ public class BlockMgrCache extends Block
     
     public static boolean globalLogging = false ;           // Also enable the logging level. 
     private boolean logging = false ;                       // Also enable the logging level. 
-    private String indexName ; 
     // ---- stats
     long cacheReadHits = 0 ;
     long cacheMisses = 0 ;
     long cacheWriteHits = 0 ;
     
-    static BlockMgr create(String indexName, int readSlots, int writeSlots, final BlockMgr blockMgr)
+    static BlockMgr create(int readSlots, int writeSlots, final BlockMgr blockMgr)
     {
         if ( readSlots < 0 && writeSlots < 0 )
             return blockMgr ;
-        return new BlockMgrCache(indexName, readSlots, writeSlots, blockMgr) ;
+        return new BlockMgrCache(readSlots, writeSlots, blockMgr) ;
     }
     
-    private BlockMgrCache(String indexName, int readSlots, int writeSlots, final BlockMgr blockMgr)
+    private BlockMgrCache(int readSlots, int writeSlots, final BlockMgr blockMgr)
     {
         super(blockMgr) ;
-        this.indexName = String.format("%-12s", indexName) ;
         // Caches are related so we can't use a Getter for cache management.
         if ( readSlots < -1 )
             readCache = CacheFactory.createNullCache() ;
@@ -274,8 +272,8 @@ public class BlockMgrCache extends Block
     { 
         if ( ! logging && ! globalLogging ) return ;
         String msg = String.format(fmt, args) ;
-        if ( indexName != null )
-             msg = indexName+" : "+msg ;
+        if ( getLabel() != null )
+             msg = getLabel()+" : "+msg ;
         log.debug(msg) ;
     }
     
@@ -284,8 +282,8 @@ public class BlockMgrCache extends Block
         if ( true )
         {
             String x = "" ;
-            if ( indexName != null )
-                x = indexName+" : ";
+            if ( getLabel() != null )
+                x = getLabel()+" : ";
             log("%sH=%d, M=%d, W=%d", x, cacheReadHits, cacheMisses, cacheWriteHits) ;
         }
         

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/base/block/BlockMgrFactory.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/base/block/BlockMgrFactory.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/base/block/BlockMgrFactory.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/base/block/BlockMgrFactory.java Sat Sep  6 14:24:23 2014
@@ -19,6 +19,7 @@
 package com.hp.hpl.jena.tdb.base.block;
 
 import org.apache.jena.atlas.lib.FileOps ;
+import org.apache.jena.atlas.logging.Log ;
 
 import com.hp.hpl.jena.tdb.TDBException ;
 import com.hp.hpl.jena.tdb.base.file.BlockAccess ;
@@ -50,6 +51,7 @@ public class BlockMgrFactory
         return tracker(blockMgr) ;
     }
     
+    // XXX Build then add cache. 
     public static BlockMgr create(FileSet fileSet, String ext, int blockSize, int readBlockCacheSize, int writeBlockCacheSize)
     {
         if ( fileSet.isMem() )
@@ -115,10 +117,37 @@ public class BlockMgrFactory
 
         String fn = FileOps.basename(filename) ;
         
-        blockMgr = BlockMgrCache.create(fn, readBlockCacheSize, writeBlockCacheSize, blockMgr) ;
+        blockMgr = BlockMgrCache.create(readBlockCacheSize, writeBlockCacheSize, blockMgr) ;
         return track(blockMgr) ;
     }
-    
+
+    /** Add a caching layer to a BlockMgr.
+     *  <p>
+     *  This does not make sense for memory BlockMgr or for memory mapper files.
+     *  This function always add the cache.
+     *  
+     *  @see #addCache(BlockMgr, FileSet, FileMode, int, int)
+     */
+    public static BlockMgr addCache(BlockMgr blockMgr, int readBlockCacheSize, int writeBlockCacheSize) {
+        if ( blockMgr instanceof BlockMgrCache )
+            Log.warn(BlockMgrFactory.class, "BlockMgr already has a cache: "+blockMgr.getLabel()) ;
+        return BlockMgrCache.create(readBlockCacheSize, writeBlockCacheSize, blockMgr) ;
+    }
+    
+    /** Add a caching layer to a BlockMgr if appropriate.
+     *  This does not make sense for memory BlockMgr or for memory mapper files.
+     *  These are skipped. 
+     */
+    public static BlockMgr addCache(BlockMgr blockMgr, FileSet fileSet, FileMode fileMode, int readBlockCacheSize, int writeBlockCacheSize) {
+        if ( fileSet.isMem() )
+            return blockMgr ;
+        if ( fileMode == null )
+            fileMode = SystemTDB.fileMode() ;
+        if ( fileMode == FileMode.mapped )
+            return blockMgr ;
+        return addCache(blockMgr, readBlockCacheSize, writeBlockCacheSize) ;
+    }
+
     /** Create a Block Manager using direct access, no caching, no nothing. */
     public static BlockMgr createStdFileNoCache(String filename, int blockSize)
     {

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexBuilder.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexBuilder.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexBuilder.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexBuilder.java Sat Sep  6 14:24:23 2014
@@ -35,8 +35,8 @@ public class IndexBuilder
     private static IndexBuilder builder         = chooseIndexBuilder() ;
     public static IndexBuilder get()            { return builder ; }
     
-    public static IndexBuilder getBPlusTree()   { return createIndexBuilder(IndexType.BPlusTree) ; }
-    public static IndexBuilder getExtHash()     { return createIndexBuilder(IndexType.ExtHash) ; }
+//    public static IndexBuilder getBPlusTree()   { return createIndexBuilder(IndexType.BPlusTree) ; }
+//    public static IndexBuilder getExtHash()     { return createIndexBuilder(IndexType.ExtHash) ; }
     
     private static IndexBuilder builderMem = null ;
     
@@ -45,11 +45,11 @@ public class IndexBuilder
         return createIndexBuilderMem(SystemTDB.getIndexType()) ;
     }
 
-    /** Convert the index name to a file name */
-    public static String filenameForIndex(String indexName) { return indexName ; }
-
-    /** Convert the index name to a file name */
-    public static FileSet filesetForIndex(Location location, String indexName) { return new FileSet(location, filenameForIndex(indexName)) ; }
+//    /** Convert the index name to a file name */
+//    public static String filenameForIndex(String indexName) { return indexName ; }
+//
+//    /** Convert the index name to a file name */
+//    public static FileSet filesetForIndex(Location location, String indexName) { return new FileSet(location, filenameForIndex(indexName)) ; }
     
     private static synchronized IndexBuilder chooseIndexBuilder()
     {
@@ -80,41 +80,41 @@ public class IndexBuilder
         throw new TDBException("Unrecognized index type: " + indexType) ;
     }
 
-    /** Create an index at the specified place
-     * @param fileset   Place and basename where the file or files needed are found/created.
-     * @return Index
-     */ 
-    static public Index createIndex(FileSet fileset, RecordFactory recordFactory)
-    {
-        return builder.newIndex(fileset, recordFactory) ;
-    }
-
-    /** Create a range index at the specified place
-     * @param fileset   Place and basename where the file or files needed are found/created.
-     * @return RangeIndex
-     */ 
-    static public RangeIndex createRangeIndex(FileSet fileset, RecordFactory recordFactory)
-    {
-        return builder.newRangeIndex(fileset, recordFactory) ;
-    }
+//    /** Create an index at the specified place
+//     * @param fileset   Place and basename where the file or files needed are found/created.
+//     * @return Index
+//     */ 
+//    static public Index createIndex(FileSet fileset, RecordFactory recordFactory, IndexParams params)
+//    {
+//        return builder.newIndex(fileset, recordFactory, params) ;
+//    }
+//
+//    /** Create a range index at the specified place
+//     * @param fileset   Place and basename where the file or files needed are found/created.
+//     * @return RangeIndex
+//     */ 
+//    static public RangeIndex createRangeIndex(FileSet fileset, RecordFactory recordFactory, IndexParams params)
+//    {
+//        return builder.newRangeIndex(fileset, recordFactory, params) ;
+//    }
 
     // ---- The class .... a pairing of an index builder and a range index builder.
     IndexFactory factoryIndex = null ;
     RangeIndexFactory builderRangeIndex = null ;
 
-    public IndexBuilder(IndexFactory indexBuilder, RangeIndexFactory rangeIndexBuilder)
+    private IndexBuilder(IndexFactory indexBuilder, RangeIndexFactory rangeIndexBuilder)
     {
         factoryIndex = indexBuilder ;
         builderRangeIndex = rangeIndexBuilder ;
     }
     
-    public Index newIndex(FileSet fileset, RecordFactory factory)
+    public Index newIndex(FileSet fileset, RecordFactory factory, IndexParams params)
     {
-        return factoryIndex.createIndex(fileset, factory) ;
+        return factoryIndex.createIndex(fileset, factory, params) ;
     }
     
-    public RangeIndex newRangeIndex(FileSet fileset , RecordFactory factory)
+    public RangeIndex newRangeIndex(FileSet fileset , RecordFactory factory, IndexParams params)
     {
-        return builderRangeIndex.createRangeIndex(fileset, factory) ;
+        return builderRangeIndex.createRangeIndex(fileset, factory, params) ;
     }
 }

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexFactory.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexFactory.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexFactory.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexFactory.java Sat Sep  6 14:24:23 2014
@@ -25,5 +25,5 @@ import com.hp.hpl.jena.tdb.base.record.R
 public interface IndexFactory
 {
     // The implementations are in the factories subpackage.
-    public Index createIndex(FileSet fileset, RecordFactory recordFactory) ;
+    public Index createIndex(FileSet fileset, RecordFactory recordFactory, IndexParams params) ;
 }

Added: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexParams.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexParams.java?rev=1622864&view=auto
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexParams.java (added)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexParams.java Sat Sep  6 14:24:23 2014
@@ -0,0 +1,26 @@
+/*
+ *  Copyright 2014 Andy Seaborne
+ *
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package com.hp.hpl.jena.tdb.index;
+
+import com.hp.hpl.jena.tdb.base.block.FileMode ;
+
+public interface IndexParams {
+    public FileMode getFileMode() ;
+    public int getBlockSize() ;
+    public int getBlockReadCacheSize() ;
+    public int getBlockWriteCacheSize() ;
+}

Propchange: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexParams.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/RangeIndexFactory.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/RangeIndexFactory.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/RangeIndexFactory.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/RangeIndexFactory.java Sat Sep  6 14:24:23 2014
@@ -24,5 +24,5 @@ import com.hp.hpl.jena.tdb.base.record.R
 
 public interface RangeIndexFactory
 {
-    public RangeIndex createRangeIndex(FileSet fileset, RecordFactory recordFactory) ;
+    public RangeIndex createRangeIndex(FileSet fileset, RecordFactory recordFactory, IndexParams params) ;
 }

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/factories/IndexFactoryBPlusTree.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/factories/IndexFactoryBPlusTree.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/factories/IndexFactoryBPlusTree.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/factories/IndexFactoryBPlusTree.java Sat Sep  6 14:24:23 2014
@@ -26,56 +26,48 @@ import com.hp.hpl.jena.tdb.base.block.Bl
 import com.hp.hpl.jena.tdb.base.block.BlockMgrFactory ;
 import com.hp.hpl.jena.tdb.base.file.FileSet ;
 import com.hp.hpl.jena.tdb.base.record.RecordFactory ;
-import com.hp.hpl.jena.tdb.index.Index ;
-import com.hp.hpl.jena.tdb.index.IndexFactory ;
-import com.hp.hpl.jena.tdb.index.RangeIndexFactory ;
-import com.hp.hpl.jena.tdb.index.RangeIndex ;
+import com.hp.hpl.jena.tdb.index.* ;
 import com.hp.hpl.jena.tdb.index.bplustree.BPlusTree ;
 import com.hp.hpl.jena.tdb.index.bplustree.BPlusTreeParams ;
 import com.hp.hpl.jena.tdb.sys.Names ;
-import com.hp.hpl.jena.tdb.sys.SystemTDB ;
 
 public class IndexFactoryBPlusTree implements IndexFactory, RangeIndexFactory
 {
     private static Logger log = LoggerFactory.getLogger(IndexFactoryBPlusTree.class) ;
-    private final int blockSize ;
 
-    public IndexFactoryBPlusTree()
-    { this(SystemTDB.BlockSize) ; }
-    
-    public IndexFactoryBPlusTree(int blockSize)
-    {
-        this.blockSize = blockSize ;
-    }
+    public IndexFactoryBPlusTree() { }
     
     @Override
-    public Index createIndex(FileSet fileset, RecordFactory factory)
+    public Index createIndex(FileSet fileset, RecordFactory factory, IndexParams params)
     {
-        return createRangeIndex(fileset, factory) ;
+        return createRangeIndex(fileset, factory, params) ;
     }
     
     @Override
-    public RangeIndex createRangeIndex(FileSet fileset, RecordFactory factory)
+    public RangeIndex createRangeIndex(FileSet fileset, RecordFactory factory, IndexParams idxParams)
     {
+        int blockSize = idxParams.getBlockSize() ; 
+        int readCacheSize = idxParams.getBlockReadCacheSize() ;
+        int writeCacheSize = idxParams.getBlockWriteCacheSize() ;
         int order = BPlusTreeParams.calcOrder(blockSize, factory) ;
         
         BPlusTreeParams params = new BPlusTreeParams(order, factory) ;
-        if ( params.getCalcBlockSize() > blockSize )
+        if ( params.getCalcBlockSize() > idxParams.getBlockSize() )
             throw new TDBException("Calculated block size is greater than required size") ;
         
-        BlockMgr blkMgrNodes = createBlockMgr(fileset, Names.bptExtTree, blockSize) ;
-        BlockMgr blkMgrRecords = createBlockMgr(fileset, Names.bptExtRecords, blockSize) ;
+        BlockMgr blkMgrNodes = createBlockMgr(fileset, Names.bptExtTree, blockSize, readCacheSize, writeCacheSize) ;
+        BlockMgr blkMgrRecords = createBlockMgr(fileset, Names.bptExtRecords, blockSize, readCacheSize, writeCacheSize) ;
         return BPlusTree.create(params, blkMgrNodes, blkMgrRecords) ;
     }
     
-    static BlockMgr createBlockMgr(FileSet fileset, String filename, int blockSize)
+    static BlockMgr createBlockMgr(FileSet fileset, String filename, int blockSize,
+                                   int readCacheSize, int writeCacheSize)
     {
         if ( fileset.isMem() )
             return BlockMgrFactory.createMem(filename, blockSize) ;
         
         String fnNodes = fileset.filename(filename) ;
         return BlockMgrFactory.createFile(fnNodes, blockSize, 
-                                          SystemTDB.BlockReadCacheSize,
-                                          SystemTDB.BlockWriteCacheSize) ;
+                                          readCacheSize, writeCacheSize) ;
     }
 }

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/factories/IndexFactoryExtHash.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/factories/IndexFactoryExtHash.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/factories/IndexFactoryExtHash.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/factories/IndexFactoryExtHash.java Sat Sep  6 14:24:23 2014
@@ -26,6 +26,7 @@ import com.hp.hpl.jena.tdb.base.file.Pla
 import com.hp.hpl.jena.tdb.base.record.RecordFactory ;
 import com.hp.hpl.jena.tdb.index.Index ;
 import com.hp.hpl.jena.tdb.index.IndexFactory ;
+import com.hp.hpl.jena.tdb.index.IndexParams ;
 import com.hp.hpl.jena.tdb.index.ext.ExtHash ;
 import com.hp.hpl.jena.tdb.sys.Names ;
 import com.hp.hpl.jena.tdb.sys.SystemTDB ;
@@ -47,7 +48,7 @@ public class IndexFactoryExtHash impleme
     }
     
     @Override
-    public Index createIndex(FileSet fileset, RecordFactory recordFactory)
+    public Index createIndex(FileSet fileset, RecordFactory recordFactory, IndexParams params)
     {
         String fnDictionary = fileset.filename(Names.extHashExt) ;
         PlainFile dictionary = FileFactory.createPlainFileDisk(fnDictionary) ;

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/factories/IndexFactoryExtHashMem.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/factories/IndexFactoryExtHashMem.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/factories/IndexFactoryExtHashMem.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/factories/IndexFactoryExtHashMem.java Sat Sep  6 14:24:23 2014
@@ -22,10 +22,10 @@ import com.hp.hpl.jena.tdb.base.file.Fil
 import com.hp.hpl.jena.tdb.base.record.RecordFactory;
 import com.hp.hpl.jena.tdb.index.Index;
 import com.hp.hpl.jena.tdb.index.IndexFactory;
+import com.hp.hpl.jena.tdb.index.IndexParams ;
 import com.hp.hpl.jena.tdb.index.ext.ExtHash;
-import com.hp.hpl.jena.tdb.sys.SystemTDB;
 
-/** Index factory for extendible hash tables in memory (for testing).
+/** Index factory for extensible hash tables in memory (for testing).
  *  Only an index, not a range index
  */
 
@@ -35,8 +35,8 @@ public class IndexFactoryExtHashMem impl
     { }
     
     @Override
-    public Index createIndex(FileSet fileset, RecordFactory recordFactory)
+    public Index createIndex(FileSet fileset, RecordFactory recordFactory, IndexParams params)
     {
-        return ExtHash.createMem(recordFactory, SystemTDB.BlockSize) ;
+        return ExtHash.createMem(recordFactory, params.getBlockSize()) ;
     }
 }

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/B.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/B.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/B.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/B.java Sat Sep  6 14:24:23 2014
@@ -21,29 +21,78 @@ package com.hp.hpl.jena.tdb.setup;
 import com.hp.hpl.jena.tdb.base.file.FileSet ;
 import com.hp.hpl.jena.tdb.base.record.RecordFactory ;
 import com.hp.hpl.jena.tdb.index.Index ;
+import com.hp.hpl.jena.tdb.index.IndexParams ;
 //import com.hp.hpl.jena.tdb.index.IndexBuilder ;
 import com.hp.hpl.jena.tdb.index.RangeIndex ;
 
 public class B {
-    // Build things.
+    
+    public static IndexBuilder createIndexBuilder() { return null ; }
+    public static IndexBuilder createIndexBuilderMem() { return null ; }
+    public static RangeIndexBuilder createRangeIndexBuilder() { return null ; }
+    public static RangeIndexBuilder createRangeIndexBuilderMem() { return null ; }
+    
+    
+    // c.f. setupTDB
+    // BlockMgrSync needed?  Outer sync?
+    // XXX Merge with com.hp.hpl.jena.tdb.index.IndexFactory
+    // Two levels - with params and with (lots of) arguments for indexes, blockMgrs
+    // IndexParams
+    
+    // Rework Build.* classes first.
     
     //RecordFactory recordFactory = new RecordFactory(SizeOfNodeId*colMap.length(),0) ;
     
+    public static RangeIndex buildRangeIndexMem(RecordFactory recordFactory) {
+        FileSet fileSet = FileSet.mem() ;
+        return buildRangeIndex(fileSet, recordFactory) ;
+    }
+    
     public static RangeIndex buildRangeIndex(FileSet fileset, RecordFactory recordFactory) {
-        BlockMgrBuilder nodeBld = new Builder.BlockMgrBuilderStd() ;
-        BlockMgrBuilder leavesBld = new Builder.BlockMgrBuilderStd() ;
-        RangeIndexBuilder builder = new Builder.RangeIndexBuilderStd(nodeBld, leavesBld) ;
-        return builder.buildRangeIndex(fileset, recordFactory) ; 
+        IndexParams indexParams = SystemParams.getDftSystemParams() ;
+        return buildRangeIndex(fileset, recordFactory, indexParams) ;
+    }
+        
+    public static RangeIndex buildRangeIndex(FileSet fileset, RecordFactory recordFactory, IndexParams indexParams) {
+        BlockMgrBuilder nodeBld = new BuilderIndex.BlockMgrBuilderStd() ;
+        BlockMgrBuilder leavesBld = new BuilderIndex.BlockMgrBuilderStd() ;
+        RangeIndexBuilder builder = new BuilderIndex.RangeIndexBuilderStd(nodeBld, leavesBld) ;
+        return builder.buildRangeIndex(fileset, recordFactory, indexParams) ; 
     }
     
+    public static Index buildIndexMem(RecordFactory recordFactory) {
+        FileSet fileSet = FileSet.mem() ;
+        return buildIndex(fileSet, recordFactory) ;
+    }
+
     public static Index buildIndex(FileSet fileset, RecordFactory recordFactory) {
-        BlockMgrBuilder nodeBld = new Builder.BlockMgrBuilderStd() ;
-        BlockMgrBuilder leavesBld = new Builder.BlockMgrBuilderStd() ;
-        IndexBuilder builder = new Builder.IndexBuilderStd(nodeBld, leavesBld) ;
-        return builder.buildIndex(fileset, recordFactory) ; 
+        IndexParams indexParams = SystemParams.getDftSystemParams() ;
+        return buildIndex(fileset, recordFactory, indexParams) ;
     }
     
+    public static Index buildIndex(FileSet fileset, RecordFactory recordFactory, IndexParams indexParams) {
+        BlockMgrBuilder nodeBld = new BuilderIndex.BlockMgrBuilderStd() ;
+        BlockMgrBuilder leavesBld = new BuilderIndex.BlockMgrBuilderStd() ;
+        IndexBuilder builder = new BuilderIndex.IndexBuilderStd(nodeBld, leavesBld) ;
+        return builder.buildIndex(fileset, recordFactory, indexParams) ; 
+    }
     
-    
+//    public static NodeTable buildNodeTable(FileSet fileset, SystemParams params) {
+//        BlockMgrBuilder nodeBld = new Builder.BlockMgrBuilderStd() ;
+//        BlockMgrBuilder leavesBld = new Builder.BlockMgrBuilderStd() ;
+//        
+//        NodeTableBuilder ntb = new Builder.NodeTableBuilderStd(null, null) ;
+//        
+//        
+//        IndexBuilder builder = new Builder.IndexBuilderStd(nodeBld, leavesBld) ;
+//        FileSet filesetIdx = new FileSet(params) ;
+//        FileSet filesetObjFile = new FileSet(params.
+//                                                                          
+//        return ntb.buildNodeTable(fileset, 
+//                                  fileset, 
+//                                  params.getNode2NodeIdCacheSize(),
+//                                  params.getNodeId2NodeCacheSize(),
+//                                  params.getNodeMissCacheSize()) ;
+//    }
 }
 

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BlockMgrBuilder.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BlockMgrBuilder.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BlockMgrBuilder.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BlockMgrBuilder.java Sat Sep  6 14:24:23 2014
@@ -20,7 +20,8 @@ package com.hp.hpl.jena.tdb.setup;
 
 import com.hp.hpl.jena.tdb.base.block.BlockMgr ;
 import com.hp.hpl.jena.tdb.base.file.FileSet ;
+import com.hp.hpl.jena.tdb.index.IndexParams ;
 
 public interface BlockMgrBuilder {
-    BlockMgr buildBlockMgr(FileSet fileSet, String ext, int blockSize) ;
+    BlockMgr buildBlockMgr(FileSet fileSet, String ext, IndexParams indexParams) ;
 }

Copied: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BuilderDB.java (from r1622774, jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/Builder.java)
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BuilderDB.java?p2=jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BuilderDB.java&p1=jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/Builder.java&r1=1622774&r2=1622864&rev=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/Builder.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BuilderDB.java Sat Sep  6 14:24:23 2014
@@ -18,20 +18,15 @@
 
 package com.hp.hpl.jena.tdb.setup;
 
-import static com.hp.hpl.jena.tdb.sys.SystemTDB.SizeOfNodeId ;
 import org.apache.jena.atlas.lib.ColumnMap ;
 import org.slf4j.Logger ;
 import org.slf4j.LoggerFactory ;
 
-import com.hp.hpl.jena.tdb.base.block.BlockMgr ;
-import com.hp.hpl.jena.tdb.base.block.BlockMgrFactory ;
-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.objectfile.ObjectFile ;
 import com.hp.hpl.jena.tdb.base.record.RecordFactory ;
-import com.hp.hpl.jena.tdb.index.* ;
-import com.hp.hpl.jena.tdb.index.bplustree.BPlusTree ;
-import com.hp.hpl.jena.tdb.index.bplustree.BPlusTreeParams ;
+import com.hp.hpl.jena.tdb.index.Index ;
+import com.hp.hpl.jena.tdb.index.RangeIndex ;
 import com.hp.hpl.jena.tdb.store.nodetable.NodeTable ;
 import com.hp.hpl.jena.tdb.store.nodetable.NodeTableCache ;
 import com.hp.hpl.jena.tdb.store.nodetable.NodeTableInline ;
@@ -41,26 +36,27 @@ import com.hp.hpl.jena.tdb.store.tupleta
 import com.hp.hpl.jena.tdb.sys.Names ;
 import com.hp.hpl.jena.tdb.sys.SystemTDB ;
 
-public class Builder
+/** Building datastructures on top of the base file
+ *  abstractions of indexes, block managers and object files.  
+ */
+public class BuilderDB
 {
     private static boolean VERBOSE = true ;
-    private static Logger log = LoggerFactory.getLogger(Builder.class) ;
+    private static Logger log = LoggerFactory.getLogger(BuilderDB.class) ;
     
     public static class TupleIndexBuilderStd implements TupleIndexBuilder
     {
         private final RangeIndexBuilder rangeIndexBuilder ;
     
-        public TupleIndexBuilderStd(RangeIndexBuilder rangeIndexBuilder)
-        {
+        public TupleIndexBuilderStd(RangeIndexBuilder rangeIndexBuilder) {
             this.rangeIndexBuilder = rangeIndexBuilder ;
         }
         
         @Override
-        public TupleIndex buildTupleIndex(FileSet fileSet, ColumnMap colMap, String name)
-        {
-            RecordFactory recordFactory = new RecordFactory(SizeOfNodeId*colMap.length(),0) ;
-            
-            RangeIndex rIdx = rangeIndexBuilder.buildRangeIndex(fileSet, recordFactory) ;
+        public TupleIndex buildTupleIndex(FileSet fileSet, ColumnMap colMap, String name, SystemParams 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 ;
         }
@@ -71,131 +67,23 @@ public class Builder
         private final IndexBuilder indexBuilder ;
         private final ObjectFileBuilder objectFileBuilder ;
         
-        public NodeTableBuilderStd(IndexBuilder indexBuilder, ObjectFileBuilder objectFileBuilder)
-        { 
+        public NodeTableBuilderStd(IndexBuilder indexBuilder, ObjectFileBuilder objectFileBuilder) {
             this.indexBuilder = indexBuilder ;
             this.objectFileBuilder = objectFileBuilder ;
         }
-        
+
         @Override
-        public NodeTable buildNodeTable(FileSet fsIndex, FileSet fsObjectFile, int sizeNode2NodeIdCache, int sizeNodeId2NodeCache, int sizeNodeMissCacheSize)
-        {
+        public NodeTable buildNodeTable(FileSet fsIndex, FileSet fsObjectFile, SystemParams params) {
             RecordFactory recordFactory = new RecordFactory(SystemTDB.LenNodeHash, SystemTDB.SizeOfNodeId) ;
-            Index idx = indexBuilder.buildIndex(fsIndex, recordFactory) ;
+            Index idx = indexBuilder.buildIndex(fsIndex, recordFactory, params) ;
             ObjectFile objectFile = objectFileBuilder.buildObjectFile(fsObjectFile, Names.extNodeData) ;
             NodeTable nodeTable = new NodeTableNative(idx, objectFile) ;
-            nodeTable = NodeTableCache.create(nodeTable, sizeNode2NodeIdCache, sizeNodeId2NodeCache, sizeNodeMissCacheSize) ;
+            nodeTable = NodeTableCache.create(nodeTable, 
+                                              params.getNode2NodeIdCacheSize(),
+                                              params.getNodeId2NodeCacheSize(),
+                                              params.getNodeMissCacheSize()) ;
             nodeTable = NodeTableInline.create(nodeTable) ;
             return nodeTable ;
         }
     }
-    // ----
-
-    public static class IndexBuilderStd implements IndexBuilder
-    {
-        protected BlockMgrBuilder bMgrNodes ;
-        protected BlockMgrBuilder bMgrRecords ;
-        protected RangeIndexBuilderStd other ;
-    
-        public IndexBuilderStd(BlockMgrBuilder bMgrNodes, BlockMgrBuilder bMgrRecords)
-        {
-            this.bMgrNodes = bMgrNodes ;
-            this.bMgrRecords = bMgrRecords ;
-            this.other = new RangeIndexBuilderStd(bMgrNodes, bMgrRecords) ;
-        }
-        
-        @Override
-        public Index buildIndex(FileSet fileSet, RecordFactory recordFactory)
-        {
-            // Cheap.
-            return other.buildRangeIndex(fileSet, recordFactory) ;
-        }
-    }
-
-    public static class RangeIndexBuilderStd implements RangeIndexBuilder
-        {
-            private BlockMgrBuilder bMgrNodes ;
-            private BlockMgrBuilder bMgrRecords ;
-            public RangeIndexBuilderStd( BlockMgrBuilder blockMgrBuilderNodes,
-                                         BlockMgrBuilder blockMgrBuilderRecords)
-            {
-                this.bMgrNodes = blockMgrBuilderNodes ;
-                this.bMgrRecords = blockMgrBuilderRecords ;
-            }
-    
-            @Override
-            public RangeIndex buildRangeIndex(FileSet fileSet, RecordFactory recordFactory)
-            {
-                int blkSize = SystemTDB.BlockSize ;
-                int order = BPlusTreeParams.calcOrder(blkSize, recordFactory.recordLength()) ;
-                int readCacheSize = SystemTDB.BlockReadCacheSize ;
-                int writeCacheSize = SystemTDB.BlockWriteCacheSize ;
-                RangeIndex rIndex = createBPTree(fileSet, order, blkSize, readCacheSize, writeCacheSize, bMgrNodes, bMgrRecords, recordFactory) ;
-                return rIndex ;
-            }
-            
-            /** Knowing all the parameters, create a B+Tree */
-            private RangeIndex createBPTree(FileSet fileset, int order, 
-                                            int blockSize,
-                                            int readCacheSize, int writeCacheSize,
-                                            BlockMgrBuilder blockMgrBuilderNodes,
-                                            BlockMgrBuilder blockMgrBuilderRecords,
-                                            RecordFactory factory)
-            {
-                // ---- Checking
-                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) ;
-                }
-            
-                // Iffy - does not allow for slop.
-                if (blockSize < 0 && order >= 0)
-                {
-                    // Only in-memory.
-                    blockSize = BPlusTreeParams.calcBlockSize(order, factory) ;
-                }
-            
-                BPlusTreeParams params = new BPlusTreeParams(order, factory) ;
-                
-                BlockMgr blkMgrNodes = blockMgrBuilderNodes.buildBlockMgr(fileset, Names.bptExtTree, blockSize) ;
-                BlockMgr blkMgrRecords = blockMgrBuilderRecords.buildBlockMgr(fileset, Names.bptExtRecords, blockSize) ;
-                return BPlusTree.create(params, blkMgrNodes, blkMgrRecords) ;
-            }
-        }
-
-    public static class ObjectFileBuilderStd implements ObjectFileBuilder
-    {
-        @Override
-        public ObjectFile buildObjectFile(FileSet fileSet, String ext)
-        {
-            String filename = fileSet.filename(ext) ;
-            if ( fileSet.isMem() )
-                return FileFactory.createObjectFileMem(filename) ;
-            return FileFactory.createObjectFileDisk(filename) ;
-        }
-    }
-
-    public static class BlockMgrBuilderStd implements BlockMgrBuilder
-    {
-        public BlockMgrBuilderStd() {}
-    
-        @Override
-        public BlockMgr buildBlockMgr(FileSet fileset, String ext, int blockSize)
-        {
-            //int readCacheSize = PropertyUtils.getPropertyAsInteger(config, Names.pBlockReadCacheSize) ;
-            //int writeCacheSize = PropertyUtils.getPropertyAsInteger(config, Names.pBlockWriteCacheSize) ;
-            
-            int readCacheSize = SystemTDB.BlockReadCacheSize ;
-            int writeCacheSize = SystemTDB.BlockWriteCacheSize ;
-            
-            BlockMgr mgr = BlockMgrFactory.create(fileset, ext, blockSize, readCacheSize, writeCacheSize) ;
-            return mgr ;
-        }
-        
-    }
-
 }

Added: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BuilderIndex.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BuilderIndex.java?rev=1622864&view=auto
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BuilderIndex.java (added)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BuilderIndex.java Sat Sep  6 14:24:23 2014
@@ -0,0 +1,160 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.hp.hpl.jena.tdb.setup;
+
+import com.hp.hpl.jena.tdb.base.block.BlockMgr ;
+import com.hp.hpl.jena.tdb.base.block.BlockMgrFactory ;
+import com.hp.hpl.jena.tdb.base.block.FileMode ;
+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.objectfile.ObjectFile ;
+import com.hp.hpl.jena.tdb.base.record.RecordFactory ;
+import com.hp.hpl.jena.tdb.index.Index ;
+import com.hp.hpl.jena.tdb.index.IndexParams ;
+import com.hp.hpl.jena.tdb.index.RangeIndex ;
+import com.hp.hpl.jena.tdb.index.bplustree.BPlusTree ;
+import com.hp.hpl.jena.tdb.index.bplustree.BPlusTreeParams ;
+import com.hp.hpl.jena.tdb.sys.Names ;
+
+/** Building indexes, blockMgr and files */ 
+
+public class BuilderIndex {
+    // XXX NO SYSTEM PARAMS here
+    
+    public static class IndexBuilderStd implements IndexBuilder
+    {
+        protected BlockMgrBuilder bMgrNodes ;
+        protected BlockMgrBuilder bMgrRecords ;
+        protected RangeIndexBuilderStd other ;
+    
+        public IndexBuilderStd(BlockMgrBuilder bMgrNodes, BlockMgrBuilder bMgrRecords) {
+            this.bMgrNodes = bMgrNodes ;
+            this.bMgrRecords = bMgrRecords ;
+            this.other = new RangeIndexBuilderStd(bMgrNodes, bMgrRecords) ;
+        }
+
+        @Override
+        public Index buildIndex(FileSet fileSet, RecordFactory recordFactory, IndexParams indexParams) {
+            // Cheap.
+            return other.buildRangeIndex(fileSet, recordFactory, indexParams) ;
+        }
+    }
+
+    public static class RangeIndexBuilderStd implements RangeIndexBuilder
+    {
+        private BlockMgrBuilder bMgrNodes ;
+        private BlockMgrBuilder bMgrRecords ;
+
+        public RangeIndexBuilderStd(BlockMgrBuilder blockMgrBuilderNodes, BlockMgrBuilder blockMgrBuilderRecords) {
+            this.bMgrNodes = blockMgrBuilderNodes ;
+            this.bMgrRecords = blockMgrBuilderRecords ;
+        }
+    
+        @Override
+        public RangeIndex buildRangeIndex(FileSet fileSet, RecordFactory recordFactory, IndexParams indexParams) {
+            int blkSize = indexParams.getBlockSize() ;
+            int order = BPlusTreeParams.calcOrder(blkSize, recordFactory.recordLength()) ;
+            RangeIndex rIndex = createBPTree(fileSet, order, bMgrNodes, bMgrRecords, recordFactory, indexParams) ;
+            return rIndex ;
+        }
+        
+        /** 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) ;
+        }
+    }
+
+    public static class ObjectFileBuilderStd implements ObjectFileBuilder
+    {
+        public ObjectFileBuilderStd() { }
+        
+        @Override
+        public ObjectFile buildObjectFile(FileSet fileSet, String ext)
+        {
+            String filename = fileSet.filename(ext) ;
+            if ( fileSet.isMem() )
+                return FileFactory.createObjectFileMem(filename) ;
+            return FileFactory.createObjectFileDisk(filename) ;
+        }
+    }
+
+    public static class BlockMgrBuilderStd/*Base*/ implements BlockMgrBuilder
+    {
+        public BlockMgrBuilderStd() {}
+    
+        @Override
+        public BlockMgr buildBlockMgr(FileSet fileset, String ext, IndexParams indexParams)
+        {
+            BlockMgr mgr = BlockMgrFactory.create(fileset, ext, indexParams.getBlockSize(), 
+                                                  indexParams.getBlockReadCacheSize(), indexParams.getBlockWriteCacheSize()) ;
+            return mgr ;
+        }
+    }
+
+    // Add cache (iff not memory and not 
+    // XXX Activate
+    public static class BlockMgrBuilderStd2 implements BlockMgrBuilder
+    {
+        private final BlockMgrBuilder other ;
+    
+        public BlockMgrBuilderStd2(BlockMgrBuilder other) {
+            this.other = other ;
+        }
+    
+        @Override
+        public BlockMgr buildBlockMgr(FileSet fileset, String ext, IndexParams indexParams)
+        {
+            BlockMgr blkMgr = other.buildBlockMgr(fileset, ext, indexParams) ;
+            if ( fileset.isMem() )
+                return blkMgr ;
+            int readCacheSize = indexParams.getBlockReadCacheSize() ;
+            int writeCacheSize = indexParams.getBlockWriteCacheSize() ;
+            boolean addCache = 
+                ! fileset.isMem() && indexParams.getFileMode() != FileMode.mapped ;
+            if ( addCache && ( readCacheSize > 0 || writeCacheSize > 0 ) )
+                blkMgr = BlockMgrFactory.addCache(blkMgr, readCacheSize, writeCacheSize) ;
+            return blkMgr ;
+        }
+    }
+
+}
+

Propchange: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BuilderIndex.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/DatasetBuilderBasic.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/DatasetBuilderBasic.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/DatasetBuilderBasic.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/DatasetBuilderBasic.java Sat Sep  6 14:24:23 2014
@@ -51,9 +51,9 @@ public class DatasetBuilderBasic //imple
     
     private /*public*/ DatasetBuilderBasic(IndexBuilder indexBuilder, RangeIndexBuilder rangeIndexBuilder)
     {
-        ObjectFileBuilder objectFileBuilder = new Builder.ObjectFileBuilderStd()  ;
-        nodeTableBuilder    = new Builder.NodeTableBuilderStd(indexBuilder, objectFileBuilder) ;
-        tupleIndexBuilder   = new Builder.TupleIndexBuilderStd(rangeIndexBuilder) ;
+        ObjectFileBuilder objectFileBuilder = new BuilderIndex.ObjectFileBuilderStd()  ;
+        nodeTableBuilder    = new BuilderDB.NodeTableBuilderStd(indexBuilder, objectFileBuilder) ;
+        tupleIndexBuilder   = new BuilderDB.TupleIndexBuilderStd(rangeIndexBuilder) ;
     }
 
     //@Override public
@@ -97,7 +97,7 @@ public class DatasetBuilderBasic //imple
     {
         FileSet fsNodeToId = new FileSet(location, indexNode2Id) ;
         FileSet fsId2Node = new FileSet(location, indexId2Node) ;
-        NodeTable nt = nodeTableBuilder.buildNodeTable(fsNodeToId, fsId2Node, sizeNode2NodeIdCache, sizeNodeId2NodeCache, sizeNodeMissCache) ;
+        NodeTable nt = nodeTableBuilder.buildNodeTable(fsNodeToId, fsId2Node, params) ;
         return nt ;
     }
     
@@ -182,7 +182,7 @@ public class DatasetBuilderBasic //imple
         // FileSet
         FileSet fs = new FileSet(location, name) ;
         ColumnMap colMap = new ColumnMap(primary, indexOrder) ;
-        return tupleIndexBuilder.buildTupleIndex(fs, colMap, indexOrder) ;
+        return tupleIndexBuilder.buildTupleIndex(fs, colMap, indexOrder, params) ;
     }
 
     private static void error(Logger log, String msg)

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/DatasetBuilderStd.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/DatasetBuilderStd.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/DatasetBuilderStd.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/DatasetBuilderStd.java Sat Sep  6 14:24:23 2014
@@ -36,6 +36,7 @@ import com.hp.hpl.jena.tdb.base.block.Bl
 import com.hp.hpl.jena.tdb.base.file.BufferChannel ;
 import com.hp.hpl.jena.tdb.base.file.FileSet ;
 import com.hp.hpl.jena.tdb.base.file.Location ;
+import com.hp.hpl.jena.tdb.index.IndexParams ;
 import com.hp.hpl.jena.tdb.solver.OpExecutorTDB1 ;
 import com.hp.hpl.jena.tdb.store.* ;
 import com.hp.hpl.jena.tdb.store.nodetable.NodeTable ;
@@ -91,21 +92,21 @@ public class DatasetBuilderStd implement
         Recorder recorder = new Recorder(this) ;
         BlockMgrBuilder blockMgrBuilderRec = new BlockMgrBuilderRecorder(blockMgrBuilder, recorder) ;
 
-        IndexBuilder indexBuilder = new Builder.IndexBuilderStd(blockMgrBuilderRec, blockMgrBuilderRec) ;
-        RangeIndexBuilder rangeIndexBuilder = new Builder.RangeIndexBuilderStd(blockMgrBuilderRec, blockMgrBuilderRec) ;
+        IndexBuilder indexBuilder = new BuilderIndex.IndexBuilderStd(blockMgrBuilderRec, blockMgrBuilderRec) ;
+        RangeIndexBuilder rangeIndexBuilder = new BuilderIndex.RangeIndexBuilderStd(blockMgrBuilderRec, blockMgrBuilderRec) ;
 
         this.nodeTableBuilder = nodeTableBuilder ;
         nodeTableBuilder = new NodeTableBuilderRecorder(nodeTableBuilder, recorder) ;
 
-        TupleIndexBuilder tupleIndexBuilder = new Builder.TupleIndexBuilderStd(rangeIndexBuilder) ;
+        TupleIndexBuilder tupleIndexBuilder = new BuilderDB.TupleIndexBuilderStd(rangeIndexBuilder) ;
         set(nodeTableBuilder, tupleIndexBuilder) ;
     }
 
     private void standardSetup() {
-        ObjectFileBuilder objectFileBuilder = new Builder.ObjectFileBuilderStd() ;
-        BlockMgrBuilder blockMgrBuilder = new Builder.BlockMgrBuilderStd() ;
-        IndexBuilder indexBuilderNT = new Builder.IndexBuilderStd(blockMgrBuilder, blockMgrBuilder) ;
-        NodeTableBuilder nodeTableBuilder = new Builder.NodeTableBuilderStd(indexBuilderNT, objectFileBuilder) ;
+        ObjectFileBuilder objectFileBuilder = new BuilderIndex.ObjectFileBuilderStd() ;
+        BlockMgrBuilder blockMgrBuilder = new BuilderIndex.BlockMgrBuilderStd() ;
+        IndexBuilder indexBuilderNT = new BuilderIndex.IndexBuilderStd(blockMgrBuilder, blockMgrBuilder) ;
+        NodeTableBuilder nodeTableBuilder = new BuilderDB.NodeTableBuilderStd(indexBuilderNT, objectFileBuilder) ;
 
         set(blockMgrBuilder, nodeTableBuilder) ;
     }
@@ -262,7 +263,7 @@ public class DatasetBuilderStd implement
         // FileSet
         FileSet fs = new FileSet(location, name) ;
         ColumnMap colMap = new ColumnMap(primary, indexOrder) ;
-        return tupleIndexBuilder.buildTupleIndex(fs, colMap, indexOrder) ;
+        return tupleIndexBuilder.buildTupleIndex(fs, colMap, indexOrder, params) ;
     }
 
     // ----
@@ -271,8 +272,7 @@ public class DatasetBuilderStd implement
                                       int sizeNode2NodeIdCache, int sizeNodeId2NodeCache, int sizeNodeMissCache) {
         FileSet fsNodeToId = new FileSet(location, indexNode2Id) ;
         FileSet fsId2Node = new FileSet(location, indexId2Node) ;
-        NodeTable nt = nodeTableBuilder.buildNodeTable(fsNodeToId, fsId2Node, sizeNode2NodeIdCache,
-                                                       sizeNodeId2NodeCache, sizeNodeMissCache) ;
+        NodeTable nt = nodeTableBuilder.buildNodeTable(fsNodeToId, fsId2Node, params) ;
         return nt ;
     }
 
@@ -355,10 +355,8 @@ public class DatasetBuilderStd implement
         }
 
         @Override
-        public NodeTable buildNodeTable(FileSet fsIndex, FileSet fsObjectFile, int sizeNode2NodeIdCache,
-                                        int sizeNodeId2NodeCache, int sizeNodeMissCacheSize) {
-            NodeTable nt = builder.buildNodeTable(fsIndex, fsObjectFile, sizeNode2NodeIdCache, sizeNodeId2NodeCache,
-                                                  sizeNodeMissCacheSize) ;
+        public NodeTable buildNodeTable(FileSet fsIndex, FileSet fsObjectFile, SystemParams params) {
+            NodeTable nt = builder.buildNodeTable(fsIndex, fsObjectFile, params) ;
             // It just knows, right?
             FileRef ref = FileRef.create(fsObjectFile.filename(Names.extNodeData)) ;
             recorder.record(ref, nt) ;
@@ -377,8 +375,8 @@ public class DatasetBuilderStd implement
         }
 
         @Override
-        public BlockMgr buildBlockMgr(FileSet fileSet, String ext, int blockSize) {
-            BlockMgr blkMgr = builder.buildBlockMgr(fileSet, ext, blockSize) ;
+        public BlockMgr buildBlockMgr(FileSet fileSet, String ext, IndexParams params) {
+            BlockMgr blkMgr = builder.buildBlockMgr(fileSet, ext, params) ;
             FileRef ref = FileRef.create(fileSet, ext) ;
             recorder.record(ref, blkMgr) ;
             return blkMgr ;

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/IndexBuilder.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/IndexBuilder.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/IndexBuilder.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/IndexBuilder.java Sat Sep  6 14:24:23 2014
@@ -21,7 +21,8 @@ package com.hp.hpl.jena.tdb.setup;
 import com.hp.hpl.jena.tdb.base.file.FileSet ;
 import com.hp.hpl.jena.tdb.base.record.RecordFactory ;
 import com.hp.hpl.jena.tdb.index.Index ;
+import com.hp.hpl.jena.tdb.index.IndexParams ;
 
 public interface IndexBuilder {
-    Index buildIndex(FileSet fileSet, RecordFactory recordfactory) ;
+    Index buildIndex(FileSet fileSet, RecordFactory recordfactory, IndexParams indexParams) ;
 }

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/NodeTableBuilder.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/NodeTableBuilder.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/NodeTableBuilder.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/NodeTableBuilder.java Sat Sep  6 14:24:23 2014
@@ -22,6 +22,5 @@ import com.hp.hpl.jena.tdb.base.file.Fil
 import com.hp.hpl.jena.tdb.store.nodetable.NodeTable ;
 
 public interface NodeTableBuilder {
-    NodeTable buildNodeTable(FileSet fsIndex, FileSet fsObjectFile, 
-                             int sizeNode2NodeIdCache, int sizeNodeId2NodeCache, int sizeNodeMissCache) ;
+    NodeTable buildNodeTable(FileSet fsIndex, FileSet fsObjectFile, SystemParams params) ;
 }

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/RangeIndexBuilder.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/RangeIndexBuilder.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/RangeIndexBuilder.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/RangeIndexBuilder.java Sat Sep  6 14:24:23 2014
@@ -20,8 +20,9 @@ package com.hp.hpl.jena.tdb.setup;
 
 import com.hp.hpl.jena.tdb.base.file.FileSet ;
 import com.hp.hpl.jena.tdb.base.record.RecordFactory ;
+import com.hp.hpl.jena.tdb.index.IndexParams ;
 import com.hp.hpl.jena.tdb.index.RangeIndex ;
 
 public interface RangeIndexBuilder {
-    RangeIndex buildRangeIndex(FileSet fileSet, RecordFactory recordfactory) ;
+    RangeIndex buildRangeIndex(FileSet fileSet, RecordFactory recordfactory, IndexParams indexParams) ;
 }

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/SystemParams.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/SystemParams.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/SystemParams.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/SystemParams.java Sat Sep  6 14:24:23 2014
@@ -21,9 +21,10 @@ package com.hp.hpl.jena.tdb.setup;
 import org.apache.jena.atlas.lib.StrUtils ;
 
 import com.hp.hpl.jena.tdb.base.block.FileMode ;
+import com.hp.hpl.jena.tdb.index.IndexParams ;
 
 /** System parameters for a TDB database instance. */
-public class SystemParams
+public class SystemParams implements IndexParams
 {
     // SystemParams are built with a SystemParamsBuilder
     
@@ -91,19 +92,22 @@ public class SystemParams
         return dftSystemParams ;
     }
 
+    @Override
     public FileMode getFileMode() {
         return fileMode ;
     }
-
     
+    @Override
     public int getBlockSize() {
         return blockSize ;
     }
 
+    @Override
     public int getBlockReadCacheSize() {
         return blockReadCacheSize ;
     }
 
+    @Override
     public int getBlockWriteCacheSize() {
         return blockWriteCacheSize ;
     }

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/TupleIndexBuilder.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/TupleIndexBuilder.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/TupleIndexBuilder.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/TupleIndexBuilder.java Sat Sep  6 14:24:23 2014
@@ -24,5 +24,5 @@ import com.hp.hpl.jena.tdb.base.file.Fil
 import com.hp.hpl.jena.tdb.store.tupletable.TupleIndex ;
 
 public interface TupleIndexBuilder {
-    TupleIndex buildTupleIndex(FileSet fileSet, ColumnMap colMap, String name) ;
+    TupleIndex buildTupleIndex(FileSet fileSet, ColumnMap colMap, String name, SystemParams params) ;
 }

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/store/DatasetPrefixesTDB.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/store/DatasetPrefixesTDB.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/store/DatasetPrefixesTDB.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/store/DatasetPrefixesTDB.java Sat Sep  6 14:24:23 2014
@@ -18,12 +18,7 @@
 
 package com.hp.hpl.jena.tdb.store;
 
-import java.util.HashMap ;
-import java.util.HashSet ;
-import java.util.Iterator ;
-import java.util.List ;
-import java.util.Map ;
-import java.util.Set ;
+import java.util.* ;
 
 import org.apache.jena.atlas.iterator.Iter ;
 import org.apache.jena.atlas.lib.ColumnMap ;
@@ -39,6 +34,8 @@ import com.hp.hpl.jena.tdb.base.file.Fil
 import com.hp.hpl.jena.tdb.base.file.Location ;
 import com.hp.hpl.jena.tdb.base.record.RecordFactory ;
 import com.hp.hpl.jena.tdb.index.IndexBuilder ;
+import com.hp.hpl.jena.tdb.index.IndexParams ;
+import com.hp.hpl.jena.tdb.setup.SystemParams ;
 import com.hp.hpl.jena.tdb.store.nodetable.NodeTable ;
 import com.hp.hpl.jena.tdb.store.nodetable.NodeTableFactory ;
 import com.hp.hpl.jena.tdb.store.nodetupletable.NodeTupleTable ;
@@ -72,6 +69,7 @@ public class DatasetPrefixesTDB implemen
     @Deprecated
     private DatasetPrefixesTDB(IndexBuilder indexBuilder, Location location, DatasetControl policy)
     {
+        IndexParams indexParams = SystemParams.getDftSystemParams() ;
         // TO BE REMOVED when DI sorted out.
         // This is a table "G" "P" "U" (Graph, Prefix, URI), indexed on GPU only.
         // GPU index
@@ -79,7 +77,8 @@ public class DatasetPrefixesTDB implemen
         if ( location != null )
             filesetGPU = new FileSet(location, Names.indexPrefix) ;
         
-        TupleIndex index = new TupleIndexRecord(3, colMap, Names.primaryIndexPrefix, factory, indexBuilder.newRangeIndex(filesetGPU, factory)) ;
+        TupleIndex index = new TupleIndexRecord(3, colMap, Names.primaryIndexPrefix, factory, 
+                                                indexBuilder.newRangeIndex(filesetGPU, factory, indexParams)) ;
         TupleIndex[] indexes = { index } ;
         
         // Node table.

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/store/nodetable/NodeTableFactory.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/store/nodetable/NodeTableFactory.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/store/nodetable/NodeTableFactory.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/store/nodetable/NodeTableFactory.java Sat Sep  6 14:24:23 2014
@@ -24,6 +24,7 @@ import com.hp.hpl.jena.tdb.base.file.Loc
 import com.hp.hpl.jena.tdb.base.objectfile.ObjectFile ;
 import com.hp.hpl.jena.tdb.index.Index ;
 import com.hp.hpl.jena.tdb.index.IndexBuilder ;
+import com.hp.hpl.jena.tdb.index.IndexParams ;
 import com.hp.hpl.jena.tdb.setup.SystemParams ;
 import com.hp.hpl.jena.tdb.sys.Names ;
 import com.hp.hpl.jena.tdb.sys.SystemTDB ;
@@ -59,9 +60,12 @@ public class NodeTableFactory
     {
         String filename = fsIdToNode.filename(Names.extNodeData) ;
         
+        // XXX Temp
+        IndexParams indexparams = SystemParams.getDftSystemParams() ;
+        
         if ( fsNodeToId.isMem() )
         {
-            Index nodeToId = indexBuilder.newIndex(FileSet.mem(), SystemTDB.nodeRecordFactory) ;
+            Index nodeToId = indexBuilder.newIndex(FileSet.mem(), SystemTDB.nodeRecordFactory, indexparams) ;
             ObjectFile objects = FileFactory.createObjectFileMem(filename) ;
             NodeTable nodeTable = new NodeTableNative(nodeToId, objects) ;
             
@@ -73,7 +77,7 @@ public class NodeTableFactory
             //return NodeTableIndex.createMem(indexBuilder) ;
         }
         
-        Index nodeToId = indexBuilder.newIndex(fsNodeToId, SystemTDB.nodeRecordFactory) ;
+        Index nodeToId = indexBuilder.newIndex(fsNodeToId, SystemTDB.nodeRecordFactory, indexparams) ;
         // Node table.
         ObjectFile objects = FileFactory.createObjectFileDisk(filename);
         NodeTable nodeTable = new NodeTableNative(nodeToId, objects) ;

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/transaction/DatasetBuilderTxn.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/transaction/DatasetBuilderTxn.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/transaction/DatasetBuilderTxn.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/transaction/DatasetBuilderTxn.java Sat Sep  6 14:24:23 2014
@@ -31,9 +31,11 @@ import com.hp.hpl.jena.tdb.base.objectfi
 import com.hp.hpl.jena.tdb.base.record.RecordFactory ;
 import com.hp.hpl.jena.tdb.index.Index ;
 import com.hp.hpl.jena.tdb.index.IndexMap ;
+import com.hp.hpl.jena.tdb.index.IndexParams ;
 import com.hp.hpl.jena.tdb.setup.BlockMgrBuilder ;
 import com.hp.hpl.jena.tdb.setup.DatasetBuilderStd ;
 import com.hp.hpl.jena.tdb.setup.NodeTableBuilder ;
+import com.hp.hpl.jena.tdb.setup.SystemParams ;
 import com.hp.hpl.jena.tdb.store.DatasetGraphTDB ;
 import com.hp.hpl.jena.tdb.store.nodetable.NodeTable ;
 import com.hp.hpl.jena.tdb.store.nodetable.NodeTableInline ;
@@ -55,13 +57,12 @@ public class DatasetBuilderTxn
 
     public DatasetBuilderTxn(TransactionManager txnMgr) { this.txnMgr = txnMgr ; }
     
-    public DatasetGraphTxn build(Transaction transaction, ReadWrite mode, DatasetGraphTDB dsg)
-    {
+    public DatasetGraphTxn build(Transaction transaction, ReadWrite mode, DatasetGraphTDB dsg) {
         this.blockMgrs = dsg.getConfig().blockMgrs ;
         this.nodeTables = dsg.getConfig().nodeTables ;
         this.txn = transaction ;
         this.dsg = dsg ;
-        
+
         DatasetGraphTDB dsgTDB ;
             
         switch(mode)
@@ -76,19 +77,17 @@ public class DatasetBuilderTxn
         dsgTxn.getContext().putAll(dsg.getContext()) ;
         return dsgTxn ;
     }
-    
-    private DatasetGraphTDB buildReadonly()
-    {
+
+    private DatasetGraphTDB buildReadonly() {
         BlockMgrBuilder blockMgrBuilder = new BlockMgrBuilderReadonly() ;
         NodeTableBuilder nodeTableBuilder = new NodeTableBuilderReadonly() ;
         DatasetBuilderStd x = new DatasetBuilderStd(blockMgrBuilder, nodeTableBuilder) ;
         DatasetGraphTDB dsg2 = x._build(dsg.getLocation(), dsg.getConfig().params, false, dsg.getReorderTransform()) ;
-        
+
         return dsg2 ;
     }
 
-    private DatasetGraphTDB buildWritable()
-    {
+    private DatasetGraphTDB buildWritable() {
         BlockMgrBuilder blockMgrBuilder = new BlockMgrBuilderTx() ;
         NodeTableBuilder nodeTableBuilder = new NodeTableBuilderTx() ;
         DatasetBuilderStd x = new DatasetBuilderStd(blockMgrBuilder, nodeTableBuilder) ;
@@ -100,49 +99,45 @@ public class DatasetBuilderTxn
     // ---- Add logging to a BlockMgr when built.
     static BlockMgrBuilder logging(BlockMgrBuilder other) { return new BlockMgrBuilderLogger(other) ; }
     
-    static class BlockMgrBuilderLogger implements BlockMgrBuilder
-    {
+    static class BlockMgrBuilderLogger implements BlockMgrBuilder {
         public BlockMgrBuilder other ;
-        public BlockMgrBuilderLogger(BlockMgrBuilder other)
-        { 
+
+        public BlockMgrBuilderLogger(BlockMgrBuilder other) {
             this.other = other ;
         }
-        
+
         @Override
-        public BlockMgr buildBlockMgr(FileSet fileSet, String ext, int blockSize)
-        {
-            BlockMgr blkMgr = other.buildBlockMgr(fileSet, ext, blockSize) ;
+        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, int sizeNode2NodeIdCache,
-                                        int sizeNodeId2NodeCache, int sizeNodeMissCacheSize)
-        {
+        public NodeTable buildNodeTable(FileSet fsIndex, FileSet fsObjectFile, SystemParams params) {
             FileRef ref = FileRef.create(fsObjectFile.filename(Names.extNodeData)) ;
             NodeTable ntBase = nodeTables.get(ref) ;
             if ( ntBase == null )
-                throw new TDBException("No NodeTable for "+ref) ;
-            
+                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) ;
+            String objFilename = fsObjectFile.filename(Names.extNodeData + "-" + Names.extJournal) ;
             ObjectFile objectFile ;
-            
+
             if ( fsObjectFile.isMem() )
                 objectFile = FileFactory.createObjectFileMem(objFilename) ;
             else
                 objectFile = FileFactory.createObjectFileDisk(objFilename) ;
 
-            NodeTableTrans ntt = new NodeTableTrans(txn ,fsObjectFile.getBasename(), ntBase, idx, objectFile) ;
+            NodeTableTrans ntt = new NodeTableTrans(txn, fsObjectFile.getBasename(), ntBase, idx, objectFile) ;
             txn.addComponent(ntt) ;
-            
+
             // Add inline wrapper.
             NodeTable nt = NodeTableInline.create(ntt) ;
             return nt ;
@@ -152,13 +147,12 @@ public class DatasetBuilderTxn
     class BlockMgrBuilderTx implements BlockMgrBuilder
     {
         @Override
-        public BlockMgr buildBlockMgr(FileSet fileSet, String ext, int blockSize)
-        {
+        public BlockMgr buildBlockMgr(FileSet fileSet, String ext, IndexParams params) {
             // 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) ;
+                throw new TDBException("No BlockMgr for " + ref) ;
             BlockMgrJournal blkMgr = new BlockMgrJournal(txn, ref, baseMgr) ;
             txn.addComponent(blkMgr) ;
             return blkMgr ;
@@ -170,12 +164,11 @@ public class DatasetBuilderTxn
     class BlockMgrBuilderReadonly implements BlockMgrBuilder
     {
         @Override
-        public BlockMgr buildBlockMgr(FileSet fileSet, String ext, int blockSize)
-        {
+        public BlockMgr buildBlockMgr(FileSet fileSet, String ext, IndexParams params) {
             FileRef ref = FileRef.create(fileSet, ext) ;
             BlockMgr blockMgr = blockMgrs.get(ref) ;
             if ( blockMgr == null )
-                throw new TDBException("No BlockMgr for "+ref) ;
+                throw new TDBException("No BlockMgr for " + ref) ;
             blockMgr = new BlockMgrReadonly(blockMgr) ;
             return blockMgr ;
         }
@@ -183,11 +176,8 @@ public class DatasetBuilderTxn
     
     class NodeTableBuilderReadonly implements NodeTableBuilder
     {
-
         @Override
-        public NodeTable buildNodeTable(FileSet fsIndex, FileSet fsObjectFile, 
-                                        int sizeNode2NodeIdCache, int sizeNodeId2NodeCache, int sizeNodeMissCacheSize)
-        {
+        public NodeTable buildNodeTable(FileSet fsIndex, FileSet fsObjectFile, SystemParams params) {
             FileRef ref = FileRef.create(fsObjectFile.filename(Names.extNodeData)) ;
             NodeTable nt = nodeTables.get(ref) ;
             nt = new NodeTableReadonly(nt) ;

Modified: jena/trunk/jena-tdb/src/test/java/com/hp/hpl/jena/tdb/store/tupletable/TestTupleIndexRecord.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/test/java/com/hp/hpl/jena/tdb/store/tupletable/TestTupleIndexRecord.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/test/java/com/hp/hpl/jena/tdb/store/tupletable/TestTupleIndexRecord.java (original)
+++ jena/trunk/jena-tdb/src/test/java/com/hp/hpl/jena/tdb/store/tupletable/TestTupleIndexRecord.java Sat Sep  6 14:24:23 2014
@@ -21,12 +21,12 @@ package com.hp.hpl.jena.tdb.store.tuplet
 
 import org.apache.jena.atlas.lib.ColumnMap ;
 
-import com.hp.hpl.jena.tdb.base.file.FileSet ;
 import com.hp.hpl.jena.tdb.base.record.RecordFactory ;
 import com.hp.hpl.jena.tdb.index.AbstractTestTupleIndex ;
-import com.hp.hpl.jena.tdb.index.IndexBuilder ;
+import com.hp.hpl.jena.tdb.index.IndexParams ;
 import com.hp.hpl.jena.tdb.index.RangeIndex ;
-import com.hp.hpl.jena.tdb.store.tupletable.TupleIndexRecord ;
+import com.hp.hpl.jena.tdb.setup.B ;
+import com.hp.hpl.jena.tdb.setup.SystemParams ;
 import com.hp.hpl.jena.tdb.sys.SystemTDB ;
 
 public class TestTupleIndexRecord extends AbstractTestTupleIndex
@@ -36,7 +36,8 @@ public class TestTupleIndexRecord extend
     @Override
     protected TupleIndexRecord create(String description)
     {
-        RangeIndex rIdx = IndexBuilder.mem().newRangeIndex(FileSet.mem(), factory) ;
+        IndexParams indexParams = SystemParams.getDftSystemParams() ; 
+        RangeIndex rIdx = B.buildRangeIndex(null, factory, indexParams) ;
         ColumnMap cmap = new ColumnMap("SPO", description) ;
         TupleIndexRecord index = new TupleIndexRecord(3, cmap, description, factory, rIdx) ;
         return index ;

Modified: jena/trunk/jena-tdb/src/test/java/com/hp/hpl/jena/tdb/store/tupletable/TestTupleIndexRecordDirect.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/test/java/com/hp/hpl/jena/tdb/store/tupletable/TestTupleIndexRecordDirect.java?rev=1622864&r1=1622863&r2=1622864&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/test/java/com/hp/hpl/jena/tdb/store/tupletable/TestTupleIndexRecordDirect.java (original)
+++ jena/trunk/jena-tdb/src/test/java/com/hp/hpl/jena/tdb/store/tupletable/TestTupleIndexRecordDirect.java Sat Sep  6 14:24:23 2014
@@ -18,6 +18,8 @@
 
 package com.hp.hpl.jena.tdb.store.tupletable;
 
+import static org.apache.jena.atlas.lib.Tuple.createTuple ;
+
 import java.util.Iterator ;
 import java.util.Set ;
 
@@ -25,15 +27,15 @@ import org.apache.jena.atlas.iterator.It
 import org.apache.jena.atlas.junit.BaseTest ;
 import org.apache.jena.atlas.lib.ColumnMap ;
 import org.apache.jena.atlas.lib.Tuple ;
-import static org.apache.jena.atlas.lib.Tuple.* ;
 import org.junit.Test ;
 
 import com.hp.hpl.jena.tdb.base.file.FileSet ;
 import com.hp.hpl.jena.tdb.base.record.RecordFactory ;
-import com.hp.hpl.jena.tdb.index.IndexBuilder ;
+import com.hp.hpl.jena.tdb.index.IndexParams ;
 import com.hp.hpl.jena.tdb.index.RangeIndex ;
+import com.hp.hpl.jena.tdb.setup.B ;
+import com.hp.hpl.jena.tdb.setup.SystemParams ;
 import com.hp.hpl.jena.tdb.store.NodeId ;
-import com.hp.hpl.jena.tdb.store.tupletable.TupleIndexRecord ;
 import com.hp.hpl.jena.tdb.sys.SystemTDB ;
 
 public class TestTupleIndexRecordDirect extends BaseTest
@@ -48,7 +50,8 @@ public class TestTupleIndexRecordDirect 
     
     static TupleIndexRecord create(String description)
     {
-        RangeIndex rIdx = IndexBuilder.mem().newRangeIndex(FileSet.mem(), factory) ;
+        IndexParams indexParams = SystemParams.getDftSystemParams() ; 
+        RangeIndex rIdx = B.buildRangeIndex(FileSet.mem(), factory, indexParams) ;
         ColumnMap cmap = new ColumnMap("SPO", description) ;
         TupleIndexRecord index = new TupleIndexRecord(3, cmap, description, factory, rIdx) ;
         return index ;