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/08 16:04:30 UTC

svn commit: r1623401 - in /jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb: base/block/ index/ index/factories/ setup/ store/nodetable/ sys/

Author: andy
Date: Mon Sep  8 14:04:30 2014
New Revision: 1623401

URL: http://svn.apache.org/r1623401
Log:
Extract 'BlockParams'; clean-up DB construction

Added:
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/base/block/BlockParams.java
Removed:
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/store/nodetable/NodeTableFactory.java
Modified:
    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/IndexParams.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/setup/BuilderIndex.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/SystemParamsBuilder.java
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/sys/FileRef.java

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=1623401&r1=1623400&r2=1623401&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 Mon Sep  8 14:04:30 2014
@@ -16,127 +16,120 @@
  * limitations under the License.
  */
 
-package com.hp.hpl.jena.tdb.base.block;
+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 ;
-import com.hp.hpl.jena.tdb.base.file.BlockAccessDirect ;
-import com.hp.hpl.jena.tdb.base.file.BlockAccessMapped ;
-import com.hp.hpl.jena.tdb.base.file.BlockAccessMem ;
-import com.hp.hpl.jena.tdb.base.file.FileSet ;
+import com.hp.hpl.jena.tdb.base.file.* ;
 import com.hp.hpl.jena.tdb.sys.SystemTDB ;
 
