You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jena.apache.org by ca...@apache.org on 2011/11/04 22:45:42 UTC

svn commit: r1197798 - in /incubator/jena/Jena2/TDB/branches/hash-ids: src-dev/dev/ src/main/java/com/hp/hpl/jena/tdb/nodetable/ src/main/java/com/hp/hpl/jena/tdb/setup/ src/main/java/com/hp/hpl/jena/tdb/sys/ src/main/java/com/hp/hpl/jena/tdb/transaction/

Author: castagna
Date: Fri Nov  4 21:45:42 2011
New Revision: 1197798

URL: http://svn.apache.org/viewvc?rev=1197798&view=rev
Log:
NodeId need to go from 64 to 128 bits, tests are failing...

Added:
    incubator/jena/Jena2/TDB/branches/hash-ids/src-dev/dev/NodeTableNative2Run.java   (with props)
    incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/nodetable/NodeTableNative2.java   (with props)
Modified:
    incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/nodetable/NodeTableFactory.java
    incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/setup/Builder.java
    incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/sys/SetupTDB.java
    incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/transaction/NodeTableTrans.java

Added: incubator/jena/Jena2/TDB/branches/hash-ids/src-dev/dev/NodeTableNative2Run.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/branches/hash-ids/src-dev/dev/NodeTableNative2Run.java?rev=1197798&view=auto
==============================================================================
--- incubator/jena/Jena2/TDB/branches/hash-ids/src-dev/dev/NodeTableNative2Run.java (added)
+++ incubator/jena/Jena2/TDB/branches/hash-ids/src-dev/dev/NodeTableNative2Run.java Fri Nov  4 21:45:42 2011
@@ -0,0 +1,66 @@
+/**
+ * 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 dev;
+
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+
+import org.openjena.atlas.lib.Pair;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.tdb.base.file.FileFactory;
+import com.hp.hpl.jena.tdb.base.file.FileSet;
+import com.hp.hpl.jena.tdb.base.file.Location;
+import com.hp.hpl.jena.tdb.base.objectfile.ObjectFile;
+import com.hp.hpl.jena.tdb.base.record.Record;
+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.nodetable.NodeTable;
+import com.hp.hpl.jena.tdb.nodetable.NodeTableNative2;
+import com.hp.hpl.jena.tdb.store.NodeId;
+import com.hp.hpl.jena.tdb.sys.Names;
+import com.hp.hpl.jena.tdb.sys.SystemTDB;
+
+public class NodeTableNative2Run {
+
+	public static void main(String[] args) {
+		Location location = new Location("/tmp/tdb");
+		RecordFactory nodeTableRecordFactory = new RecordFactory(SystemTDB.SizeOfNodeId, SystemTDB.SizeOfNodeId) ;
+		Index nodeToIdIndex = IndexBuilder.createIndex(new FileSet(location, Names.indexNode2Id), nodeTableRecordFactory) ;
+		ObjectFile objects = FileFactory.createObjectFileDisk(Names.indexId2Node) ;
+		NodeTable nt = new NodeTableNative2(nodeToIdIndex, objects);
+		NodeId nodeId = nt.getAllocateNodeId(Node.createLiteral("Foo"));
+		nt.sync();
+		
+		System.out.println(nodeId);
+		
+		Iterator<Pair<Long, ByteBuffer>> iter = objects.all() ;
+		while ( iter.hasNext() ) {
+			System.out.println(iter.next()) ;
+		}
+		
+		Iterator<Record> iter2 = nodeToIdIndex.iterator() ;
+		while ( iter2.hasNext() ) {
+			System.out.println(iter2.next()) ;
+		}
+	}
+
+}
+

Propchange: incubator/jena/Jena2/TDB/branches/hash-ids/src-dev/dev/NodeTableNative2Run.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/nodetable/NodeTableFactory.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/nodetable/NodeTableFactory.java?rev=1197798&r1=1197797&r2=1197798&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/nodetable/NodeTableFactory.java (original)
+++ incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/nodetable/NodeTableFactory.java Fri Nov  4 21:45:42 2011
@@ -75,7 +75,7 @@ public class NodeTableFactory
         {
             Index nodeToId = indexBuilder.newIndex(FileSet.mem(), SystemTDB.nodeRecordFactory) ;
             ObjectFile objects = FileFactory.createObjectFileMem(filename) ;
-            NodeTable nodeTable = new NodeTableNative(nodeToId, objects) ;
+            NodeTable nodeTable = new NodeTableNative2(nodeToId, objects) ;
             
             nodeTable = NodeTableCache.create(nodeTable, 100, 100) ; 
             nodeTable =  NodeTableInline.create(nodeTable) ;
@@ -88,7 +88,7 @@ public class NodeTableFactory
         Index nodeToId = indexBuilder.newIndex(fsNodeToId, SystemTDB.nodeRecordFactory) ;
         // Node table.
         ObjectFile objects = FileFactory.createObjectFileDisk(filename);
-        NodeTable nodeTable = new NodeTableNative(nodeToId, objects) ;
+        NodeTable nodeTable = new NodeTableNative2(nodeToId, objects) ;
         nodeTable = NodeTableCache.create(nodeTable, nodeToIdCacheSize, idToNodeCacheSize) ; 
         nodeTable = NodeTableInline.create(nodeTable) ;
         return nodeTable ;

Added: incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/nodetable/NodeTableNative2.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/nodetable/NodeTableNative2.java?rev=1197798&view=auto
==============================================================================
--- incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/nodetable/NodeTableNative2.java (added)
+++ incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/nodetable/NodeTableNative2.java Fri Nov  4 21:45:42 2011
@@ -0,0 +1,262 @@
+/**
+ * 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.nodetable;
+
+import static com.hp.hpl.jena.tdb.lib.NodeLib.setHash ;
+
+import java.util.Iterator ;
+
+import org.openjena.atlas.iterator.Iter ;
+import org.openjena.atlas.iterator.Transform ;
+import org.openjena.atlas.lib.Bytes;
+import org.openjena.atlas.lib.Pair ;
+
+import com.hp.hpl.jena.graph.Node ;
+import com.hp.hpl.jena.tdb.TDBException ;
+import com.hp.hpl.jena.tdb.base.objectfile.ObjectFile ;
+import com.hp.hpl.jena.tdb.base.record.Record ;
+import com.hp.hpl.jena.tdb.index.Index ;
+import com.hp.hpl.jena.tdb.lib.NodeLib ;
+import com.hp.hpl.jena.tdb.store.Hash ;
+import com.hp.hpl.jena.tdb.store.NodeId ;
+
+public class NodeTableNative2 implements NodeTable
+{
+    // Assumes an StringFile and an Indexer, which may be an Index but allows
+    // this to be overriden for a direct use of BDB.
+
+    protected ObjectFile objects ;
+    protected Index nodeHashToOffset ;        // hash -> int
+    private boolean syncNeeded = false ;
+    
+    // Delayed construction - must call init explicitly.
+    protected NodeTableNative2() {}
+    
+    // Combined into one constructor.
+    public NodeTableNative2(Index nodeToId, ObjectFile objectFile)
+    {
+        this() ;
+        init(nodeToId, objectFile) ;
+    }
+    
+    protected void init(Index nodeToId, ObjectFile objectFile)
+    {
+        this.nodeHashToOffset = nodeToId ;
+        this.objects = objectFile;
+    }
+
+    // ---- Public interface for Node <==> NodeId
+
+    /** Get the Node for this NodeId, or null if none */
+    @Override
+    public Node getNodeForNodeId(NodeId id)
+    {
+        return _retrieveNodeByNodeId(id) ;
+    }
+
+    /** Find the NodeId for a node, or return NodeId.NodeDoesNotExist */ 
+    @Override
+    public NodeId getNodeIdForNode(Node node)  { return _idForNode(node, false) ; }
+
+    /** Find the NodeId for a node, allocating a new NodeId if the Node does not yet have a NodeId */ 
+    @Override
+    public NodeId getAllocateNodeId(Node node)  { return _idForNode(node, true) ; }
+
+    // ---- The worker functions
+    // Synchronization:
+    // accesIndex and readNodeFromTable
+    
+    // Cache around this class further out in NodeTableCache are synchronized
+    // to maintain cache validatity which indirectly sync access to the NodeTable.
+    // But to be sure, we provide MRSW guarantees on this class.
+    // (otherwise if no cache => disaster)
+    // synchonization happens in accessIndex() and readNodeByNodeId
+    
+    // NodeId to Node worker.
+    private Node _retrieveNodeByNodeId(NodeId id)
+    {
+        if ( NodeId.doesNotExist(id) )
+            return null ;
+        if ( NodeId.isAny(id) )
+            return null ;
+        
+        Node n = readNodeFromTable(id) ;
+        return n ;
+    }
+
+    // ----------------
+    
+    // Node to NodeId worker
+    // Find a node, possibly placing it in the node file as well
+    private NodeId _idForNode(Node node, boolean allocate)
+    {
+        if ( node == Node.ANY )
+            return NodeId.NodeIdAny ;
+        
+        // synchronized in accessIndex
+        NodeId nodeId = accessIndex(node, allocate) ;
+        return nodeId ;
+    }
+    
+    protected final NodeId accessIndex(Node node, boolean create)
+    {
+        Hash hash = new Hash(nodeHashToOffset.getRecordFactory().keyLength()) ;
+        setHash(hash, node) ;
+        byte k[] = hash.getBytes() ;        
+        // Key only.
+        Record r = nodeHashToOffset.getRecordFactory().create(k) ;
+        
+        synchronized (this)  // Pair to readNodeFromTable.
+        {
+            // Key and value, or null
+            Record r2 = nodeHashToOffset.find(r) ;
+            if ( r2 != null )
+            {
+                // Found.  Get the NodeId.
+            	NodeId id = NodeId.create(k, 0) ;
+                return id ;
+            }
+
+            // Not found.
+            if ( ! create )
+                return NodeId.NodeDoesNotExist ;
+
+            // Write the node, which allocates an id for it.
+            long offset = writeNodeToTable(node) ;
+
+            // Update the r record with the new id.
+            // r.value := id bytes ; 
+            Bytes.setLong(offset, r.getValue(), 0) ;
+
+            // Put in index - may appear because of concurrency
+            if ( ! nodeHashToOffset.add(r) )
+                throw new TDBException("NodeTableBase::nodeToId - record mysteriously appeared") ;
+            
+            NodeId id = NodeId.create(k, 0) ;
+            return id ;
+        }
+    }
+    
+    // -------- NodeId<->Node
+    // Synchronization:
+    //   write: in accessIndex
+    //   read: synchronized here.
+    // Only places for accessing the StringFile.
+    
+    private final long writeNodeToTable(Node node)
+    {
+        syncNeeded = true ;
+        // Synchroized in accessIndex
+        long offset = NodeLib.encodeStore(node, getObjects()) ;
+        return offset;
+    }
+    
+
+    private final Node readNodeFromTable(NodeId id)
+    {
+        synchronized (this) // Pair to accessIndex
+        {
+            byte k[] = Bytes.packLong(id.getId()) ;        
+            // Key only.
+            Record r = nodeHashToOffset.getRecordFactory().create(k) ;
+        	
+            Record r2 = nodeHashToOffset.find(r) ;
+            if ( r2 != null )
+            {
+                long offset = Bytes.getLong(r2.getValue()) ;
+                if ( offset >= getObjects().length() )
+                    return null ;
+                return NodeLib.fetchDecode(offset, getObjects()) ;
+            } 
+            else
+            	return null ;
+        }
+    }
+    // -------- NodeId<->Node
+
+    @Override
+    public synchronized void close()
+    {
+        // Close once.  This may be shared (e.g. triples table and quads table). 
+        if ( nodeHashToOffset != null )
+        {
+            nodeHashToOffset.close() ;
+            nodeHashToOffset = null ;
+        }
+        if ( getObjects() != null )
+        {
+            getObjects().close() ;
+            objects = null ;
+        }
+    }
+
+    @Override
+    public NodeId allocOffset()
+    {
+        return NodeId.create(getObjects().length()) ;
+    }
+    
+    // Not synchronized
+    @Override
+    public Iterator<Pair<NodeId, Node>> all() { return all1() ; }
+    
+    private Iterator<Pair<NodeId, Node>> all1()
+    
+    {
+        Iterator<Record> iter = nodeHashToOffset.iterator() ; ;
+
+        Transform<Record, Pair<NodeId, Node>> transform = new Transform<Record, Pair<NodeId, Node>>() {
+            @Override
+            public Pair<NodeId, Node> convert(Record item)
+            {
+            	NodeId id = NodeId.create(item.getKey()) ;
+            	long offset = Bytes.getLong(item.getValue()) ;
+                Node n = NodeLib.fetchDecode(offset, getObjects()) ;
+                return new Pair<NodeId, Node>(id, n) ;
+            }};
+        return Iter.map(iter, transform) ;
+    }
+
+    @Override
+    public void sync() 
+    { 
+        if ( syncNeeded )
+        {
+            if ( nodeHashToOffset != null )
+                nodeHashToOffset.sync() ;
+            if ( getObjects() != null )
+                getObjects().sync() ;
+            syncNeeded = false ;
+        }
+    }
+
+    public ObjectFile getObjects()
+    {
+        return objects;
+    }
+    
+    @Override
+    public String toString() { return objects.getLabel() ; }
+
+    @Override
+    public boolean isEmpty()
+    {
+        return false ;
+    }
+}

