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

svn commit: r1622899 - in /jena/trunk/jena-tdb/src: main/java/com/hp/hpl/jena/tdb/index/ main/java/com/hp/hpl/jena/tdb/setup/ main/java/com/hp/hpl/jena/tdb/store/ main/java/com/hp/hpl/jena/tdb/store/nodetable/ main/java/com/hp/hpl/jena/tdb/sys/ test/ja...

Author: andy
Date: Sat Sep  6 18:04:59 2014
New Revision: 1622899

URL: http://svn.apache.org/r1622899
Log:
Step in refactoring of DB building

Removed:
    jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/store/nodetable/NodeTableSink.java
Modified:
    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/setup/B.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/NodeTableCache.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/sys/SetupTDB.java
    jena/trunk/jena-tdb/src/test/java/com/hp/hpl/jena/tdb/graph/TestPrefixMappingTDB.java
    jena/trunk/jena-tdb/src/test/java/com/hp/hpl/jena/tdb/store/tupletable/TestTupleIndexRecord.java

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=1622899&r1=1622898&r2=1622899&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 18:04:59 2014
@@ -20,7 +20,6 @@ package com.hp.hpl.jena.tdb.index;
 
 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.factories.IndexFactoryBPlusTree ;
 import com.hp.hpl.jena.tdb.index.factories.IndexFactoryExtHash ;
@@ -28,7 +27,7 @@ import com.hp.hpl.jena.tdb.sys.SystemTDB
 
 /** A policy holder for making indexes and range indexes.  */
  
-public class IndexBuilder
+class IndexBuilder
 {
     // Migrate to be a general policy place for files.
     
@@ -108,12 +107,12 @@ public class IndexBuilder
         builderRangeIndex = rangeIndexBuilder ;
     }
     
-    public Index newIndex(FileSet fileset, RecordFactory factory, IndexParams params)
+    public Index newIndex$(FileSet fileset, RecordFactory factory, IndexParams params)
     {
         return factoryIndex.createIndex(fileset, factory, params) ;
     }
     
-    public RangeIndex newRangeIndex(FileSet fileset , RecordFactory factory, IndexParams params)
+    public RangeIndex newRangeIndex$(FileSet fileset , RecordFactory factory, IndexParams params)
     {
         return builderRangeIndex.createRangeIndex(fileset, factory, params) ;
     }

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=1622899&r1=1622898&r2=1622899&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 18:04:59 2014
@@ -18,20 +18,37 @@
 
 package com.hp.hpl.jena.tdb.setup;
 
+import org.apache.jena.atlas.lib.NotImplemented ;
+
 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 ;