-
-public class BlockMgrFactory
-{
+public class BlockMgrFactory {
     // This isn't always helpful so be careful if setting the default to "true".
     // Sometimes the tracking is too strict
-    //     e.g. transactions keep blocks and not release them down the layers.
-    //     But journal layers over a tracked BlockMgr is this is on. 
-    public /*final*/ static boolean AddTracker = false ;
-    
-    public static BlockMgr tracker(BlockMgr blockMgr)
-    {
-        if ( blockMgr instanceof BlockMgrTracker ) return blockMgr ;
+    // e.g. transactions keep blocks and not release them down the layers.
+    public/* final */static boolean AddTracker = false ;
+
+    public static BlockMgr tracker(BlockMgr blockMgr) {
+        if ( blockMgr instanceof BlockMgrTracker )
+            return blockMgr ;
         return BlockMgrTracker.track(blockMgr) ;
     }
-    
+
     /** Add a tracker if the system default is to do so */
-    private static BlockMgr track(BlockMgr blockMgr)
-    {
-        if ( ! AddTracker ) return blockMgr ;
+    private static BlockMgr track(BlockMgr blockMgr) {
+        if ( !AddTracker )
+            return blockMgr ;
         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() )
-            return createMem(fileSet.filename(ext), blockSize) ;
-        else
-            return createFile(fileSet.filename(ext), null, blockSize, readBlockCacheSize, writeBlockCacheSize) ;
+
+    public static BlockMgr create(FileSet fileSet, String ext, BlockParams params) {
+        return createFile(fileSet.filename(ext), params) ;
     }
     
-    public static BlockMgr create(FileSet fileSet, FileMode fileMode, String ext, int blockSize, int readBlockCacheSize, int writeBlockCacheSize)
-    {
+    // XXX Build then add cache.
+    // XXX Deprecate?
+    public static BlockMgr create(FileSet fileSet, String ext, int blockSize, int readBlockCacheSize, int writeBlockCacheSize) {
+        return create(fileSet, ext, null, blockSize, readBlockCacheSize, writeBlockCacheSize) ;
+    }
+
+    // XXX Deprecate?
+    public static BlockMgr create(FileSet fileSet, String ext, FileMode fileMode, int blockSize, int readBlockCacheSize, int writeBlockCacheSize) {
         if ( fileSet.isMem() )
             return createMem(fileSet.filename(ext), blockSize) ;
         else
             return createFile(fileSet.filename(ext), fileMode, blockSize, readBlockCacheSize, writeBlockCacheSize) ;
     }
 
-    /** Create an in-memory block manager */ 
-    public static BlockMgr createMem(String indexName, int blockSize)
-    {
+    /** Create an in-memory block manager */
+    public static BlockMgr createMem(String indexName, int blockSize) {
         BlockAccess file = new BlockAccessMem(indexName, blockSize) ;
         BlockMgr blockMgr = new BlockMgrFileAccess(file, blockSize) ;
         blockMgr = new BlockMgrFreeChain(blockMgr) ;
-
         // Small cache - testing.
-        //blockMgr = new BlockMgrCache(indexName, 3, 3, blockMgr) ;
-        
+        // blockMgr = new BlockMgrCache(indexName, 3, 3, blockMgr) ;
         return track(blockMgr) ;
     }
-    
-    /** Create a BlockMgr backed by a file */
-    public static BlockMgr createFile(String filename, int blockSize, int readBlockCacheSize, int writeBlockCacheSize) {
-        return  createFile(filename, null,blockSize, readBlockCacheSize, writeBlockCacheSize) ;
+
+    /** Create a BlockMgr backed by a real file */
+    public static BlockMgr createFile(String filename, BlockParams params) {
+        return createFile(filename, 
+                          params.getFileMode(), params.getBlockSize(),
+                          params.getBlockReadCacheSize(), params.getBlockWriteCacheSize()) ;
     }
 
-    /** Create a BlockMgr backed by a file */
-    public static BlockMgr createFile(String filename, FileMode fileMode, int blockSize, int readBlockCacheSize, int writeBlockCacheSize)
-    {
+        /** Create a BlockMgr backed by a real file */
+    public static BlockMgr createFile(String filename, FileMode fileMode, int blockSize, int readBlockCacheSize, int writeBlockCacheSize) {
         if ( fileMode == null )
             fileMode = SystemTDB.fileMode() ;
-        switch ( fileMode )
-        {
-            case mapped:
+        switch (fileMode) {
+            case mapped :
                 return createMMapFile(filename, blockSize) ;
-            case direct:
+            case direct :
                 return createStdFile(filename, blockSize, readBlockCacheSize, writeBlockCacheSize) ;
         }
-        throw new TDBException("Unknown file mode: "+fileMode) ;
-    }        
+        throw new TDBException("Unknown file mode: " + fileMode) ;
+    }
 
     /** Create a NIO Block Manager */
-    public static BlockMgr createMMapFile(String filename, int blockSize)
-    {
+    public static BlockMgr createMMapFile(String filename, int blockSize) {
         BlockAccess file = new BlockAccessMapped(filename, blockSize) ;
-        BlockMgr blockMgr =  wrapFileAccess(file, blockSize) ;
+        BlockMgr blockMgr = wrapFileAccess(file, blockSize) ;
         return track(blockMgr) ;
     }
-    
+
     /** Create a Block Manager using direct access (and a cache) */
-    public static BlockMgr createStdFile(String filename, int blockSize, int readBlockCacheSize, int writeBlockCacheSize)
-    {
+    public static BlockMgr createStdFile(String filename, int blockSize, int readBlockCacheSize, int writeBlockCacheSize) {
         BlockAccess file = new BlockAccessDirect(filename, blockSize) ;
-        BlockMgr blockMgr =  wrapFileAccess(file, blockSize) ;
-
-        String fn = FileOps.basename(filename) ;
-        
-        blockMgr = BlockMgrCache.create(readBlockCacheSize, writeBlockCacheSize, blockMgr) ;
+        BlockMgr blockMgr = wrapFileAccess(file, blockSize) ;
+        blockMgr = addCache(blockMgr, readBlockCacheSize, writeBlockCacheSize) ;
         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)
+    /** Create a Block Manager using direct access, no caching, no nothing. */
+    public static BlockMgr createStdFileNoCache(String filename, int blockSize) {
+        BlockAccess blockAccess = new BlockAccessDirect(filename, blockSize) ;
+        BlockMgr blockMgr = new BlockMgrFileAccess(blockAccess, blockSize) ;
+        return 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()) ;
+            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. 
+
+    /**
+     * 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() )
@@ -148,20 +141,10 @@ public class BlockMgrFactory
         return addCache(blockMgr, readBlockCacheSize, writeBlockCacheSize) ;
     }
 
-    /** Create a Block Manager using direct access, no caching, no nothing. */
-    public static BlockMgr createStdFileNoCache(String filename, int blockSize)
-    {
-        BlockAccess blockAccess = new BlockAccessDirect(filename, blockSize) ;
-        BlockMgr blockMgr = new BlockMgrFileAccess(blockAccess, blockSize) ;
-        return blockMgr ;
-    }
-    
-    private static BlockMgr wrapFileAccess(BlockAccess blockAccess, int blockSize)
-    {
+    private static BlockMgr wrapFileAccess(BlockAccess blockAccess, int blockSize) {
         BlockMgr blockMgr = new BlockMgrFileAccess(blockAccess, blockSize) ;
-        // This is a temporary fix to the problem 
+        // This is a temporary fix to the problem
         blockMgr = new BlockMgrFreeChain(blockMgr) ;
         return blockMgr ;
-        
     }
 }

Added: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/base/block/BlockParams.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/base/block/BlockParams.java?rev=1623401&view=auto
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/base/block/BlockParams.java (added)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/base/block/BlockParams.java Mon Sep  8 14:04:30 2014
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.hp.hpl.jena.tdb.base.block;
+
+public interface BlockParams {
+    public FileMode getFileMode() ;
+    public int getBlockSize() ;
+    public int getBlockReadCacheSize() ;
+    public int getBlockWriteCacheSize() ;
+}
+

Modified: 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=1623401&r1=1623400&r2=1623401&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexParams.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/index/IndexParams.java Mon Sep  8 14:04:30 2014
@@ -16,11 +16,12 @@
 
 package com.hp.hpl.jena.tdb.index;
 
+import com.hp.hpl.jena.tdb.base.block.BlockParams ;
 import com.hp.hpl.jena.tdb.base.block.FileMode ;
 
-public interface IndexParams {
-    public FileMode getFileMode() ;
-    public int getBlockSize() ;
-    public int getBlockReadCacheSize() ;
-    public int getBlockWriteCacheSize() ;
+public interface IndexParams extends BlockParams {
+    @Override public FileMode getFileMode() ;
+    @Override public int getBlockSize() ;
+    @Override public int getBlockReadCacheSize() ;
+    @Override public int getBlockWriteCacheSize() ;
 }

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=1623401&r1=1623400&r2=1623401&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 Mon Sep  8 14:04:30 2014
@@ -55,19 +55,17 @@ public class IndexFactoryBPlusTree imple
         if ( params.getCalcBlockSize() > idxParams.getBlockSize() )
             throw new TDBException("Calculated block size is greater than required size") ;
         
-        BlockMgr blkMgrNodes = createBlockMgr(fileset, Names.bptExtTree, blockSize, readCacheSize, writeCacheSize) ;
-        BlockMgr blkMgrRecords = createBlockMgr(fileset, Names.bptExtRecords, blockSize, readCacheSize, writeCacheSize) ;
+        BlockMgr blkMgrNodes = createBlockMgr(fileset, Names.bptExtTree, idxParams) ;
+        BlockMgr blkMgrRecords = createBlockMgr(fileset, Names.bptExtRecords, idxParams) ;
         return BPlusTree.create(params, blkMgrNodes, blkMgrRecords) ;
     }
     
-    static BlockMgr createBlockMgr(FileSet fileset, String filename, int blockSize,
-                                   int readCacheSize, int writeCacheSize)
+    static BlockMgr createBlockMgr(FileSet fileset, String filename, IndexParams idxParams)
     {
         if ( fileset.isMem() )
-            return BlockMgrFactory.createMem(filename, blockSize) ;
+            return BlockMgrFactory.createMem(filename, idxParams.getBlockSize()) ;
         
         String fnNodes = fileset.filename(filename) ;
-        return BlockMgrFactory.createFile(fnNodes, blockSize, 
-                                          readCacheSize, writeCacheSize) ;
+        return BlockMgrFactory.createFile(fnNodes, idxParams) ;
     }
 }

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=1623401&r1=1623400&r2=1623401&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 Mon Sep  8 14:04:30 2014
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package com.hp.hpl.jena.tdb.index.factories;
+package com.hp.hpl.jena.tdb.index.factories ;
 
 import com.hp.hpl.jena.tdb.base.block.BlockMgr ;
 import com.hp.hpl.jena.tdb.base.block.BlockMgrFactory ;
@@ -29,40 +29,26 @@ import com.hp.hpl.jena.tdb.index.IndexFa
 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 ;
 
-/** Index factory for extendible hash tables.
- *  Only an index, not a range index
+/**
+ * Index factory for extensible hash tables. Only an index, not a range index
  */
 
-public class IndexFactoryExtHash implements IndexFactory
-{
-    private final int blockSize ;
-
-    public IndexFactoryExtHash()
-    { this(SystemTDB.BlockSize) ; }
-    
-    public IndexFactoryExtHash(int blockSize)
-    {
-        this.blockSize = blockSize ;
-    }
-    
+public class IndexFactoryExtHash implements IndexFactory {
+    public IndexFactoryExtHash() {}
+
     @Override
-    public Index createIndex(FileSet fileset, RecordFactory recordFactory, IndexParams params)
-    {
+    public Index createIndex(FileSet fileset, RecordFactory recordFactory, IndexParams idxParams) {
         String fnDictionary = fileset.filename(Names.extHashExt) ;
         PlainFile dictionary = FileFactory.createPlainFileDisk(fnDictionary) ;
-        
+
         String fnBuckets = fileset.filename(Names.extHashBucketExt) ;
-        BlockMgr mgr =  createBlockMgr(fnBuckets, blockSize) ;
+        BlockMgr mgr = createBlockMgr(fnBuckets, idxParams) ;
         ExtHash eHash = new ExtHash(dictionary, recordFactory, mgr) ;
         return eHash ;
     }
-    
-    protected BlockMgr createBlockMgr(String filename, int blockSize)
-    {
-        return BlockMgrFactory.createFile(filename, blockSize, 
-                                          SystemTDB.BlockReadCacheSize,
-                                          SystemTDB.BlockWriteCacheSize) ;
+
+    protected BlockMgr createBlockMgr(String filename, IndexParams idxParams) {
+        return BlockMgrFactory.createFile(filename, idxParams) ;
     }
 }

Modified: 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=1623401&r1=1623400&r2=1623401&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BuilderIndex.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/BuilderIndex.java Mon Sep  8 14:04:30 2014
@@ -20,7 +20,6 @@ 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 ;
@@ -32,8 +31,7 @@ import com.hp.hpl.jena.tdb.index.bplustr
 import com.hp.hpl.jena.tdb.index.bplustree.BPlusTreeParams ;
 import com.hp.hpl.jena.tdb.sys.Names ;
 
-/** Building indexes, blockMgr and files */ 
-
+/** Building indexes, blockMgr and object files */ 
 public class BuilderIndex {
     public static class IndexBuilderStd implements IndexBuilder
     {
@@ -115,44 +113,15 @@ public class BuilderIndex {
         }
     }
 
-    public static class BlockMgrBuilderStd/*Base*/ implements BlockMgrBuilder
+    public static class BlockMgrBuilderStd 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 ;
+            return BlockMgrFactory.create(fileset, ext, indexParams) ;
         }
     }
-
 }
 

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=1623401&r1=1623400&r2=1623401&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 Mon Sep  8 14:04:30 2014
@@ -18,10 +18,16 @@
 
 package com.hp.hpl.jena.tdb.setup ;
 
+import java.io.File ;
+import java.io.IOException ;
 import java.util.HashMap ;
 import java.util.Map ;
+import java.util.Properties ;
 
+import org.apache.jena.atlas.io.IO ;
 import org.apache.jena.atlas.lib.ColumnMap ;
+import org.apache.jena.atlas.lib.FileOps ;
+import org.apache.jena.atlas.lib.PropertyUtils ;
 import org.apache.jena.atlas.lib.StrUtils ;
 import org.slf4j.Logger ;
 
@@ -56,6 +62,7 @@ public class DatasetBuilderStd implement
     private NodeTableBuilder            nodeTableBuilder ;
     private TupleIndexBuilder           tupleIndexBuilder ;
 
+    // XXX Should not be an object field. 
     private SystemParams                params ;
 
     private Map<FileRef, BlockMgr>      blockMgrs      = new HashMap<>() ;
@@ -63,7 +70,8 @@ public class DatasetBuilderStd implement
     private Map<FileRef, NodeTable>     nodeTables     = new HashMap<>() ;
 
     public static DatasetGraphTDB create(Location location) {
-        DatasetBuilderStd x = new DatasetBuilderStd(SystemParams.getDftSystemParams()) ;
+        SystemParams params = paramsForLocation(location) ;
+        DatasetBuilderStd x = new DatasetBuilderStd(params) ;
         x.standardSetup() ;
         return x.build(location) ;
     }
@@ -78,8 +86,11 @@ public class DatasetBuilderStd implement
         return x ;
     }
 
-    public static DatasetBuilderStd stdBuilder(SystemParams params) {
-        DatasetBuilderStd x = new DatasetBuilderStd(params) ;
+    /** Create a building : if database settings already at the  location,
+     * use those otherwiese use the provided defaults.  
+     */
+    public static DatasetBuilderStd stdBuilder(SystemParams dftParams) {
+        DatasetBuilderStd x = new DatasetBuilderStd(dftParams) ;
         x.standardSetup() ;
         return x ;
     }
@@ -88,7 +99,11 @@ public class DatasetBuilderStd implement
         this(SystemParams.getDftSystemParams()) ;
     }
 
+    // XXX Take params out of constructor.  They are location sensitive.
     protected DatasetBuilderStd(SystemParams params) {
+        //Objects.requireNonNull(params) ;
+        if ( params == null )
+            params = SystemParams.getDftSystemParams() ;
         this.params = params ;
     }
 
@@ -116,6 +131,54 @@ public class DatasetBuilderStd implement
         set(nodeTableBuilder, tupleIndexBuilder) ;
     }
 
+    private static SystemParams paramsForLocation(Location location) {
+        if ( location.exists(DB_CONFIG_FILE) ) {
+            log.debug("Existing configuration file found") ;
+            Properties properties = new Properties() ;
+            try { 
+                PropertyUtils.loadFromFile(properties, DB_CONFIG_FILE) ;
+            } catch (IOException ex) { IO.exception(ex) ; throw new TDBException("Bad configuration file", ex) ; }
+        }
+        return SystemParams.getDftSystemParams() ;
+    }
+
+//    private void checkIfConfig(Location location) {
+//    }
+
+    private void checkIfNew(Location location) {
+        if ( location.isMem() ) {
+            return ;
+        }
+        
+        if ( FileOps.existsAnyFiles(location.getDirectoryPath()) ) {
+            
+        }
+
+        if ( location.exists(DB_CONFIG_FILE) ) {
+            log.debug("Existing config file") ;
+            return ;
+        }
+        
+    }
+    
+    private void checkConfiguration() { } 
+    
+    private static void checkLocation(Location location) { 
+        if ( location.isMem() )
+            return ;
+        String dirname = location.getDirectoryPath() ;
+        File dir = new File(dirname) ;
+        // File location.
+        if ( ! dir.exists() )
+            error(log, "Does not exist: "+dirname) ;
+        if ( ! dir.isDirectory() )
+            error(log, "Not a directory: "+dirname) ;
+        if ( ! dir.canRead() )
+            error(log, "Directory not readable: "+dirname) ;
+        if ( ! dir.canWrite() )
+            error(log, "Directory not writeable: "+dirname) ;
+    }
+
     private void standardSetup() {
         ObjectFileBuilder objectFileBuilder = new BuilderIndex.ObjectFileBuilderStd() ;
         BlockMgrBuilder blockMgrBuilder = new BuilderIndex.BlockMgrBuilderStd() ;
@@ -129,21 +192,35 @@ public class DatasetBuilderStd implement
     public DatasetGraphTDB build(Location location) {
         // Ensure that there is global synchronization
         synchronized (DatasetBuilderStd.class) {
+            log.debug("Build database: "+location.getDirectoryPath()) ;
+            checkIfNew(location) ;
+            checkLocation(location) ;
             return _build(location, params, true, null) ;
         }
     }
 
+    private static String DB_CONFIG_FILE = "tdb.cfg" ; 
+    
     // Main engine for building.
     // Called by DatasetBuilderTxn
     // XXX Rework - provide a cloning constructor (copies maps).
     // Or "reset"
-    public DatasetGraphTDB _build(Location location, SystemParams _params, boolean readonly,
-                                  ReorderTransformation _transform) {
-        if ( _params != null )
-            params = _params ;
-        init(location) ;
+    public synchronized DatasetGraphTDB _build(Location location, SystemParams _params, boolean writeable,
+                                               ReorderTransformation _transform) {
+        // This should create a new DatabseBuilderStd as a clone and use that!
+        SystemParams dftParams = params ;
+        try { 
+            if ( _params != null )
+                params = _params ;
+            init(location) ;
+            return buildWorker(location, writeable, _transform) ;
+        } finally {
+            params = dftParams ;
+        }
+    }
+    
+    private DatasetGraphTDB buildWorker(Location location, boolean writeable, ReorderTransformation _transform) {
         DatasetControl policy = createConcurrencyPolicy() ;
-
         NodeTable nodeTable = makeNodeTable(location, params) ;
         TripleTable tripleTable = makeTripleTable(location, nodeTable, policy) ;
         QuadTable quadTable = makeQuadTable(location, nodeTable, policy) ;
@@ -151,7 +228,7 @@ public class DatasetBuilderStd implement
 
         ReorderTransformation transform = (_transform == null) ? chooseReorderTransformation(location) : _transform ;
 
-        StorageConfig storageConfig = new StorageConfig(location, params, readonly, blockMgrs, bufferChannels,
+        StorageConfig storageConfig = new StorageConfig(location, params, writeable, blockMgrs, bufferChannels,
                                                         nodeTables) ;
         DatasetGraphTDB dsg = new DatasetGraphTDB(tripleTable, quadTable, prefixes, transform, storageConfig) ;
         // TDB does filter placement on BGPs itself.

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/SystemParamsBuilder.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/SystemParamsBuilder.java?rev=1623401&r1=1623400&r2=1623401&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/SystemParamsBuilder.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/setup/SystemParamsBuilder.java Mon Sep  8 14:04:30 2014
@@ -27,60 +27,60 @@ public class SystemParamsBuilder {
     
     /** Database and query configuration */ 
     
-    public static String fFileMode             = "FileMode" ;
+    // public static String fFileMode             = "FileMode" ;
     private FileMode     fileMode              = SystemTDB.fileMode() ;
 
-    public static String fReadCacheSize        = "blockReadCacheSize" ;
+    // public static String fReadCacheSize        = "blockReadCacheSize" ;
     private int          blockReadCacheSize    = SystemTDB.BlockReadCacheSize ;
 
-    public static String fWriteCacheSize       = "blockWriteCacheSize" ;
+    // public static String fWriteCacheSize       = "blockWriteCacheSize" ;
     private int          blockWriteCacheSize   = SystemTDB.BlockWriteCacheSize ;
 
-    public static String fNode2NodeIdCacheSize = "Node2NodeIdCacheSize" ;
+    // public static String fNode2NodeIdCacheSize = "Node2NodeIdCacheSize" ;
     private int          Node2NodeIdCacheSize  = SystemTDB.Node2NodeIdCacheSize ;
 
-    public static String fNodeId2NodeCacheSize = "NodeId2NodeCacheSize" ;
+    // public static String fNodeId2NodeCacheSize = "NodeId2NodeCacheSize" ;
     private int          NodeId2NodeCacheSize  = SystemTDB.NodeId2NodeCacheSize ;
 
-    public static String fNodeMissCacheSize    = "NodeMissCacheSize" ;
+    // public static String fNodeMissCacheSize    = "NodeMissCacheSize" ;
     private int          NodeMissCacheSize     = SystemTDB.NodeMissCacheSize ;
 
     /** Database layout - ignored after a database is created */
 
-    public static String fBlockSize            = "blockSize" ;
+    // public static String fBlockSize            = "blockSize" ;
     private int          blockSize             = SystemTDB.BlockSize ;
 
-    public static String fIndexNode2Id         = "indexNode2Id" ;
+    // public static String fIndexNode2Id         = "indexNode2Id" ;
     private String       indexNode2Id          = Names.indexNode2Id ;
 
-    public static String fIndexId2Node         = "indexId2Node" ;
+    // public static String fIndexId2Node         = "indexId2Node" ;
     private String       indexId2Node          = Names.indexId2Node ;
 
-    public static String fPrimaryIndexTriples  = "primaryIndexTriples" ;
+    // public static String fPrimaryIndexTriples  = "primaryIndexTriples" ;
     private String       primaryIndexTriples   = Names.primaryIndexTriples ;
 
-    public static String fTripleIndexes        = "tripleIndexes" ;
+    // public static String fTripleIndexes        = "tripleIndexes" ;
     private String[]     tripleIndexes         = Names.tripleIndexes ;
 
-    public static String fPrimaryIndexQuads    = "primaryIndexQuads" ;
+    // public static String fPrimaryIndexQuads    = "primaryIndexQuads" ;
     private String       primaryIndexQuads     = Names.primaryIndexQuads ;
 
-    public static String fQuadIndexes          = "quadIndexes" ;
+    // public static String fQuadIndexes          = "quadIndexes" ;
     private String[]     quadIndexes           = Names.quadIndexes ;
 
-    public static String fPrimaryIndexPrefix   = "primaryIndexPrefix" ;
+    // public static String fPrimaryIndexPrefix   = "primaryIndexPrefix" ;
     private String       primaryIndexPrefix    = Names.primaryIndexPrefix ;
 
-    public static String fPrefixIndexes        = "prefixIndexes" ;
+    // public static String fPrefixIndexes        = "prefixIndexes" ;
     private String[]     prefixIndexes         = Names.prefixIndexes ;
 
-    public static String fIndexPrefix          = "indexPrefix" ;
+    // public static String fIndexPrefix          = "indexPrefix" ;
     private String       indexPrefix           = Names.indexPrefix ;
 
-    public static String fPrefixNode2Id        = "prefixNode2Id" ;
+    // public static String fPrefixNode2Id        = "prefixNode2Id" ;
     private String       prefixNode2Id         = Names.prefixNode2Id ;
 
-    public static String fPrefixId2Node        = "prefixId2Node" ;
+    // public static String fPrefixId2Node        = "prefixId2Node" ;
     private String       prefixId2Node         = Names.prefixId2Node ;
 
     public static SystemParamsBuilder create() { return new SystemParamsBuilder() ; }

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/sys/FileRef.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/sys/FileRef.java?rev=1623401&r1=1623400&r2=1623401&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/sys/FileRef.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/sys/FileRef.java Mon Sep  8 14:04:30 2014
@@ -49,30 +49,6 @@ public class FileRef
     static Map<String, Integer> name2id = new HashMap<>() ;
     static Map<Integer, FileRef> id2name = new TreeMap<>() ;// new HashMap<Integer, FileRef>() ;
     
-    static private void printTable()
-    {
-        for ( String name : Names.tripleIndexes )
-            bTree(name) ;
-        for ( String name : Names.quadIndexes )
-            bTree(name) ;
-//        // Not the name of the index.
-//        for ( String name : Names.prefixIndexes )
-//            bTree(name) ;
-
-        bTree(Names.prefixId2Node) ;
-        bTree(Names.prefixNode2Id) ;
-        file(Names.indexId2Node+".dat") ;
-        bTree(Names.indexNode2Id) ;
-        bTree(Names.indexPrefix) ;
-        
-        for ( Map.Entry<Integer, FileRef> e : id2name.entrySet() )
-        {
-            System.out.printf("    add(%s+idxOffset , %s) ;\n", e.getKey()-idxOffset, '"'+e.getValue().filename+'"') ;
-        }
-        
-        
-    }
-    
     static {
         //printTable() ;
         add(journalIdx , journalName) ;