Propchange: incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/nodetable/NodeTableNative2.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/setup/Builder.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/setup/Builder.java?rev=1197798&r1=1197797&r2=1197798&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/setup/Builder.java (original)
+++ incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/setup/Builder.java Fri Nov  4 21:45:42 2011
@@ -38,7 +38,7 @@ import com.hp.hpl.jena.tdb.index.bplustr
 import com.hp.hpl.jena.tdb.nodetable.NodeTable ;
 import com.hp.hpl.jena.tdb.nodetable.NodeTableCache ;
 import com.hp.hpl.jena.tdb.nodetable.NodeTableInline ;
-import com.hp.hpl.jena.tdb.nodetable.NodeTableNative ;
+import com.hp.hpl.jena.tdb.nodetable.NodeTableNative2 ;
 import com.hp.hpl.jena.tdb.sys.Names ;
 import com.hp.hpl.jena.tdb.sys.SystemTDB ;
 
@@ -84,7 +84,7 @@ public class Builder
             RecordFactory recordFactory = new RecordFactory(SystemTDB.LenNodeHash, SystemTDB.SizeOfNodeId) ;
             Index idx = indexBuilder.buildIndex(fsIndex, recordFactory) ;
             ObjectFile objectFile = objectFileBuilder.buildObjectFile(fsObjectFile, Names.extNodeData) ;