+import com.hp.hpl.jena.tdb.setup.BuilderIndex.BlockMgrBuilderStd ;
 
 public class B {
     
-    public static IndexBuilder createIndexBuilder() { return null ; }
-    public static IndexBuilder createIndexBuilderMem() { return null ; }
-    public static RangeIndexBuilder createRangeIndexBuilder() { return null ; }
-    public static RangeIndexBuilder createRangeIndexBuilderMem() { return null ; }
+    public static IndexBuilder createIndexBuilderMem() { 
+        return createIndexBuilder(createRangeIndexBuilderMem()) ;
+    }
     
+    public static IndexBuilder createIndexBuilder(final RangeIndexBuilder other) 
+    { 
+        return new IndexBuilder() {
+            @Override
+            public Index buildIndex(FileSet fileSet, RecordFactory recordfactory, IndexParams indexParams) {
+                return other.buildRangeIndex(fileSet, recordfactory, indexParams) ;
+            }
+        } ;
+    }
+
+    public static RangeIndexBuilder createRangeIndexBuilderMem() {
+        BlockMgrBuilder blockMgrBuilderNodes = new BlockMgrBuilderStd() ;
+        BlockMgrBuilder blockMgrBuilderRecords = new BlockMgrBuilderStd() ;
+        return new BuilderIndex.RangeIndexBuilderStd(blockMgrBuilderNodes, blockMgrBuilderRecords) ;
+    }
     
     // c.f. setupTDB
     // BlockMgrSync needed?  Outer sync?

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=1622899&r1=1622898&r2=1622899&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 18:04:59 2014
@@ -33,9 +33,7 @@ import com.hp.hpl.jena.sparql.graph.Grap
 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.IndexParams ;
-import com.hp.hpl.jena.tdb.setup.SystemParams ;
+import com.hp.hpl.jena.tdb.setup.* ;
 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 ;
@@ -51,25 +49,29 @@ public class DatasetPrefixesTDB implemen
     // Index on GPU and a nodetable.
     // The nodetable is itself an index and a data file.
     
-    static final ColumnMap colMap = new ColumnMap(Names.primaryIndexPrefix, Names.primaryIndexPrefix) ;
     static final RecordFactory factory = new RecordFactory(3*NodeId.SIZE, 0) ;
     static final String unamedGraphURI = "" ; //Quad.defaultGraphNode.getURI() ;
     
     // Use NodeTupleTableView?
     private final NodeTupleTable nodeTupleTable ;
     
-    @Deprecated
-    public static DatasetPrefixesTDB createTesting(Location location, DatasetControl policy) 
-    { return create(IndexBuilder.get(), location, policy) ; }
+    /** Testing - dataset prefixes in-memory */
+    public static DatasetPrefixesTDB createTesting() { 
+        return createTesting(Location.mem(), new DatasetControlMRSW()) ;
+    }
+    
+    public static DatasetPrefixesTDB createTesting(Location location, DatasetControl policy) {
+        return new DatasetPrefixesTDB(B.createRangeIndexBuilderMem(), 
+                                      B.createIndexBuilderMem(),
+                                      location, policy) ; 
+    }
     
     @Deprecated
-    private static DatasetPrefixesTDB create(IndexBuilder indexBuilder, Location location, DatasetControl policy)
-    { return new DatasetPrefixesTDB(indexBuilder, location, policy) ; }
-
-    @Deprecated
-    private DatasetPrefixesTDB(IndexBuilder indexBuilder, Location location, DatasetControl policy)
+    private DatasetPrefixesTDB(RangeIndexBuilder indexRangeBuilder, 
+                               IndexBuilder indexBuilder,
+                               Location location, DatasetControl policy)
     {
-        IndexParams indexParams = SystemParams.getDftSystemParams() ;
+        SystemParams params = 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
@@ -77,9 +79,15 @@ 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, indexParams)) ;
-        TupleIndex[] indexes = { index } ;
+        TupleIndex[] indexes = new TupleIndex[params.getPrefixIndexes().length] ;
+        
+        int i = 0 ;
+        for ( String indexName : params.getPrefixIndexes() ) {
+            ColumnMap colMap = new ColumnMap("GPU", params.getPrimaryIndexPrefix()) ;
+            TupleIndex index = new TupleIndexRecord(3, colMap, indexName, factory, 
+                                                    indexRangeBuilder.buildRangeIndex(filesetGPU, factory, params)) ;
+            indexes[i++] = index ;
+        }
         
         // Node table.
         FileSet filesetNodeTableIdx = null ;
@@ -90,7 +98,13 @@ public class DatasetPrefixesTDB implemen
         if ( location != null )
             filesetNodeTable = new FileSet(location, Names.prefixId2Node) ;
         
-        NodeTable nodes = NodeTableFactory.create(indexBuilder, filesetNodeTable, filesetNodeTableIdx, -1, -1, -1) ;
+        SystemParamsBuilder spBuild = new SystemParamsBuilder() ;
+        spBuild.node2NodeIdCacheSize(-1) ;
+        spBuild.nodeId2NodeCacheSize(-1) ;
+        spBuild.nodeMissCacheSize(-1) ;
+        SystemParams params2 = spBuild.buildParams() ;
+        // No cache.
+        NodeTable nodes = NodeTableFactory.create(indexBuilder, filesetNodeTable, filesetNodeTableIdx, params2) ;
         nodeTupleTable = new NodeTupleTableConcrete(3, indexes, nodes, policy) ;
     }
 
