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 2016/07/27 12:06:36 UTC

[07/11] jena git commit: Draft for testing cancellable sort()s.

Draft for testing cancellable sort()s.

The overall idea is to set up a query iterator that delivers
some bunch of values but after only a few (== 1) values have
been pulled the iterator is cancelled; we then show that
no more values can be pulled from the iterator and that the
sort has been abandoned.

This is a draft.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/512b2a5e
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/512b2a5e
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/512b2a5e

Branch: refs/heads/master
Commit: 512b2a5e0e470732b8e3405c124df61d91a7cfab
Parents: a3f3fdc
Author: chris dollin <eh...@gmail.com>
Authored: Fri Jul 22 16:10:07 2016 +0100
Committer: chris dollin <eh...@gmail.com>
Committed: Fri Jul 22 16:10:07 2016 +0100

----------------------------------------------------------------------
 .../apache/jena/atlas/data/SortedDataBag.java   |  17 +-
 .../iterator/TestSortedDataBagCancellation.java | 244 +++++++++++++++++++
 2 files changed, 257 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/512b2a5e/jena-arq/src/main/java/org/apache/jena/atlas/data/SortedDataBag.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/atlas/data/SortedDataBag.java b/jena-arq/src/main/java/org/apache/jena/atlas/data/SortedDataBag.java
index b248006..b7c74ae 100644
--- a/jena-arq/src/main/java/org/apache/jena/atlas/data/SortedDataBag.java
+++ b/jena-arq/src/main/java/org/apache/jena/atlas/data/SortedDataBag.java
@@ -83,14 +83,23 @@ public class SortedDataBag<E> extends AbstractDataBag<E>
         this.comparator = new AbortableComparator<E>(comparator);
     }
     
-	/**
-	    cancel arranges that further comparisons using the supplied
-	    comparator will abandon the sort in progress.
-	*/
+    /**
+	cancel arranges that further comparisons using the supplied
+	comparator will abandon the sort in progress.
+     */
 	public void cancel() 
 	{
 		comparator.cancel();
 	}