-            NodeTable nodeTable = new NodeTableNative(idx, objectFile) ;
+            NodeTable nodeTable = new NodeTableNative2(idx, objectFile) ;
             nodeTable = NodeTableCache.create(nodeTable, sizeNode2NodeIdCache, sizeNodeId2NodeCache) ;
             nodeTable = NodeTableInline.create(nodeTable) ;
             return nodeTable ;

Modified: incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/sys/SetupTDB.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/sys/SetupTDB.java?rev=1197798&r1=1197797&r2=1197798&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/sys/SetupTDB.java (original)
+++ incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/sys/SetupTDB.java Fri Nov  4 21:45:42 2011
@@ -64,7 +64,7 @@ import com.hp.hpl.jena.tdb.nodetable.Nod
 import com.hp.hpl.jena.tdb.nodetable.NodeTableCache ;
 import com.hp.hpl.jena.tdb.nodetable.NodeTableFactory ;
 import com.hp.hpl.jena.tdb.nodetable.NodeTableInline ;
-import com.hp.hpl.jena.tdb.nodetable.NodeTableNative ;
+import com.hp.hpl.jena.tdb.nodetable.NodeTableNative2 ;
 import com.hp.hpl.jena.tdb.store.DatasetGraphTDB ;
 import com.hp.hpl.jena.tdb.store.DatasetPrefixesTDB ;
 import com.hp.hpl.jena.tdb.store.NodeId ;