@@ -101,14 +115,6 @@ public class DatasetPrefixesTDB implemen
         this.nodeTupleTable = new NodeTupleTableConcrete(3, indexes, nodes, policy) ;
     }
     
-    private DatasetPrefixesTDB()
-    {
-        this(IndexBuilder.mem(), Location.mem(), new DatasetControlMRSW()) ;
-    }
-    
-    /** Testing - dataset prefixes in-memory */
-    public static DatasetPrefixesTDB testing() { return new DatasetPrefixesTDB() ; }
-    
     // Use DatasetControl
 //    public boolean isReadOnly() { return nodeTupleTable.isReadOnly() ; }
 //    public void setReadOnly(boolean mode) { nodeTupleTable.setReadOnly(mode) ; }

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/store/nodetable/NodeTableCache.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/store/nodetable/NodeTableCache.java?rev=1622899&r1=1622898&r2=1622899&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/store/nodetable/NodeTableCache.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/store/nodetable/NodeTableCache.java Sat Sep  6 18:04:59 2014
@@ -27,9 +27,9 @@ import org.apache.jena.atlas.lib.CacheSe
 import org.apache.jena.atlas.lib.Pair ;
 import org.apache.jena.atlas.logging.Log ;
 
-
 import com.hp.hpl.jena.graph.Node ;
 import com.hp.hpl.jena.tdb.TDBException ;
