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

svn commit: r1197689 - in /incubator/jena/Jena2/ARQ/trunk/src: main/java/com/hp/hpl/jena/sparql/engine/ main/java/com/hp/hpl/jena/sparql/graph/ main/java/com/hp/hpl/jena/sparql/util/ main/java/com/hp/hpl/jena/sparql/util/graph/ test/java/com/hp/hpl/jen...

Author: sallen
Date: Fri Nov  4 17:47:38 2011
New Revision: 1197689

URL: http://svn.apache.org/viewvc?rev=1197689&view=rev
Log:
JENA-119 (Eliminate memory bounds during query execution).  Added a Graph backed by a DataBag to provide a non-memory bound container for CONSTRUCT queries.

Added:
    incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/graph/GraphDataBag.java
    incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/graph/GraphDefaultDataBag.java
    incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/graph/GraphDistinctDataBag.java
    incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/util/NodeComparator.java
    incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/util/TripleComparator.java
    incubator/jena/Jena2/ARQ/trunk/src/test/java/com/hp/hpl/jena/sparql/graph/TestGraphsDataBag.java
Modified:
    incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/engine/QueryExecutionBase.java
    incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/util/graph/GraphFactory.java
    incubator/jena/Jena2/ARQ/trunk/src/test/java/com/hp/hpl/jena/sparql/graph/TS_Graph.java

Modified: incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/engine/QueryExecutionBase.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/engine/QueryExecutionBase.java?rev=1197689&r1=1197688&r2=1197689&view=diff
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/engine/QueryExecutionBase.java (original)
+++ incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/engine/QueryExecutionBase.java Fri Nov  4 17:47:38 2011
@@ -18,52 +18,52 @@
 
 package com.hp.hpl.jena.sparql.engine;
 
-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.concurrent.TimeUnit ;
-
-import org.openjena.atlas.lib.AlarmClock ;
-import org.openjena.atlas.lib.Callback ;
-import org.openjena.atlas.lib.Pingback ;
-import org.openjena.atlas.logging.Log ;
-
-import com.hp.hpl.jena.graph.Node ;
-import com.hp.hpl.jena.graph.Triple ;
-import com.hp.hpl.jena.n3.IRIResolver ;
-import com.hp.hpl.jena.query.ARQ ;
-import com.hp.hpl.jena.query.Dataset ;
-import com.hp.hpl.jena.query.Query ;
-import com.hp.hpl.jena.query.QueryExecException ;
-import com.hp.hpl.jena.query.QueryExecution ;
-import com.hp.hpl.jena.query.QuerySolution ;
-import com.hp.hpl.jena.query.ResultSet ;
-import com.hp.hpl.jena.rdf.model.Model ;
-import com.hp.hpl.jena.rdf.model.ModelFactory ;
-import com.hp.hpl.jena.rdf.model.RDFNode ;
-import com.hp.hpl.jena.rdf.model.Resource ;
-import com.hp.hpl.jena.rdf.model.Statement ;
-import com.hp.hpl.jena.shared.PrefixMapping ;
-import com.hp.hpl.jena.sparql.ARQConstants ;
-import com.hp.hpl.jena.sparql.core.DatasetGraph ;
-import com.hp.hpl.jena.sparql.core.describe.DescribeHandler ;
-import com.hp.hpl.jena.sparql.core.describe.DescribeHandlerRegistry ;
-import com.hp.hpl.jena.sparql.engine.binding.Binding ;
-import com.hp.hpl.jena.sparql.engine.binding.BindingRoot ;
-import com.hp.hpl.jena.sparql.engine.binding.BindingUtils ;
-import com.hp.hpl.jena.sparql.engine.iterator.QueryIteratorBase ;
-import com.hp.hpl.jena.sparql.engine.iterator.QueryIteratorWrapper ;
-import com.hp.hpl.jena.sparql.syntax.ElementGroup ;
-import com.hp.hpl.jena.sparql.syntax.Template ;
-import com.hp.hpl.jena.sparql.util.Context ;
-import com.hp.hpl.jena.sparql.util.DatasetUtils ;
-import com.hp.hpl.jena.sparql.util.ModelUtils ;
-import com.hp.hpl.jena.sparql.util.NodeFactory ;
-import com.hp.hpl.jena.sparql.util.graph.GraphFactory ;
-import com.hp.hpl.jena.util.FileManager ;
+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.concurrent.TimeUnit ;
+
+import org.openjena.atlas.lib.AlarmClock ;
+import org.openjena.atlas.lib.Callback ;
+import org.openjena.atlas.lib.Pingback ;
+import org.openjena.atlas.logging.Log ;
+
+import com.hp.hpl.jena.graph.Node ;
+import com.hp.hpl.jena.graph.Triple ;
+import com.hp.hpl.jena.n3.IRIResolver ;
+import com.hp.hpl.jena.query.ARQ ;
+import com.hp.hpl.jena.query.Dataset ;
+import com.hp.hpl.jena.query.Query ;
+import com.hp.hpl.jena.query.QueryExecException ;
+import com.hp.hpl.jena.query.QueryExecution ;
+import com.hp.hpl.jena.query.QuerySolution ;
+import com.hp.hpl.jena.query.ResultSet ;
+import com.hp.hpl.jena.rdf.model.Model ;
+import com.hp.hpl.jena.rdf.model.ModelFactory ;
+import com.hp.hpl.jena.rdf.model.RDFNode ;
+import com.hp.hpl.jena.rdf.model.Resource ;
+import com.hp.hpl.jena.rdf.model.Statement ;
+import com.hp.hpl.jena.shared.PrefixMapping ;
+import com.hp.hpl.jena.sparql.ARQConstants ;
+import com.hp.hpl.jena.sparql.core.DatasetGraph ;
+import com.hp.hpl.jena.sparql.core.describe.DescribeHandler ;
+import com.hp.hpl.jena.sparql.core.describe.DescribeHandlerRegistry ;
+import com.hp.hpl.jena.sparql.engine.binding.Binding ;
+import com.hp.hpl.jena.sparql.engine.binding.BindingRoot ;
+import com.hp.hpl.jena.sparql.engine.binding.BindingUtils ;
+import com.hp.hpl.jena.sparql.engine.iterator.QueryIteratorBase ;
+import com.hp.hpl.jena.sparql.engine.iterator.QueryIteratorWrapper ;
+import com.hp.hpl.jena.sparql.syntax.ElementGroup ;
+import com.hp.hpl.jena.sparql.syntax.Template ;
+import com.hp.hpl.jena.sparql.util.Context ;
+import com.hp.hpl.jena.sparql.util.DatasetUtils ;
+import com.hp.hpl.jena.sparql.util.ModelUtils ;
+import com.hp.hpl.jena.sparql.util.NodeFactory ;
+import com.hp.hpl.jena.sparql.util.graph.GraphFactory ;
+import com.hp.hpl.jena.util.FileManager ;
 
 /** All the SPARQL query result forms made from a graph-level execution object */ 
 