@@ -566,7 +566,7 @@ public class SetupTDB
         Index nodeToId = makeIndex(location, indexNode2Id, LenNodeHash, SizeOfNodeId, -1 ,-1) ;
         
         // -- Make the node table using the components established above.
-        NodeTable nodeTable = new NodeTableNative(nodeToId, stringFile) ;
+        NodeTable nodeTable = new NodeTableNative2(nodeToId, stringFile) ;
         return nodeTable ;
     }
     

Modified: incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/transaction/NodeTableTrans.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/transaction/NodeTableTrans.java?rev=1197798&r1=1197797&r2=1197798&view=diff
==============================================================================
--- incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/transaction/NodeTableTrans.java (original)
+++ incubator/jena/Jena2/TDB/branches/hash-ids/src/main/java/com/hp/hpl/jena/tdb/transaction/NodeTableTrans.java Fri Nov  4 21:45:42 2011
@@ -31,7 +31,7 @@ import com.hp.hpl.jena.tdb.base.objectfi
 import com.hp.hpl.jena.tdb.index.Index ;
 import com.hp.hpl.jena.tdb.nodetable.NodeTable ;
 import com.hp.hpl.jena.tdb.nodetable.NodeTableCache ;
-import com.hp.hpl.jena.tdb.nodetable.NodeTableNative ;
+import com.hp.hpl.jena.tdb.nodetable.NodeTableNative2 ;
 import com.hp.hpl.jena.tdb.store.NodeId ;
 
 public class NodeTableTrans implements NodeTable, TransactionLifecycle
@@ -154,7 +154,7 @@ public class NodeTableTrans implements N
         //debug("begin: base=%s  offset=0x%X journalOffset=0x%X", base, offset, journalOffset) ;
         
         offset += journalOffset ;
-        this.nodeTableJournal = new NodeTableNative(nodeIndex, journal) ;
+        this.nodeTableJournal = new NodeTableNative2(nodeIndex, journal) ;
         this.nodeTableJournal = NodeTableCache.create(nodeTableJournal, CacheSize, CacheSize) ;
         // Do not add the inline NodeTable here - don't convert it's values by the offset!  
     }