+import com.hp.hpl.jena.tdb.setup.SystemParams ;
 import com.hp.hpl.jena.tdb.store.NodeId ;
 
 /** Cache wrapper around a NodeTable.  
@@ -49,6 +49,14 @@ public class NodeTableCache implements N
     private NodeTable baseTable ;
     private Object lock = new Object() ;
 
+    public static NodeTable create(NodeTable nodeTable, SystemParams params) {
+        int nodeToIdCacheSize = params.getNode2NodeIdCacheSize() ;
+        int idToNodeCacheSize = params.getNodeId2NodeCacheSize() ;
+        if ( nodeToIdCacheSize <= 0 && idToNodeCacheSize <= 0 )
+            return nodeTable ;
+        return new NodeTableCache(nodeTable, nodeToIdCacheSize, idToNodeCacheSize, params.getNodeMissCacheSize()) ;
+    }
+
     public static NodeTable create(NodeTable nodeTable, int nodeToIdCacheSize, int idToNodeCacheSize, int nodeMissesCacheSize)
     {
         if ( nodeToIdCacheSize <= 0 && idToNodeCacheSize <= 0 )

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=1622899&r1=1622898&r2=1622899&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 18:04:59 2014
@@ -23,8 +23,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.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.B ;
+import com.hp.hpl.jena.tdb.setup.IndexBuilder ;
 import com.hp.hpl.jena.tdb.setup.SystemParams ;
 import com.hp.hpl.jena.tdb.sys.Names ;
 import com.hp.hpl.jena.tdb.sys.SystemTDB ;
@@ -35,6 +35,7 @@ public class NodeTableFactory
     @Deprecated
     public static NodeTable create(IndexBuilder indexBuilder, Location location)
     {
+        // XXX FIX up names - add SystemParams arg
         // The node table (id to node).
         FileSet filesetNodeTable = null ;
         if ( location != null )
@@ -47,53 +48,39 @@ public class NodeTableFactory
         
         SystemParams params = SystemParams.getDftSystemParams() ;
         
-        return  create(indexBuilder, filesetNodeTable, filesetIdx,
-                       params.getNode2NodeIdCacheSize() ,
-                       params.getNodeId2NodeCacheSize() ,
-                       params.getNodeMissCacheSize() ) ;
+        return create(indexBuilder, filesetNodeTable, filesetIdx, params) ;
     }
 
     /** Custom node table */
     public static NodeTable create(IndexBuilder indexBuilder, 
                                    FileSet fsIdToNode, FileSet fsNodeToId,
-                                   int nodeToIdCacheSize, int idToNodeCacheSize, int nodeMissCacheSize)
+                                   SystemParams params)
     {
         String filename = fsIdToNode.filename(Names.extNodeData) ;
         
-        // XXX Temp
-        IndexParams indexparams = SystemParams.getDftSystemParams() ;
-        
         if ( fsNodeToId.isMem() )
         {
-            Index nodeToId = indexBuilder.newIndex(FileSet.mem(), SystemTDB.nodeRecordFactory, indexparams) ;
+            Index nodeToId = indexBuilder.buildIndex(fsNodeToId, SystemTDB.nodeRecordFactory, params) ;
             ObjectFile objects = FileFactory.createObjectFileMem(filename) ;
             NodeTable nodeTable = new NodeTableNative(nodeToId, objects) ;
             
-            nodeTable = NodeTableCache.create(nodeTable, 100, 100, 100) ; 
+            nodeTable = NodeTableCache.create(nodeTable, 100, 100, 10) ; 
             nodeTable =  NodeTableInline.create(nodeTable) ;
             
             return nodeTable ;
-            
-            //return NodeTableIndex.createMem(indexBuilder) ;
         }
         
-        Index nodeToId = indexBuilder.newIndex(fsNodeToId, SystemTDB.nodeRecordFactory, indexparams) ;
+        Index nodeToId = indexBuilder.buildIndex(fsNodeToId, SystemTDB.nodeRecordFactory, params) ;
         // Node table.
         ObjectFile objects = FileFactory.createObjectFileDisk(filename);
         NodeTable nodeTable = new NodeTableNative(nodeToId, objects) ;
-        nodeTable = NodeTableCache.create(nodeTable, nodeToIdCacheSize, idToNodeCacheSize, nodeMissCacheSize) ; 
+        nodeTable = NodeTableCache.create(nodeTable, params) ; 
         nodeTable = NodeTableInline.create(nodeTable) ;
         return nodeTable ;
         
     }
 
-    public static NodeTable createMem(IndexBuilder indexBuilder)
-    {
-        return create(indexBuilder, FileSet.mem(), FileSet.mem(), 100, 100, 10) ;
-    }
-    
-    public static NodeTable createSink(IndexBuilder indexBuilder, Location location)
-    {
-        return new NodeTableSink() ;
+    public static NodeTable createMem() {
+        return create(B.createIndexBuilderMem(), FileSet.mem(), FileSet.mem(), SystemParams.getDftSystemParams()) ;
     }
 }

Modified: jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/sys/SetupTDB.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/sys/SetupTDB.java?rev=1622899&r1=1622898&r2=1622899&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/sys/SetupTDB.java (original)
+++ jena/trunk/jena-tdb/src/main/java/com/hp/hpl/jena/tdb/sys/SetupTDB.java Sat Sep  6 18:04:59 2014
@@ -34,17 +34,12 @@ import com.hp.hpl.jena.tdb.base.file.Loc
 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.IndexBuilder ;
 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.setup.DatasetBuilderStd ;
 import com.hp.hpl.jena.tdb.setup.SystemParams ;
-import com.hp.hpl.jena.tdb.store.DatasetGraphTDB ;
-import com.hp.hpl.jena.tdb.store.DatasetPrefixesTDB ;
-import com.hp.hpl.jena.tdb.store.NodeId ;
-import com.hp.hpl.jena.tdb.store.QuadTable ;
-import com.hp.hpl.jena.tdb.store.TripleTable ;
+import com.hp.hpl.jena.tdb.store.* ;
 import com.hp.hpl.jena.tdb.store.nodetable.* ;
 import com.hp.hpl.jena.tdb.store.tupletable.TupleIndex ;
 import com.hp.hpl.jena.tdb.store.tupletable.TupleIndexRecord ;