@@ -219,8 +219,22 @@ public class QueryExecutionBase implemen
     // Construct
     @Override
     public Model execConstruct()
-    { return execConstruct(GraphFactory.makeJenaDefaultModel()) ; }
-
+    {
+        return execConstruct(GraphFactory.makeJenaDefaultModel()) ;
+    }
+
+//  TODO: enable this?
+//    /**
+//     * Executes as a construct query, placing the results into a newly constructed {@link com.hp.hpl.jena.sparql.graph.GraphDistinctDataBag}.
+//     * The threshold policy is set from the current context.
+//     */
+//    @Override
+//    public Model execConstructDataBag()
+//    {
+//        ThresholdPolicy<Triple> thresholdPolicy = ThresholdPolicyFactory.policyFromContext(context);
+//        return execConstruct(GraphFactory.makeDataBagModel(thresholdPolicy)) ;
+//    }
+	
     @Override
     public Model execConstruct(Model model)
     {

Added: incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/graph/GraphDataBag.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/graph/GraphDataBag.java?rev=1197689&view=auto
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/graph/GraphDataBag.java (added)
+++ incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/graph/GraphDataBag.java Fri Nov  4 17:47:38 2011
@@ -0,0 +1,135 @@
+/**
+ * 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.sparql.graph;
+
+import org.openjena.atlas.data.DataBag ;
+import org.openjena.atlas.data.ThresholdPolicy ;
+
+import com.hp.hpl.jena.graph.Capabilities ;
+import com.hp.hpl.jena.graph.Triple ;
+import com.hp.hpl.jena.graph.TripleMatch ;
+import com.hp.hpl.jena.graph.TripleMatchIterator ;
+import com.hp.hpl.jena.graph.impl.SimpleReifier ;
+import com.hp.hpl.jena.graph.impl.TripleStore ;
+import com.hp.hpl.jena.mem.GraphMemBase ;
+import com.hp.hpl.jena.shared.ReificationStyle ;
+import com.hp.hpl.jena.util.iterator.ExtendedIterator ;
+
+/**
+ * A Graph based on top of a DataBag.  This means it has some limitations:
+ * <ol>
+ * <li>Cannot add any triples after you call find() unless you first call clear() or getBulkUpdateHandler().removeAll().</li>
+ * <li>Cannot remove any triples except by calling clear() or getBulkUpdateHandler().removeAll().</li>
+ * <li>There is no indexing, so find() will always scan all triples.</li>
+ * <li>The size() method is not guaranteed to be accurate, treat it as an estimate.</li>
+ * <li>You must call close() in order to release any resources (such as spill files).</li>
+ * </ol>
+ */
+public abstract class GraphDataBag extends GraphMemBase
+{
+    private final ThresholdPolicy<Triple> thresholdPolicy ;
+    private DataBag<Triple> db ;
+    
+    public GraphDataBag(ThresholdPolicy<Triple> thresholdPolicy)
+    {
+        this(thresholdPolicy, ReificationStyle.Minimal) ;
+    }
+
+    public GraphDataBag(ThresholdPolicy<Triple> thresholdPolicy, ReificationStyle style)
+    {
+        super(style) ;
+        
+        this.thresholdPolicy = thresholdPolicy;
+        
+        capabilities = new Capabilities()
+        {
+            @Override
+            public boolean sizeAccurate() { return false; }
+            @Override
+            public boolean addAllowed() { return addAllowed( false ); }
+            @Override
+            public boolean addAllowed( boolean every ) { return true; } 
+            @Override
+            public boolean deleteAllowed() { return deleteAllowed( false ); }
+            @Override
+            public boolean deleteAllowed( boolean every ) { return false; } 
+            @Override
+            public boolean canBeEmpty() { return true; }
+            @Override
+            public boolean iteratorRemoveAllowed() { return false; }
+            @Override
+            public boolean findContractSafe() { return true; }
+            @Override
+            public boolean handlesLiteralTyping() { return true; }
+        };
+        
+        this.db = createDataBag();
+    }
+    
+    protected abstract DataBag<Triple> createDataBag();
+    
+    protected ThresholdPolicy<Triple> getThresholdPolicy()
+    {
+        return thresholdPolicy;
+    }
+
+    @Override
+    protected TripleStore createTripleStore()
+    {
+        // No TripleStore for us
+        return null ;
+    }
+    
+    @Override
+    public void performAdd(Triple t)
+    {
+        if ( !getReifier().handledAdd(t) )
+        {
+            db.add(t) ;
+        }
+    }
+
+    @Override
+    public int graphBaseSize()
+    {
+        return (int) db.size() ;
+    }
+
+    @Override
+    protected void destroy()
+    {
+        db.close() ;
+    }
+
+    @Override
+    public void clear()
+    {
+        db.close() ;
+        ((SimpleReifier) getReifier()).clear() ;
+        db = createDataBag() ;
+    }
+
+    @Override
+    protected ExtendedIterator<Triple> graphBaseFind(TripleMatch m)
+    {
+        return new TripleMatchIterator(m.asTriple(), db.iterator()) ;
+    }
+
+}
+

Added: incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/graph/GraphDefaultDataBag.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/graph/GraphDefaultDataBag.java?rev=1197689&view=auto
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/graph/GraphDefaultDataBag.java (added)
+++ incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/graph/GraphDefaultDataBag.java Fri Nov  4 17:47:38 2011
@@ -0,0 +1,50 @@
+/**
+ * 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.sparql.graph;
+
+import org.openjena.atlas.data.BagFactory ;
+import org.openjena.atlas.data.DataBag ;
+import org.openjena.atlas.data.ThresholdPolicy ;
+import org.openjena.riot.SerializationFactoryFinder ;
+
+import com.hp.hpl.jena.graph.Triple ;
+import com.hp.hpl.jena.shared.ReificationStyle ;
+
+/**
+ * An implementation of {@link GraphDataBag} that may return duplicate triples from the find() operation.
+ */
+public class GraphDefaultDataBag extends GraphDataBag
+{
+    public GraphDefaultDataBag(ThresholdPolicy<Triple> thresholdPolicy)
+    {
+        super(thresholdPolicy) ;
+    }
+    
+    public GraphDefaultDataBag(ThresholdPolicy<Triple> thresholdPolicy, ReificationStyle style)
+    {
+        super(thresholdPolicy, style) ;
+    }
+    
+    @Override
+    protected DataBag<Triple> createDataBag()
+    {
+        return BagFactory.newDefaultBag(getThresholdPolicy(), SerializationFactoryFinder.tripleSerializationFactory()) ;
+    }
+}
+

Added: incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/graph/GraphDistinctDataBag.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/graph/GraphDistinctDataBag.java?rev=1197689&view=auto
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/graph/GraphDistinctDataBag.java (added)
+++ incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/graph/GraphDistinctDataBag.java Fri Nov  4 17:47:38 2011
@@ -0,0 +1,51 @@
+/**
+ * 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.sparql.graph;
+
+import org.openjena.atlas.data.BagFactory ;
+import org.openjena.atlas.data.DataBag ;
+import org.openjena.atlas.data.ThresholdPolicy ;
+import org.openjena.riot.SerializationFactoryFinder ;
+
+import com.hp.hpl.jena.graph.Triple ;
+import com.hp.hpl.jena.shared.ReificationStyle ;
+import com.hp.hpl.jena.sparql.util.TripleComparator ;
+
+/**
+ * An implementation of {@link GraphDataBag} with a guarantee that there will be no duplicate triples in the find() operation.
+ */
+public class GraphDistinctDataBag extends GraphDataBag
+{
+    public GraphDistinctDataBag(ThresholdPolicy<Triple> thresholdPolicy)
+    {
+        super(thresholdPolicy) ;
+    }
+    
+    public GraphDistinctDataBag(ThresholdPolicy<Triple> thresholdPolicy, ReificationStyle style)
+    {
+        super(thresholdPolicy, style) ;
+    }
+    
+    @Override
+    protected DataBag<Triple> createDataBag()
+    {
+        return BagFactory.newDistinctBag(getThresholdPolicy(), SerializationFactoryFinder.tripleSerializationFactory(), new TripleComparator()) ;
+    }
+}
+

Added: incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/util/NodeComparator.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/util/NodeComparator.java?rev=1197689&view=auto
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/util/NodeComparator.java (added)
+++ incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/util/NodeComparator.java Fri Nov  4 17:47:38 2011
@@ -0,0 +1,34 @@
+/**
+ * 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.sparql.util;
+
+import java.util.Comparator ;
+
+import com.hp.hpl.jena.graph.Node ;
+
+public class NodeComparator implements Comparator<Node>
+{
+    @Override
+    public int compare(Node o1, Node o2)
+    {
+        return NodeUtils.compareRDFTerms(o1, o2);
+        //return NodeValue.compareAlways(NodeValue.makeNode(o1), NodeValue.makeNode(o2));
+    }
+}
+

Added: incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/util/TripleComparator.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/util/TripleComparator.java?rev=1197689&view=auto
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/util/TripleComparator.java (added)
+++ incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/util/TripleComparator.java Fri Nov  4 17:47:38 2011
@@ -0,0 +1,44 @@
+/**
+ * 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.sparql.util;
+
+import java.util.Comparator ;
+
+import com.hp.hpl.jena.graph.Triple ;
+
+public class TripleComparator implements Comparator<Triple>
+{
+    private static final NodeComparator nc = new NodeComparator();
+    
+	@Override
+    public int compare(Triple o1, Triple o2)
+    {
+        int toReturn = nc.compare(o1.getSubject(), o2.getSubject());
+        if (toReturn == 0)
+        {
+            toReturn = nc.compare(o1.getPredicate(), o2.getPredicate());
+            if (toReturn == 0)
+            {
+                toReturn = nc.compare(o1.getObject(), o2.getObject());
+            }
+        }
+        
+        return toReturn;
+    }
+}

Modified: incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/util/graph/GraphFactory.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/util/graph/GraphFactory.java?rev=1197689&r1=1197688&r2=1197689&view=diff
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/util/graph/GraphFactory.java (original)
+++ incubator/jena/Jena2/ARQ/trunk/src/main/java/com/hp/hpl/jena/sparql/util/graph/GraphFactory.java Fri Nov  4 17:47:38 2011
@@ -18,11 +18,15 @@
 
 package com.hp.hpl.jena.sparql.util.graph;
 
+import org.openjena.atlas.data.ThresholdPolicy ;
+
 import com.hp.hpl.jena.graph.Factory ;
 import com.hp.hpl.jena.graph.Graph ;
+import com.hp.hpl.jena.graph.Triple ;
 import com.hp.hpl.jena.query.ARQ ;
 import com.hp.hpl.jena.rdf.model.Model ;
 import com.hp.hpl.jena.rdf.model.ModelFactory ;
+import com.hp.hpl.jena.sparql.graph.GraphDistinctDataBag ;
 import com.hp.hpl.jena.sparql.graph.GraphMemSimple2 ;
 import com.hp.hpl.jena.sparql.util.RefBoolean ;
 
@@ -61,6 +65,12 @@ public class GraphFactory
     {
         return new GraphSink() ;
     }
+    
+    /** A graph backed by a DistinctDataBag&lt;Triple&gt;. */
+    public static Graph createDataBagGraph(ThresholdPolicy<Triple> thresholdPolicy)
+    {
+        return new GraphDistinctDataBag(thresholdPolicy) ;
+    }
 
     /** Guaranteed call-through to Jena's ModelFactory operation */
     public static Model makeJenaDefaultModel() { return ModelFactory.createDefaultModel() ; }
@@ -76,4 +86,10 @@ public class GraphFactory
     {
         return ModelFactory.createModelForGraph(createPlainGraph()) ;
     }
+    
+    /** Create a model over a DataBag graph (will spill to disk when it get large) */
+    public static Model makeDataBagModel(ThresholdPolicy<Triple> thresholdPolicy)
+    {
+        return ModelFactory.createModelForGraph(createDataBagGraph(thresholdPolicy)) ;
+    }
 }