+	
+	/**
+		isCancelled is true iff cancel has been called on this
+		bags comparator. (Used in testing.)
+	*/
+	public boolean isCancelled()
+	{
+		return comparator.cancelled;
+	}
     
     protected void checkClosed()
     {

http://git-wip-us.apache.org/repos/asf/jena/blob/512b2a5e/jena-arq/src/test/java/org/apache/jena/sparql/engine/iterator/TestSortedDataBagCancellation.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/iterator/TestSortedDataBagCancellation.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/iterator/TestSortedDataBagCancellation.java
new file mode 100644
index 0000000..c29552e
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/iterator/TestSortedDataBagCancellation.java
@@ -0,0 +1,244 @@
+/*
+ * 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 org.apache.jena.sparql.engine.iterator;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.jena.atlas.io.IndentedWriter;
+import org.apache.jena.graph.Graph;
+import org.apache.jena.graph.NodeFactory;
+import org.apache.jena.query.QueryCancelledException;
+import org.apache.jena.query.SortCondition;
+import org.apache.jena.sparql.core.DatasetGraph;
+import org.apache.jena.sparql.core.DatasetGraphFactory;
+import org.apache.jena.sparql.core.Var;
+import org.apache.jena.sparql.engine.ExecutionContext;
+import org.apache.jena.sparql.engine.binding.Binding;
+import org.apache.jena.sparql.engine.binding.BindingComparator;
+import org.apache.jena.sparql.engine.binding.BindingFactory;
+import org.apache.jena.sparql.engine.binding.BindingMap;
+import org.apache.jena.sparql.engine.iterator.QueryIterSort;
+import org.apache.jena.sparql.engine.iterator.QueryIteratorBase;
+import org.apache.jena.sparql.engine.main.OpExecutor;
+import org.apache.jena.sparql.engine.main.OpExecutorFactory;
+import org.apache.jena.sparql.graph.GraphMemPlain;
+import org.apache.jena.sparql.serializer.SerializationContext;
+import org.apache.jena.sparql.util.Context;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+/*
+	Test that a SortedDataBag used inside a QueryIterSort
+	does indeed cut off when cancelled.
+	
+	This is horribly clunky because of the effort of
+	setting up. Maybe we should instead be content to
+	test the SortedDataBag correctly?
+ 
+*/
+public class TestSortedDataBagCancellation extends TestCase 
+{	
+	
+	static final BindingMap b1 = BindingFactory.create();
+	static final BindingMap b2 = BindingFactory.create();
+	static final BindingMap b3 = BindingFactory.create();
+	static final BindingMap b4 = BindingFactory.create();
+	
+	static
+	{
+		b1.add(Var.alloc("v1"), NodeFactory.createLiteral("alpha")); 
+		b2.add(Var.alloc("v2"), NodeFactory.createLiteral("beta"));
+		b3.add(Var.alloc("v3"), NodeFactory.createLiteral("gamma"));
+		b4.add(Var.alloc("v4"), NodeFactory.createLiteral("delta"));		
+	}
+	
+	final Context params = new Context();
+	
+	final OpExecutorFactory factory = new OpExecutorFactory() {
+		
+		@Override public OpExecutor create(ExecutionContext ec) {
+			throw new UnsupportedOperationException();
+		}
+	};
+	
+	final Graph activeGraph = new GraphMemPlain();
+	
+	final DatasetGraph dataset = DatasetGraphFactory.create();
+	
+	final List<SortCondition> conditions = new ArrayList<SortCondition>();
+	
+	final ExecutionContext ec = new ExecutionContext
+		(		
+		params,
+		activeGraph,
+		dataset,
+		factory
+		);
+	
+	final BindingComparator base_bc = new BindingComparator(conditions, ec);
+	final SpecialBindingComparator bc = new SpecialBindingComparator(base_bc, ec);
+	
+	QueryIteratorItems baseIter = new QueryIteratorItems();
+	
+	{
+		baseIter.bindings.add(b1);
+		baseIter.bindings.add(b2);
+		baseIter.bindings.add(b3);
+		baseIter.bindings.add(b4);;
+	}
+	
+	QueryIterSort qs = new QueryIterSort(baseIter, bc, ec);
+
+	/**
+		In this test, the iterator is not cancelled; 
+		all items should be delivered, and the compare
+		count should be monotonic-nondecreasing.
+	*/
+	@Test public void testIteratesToCompletion()
+	{
+		int count = 0;
+		assertEquals(0, count = bc.count);
+		Set<Binding> results = new HashSet<Binding>();
+		
+		assertTrue(qs.hasNext());
+		assertTrue(bc.count >= count); count = bc.count;
+		results.add(qs.next());
+
+		assertTrue(qs.hasNext());
+		assertTrue(bc.count >= count); count = bc.count;
+		results.add(qs.next());
+
+		assertTrue(qs.hasNext());
+		assertTrue(bc.count >= count); count = bc.count;
+		results.add(qs.next());
+
+		assertTrue(qs.hasNext());
+		assertTrue(bc.count >= count); count = bc.count;
+		results.add(qs.next());
+
+		assertFalse(qs.hasNext());
+		
+		Set<Binding> expected = new HashSet<Binding>();
+		expected.add(b1);
+		expected.add(b2);
+		expected.add(b3);
+		expected.add(b4);
+		
+		assertEquals(expected, results);
+	}	
+	
+	/**
+		In this test, the iterator is cancelled after
+		the first result is delivered. Any attempt to
+		run the comparator should be trapped an exception
+		thrown. The iterators should deliver no more values.
+	 */
+	@Test public void testIteratesWithCancellation()
+	{
+		int count = 0;
+		assertEquals(0, count = bc.count);
+		Set<Binding> results = new HashSet<Binding>();
+		
+		assertTrue(qs.hasNext());
+		assertTrue(bc.count >= count); count = bc.count;
+		results.add(qs.next());
+
+		qs.cancel();
+		try 
+		{
+			bc.noMoreCalls();
+			while (qs.hasNext()) qs.next();
+		} 
+		catch (QueryCancelledException qe) 
+		{
+			assertTrue(qs.db.isCancelled());
+			return;
+			
+		}
+		fail("query was not cancelled");		
+	}	
+	
+	/**
+		A QueryIterator that delivers the elements of a list of bindings. 
+	*/
+	private static final class QueryIteratorItems extends QueryIteratorBase 
+	{
+		List<Binding> bindings = new ArrayList<Binding>();
+		int index = 0;
+
+		@Override
+		public void output(IndentedWriter out, SerializationContext sCxt) {
+			out.write("a QueryIteratorItems");
+		}
+
+		@Override
+		protected boolean hasNextBinding() 
+		{
+			return index < bindings.size();
+		}
+
+		@Override
+		protected Binding moveToNextBinding()
+		{
+			index += 1;
+			return bindings.get(index - 1);
+		}
+
+		@Override
+		protected void closeIterator() 
+		{			
+		}
+
+		@Override
+		protected void requestCancel() 
+		{			
+		}
+	}
+	
+	/**
+		A BindingComparator that wraps another BindingComparator
+		and counts how many times compare() is called.
+	*/
+	static class SpecialBindingComparator extends BindingComparator 
+	{
+		final BindingComparator base;
+		int count = 0;
+		boolean trapCompare = false;
+		
+		public SpecialBindingComparator(BindingComparator base, ExecutionContext ec)
+		{
+			super(base.getConditions(), ec);
+			this.base = base;
+		}
+		
+		public void noMoreCalls() {
+			trapCompare = true;
+		}
+
+		public int compare(Binding x, Binding y) 
+		{
+			if (trapCompare) throw new RuntimeException("compare() no longer allowed.");
+			count += 1;
+			return base.compare(x, y);
+		}
+	}
+}
\ No newline at end of file