@@ -198,7 +193,7 @@ public class SetupTDB
     public static NodeTable makeNodeTableBase(Location location, String indexNode2Id, String indexId2Node)
     {
         if (location.isMem()) 
-            return NodeTableFactory.createMem(IndexBuilder.mem()) ;
+            return NodeTableFactory.createMem() ;
 
         // -- make id to node mapping -- Names.indexId2Node
         FileSet fsIdToNode = new FileSet(location, indexId2Node) ;

Modified: jena/trunk/jena-tdb/src/test/java/com/hp/hpl/jena/tdb/graph/TestPrefixMappingTDB.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-tdb/src/test/java/com/hp/hpl/jena/tdb/graph/TestPrefixMappingTDB.java?rev=1622899&r1=1622898&r2=1622899&view=diff
==============================================================================
--- jena/trunk/jena-tdb/src/test/java/com/hp/hpl/jena/tdb/graph/TestPrefixMappingTDB.java (original)
+++ jena/trunk/jena-tdb/src/test/java/com/hp/hpl/jena/tdb/graph/TestPrefixMappingTDB.java Sat Sep  6 18:04:59 2014
@@ -49,7 +49,7 @@ public class TestPrefixMappingTDB extend
     @Override
     protected PrefixMapping create()
     {
-        last = DatasetPrefixesTDB.testing() ;
+        last = DatasetPrefixesTDB.createTesting() ;
         return view() ;
     }
 
@@ -61,7 +61,7 @@ public class TestPrefixMappingTDB extend
 
     @Test public void multiple1()
     {
-        DatasetPrefixesTDB prefixes = DatasetPrefixesTDB.testing() ;
+        DatasetPrefixesTDB prefixes = DatasetPrefixesTDB.createTesting() ;
         PrefixMapping pmap1 = prefixes.getPrefixMapping() ;
         PrefixMapping pmap2 = prefixes.getPrefixMapping("http://graph/") ;
         pmap1.setNsPrefix("x", "http://foo/") ;
@@ -71,7 +71,7 @@ public class TestPrefixMappingTDB extend
     
     @Test public void multiple2()
     {
-        DatasetPrefixesTDB prefixes = DatasetPrefixesTDB.testing() ;
+        DatasetPrefixesTDB prefixes = DatasetPrefixesTDB.createTesting() ;
         PrefixMapping pmap1 = prefixes.getPrefixMapping("http://graph/") ;  // Same
         PrefixMapping pmap2 = prefixes.getPrefixMapping("http://graph/") ;
         pmap1.setNsPrefix("x", "http://foo/") ;
@@ -80,7 +80,6 @@ public class TestPrefixMappingTDB extend
     }
     
     // Persistent.
-    @SuppressWarnings("deprecation")
     @Test public void persistent1()
     {
         String dir = ConfigTest.getTestingDir() ;
@@ -95,7 +94,6 @@ public class TestPrefixMappingTDB extend
     }
     
     // Persistent.
-    @SuppressWarnings("deprecation")
     @Test public void persistent2()
     {
         String dir = ConfigTest.getTestingDir() ;

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=1622899&r1=1622898&r2=1622899&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 18:04:59 2014
@@ -21,6 +21,7 @@ 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.IndexParams ;
@@ -37,7 +38,7 @@ public class TestTupleIndexRecord extend
     protected TupleIndexRecord create(String description)
     {
         IndexParams indexParams = SystemParams.getDftSystemParams() ; 
-        RangeIndex rIdx = B.buildRangeIndex(null, factory, indexParams) ;
+        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 ;