Modified: incubator/jena/Jena2/ARQ/trunk/src/test/java/com/hp/hpl/jena/sparql/graph/TS_Graph.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/test/java/com/hp/hpl/jena/sparql/graph/TS_Graph.java?rev=1197689&r1=1197688&r2=1197689&view=diff
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/test/java/com/hp/hpl/jena/sparql/graph/TS_Graph.java (original)
+++ incubator/jena/Jena2/ARQ/trunk/src/test/java/com/hp/hpl/jena/sparql/graph/TS_Graph.java Fri Nov  4 17:47:38 2011
@@ -24,7 +24,8 @@ import org.junit.runners.Suite ;
 @RunWith(Suite.class)
 @Suite.SuiteClasses( {
       TestGraphsMem.class
-    , TestDatasetGraphMem.class
+    , TestDatasetGraphMem.class
+    , TestGraphsDataBag.class
 })
 public class TS_Graph
 {

Added: incubator/jena/Jena2/ARQ/trunk/src/test/java/com/hp/hpl/jena/sparql/graph/TestGraphsDataBag.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/test/java/com/hp/hpl/jena/sparql/graph/TestGraphsDataBag.java?rev=1197689&view=auto
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/test/java/com/hp/hpl/jena/sparql/graph/TestGraphsDataBag.java (added)
+++ incubator/jena/Jena2/ARQ/trunk/src/test/java/com/hp/hpl/jena/sparql/graph/TestGraphsDataBag.java Fri Nov  4 17:47:38 2011
@@ -0,0 +1,180 @@
+/**
+ * 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.sparql.graph;
+
+import org.junit.After ;
+import org.junit.Before ;
+import org.junit.Test ;
+import org.openjena.atlas.data.ThresholdPolicy ;
+import org.openjena.atlas.data.ThresholdPolicyFactory ;
+import org.openjena.atlas.junit.BaseTest ;
+
+import com.hp.hpl.jena.graph.Graph ;
+import com.hp.hpl.jena.graph.Triple ;
+import com.hp.hpl.jena.query.Query ;
+import com.hp.hpl.jena.query.QueryExecution ;
+import com.hp.hpl.jena.query.QueryExecutionFactory ;
+import com.hp.hpl.jena.query.QueryFactory ;
+import com.hp.hpl.jena.query.ResultSet ;
+import com.hp.hpl.jena.query.ResultSetFormatter ;
+import com.hp.hpl.jena.query.Syntax ;
+import com.hp.hpl.jena.rdf.model.Model ;
+import com.hp.hpl.jena.rdf.model.ModelFactory ;
+import com.hp.hpl.jena.shared.DeleteDeniedException ;
+import com.hp.hpl.jena.sparql.sse.SSE ;
+import com.hp.hpl.jena.util.iterator.ExtendedIterator ;
+
+/** Tests for DataBag graphs */
+public class TestGraphsDataBag extends BaseTest
+{
+    protected Graph distinct;
+    protected Graph duplicates;
+    
+    @Before
+    public void setup()
+    {
+        ThresholdPolicy<Triple> policy = ThresholdPolicyFactory.never();
+        distinct = new GraphDistinctDataBag(policy);
+        
+        ThresholdPolicy<Triple> policy2 = ThresholdPolicyFactory.never();
+        duplicates = new GraphDefaultDataBag(policy2);
+    }
+    
+    @After
+    public void shutdown()
+    {
+        if (null != distinct)
+        {
+            distinct.close() ;
+        }
+        
+        if (null != duplicates)
+        {
+            duplicates.close();
+        }
+    }
+    
+    @Test
+    public void add_1()
+    {
+        distinct.add(SSE.parseTriple("(<x> <p> 'ZZZ')"));
+        assertEquals(1, count(distinct));
+    }
+    
+    @Test
+    public void add_2()
+    {
+        distinct.add(SSE.parseTriple("(<x> <p> 'ZZZ')"));
+        distinct.add(SSE.parseTriple("(<x> <p> 'ZZZ')"));
+        assertEquals(1, count(distinct));
+    }
+    
+    @Test
+    public void add_3()
+    {
+        duplicates.add(SSE.parseTriple("(<x> <p> 'ZZZ')"));
+        duplicates.add(SSE.parseTriple("(<x> <p> 'ZZZ')"));
+        assertEquals(2, count(duplicates));
+    }
+    
+    @Test
+    public void empty_0()
+    {
+        assertEquals(0, count(distinct));
+    }
+    
+    @Test
+    public void removeAll_1()
+    {
+        distinct.add(SSE.parseTriple("(<x> <p> 'ZZZ')"));
+        assertEquals(1, count(distinct));
+        
+        distinct.getBulkUpdateHandler().removeAll();
+        assertEquals(0, count(distinct));
+    }
+    
+    @Test(expected=DeleteDeniedException.class)
+    public void delete_1()
+    {
+        Triple t = SSE.parseTriple("(<x> <p> 'ZZZ')");
+        distinct.add(t);
+        distinct.delete(t);
+    }
+    
+    @Test
+    public void complexQuery_1()
+    {
+        for (int i=0; i<2; i++)
+        {
+            distinct.add(SSE.parseTriple("(<http://example.org/a> <http://example.org/p> 'YYY')"));
+            distinct.add(SSE.parseTriple("(<http://example.org/a> <http://example.org/p2> 'ZZZ')"));
+            distinct.add(SSE.parseTriple("(<http://example.org/b> <http://example.org/p> 'YYY')"));
+            distinct.add(SSE.parseTriple("(<http://example.org/b> <http://example.org/p2> 'ZZZ')"));
+        }
+        
+        assertEquals(2, query("select * where { ?a <http://example.org/p> ?v ; <http://example.org/p2> ?v2 }", distinct));
+        assertEquals(2, query("select distinct * where { ?a <http://example.org/p> ?v ; <http://example.org/p2> ?v2 }", distinct));
+    }
+    
+    @Test
+    public void complexQuery_2()
+    {
+        for (int i=0; i<2; i++)
+        {
+            duplicates.add(SSE.parseTriple("(<http://example.org/a> <http://example.org/p> 'YYY')"));
+            duplicates.add(SSE.parseTriple("(<http://example.org/a> <http://example.org/p2> 'ZZZ')"));
+            duplicates.add(SSE.parseTriple("(<http://example.org/b> <http://example.org/p> 'YYY')"));
+            duplicates.add(SSE.parseTriple("(<http://example.org/b> <http://example.org/p2> 'ZZZ')"));
+        }
+        
+        assertEquals(8, query("select * where { ?a <http://example.org/p> ?v ; <http://example.org/p2> ?v2 }", duplicates));
+        assertEquals(2, query("select distinct * where { ?a <http://example.org/p> ?v ; <http://example.org/p2> ?v2 }", duplicates));
+    }
+    
+    private int query(String str, Graph g)
+    {
+        Model model = ModelFactory.createModelForGraph(g);
+        Query q = QueryFactory.create(str, Syntax.syntaxARQ) ;
+        QueryExecution qexec = QueryExecutionFactory.create(q, model) ;
+        ResultSet rs = qexec.execSelect() ;
+        int x = ResultSetFormatter.consume(rs) ;
+        qexec.close() ;
+        return x ;
+    }
+    
+    private int count(Graph g)
+    {
+        int toReturn = 0;
+        ExtendedIterator<Triple> it = g.find(null, null, null);
+        try
+        {
+            while (it.hasNext())
+            {
+                it.next();
+                toReturn++;
+            }
+        }
+        finally
+        {
+            it.close();
+        }
+        
+        return toReturn;
+    }
+}