From: <tho...@us...> - 2010-08-27 14:57:31
|
Revision: 3467 http://bigdata.svn.sourceforge.net/bigdata/?rev=3467&view=rev Author: thompsonbry Date: 2010-08-27 14:57:24 +0000 (Fri, 27 Aug 2010) Log Message: ----------- Working on query engine handling of the output chunks by wrapping the FutureTask for the operator computation. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineStartOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/eval/JoinGraph.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ndx/SampleLocalBTree.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ndx/SampleLocalShard.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/MapBindingSetsOverShards.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/PipelineDelayOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/QueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/ReceiveBindingSets.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/join/TestPipelineJoin.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java 2010-08-27 14:28:04 UTC (rev 3466) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java 2010-08-27 14:57:24 UTC (rev 3467) @@ -27,7 +27,7 @@ package com.bigdata.bop; -import java.util.concurrent.Future; +import java.util.concurrent.FutureTask; import com.bigdata.bop.engine.BOpStats; import com.bigdata.relation.accesspath.BlockingBuffer; @@ -111,14 +111,17 @@ IBlockingBuffer<E[]> newBuffer(); /** - * Initiate execution for the operator, returning a {@link Future} which for - * that evaluation. + * Return a {@link FutureTask} which computes the operator against the + * evaluation context. The caller is responsible for executing the + * {@link FutureTask} (this gives them the ability to hook the completion of + * the computation). * * @param context * The evaluation context. * - * @return The {@link Future} for the operator's evaluation. + * @return The {@link FutureTask} which will compute the operator's + * evaluation. */ - Future<Void> eval(BOpContext<E> context); + FutureTask<Void> eval(BOpContext<E> context); } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineStartOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineStartOp.java 2010-08-27 14:28:04 UTC (rev 3466) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineStartOp.java 2010-08-27 14:57:24 UTC (rev 3467) @@ -71,13 +71,9 @@ super(args, annotations); } - public Future<Void> eval(final BOpContext<IBindingSet> context) { + public FutureTask<Void> eval(final BOpContext<IBindingSet> context) { - final FutureTask<Void> ft = new FutureTask<Void>(new CopyTask(context)); - - context.getIndexManager().getExecutorService().execute(ft); - - return ft; + return new FutureTask<Void>(new CopyTask(context)); } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/eval/JoinGraph.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/eval/JoinGraph.java 2010-08-27 14:28:04 UTC (rev 3466) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/eval/JoinGraph.java 2010-08-27 14:57:24 UTC (rev 3467) @@ -32,7 +32,6 @@ import java.util.List; import java.util.Set; import java.util.concurrent.Callable; -import java.util.concurrent.Future; import java.util.concurrent.FutureTask; import com.bigdata.bop.AbstractPipelineOp; @@ -233,14 +232,10 @@ } - public Future<Void> eval(final BOpContext<IBindingSet> context) { + public FutureTask<Void> eval(final BOpContext<IBindingSet> context) { - final FutureTask<Void> ft = new FutureTask<Void>(new JoinGraphTask(context)); + return new FutureTask<Void>(new JoinGraphTask(context)); - context.getIndexManager().getExecutorService().execute(ft); - - return ft; - } /** Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 2010-08-27 14:28:04 UTC (rev 3466) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 2010-08-27 14:57:24 UTC (rev 3467) @@ -320,14 +320,9 @@ } - public Future<Void> eval(final BOpContext<IBindingSet> context) { + public FutureTask<Void> eval(final BOpContext<IBindingSet> context) { - final FutureTask<Void> ft = new FutureTask<Void>(new JoinTask(this, - context)); - - context.getIndexManager().getExecutorService().execute(ft); - - return ft; + return new FutureTask<Void>(new JoinTask(this, context)); } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ndx/SampleLocalBTree.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ndx/SampleLocalBTree.java 2010-08-27 14:28:04 UTC (rev 3466) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ndx/SampleLocalBTree.java 2010-08-27 14:57:24 UTC (rev 3467) @@ -1,7 +1,6 @@ package com.bigdata.bop.ndx; import java.util.concurrent.Callable; -import java.util.concurrent.Future; import java.util.concurrent.FutureTask; import com.bigdata.bop.BOpContext; @@ -28,20 +27,15 @@ } - public Future<Void> eval(final BOpContext<E> context) { + public FutureTask<Void> eval(final BOpContext<E> context) { if (context.getPartitionId() != -1) { // Must not be specific to a shard. throw new UnsupportedOperationException(); } - final FutureTask<Void> ft = new FutureTask<Void>( - new LocalBTreeSampleTask(context)); + return new FutureTask<Void>(new LocalBTreeSampleTask(context)); - context.getIndexManager().getExecutorService().execute(ft); - - return ft; - } /** Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ndx/SampleLocalShard.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ndx/SampleLocalShard.java 2010-08-27 14:28:04 UTC (rev 3466) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ndx/SampleLocalShard.java 2010-08-27 14:57:24 UTC (rev 3467) @@ -33,20 +33,15 @@ /* * Note: This is done at evaluation time, local to the data. */ - public Future<Void> eval(final BOpContext<E> context) { + public FutureTask<Void> eval(final BOpContext<E> context) { if (context.getPartitionId() == -1) { // Must be specific to a shard. throw new UnsupportedOperationException(); } - final FutureTask<Void> ft = new FutureTask<Void>( - new LocalShardSampleTask(context)); + return new FutureTask<Void>(new LocalShardSampleTask(context)); - context.getIndexManager().getExecutorService().execute(ft); - - return ft; - } /** Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/MapBindingSetsOverShards.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/MapBindingSetsOverShards.java 2010-08-27 14:28:04 UTC (rev 3466) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/MapBindingSetsOverShards.java 2010-08-27 14:57:24 UTC (rev 3467) @@ -154,7 +154,7 @@ } - public Future<Void> eval(final BOpContext<IBindingSet> context) { + public FutureTask<Void> eval(final BOpContext<IBindingSet> context) { if (context.getFederation() == null) { @@ -169,16 +169,9 @@ } - /* - * Note: The caller's BlockingBuffer is ignored. - */ - final FutureTask<Void> ft = new FutureTask<Void>(new MapShardsTask( - context, sourceOp(), targetPred())); + return new FutureTask<Void>(new MapShardsTask(context, sourceOp(), + targetPred())); - context.getIndexManager().getExecutorService().execute(ft); - - return ft; - } /** Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/PipelineDelayOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/PipelineDelayOp.java 2010-08-27 14:28:04 UTC (rev 3466) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/PipelineDelayOp.java 2010-08-27 14:57:24 UTC (rev 3467) @@ -2,9 +2,10 @@ import java.util.Map; import java.util.concurrent.Callable; -import java.util.concurrent.Future; import java.util.concurrent.FutureTask; +import junit.framework.AssertionFailedError; + import com.bigdata.bop.AbstractPipelineOp; import com.bigdata.bop.BOp; import com.bigdata.bop.BOpContext; @@ -12,8 +13,8 @@ import com.bigdata.bop.IBindingSet; /** - * This operator is used to feed the first join in the pipeline. The operator - * should have no children but may be decorated with annotations as necessary. + * Operator block evaluation for the specified {@link Annotations#DELAY} and + * then throws an {@link AssertionFailedError}. * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id$ @@ -66,23 +67,17 @@ return delay.longValue(); } - /** - * Blocks for the specified {@link Annotations#DELAY}. - */ - public Future<Void> eval(final BOpContext<IBindingSet> context) { + public FutureTask<Void> eval(final BOpContext<IBindingSet> context) { final FutureTask<Void> ft = new FutureTask<Void>(new Callable<Void>() { public Void call() throws Exception { Thread.sleep(delayMillis()); - TestQueryEngine.fail(); - return null; + throw new AssertionFailedError(); } }); - context.getIndexManager().getExecutorService().execute(ft); - return ft; } -} \ No newline at end of file +} Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/QueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/QueryEngine.java 2010-08-27 14:28:04 UTC (rev 3466) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/QueryEngine.java 2010-08-27 14:57:24 UTC (rev 3467) @@ -59,7 +59,6 @@ import com.bigdata.relation.accesspath.IBlockingBuffer; import com.bigdata.resources.ResourceManager; import com.bigdata.service.IBigdataFederation; -import com.bigdata.striterator.IChunkedIterator; import com.bigdata.striterator.ICloseableIterator; /** @@ -429,10 +428,12 @@ private final Map<Integer,BOp> bopIndex; /** - * A collection of the currently executing futures. {@link Future}s are - * added to this collection + * A collection of the currently executing futures. {@link Future}s are + * added to this collection by {@link #newChunkTask(BindingSetChunk)}. + * They are removed when they are {@link Future#isDone()}. + * {@link Future}s are cancelled if the {@link RunningQuery} is halted. */ - private final ConcurrentHashMap<Future<?>,Future<?>> futures = new ConcurrentHashMap<Future<?>, Future<?>>(); + private final ConcurrentHashMap<Future<?>, Future<?>> futures = new ConcurrentHashMap<Future<?>, Future<?>>(); /** * @@ -459,6 +460,19 @@ } /** + * Create a {@link BindingSetChunk} from a sink and add it to the queue. + * + * @param sinkId + * @param sink + * + * @todo In scale-out, this is where we need to map the binding sets + * over the shards for the target operator. + */ + private void add(final int sinkId, final IBlockingBuffer<?> sink) { + throw new UnsupportedOperationException(); + } + + /** * Make a chunk of binding sets available for consumption by the query. * * @param chunk @@ -508,6 +522,9 @@ * nodes). * * @todo If eval of the chunk fails, halt() the query. + * + * @todo evaluation of element[] pipelines might run into type + * problems with the [queryBuffer]. */ final BOp bop = bopIndex.get(chunk.bopId); if (bop == null) { @@ -517,18 +534,47 @@ throw new UnsupportedOperationException(bop.getClass() .getName()); } + // sink + final Integer sinkId = null;// @todo from annotation (it is the parent). final IBlockingBuffer<?> sink = ((PipelineOp<?>) bop).newBuffer(); + // altSink final Integer altSinkId = (Integer) bop .getProperty(BindingSetPipelineOp.Annotations.ALT_SINK_REF); if (altSinkId != null && !bopIndex.containsKey(altSinkId)) { throw new NoSuchBOpException(altSinkId); } - final IBlockingBuffer<?> sink2 = altSinkId == null ? null + final IBlockingBuffer<?> altSink = altSinkId == null ? null : ((PipelineOp<?>) bop).newBuffer(); + // context final BOpContext context = new BOpContext(queryEngine.fed, queryEngine.indexManager, readTimestamp, writeTimestamp, - chunk.partitionId, stats, chunk.source, sink, sink2); - return ((PipelineOp)bop).eval(context); + chunk.partitionId, stats, chunk.source, sink, altSink); + // FutureTask for operator execution (not running yet). + final FutureTask<Void> f = ((PipelineOp)bop).eval(context); + // Hook the FutureTask. + final Runnable r = new Runnable() { + public void run() { + try { + f.run(); // run + f.get(); // verify success + add(sinkId, sink); // handle output chunk. + if (altSink != null) // handle alt sink output chunk. + add(altSinkId, altSink); + } catch (Throwable t) { + // operator failed on this chunk. + RunningQuery.this + .cancel(true/* mayInterruptIfRunning */); + log.error("queryId=" + queryId + ",bopId=" + + chunk.bopId + ",partitionId=" + + chunk.partitionId + " : " + t); + } + } + }; + // wrap runnable. + final FutureTask<Void> f2 = new FutureTask(r, null/* result */); + // add to list of active futures. + futures.put(f2, f2); + return f; } /** @@ -541,8 +587,8 @@ * @todo Do all queries produce solutions (mutation operations might * return a mutation count, but they do not return solutions). */ - public IChunkedIterator<IBindingSet> iterator() { - throw new UnsupportedOperationException(); + public IAsynchronousIterator<IBindingSet[]> iterator() { + return queryBuffer.iterator(); } /* @@ -553,7 +599,10 @@ * query. */ - final public boolean cancel(boolean mayInterruptIfRunning) { + final public boolean cancel(final boolean mayInterruptIfRunning) { + for (Future<?> f : futures.keySet()) { + f.cancel(mayInterruptIfRunning); + } return future.cancel(mayInterruptIfRunning); } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/ReceiveBindingSets.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/ReceiveBindingSets.java 2010-08-27 14:28:04 UTC (rev 3466) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/ReceiveBindingSets.java 2010-08-27 14:57:24 UTC (rev 3467) @@ -28,7 +28,7 @@ package com.bigdata.bop.engine; import java.util.Map; -import java.util.concurrent.Future; +import java.util.concurrent.FutureTask; import org.apache.log4j.Logger; @@ -74,7 +74,7 @@ } - public Future<Void> eval(final BOpContext<IBindingSet> context) { + public FutureTask<Void> eval(final BOpContext<IBindingSet> context) { if (context.getFederation() == null) { Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-08-27 14:28:04 UTC (rev 3466) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-08-27 14:57:24 UTC (rev 3467) @@ -247,7 +247,7 @@ assertEquals(1L, stats.chunksOut.get()); // Verify no results. - final IChunkedIterator<IBindingSet> itr = runningQuery.iterator(); + final IAsynchronousIterator<IBindingSet[]> itr = runningQuery.iterator(); try { if (itr.hasNext()) fail("Not expecting any solutions"); @@ -323,14 +323,18 @@ -1, //partitionId newBindingSetIterator(EmptyBindingSet.INSTANCE))); - final IChunkedIterator<IBindingSet> itr = runningQuery.iterator(); + final IAsynchronousIterator<IBindingSet[]> itr = runningQuery.iterator(); try { int n = 0; while (itr.hasNext()) { - final IBindingSet e = itr.next(); + final IBindingSet[] e = itr.next(); if (log.isInfoEnabled()) - log.info(n + " : " + e); - assertEquals(expected[n], e); + log.info(n + " : chunkSize=" + e.length); + for (int i = 0; i < e.length; i++) { + if (log.isInfoEnabled()) + log.info(n + " : " + e[i]); + assertEquals(expected[n], e[i]); + } n++; } } finally { Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/join/TestPipelineJoin.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/join/TestPipelineJoin.java 2010-08-27 14:28:04 UTC (rev 3466) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/join/TestPipelineJoin.java 2010-08-27 14:57:24 UTC (rev 3467) @@ -28,6 +28,8 @@ package com.bigdata.bop.join; import java.util.Properties; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.FutureTask; import junit.framework.TestCase2; @@ -189,8 +191,10 @@ * (T2, B3) // T2:(Paul loves Leon) with B3:[A=Leon, B=Paul, ...]. * (T3, B2) // T3:(Leon loves Leon) with T2:[A=Paul, B=Leon, ...]. * </pre> + * @throws ExecutionException + * @throws InterruptedException */ - public void test_pipelineJoin() { + public void test_pipelineJoin() throws InterruptedException, ExecutionException { final int startId = 1; final int joinId = 2; @@ -241,7 +245,11 @@ ITx.UNISOLATED/* writeTimestamp */, -1/* partitionId */, stats, source, sink, null/* sink2 */); - query.eval(context); + // get task. + final FutureTask<Void> ft = query.eval(context); + + // execute task. + jnl.getExecutorService().execute(ft); final IAsynchronousIterator<IBindingSet[]> itr = sink.iterator(); try { @@ -268,6 +276,10 @@ assertEquals(1L, stats.accessPathCount.get()); assertEquals(1L, stats.chunkCount.get()); assertEquals(1L, stats.elementCount.get()); + + assertTrue(ft.isDone()); + assertFalse(ft.isCancelled()); + ft.get(); // verify nothing thrown. } This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2010-09-01 18:27:44
|
Revision: 3489 http://bigdata.svn.sourceforge.net/bigdata/?rev=3489&view=rev Author: thompsonbry Date: 2010-09-01 18:27:35 +0000 (Wed, 01 Sep 2010) Log Message: ----------- Added life cycle hooks for operator evaluation to the query engine. Modified the operator evaluation context to use the bop annotations for the access path. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpUtility.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestBOpUtility.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/IQueryClient.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/QueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/RunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestAll.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/join/TestPipelineJoin.java Added Paths: ----------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/DuplicateBOpException.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/BOpShard.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/HaltOpMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/PipelineUtility.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/StartOpMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestPipelineUtility.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine2.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java 2010-09-01 18:27:08 UTC (rev 3488) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java 2010-09-01 18:27:35 UTC (rev 3489) @@ -152,6 +152,16 @@ * The default timeout for operator evaluation. */ long DEFAULT_TIMEOUT = Long.MAX_VALUE; + + /** + * For hash partitioned operators, this is the set of the member nodes + * for the operator. + * <p> + * This annotation is required for such operators since the set of known + * nodes of a given type (such as all data services) can otherwise + * change at runtime. + */ + String MEMBER_SERVICES = "memberServices"; } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java 2010-09-01 18:27:08 UTC (rev 3488) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java 2010-09-01 18:27:35 UTC (rev 3489) @@ -30,29 +30,29 @@ import org.apache.log4j.Logger; import com.bigdata.bop.engine.BOpStats; +import com.bigdata.btree.IIndex; import com.bigdata.btree.ILocalBTreeView; +import com.bigdata.btree.IRangeQuery; import com.bigdata.journal.IIndexManager; import com.bigdata.journal.ITx; import com.bigdata.journal.TimestampUtility; -import com.bigdata.relation.AbstractRelation; import com.bigdata.relation.IRelation; +import com.bigdata.relation.accesspath.AccessPath; import com.bigdata.relation.accesspath.IAccessPath; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; import com.bigdata.relation.locator.IResourceLocator; import com.bigdata.relation.rule.IRule; import com.bigdata.relation.rule.eval.IJoinNexus; +import com.bigdata.service.DataService; import com.bigdata.service.IBigdataFederation; +import com.bigdata.striterator.IKeyOrder; /** * The evaluation context for the operator (NOT serializable). * * @param <E> * The generic type of the objects processed by the operator. - * - * @todo Make it easy to obtain another {@link BOpContext} in which the source - * or sink are different? E.g., for the evaluation of the right operand in - * a join. */ public class BOpContext<E> { @@ -189,6 +189,9 @@ * @throws IllegalArgumentException * if the <i>indexManager</i> is <code>null</code> * @throws IllegalArgumentException + * if the <i>indexManager</i> is is not a <em>local</em> index + * manager. + * @throws IllegalArgumentException * if the <i>readTimestamp</i> is {@link ITx#UNISOLATED} * (queries may not read on the unisolated indices). * @throws IllegalArgumentException @@ -210,6 +213,16 @@ final IBlockingBuffer<E[]> sink, final IBlockingBuffer<E[]> sink2) { if (indexManager == null) throw new IllegalArgumentException(); + if (indexManager instanceof IBigdataFederation<?>) { + /* + * This is disallowed because the predicate specifies an index + * partition and expects to have access to the local index objects + * for that index partition. + */ + throw new IllegalArgumentException( + "Expecting a local index manager, not: " + + indexManager.getClass().toString()); + } if (readTimestamp == ITx.UNISOLATED) throw new IllegalArgumentException(); if (TimestampUtility.isReadOnly(writeTimestamp)) @@ -263,7 +276,6 @@ } /** - /** * Obtain an access path reading from relation for the specified predicate * (from the tail of some rule). * <p> @@ -282,12 +294,44 @@ * * @return The access path. * - * @todo replaces {@link IJoinNexus#getTailAccessPath(IRelation, IPredicate)}. + * @todo replaces + * {@link IJoinNexus#getTailAccessPath(IRelation, IPredicate)}. */ @SuppressWarnings("unchecked") public IAccessPath<?> getAccessPath(final IRelation<?> relation, final IPredicate<?> predicate) { + if (relation == null) + throw new IllegalArgumentException(); + + if (predicate == null) + throw new IllegalArgumentException(); + + final IKeyOrder keyOrder = relation.getKeyOrder((IPredicate) predicate); + + if (keyOrder == null) + throw new RuntimeException("No access path: " + predicate); + + final int partitionId = predicate.getPartitionId(); + + final int flags = predicate.getProperty( + PipelineOp.Annotations.FLAGS, + PipelineOp.Annotations.DEFAULT_FLAGS) + | (TimestampUtility.isReadOnly(getReadTimestamp()) ? IRangeQuery.READONLY + : 0); + + final int chunkOfChunksCapacity = predicate.getProperty( + PipelineOp.Annotations.CHUNK_OF_CHUNKS_CAPACITY, + PipelineOp.Annotations.DEFAULT_CHUNK_OF_CHUNKS_CAPACITY); + + final int chunkCapacity = predicate.getProperty( + PipelineOp.Annotations.CHUNK_CAPACITY, + PipelineOp.Annotations.DEFAULT_CHUNK_CAPACITY); + + final int fullyBufferedReadThreshold = predicate.getProperty( + PipelineOp.Annotations.FULLY_BUFFERED_READ_THRESHOLD, + PipelineOp.Annotations.DEFAULT_FULLY_BUFFERED_READ_THRESHOLD); + if (predicate.getPartitionId() != -1) { /* @@ -299,18 +343,64 @@ * require a total view of the relation, which is not available * during scale-out pipeline joins. Likewise, the [backchain] * property will be ignored since it is handled by an expander. + * + * @todo Replace this with IRelation#getAccessPathForIndexPartition() */ +// return ((AbstractRelation<?>) relation) +// .getAccessPathForIndexPartition(indexManager, +// (IPredicate) predicate); + /* + * @todo This condition should probably be an error since the expander + * will be ignored. + */ +// if (predicate.getSolutionExpander() != null) +// throw new IllegalArgumentException(); + + final String namespace = relation.getNamespace();//predicate.getOnlyRelationName(); - return ((AbstractRelation<?>) relation) - .getAccessPathForIndexPartition(indexManager, - (IPredicate) predicate); + // The name of the desired index partition. + final String name = DataService.getIndexPartitionName(namespace + + "." + keyOrder.getIndexName(), partitionId); + // MUST be a local index view. + final ILocalBTreeView ndx = (ILocalBTreeView) indexManager + .getIndex(name, readTimestamp); + + return new AccessPath(relation, indexManager, readTimestamp, + predicate, keyOrder, ndx, flags, chunkOfChunksCapacity, + chunkCapacity, fullyBufferedReadThreshold).init(); + } - // Find the best access path for the predicate for that relation. - final IAccessPath<?> accessPath = relation - .getAccessPath((IPredicate) predicate); + /* + * Find the best access path for the predicate for that relation. + * + * @todo Replace this with IRelation#getAccessPath(IPredicate) once the + * bop conversion is done. It is the same logic. + */ + IAccessPath accessPath; + { +// accessPath = relation.getAccessPath((IPredicate) predicate); + + final IIndex ndx = relation.getIndex(keyOrder); + + if (ndx == null) { + + throw new IllegalArgumentException("no index? relation=" + + relation.getNamespace() + ", timestamp=" + + readTimestamp + ", keyOrder=" + keyOrder + ", pred=" + + predicate + ", indexManager=" + getIndexManager()); + + } + + accessPath = new AccessPath((IRelation) relation, indexManager, + readTimestamp, (IPredicate) predicate, + (IKeyOrder) keyOrder, ndx, flags, chunkOfChunksCapacity, + chunkCapacity, fullyBufferedReadThreshold).init(); + + } + /* * @todo No expander's for bops, at least not right now. They could be * added in easily enough, which would support additional features for Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpUtility.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpUtility.java 2010-09-01 18:27:08 UTC (rev 3488) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpUtility.java 2010-09-01 18:27:35 UTC (rev 3489) @@ -29,8 +29,12 @@ import java.util.Iterator; import java.util.LinkedHashMap; +import java.util.LinkedHashSet; import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.log4j.Logger; + import com.bigdata.bop.BOp.Annotations; import com.bigdata.btree.AbstractNode; @@ -44,12 +48,11 @@ * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id$ - * - * @todo In general recursive traversal iterators do not protect against loops - * in the operator tree, but see {@link #getIndex(BOp)}. */ public class BOpUtility { + private static final Logger log = Logger.getLogger(BOpUtility.class); + /** * Pre-order recursive visitation of the operator tree (arguments only, no * annotations). @@ -361,36 +364,53 @@ * Return an index from the {@link BOp.Annotations#BOP_ID} to the * {@link BOp} for each spanned {@link BOp} (including annotations). * {@link BOp}s without identifiers are not indexed. + * <p> + * {@link BOp}s should form directed acyclic graphs, but this is not + * strictly enforced. The recursive traversal iterators declared by this + * class do not protect against loops in the operator tree. However, + * {@link #getIndex(BOp)} detects and report loops based on duplicate + * {@link Annotations#BOP_ID}s -or- duplicate {@link BOp} references. * * @param op * A {@link BOp}. * * @return The index. * - * @todo define recursive striterator for {@link BOp}s (as top-level method) - * and then layer on an expander for the {@link BOp} annotations. - * Finally, layer in a filter for the presence of the bopId. The - * {@link BOp}s visited by the iterator should be inserted into the - * indexed. [it is an error if there is a duplicate bopId.] + * @throws DuplicateBOpIdException + * if there are two or more {@link BOp}s having the same + * {@link Annotations#BOP_ID}. + * @throws BadBOpIdTypeException + * if the {@link Annotations#BOP_ID} is not an {@link Integer}. + * @throws DuplicateBOpException + * if the same {@link BOp} appears more once in the operator + * tree and it is neither an {@link IVariable} nor an + * {@link IConstant}. */ static public Map<Integer,BOp> getIndex(final BOp op) { final LinkedHashMap<Integer, BOp> map = new LinkedHashMap<Integer, BOp>(); + final LinkedHashSet<BOp> distinct = new LinkedHashSet<BOp>(); final Iterator<BOp> itr = preOrderIteratorWithAnnotations(op); while (itr.hasNext()) { final BOp t = itr.next(); final Object x = t.getProperty(Annotations.BOP_ID); - if (x == null) { - continue; + if (x != null) { + if (!(x instanceof Integer)) { + throw new BadBOpIdTypeException("Must be Integer, not: " + + x.getClass() + ": " + Annotations.BOP_ID); + } + final Integer id = (Integer) t.getProperty(Annotations.BOP_ID); + final BOp conflict = map.put(id, t); + if (conflict != null) + throw new DuplicateBOpIdException("duplicate id=" + id + + " for " + conflict + " and " + t); } - if (!(x instanceof Integer)) { - throw new BadBOpIdTypeException("Must be Integer, not: " - + x.getClass() + ": " + Annotations.BOP_ID); + if (!distinct.add(t) && !(t instanceof IVariableOrConstant<?>)) { + /* + * BOp appears more than once. This is only allowed for + * constants and variables. + */ + throw new DuplicateBOpException(t.toString()); } - final Integer id = (Integer) t.getProperty(Annotations.BOP_ID); - final BOp conflict = map.put(id, t); - if (conflict != null) - throw new DuplicateBOpIdException("duplicate id=" + id + " for " - + conflict + " and " + t); } return map; } Added: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/DuplicateBOpException.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/DuplicateBOpException.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/DuplicateBOpException.java 2010-09-01 18:27:35 UTC (rev 3489) @@ -0,0 +1,24 @@ +package com.bigdata.bop; + +/** + * Exception thrown when a {@link BOp} appears more than once in an operator + * tree (operator trees must not contain loops and operator instances may not + * appear more than once unless they are an {@link IConstant} or an + * {@link IVariable}). + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id: DuplicateBOpIdException.java 3466 2010-08-27 14:28:04Z + * thompsonbry $ + */ +public class DuplicateBOpException extends RuntimeException { + + /** + * @param msg + */ + public DuplicateBOpException(String msg) { + super(msg); + } + + private static final long serialVersionUID = 1L; + +} \ No newline at end of file Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/DuplicateBOpException.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java 2010-09-01 18:27:08 UTC (rev 3488) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java 2010-09-01 18:27:35 UTC (rev 3489) @@ -30,6 +30,8 @@ import java.util.concurrent.FutureTask; import com.bigdata.bop.engine.BOpStats; +import com.bigdata.btree.IRangeQuery; +import com.bigdata.relation.accesspath.AccessPath; import com.bigdata.relation.accesspath.BlockingBuffer; import com.bigdata.relation.accesspath.IBlockingBuffer; import com.bigdata.relation.accesspath.IBuffer; @@ -55,6 +57,8 @@ * would block (default {@value #DEFAULT_CHUNK_OF_CHUNKS_CAPACITY}). * Note that partial chunks may be combined into full chunks whose * nominal capacity is specified by {@link #CHUNK_CAPACITY}. + * + * @see #DEFAULT_CHUNK_OF_CHUNKS_CAPACITY */ String CHUNK_OF_CHUNKS_CAPACITY = PipelineOp.class.getName() + ".chunkOfChunksCapacity"; @@ -69,6 +73,7 @@ * of {@link IBindingSet}s (default {@value #CHUNK_CAPACITY}). Partial * chunks may be automatically combined into full chunks. * + * @see #DEFAULT_CHUNK_CAPACITY * @see #CHUNK_OF_CHUNKS_CAPACITY */ String CHUNK_CAPACITY = PipelineOp.class.getName() + ".chunkCapacity"; @@ -83,6 +88,8 @@ * for another chunk to combine with the current chunk before returning * the current chunk (default {@value #DEFAULT_CHUNK_TIMEOUT}). This may * be ZERO (0) to disable the chunk combiner. + * + * @see #DEFAULT_CHUNK_TIMEOUT */ String CHUNK_TIMEOUT = PipelineOp.class.getName() + ".chunkTimeout"; @@ -93,6 +100,46 @@ */ int DEFAULT_CHUNK_TIMEOUT = 1000; + /** + * If the estimated rangeCount for an {@link AccessPath#iterator()} is + * LTE this threshold then use a fully buffered (synchronous) iterator. + * Otherwise use an asynchronous iterator whose capacity is governed by + * {@link #CHUNK_OF_CHUNKS_CAPACITY}. + * + * @see #DEFAULT_FULLY_BUFFERED_READ_THRESHOLD + */ + String FULLY_BUFFERED_READ_THRESHOLD = PipelineOp.class.getName() + + ".fullyBufferedReadThreshold"; + + /** + * Default for {@link #FULLY_BUFFERED_READ_THRESHOLD}. + * + * @todo try something closer to the branching factor, e.g., 100. + */ + int DEFAULT_FULLY_BUFFERED_READ_THRESHOLD = 1000; + + /** + * Flags for the iterator ({@link IRangeQuery#KEYS}, + * {@link IRangeQuery#VALS}, {@link IRangeQuery#PARALLEL}). + * <p> + * Note: The {@link IRangeQuery#PARALLEL} flag here is an indication + * that the iterator may run in parallel across the index partitions. + * This only effects scale-out and only for simple triple patterns since + * the pipeline join does something different (it runs inside the index + * partition using the local index, not the client's view of a + * distributed index). + * + * @see #DEFAULT_FLAGS + */ + String FLAGS = PipelineOp.class.getName() + ".flags"; + + /** + * The default flags will visit the keys and values of the non-deleted + * tuples and allows parallelism in the iterator (when supported). + */ + final int DEFAULT_FLAGS = IRangeQuery.KEYS | IRangeQuery.VALS + | IRangeQuery.PARALLEL; + } /** Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 2010-09-01 18:27:08 UTC (rev 3488) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 2010-09-01 18:27:35 UTC (rev 3489) @@ -39,7 +39,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; import java.util.concurrent.FutureTask; import java.util.concurrent.TimeUnit; @@ -59,7 +58,6 @@ import com.bigdata.btree.BytesUtil; import com.bigdata.btree.keys.IKeyBuilder; import com.bigdata.counters.CAT; -import com.bigdata.journal.IIndexManager; import com.bigdata.relation.IRelation; import com.bigdata.relation.accesspath.AbstractUnsynchronizedArrayBuffer; import com.bigdata.relation.accesspath.AccessPath; @@ -73,8 +71,6 @@ import com.bigdata.relation.rule.IStarJoin; import com.bigdata.relation.rule.IStarJoin.IStarConstraint; import com.bigdata.relation.rule.eval.ISolution; -import com.bigdata.relation.rule.eval.pipeline.DistributedJoinTask; -import com.bigdata.relation.rule.eval.pipeline.JoinMasterTask; import com.bigdata.service.DataService; import com.bigdata.striterator.IChunkedOrderedIterator; import com.bigdata.striterator.IKeyOrder; @@ -95,30 +91,14 @@ * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id$ * - * @todo There is only one source, even if scale-out, and the {@link JoinTask} - * runs only for the duration of that source. The termination conditions - * for query evaluation are handled outside of the operator - * implementation. - * <p> - * The first join dimension always has a single source - the - * initialBindingSet established by the {@link JoinMasterTask}. Downstream - * join dimensions read from {@link IAsynchronousIterator} (s) from the - * upstream join dimension. When the {@link IIndexManager} allows - * key-range partitions, then the fan-in for the sources may be larger - * than one as there will be one {@link JoinTask} for each index partition - * touched by each join dimension. - * - * @todo provide more control over the access path (fully buffered read - * thresholds). - * - * @todo Do we need to hook the source and sink {@link Future}s? - * * @todo Break the star join logic out into its own join operator. * * @todo Implement operator at a time or mega-chunk pipeline operators for high * volume query. These will differ by running across the entire shard on * the right hand operator using multi-block IO each time they process a * (mega-)chunk of bindings from the left hand operator. + * + * @todo Support SLICE via annotations. */ public class PipelineJoin extends AbstractPipelineOp<IBindingSet> implements BindingSetPipelineOp { @@ -334,12 +314,6 @@ */ private static class JoinTask extends Haltable<Void> implements Callable<Void> { -// /** -// * The federation reference is passed along when we evaluate the -// * {@link #left} operand. -// */ -// final protected IBigdataFederation<?> fed; - /** * The join that is being executed. */ @@ -361,15 +335,15 @@ * the failed joined needs to jump out of a join group rather than * routing directly to the ancestor in the operator tree. * - * @todo Support for the {@link #optionalSink} is not finished. When the - * optional target is not simply the direct ancestor in the - * operator tree then we need to have a separate thread local - * buffering in front of the optional sink for the join task. This - * means that we need to use two {@link #threadLocalBufferFactory} - * s, one for the optional path. All of this only matters when the - * binding sets are being routed out of an optional join group. - * When the tails are independent optionals then the target is the - * same as the target for binding sets which do join. + * FIXME Support for the {@link #optionalSink} is not finished. When the + * optional target is not simply the direct ancestor in the operator + * tree then we need to have a separate thread local buffering in front + * of the optional sink for the join task. This means that we need to + * use two {@link #threadLocalBufferFactory} s, one for the optional + * path. All of this only matters when the binding sets are being routed + * out of an optional join group. When the tails are independent + * optionals then the target is the same as the target for binding sets + * which do join. */ final IBlockingBuffer<IBindingSet[]> optionalSink; @@ -406,82 +380,6 @@ */ final protected BOpContext<IBindingSet> context; -// /** -// * Volatile flag is set <code>true</code> if the {@link JoinTask} -// * (including any tasks executing on its behalf) should halt. This flag -// * is monitored by the {@link BindingSetConsumerTask}, the -// * {@link AccessPathTask}, and the {@link ChunkTask}. It is set by any -// * of those tasks if they are interrupted or error out. -// * -// * @todo review handling of this flag. Should an exception always be -// * thrown if the flag is set wrapping the {@link #firstCause}? Are -// * there any cases where the behavior should be different? If not, -// * then replace tests with halt() and encapsulate the logic in -// * that method. -// */ -// volatile protected boolean halt = false; -// -// /** -// * Set by {@link BindingSetConsumerTask}, {@link AccessPathTask}, and -// * {@link ChunkTask} if they throw an error. Tasks are required to use -// * an {@link AtomicReference#compareAndSet(Object, Object)} and must -// * specify <code>null</code> as the expected value. This ensures that -// * only the first cause is recorded by this field. -// */ -// final protected AtomicReference<Throwable> firstCause = new AtomicReference<Throwable>( -// null); -// -// /** -// * Indicate that join processing should halt. This method is written -// * defensively and will not throw anything. -// * -// * @param cause -// * The cause. -// */ -// protected void halt(final Throwable cause) { -// -// halt = true; -// -// final boolean isFirstCause = firstCause.compareAndSet( -// null/* expect */, cause); -// -// if (log.isEnabledFor(Level.WARN)) -// -// try { -// -// if (!InnerCause.isInnerCause(cause, -// InterruptedException.class) -// && !InnerCause.isInnerCause(cause, -// CancellationException.class) -// && !InnerCause.isInnerCause(cause, -// ClosedByInterruptException.class) -// && !InnerCause.isInnerCause(cause, -// RejectedExecutionException.class) -// && !InnerCause.isInnerCause(cause, -// BufferClosedException.class)) { -// -// /* -// * This logs all unexpected causes, not just the first -// * one to be reported for this join task. -// * -// * Note: The master will log the firstCause that it -// * receives as an error. -// */ -// -// log.warn("joinOp=" + joinOp + ", isFirstCause=" -// + isFirstCause + " : " -// + cause.getLocalizedMessage(), cause); -// -// } -// -// } catch (Throwable ex) { -// -// // error in logging system - ignore. -// -// } -// -// } - /** * The statistics for this {@link JoinTask}. */ @@ -797,26 +695,9 @@ } catch (Throwable t) { - try { - logCallError(t); - } catch (Throwable t2) { - log.error(t2.getLocalizedMessage(), t2); - } - /* * This is used for processing errors and also if this task is - * interrupted (because a SLICE has been satisfied). - * - * @todo For a SLICE, consider that the query solution buffer - * proxy could return the #of solutions added so far so that we - * can halt each join task on the last join dimension in a - * relatively timely manner producing no more than one chunk too - * many (actually, it might not be that timely since some index - * partitions might not produce any solutions; this suggests - * that the master might need a fatter API than a Future for the - * JoinTask so that it can directly notify the JoinTasks for the - * first predicate and they can propagate that notice downstream - * to their sinks). This will be an issue when fanOut GT ONE. + * interrupted (because the sink has been closed). */ halt(t); @@ -836,13 +717,6 @@ log.error(t2.getLocalizedMessage(), t2); } -// // report join stats _before_ we close our source(s). -// try { -// reportOnce(); -// } catch (Throwable t2) { -// log.error(t2.getLocalizedMessage(), t2); -// } - /* * Close source iterators, which will cause any source JoinTasks * that are still executing to throw a CancellationException @@ -857,61 +731,11 @@ throw new RuntimeException(t); - } finally { - -// // report join stats iff they have not already been reported. -// reportOnce(); - } } /** - * Method is used to log the primary exception thrown by {@link #call()} - * . The default implementation does nothing and the exception will be - * logged by the {@link JoinMasterTask}. However, this method is - * overridden by {@link DistributedJoinTask} so that the exception can - * be logged on the host and {@link DataService} where it originates. - * This appears to be necessary in order to trace back the cause of an - * exception which can otherwise be obscured (or even lost?) in a deeply - * nested RMI stack trace. - * - * @param o - * @param t - */ - protected void logCallError(Throwable t) { - - } - -// /** -// * Method reports {@link JoinStats} to the {@link JoinMasterTask}, but -// * only if they have not already been reported. This "report once" -// * constraint is used to make it safe to invoke during error handling -// * before actions which could cause the source {@link JoinTask}s (and -// * hence the {@link JoinMasterTask}) to terminate. -// */ -// protected void reportOnce() { -// -// if (didReport.compareAndSet(false/* expect */, true/* update */)) { -// -//// try { -//// -////// @todo report statistics to the master. -//// masterProxy.report(stats); -//// -//// } catch (IOException ex) { -//// -//// log.warn("Could not report statistics to the master", ex); -//// -//// } -// -// } -// -// } -// -// private final AtomicBoolean didReport = new AtomicBoolean(false); - - /** * Consume {@link IBindingSet} chunks from the {@link #sink}. * * @throws Exception Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestBOpUtility.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestBOpUtility.java 2010-09-01 18:27:08 UTC (rev 3488) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestBOpUtility.java 2010-09-01 18:27:35 UTC (rev 3489) @@ -416,7 +416,7 @@ } /** - * Unit test for {@link BOpUtility#getIndex(BOp)}. + * Unit test for {@link BOpUtility#getIndex(BOp)} using valid inputs. */ public void test_getIndex() { @@ -489,7 +489,7 @@ /** * Unit test for {@link BOpUtility#getIndex(BOp)} in which we verify that it - * rejects operator trees operator ids which are not {@link Integer}s. + * rejects operator trees with operator ids which are not {@link Integer}s. */ public void test_getIndex_rejectsNonIntegerIds() { @@ -510,6 +510,65 @@ } /** + * Unit test for {@link BOpUtility#getIndex(BOp)} in which we verify that it + * rejects operator trees in which the same {@link BOp} reference appears + * more than once but allows duplicate {@link IVariable}s and + * {@link IConstant}s. + */ + public void test_getIndex_duplicateBOps() { + + final IConstant<Long> c1 = new Constant<Long>(12L); + final IVariable<?> v1 = Var.var("y"); + + /* + * Operator tree with duplicate variable and duplicate constant refs. + */ + { + // root + final BOp root = new BOpBase(new BOp[] { // root args[] + c1, v1 }, NV.asMap(new NV[] {// + new NV(BOp.Annotations.BOP_ID, 4),// + new NV("foo", v1), // duplicate variable. + new NV("bar", c1) // duplicate variable. + })); + + // should be Ok. + final Map<Integer, BOp> map = BOpUtility.getIndex(root); + + assertTrue(root == map.get(4)); + + } + + /* + * Operator tree with duplicate bop which is neither a var nor or a + * constant. + */ + { + + /* + * bop w/o bopId is used to verify correct detection of duplicate + * references. + */ + final BOp op2 = new BOpBase(new BOp[]{}, null/*annotations*/); + + // root + final BOp root = new BOpBase(new BOp[] { // root args[] + op2, op2 }, NV.asMap(new NV[] {// + new NV(BOp.Annotations.BOP_ID, 4),// + })); + + try { + BOpUtility.getIndex(root); + fail("Expecting: " + DuplicateBOpException.class); + } catch (DuplicateBOpException ex) { + if (log.isInfoEnabled()) + log.info("Ignoring expected exception: " + ex); + } + } + + } + + /** * Unit test for {@link BOpUtility#getParent(BOp, BOp)}. */ public void test_getParent() { Added: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/BOpShard.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/BOpShard.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/BOpShard.java 2010-09-01 18:27:35 UTC (rev 3489) @@ -0,0 +1,75 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Sep 1, 2010 + */ + +package com.bigdata.bop.engine; + +/** + * An immutable class capturing the evaluation context of an operator against a + * shard. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +public class BOpShard { + + public final int bopId; + + public final int shardId; + + public BOpShard(final int bopId, final int shardId) { + + this.bopId = bopId; + + this.shardId = shardId; + + } + + /** + * {@inheritDoc} + * + * @todo verify that this is a decent hash function. + */ + public int hashCode() { + + return (bopId * 31) + shardId; + + } + + public boolean equals(final Object o) { + + if (this == o) + return true; + + if (!(o instanceof BOpShard)) + return false; + + return bopId == ((BOpShard) o).bopId + && shardId == ((BOpShard) o).shardId; + + } + +} Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/BOpShard.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Added: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/HaltOpMessage.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/HaltOpMessage.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/HaltOpMessage.java 2010-09-01 18:27:35 UTC (rev 3489) @@ -0,0 +1,135 @@ +package com.bigdata.bop.engine; + +import java.io.Serializable; +import java.util.UUID; + +/** + * A message sent to the {@link IQueryClient} when an operator is done executing + * for some chunk of inputs. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +public class HaltOpMessage implements Serializable { + + /** + * + */ + private static final long serialVersionUID = 1L; + + /** The identifier of the query. */ + final long queryId; + + /** The identifier of the operator. */ + final int bopId; + + /** + * The index partition identifier against which the operator was + * executing. + */ + final int partitionId; + + /** + * The identifier of the service on which the operator was executing. + */ + final UUID serviceId; + + /** + * * The cause and <code>null</code> if the operator halted normally. + */ + final Throwable cause; + + /** + * The operator identifier for the primary sink -or- <code>null</code> + * if there is no primary sink (for example, if this is the last + * operator in the pipeline). + */ + final Integer sinkId; + + /** + * The number of the {@link BindingSetChunk}s that were output for the + * primary sink. (This information is used for the atomic termination + * decision.) + * <p> + * For a given downstream operator this is ONE (1) for scale-up. For + * scale-out, this is one per index partition over which the + * intermediate results were mapped. + */ + final int sinkChunksOut; + + /** + * The operator identifier for the alternative sink -or- + * <code>null</code> if there is no alternative sink. + */ + final Integer altSinkId; + + /** + * The number of the {@link BindingSetChunk}s that were output for the + * alternative sink. (This information is used for the atomic + * termination decision.) + * <p> + * For a given downstream operator this is ONE (1) for scale-up. For + * scale-out, this is one per index partition over which the + * intermediate results were mapped. It is zero if there was no + * alternative sink for the operator. + */ + final int altSinkChunksOut; + + /** + * The statistics for the execution of the bop against the partition on + * the service. + */ + final BOpStats taskStats; + + /** + * @param queryId + * The query identifier. + * @param bopId + * The operator whose execution phase has terminated for a + * specific index partition and input chunk. + * @param partitionId + * The index partition against which the operator was + * executed. + * @param serviceId + * The node which executed the operator. + * @param cause + * <code>null</code> unless execution halted abnormally. + * @param chunksOut + * A map reporting the #of binding set chunks which were + * output for each downstream operator for which at least one + * chunk of output was produced. + * @param taskStats + * The statistics for the execution of that bop on that shard + * and service. + */ + public HaltOpMessage( + // + final long queryId, final int bopId, final int partitionId, + final UUID serviceId, Throwable cause, // + final Integer sinkId, final int sinkChunksOut,// + final Integer altSinkId, final int altSinkChunksOut,// + final BOpStats taskStats) { + + if (altSinkId != null && sinkId == null) { + // The primary sink must be defined if the altSink is defined. + throw new IllegalArgumentException(); + } + + if (sinkId != null && altSinkId != null + && sinkId.intValue() == altSinkId.intValue()) { + // The primary and alternative sink may not be the same operator. + throw new IllegalArgumentException(); + } + + this.queryId = queryId; + this.bopId = bopId; + this.partitionId = partitionId; + this.serviceId = serviceId; + this.cause = cause; + this.sinkId = sinkId; + this.sinkChunksOut = sinkChunksOut; + this.altSinkId = altSinkId; + this.altSinkChunksOut = altSinkChunksOut; + this.taskStats = taskStats; + } +} \ No newline at end of file Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/HaltOpMessage.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/IQueryClient.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/IQueryClient.java 2010-09-01 18:27:08 UTC (rev 3488) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/IQueryClient.java 2010-09-01 18:27:35 UTC (rev 3489) @@ -2,7 +2,6 @@ import java.rmi.Remote; import java.rmi.RemoteException; -import java.util.UUID; import com.bigdata.bop.BOp; @@ -74,51 +73,17 @@ /** * Notify the client that execution has started for some query, operator, * node, and index partition. - * - * @param queryId - * The query identifier. - * @param opId - * The operator identifier. - * @param partitionId - * The index partition identifier. - * @param serviceId - * The node on which the operator will execute. - * @param nchunks - * The #of chunks which form the input to that operator (for the - * atomic termination condition decision). */ - public void startOp(long queryId, int opId, int partitionId, UUID serviceId, final int nchunks) + public void startOp(StartOpMessage msg) throws RemoteException; /** * Notify the client that execution has halted for some query, operator, - * node and index partition. If execution halted abnormally, then the cause - * is sent as well. - * - * @param queryId - * The query identifier. - * @param opId - * The operator whose execution phase has terminated for a - * specific index partition and input chunk. - * @param partitionId - * The index partition against which the operator was executed. - * @param serviceId - * The node which executed the operator. - * @param cause - * <code>null</code> unless execution halted abnormally. - * @param nchunks - * The #of chunks which were output by the operator (for the - * atomic termination decision). This is ONE (1) for scale-up. - * For scale-out, this is one per index partition over which the - * intermediate results were mapped. - * @param taskStats - * The statistics for the execution of that bop on that shard and - * service. + * node, shard, and source binding set chunk(s). If execution halted + * abnormally, then the cause is sent as well. */ - public void haltOp(long queryId, int opId, int partitionId, UUID serviceId, - Throwable cause, int nchunks, BOpStats taskStats) - throws RemoteException; - + public void haltOp(HaltOpMessage msg) throws RemoteException; + // /** // * Notify the query controller that a chunk of intermediate results is // * available for the query. Added: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/PipelineUtility.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/PipelineUtility.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/PipelineUtility.java 2010-09-01 18:27:35 UTC (rev 3489) @@ -0,0 +1,156 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Sep 1, 2010 + */ + +package com.bigdata.bop.engine; + +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.log4j.Logger; + +import com.bigdata.bop.BOp; +import com.bigdata.bop.BOpUtility; +import com.bigdata.bop.NoSuchBOpException; + +/** + * Utility methods relevant to pipelined operator evaluation. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +public class PipelineUtility { + + private static final Logger log = Logger.getLogger(PipelineUtility.class); + + /** + * Return <code>true</code> iff the <i>runningCountMap</i> AND + * <i>availableChunkMap</i> map are ZERO (0) for both the given operator and + * for all operators which proceed the given operator in the tree structure + * of its operands. + * <p> + * Note: The movement of the intermediate binding set chunks forms an + * acyclic directed graph. We can decide whether or not a {@link BOp} in the + * query plan can be triggered by the current activity pattern by inspecting + * the {@link BOp} and its operands recursively. If neither the {@link BOp} + * nor any of its operands (recursively) has non-zero activity then the + * {@link BOp} can not be triggered and this method will return + * <code>true</code>. + * + * @param bopId + * The identifier for an operator which appears in the query + * plan. + * @param queryPlan + * The query plan. + * @param queryIndex + * An index for the query plan as constructed by + * {@link BOpUtility#getIndex(BOp)}. + * @param runningCountMap + * A map reporting the #of instances of each operator which are + * currently being evaluated (distinct evaluations are performed + * for each chunk and shard). + * @param availableChunkCountMap + * A map reporting the #of chunks available for each operator in + * the pipeline (we only report chunks for pipeline operators). + * + * @return <code>true</code> iff the {@link BOp} can not be triggered given + * the query plan and the activity map. + * + * @throws IllegalArgumentException + * if any argument is <code>null</code>. + * @throws NoSuchBOpException + * if <i>bopId</i> is not found in the query index. + */ + static public boolean isDone(final int bopId, final BOp queryPlan, + final Map<Integer, BOp> queryIndex, + final Map<Integer, AtomicLong> runningCountMap, + final Map<Integer, AtomicLong> availableChunkCountMap) { + + if (queryPlan == null) + throw new IllegalArgumentException(); + if (queryIndex == null) + throw new IllegalArgumentException(); + if (availableChunkCountMap == null) + throw new IllegalArgumentException(); + + final BOp op = queryIndex.get(bopId); + + if (op == null) + throw new NoSuchBOpException(bopId); + + final Iterator<BOp> itr = BOpUtility.preOrderIterator(op); + + while (itr.hasNext()) { + + final BOp t = itr.next(); + + final Integer id = (Integer) t.getProperty(BOp.Annotations.BOP_ID); + + if (id == null) + continue; + { + + final AtomicLong runningCount = runningCountMap.get(id); + + if (runningCount != null && runningCount.get() != 0) { + + if (log.isInfoEnabled()) + log.info("Operator can be triggered: op=" + op + + ", possible trigger=" + t + " is running."); + + return false; + + } + + } + + { + + final AtomicLong availableChunkCount = availableChunkCountMap + .get(id); + + if (availableChunkCount != null + && availableChunkCount.get() != 0) { + + if (log.isInfoEnabled()) + log.info("Operator can be triggered: op=" + op + + ", possible trigger=" + t + " has " + + availableChunkCount + " chunks available."); + + return false; + + } + + } + + } + + return true; + + } + +} Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/PipelineUtility.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/QueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/QueryEngine.java 2010-09-01 18:27:08 UTC (rev 3488) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/QueryEngine.java 2010-09-01 18:27:35 UTC (rev 3489) @@ -517,7 +517,7 @@ public void bufferReady(IQueryClient clientProxy, InetSocketAddress serviceAddr, long queryId, int bopId) { - // TODO SCALEOUT + // @todo SCALEOUT notify peer when a buffer is ready. } @@ -538,22 +538,28 @@ return null; } - public void startOp(final long queryId, final int opId, - final int partitionId, final UUID serviceId, final int nchunks) - throws RemoteException { - final RunningQuery q = runningQueries.get(queryId); + public void startOp(final StartOpMessage msg) throws RemoteException { + + final RunningQuery q = runningQueries.get(msg.queryId); + if (q != null) { - q.startOp(opId, partitionId, serviceId, nchunks); + + q.startOp(msg); + } + } - public void haltOp(final long queryId, final int opId, - final int partitionId, final UUID serviceId, final Throwable cause, - final int nchunks, final BOpStats taskStats) throws RemoteException { - final RunningQuery q = runningQueries.get(queryId); + public void haltOp(final HaltOpMessage msg) throws RemoteException { + + final RunningQuery q = runningQueries.get(msg.queryId); + if (q != null) { - q.haltOp(opId, partitionId, serviceId, cause, nchunks, taskStats); + + q.haltOp(msg); + } + } /** Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/RunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/RunningQuery.java 2010-09-01 18:27:08 UTC (rev 3488) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/RunningQuery.java 2010-09-01 18:27:35 UTC (rev 3489) @@ -29,7 +29,10 @@ import java.nio.ByteBuffer; import java.rmi.RemoteException; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; import java.util.Map; +import java.util.Set; import java.util.UUID; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; @@ -39,8 +42,8 @@ import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import org.apache.log4j.Logger; @@ -50,6 +53,8 @@ import com.bigdata.bop.BOpUtility; import com.bigdata.bop.BindingSetPipelineOp; import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.IConstraint; +import com.bigdata.bop.IPredicate; import com.bigdata.bop.NoSuchBOpException; import com.bigdata.bop.ap.Predicate; import com.bigdata.relation.accesspath.IAsynchronousIterator; @@ -134,6 +139,86 @@ final IBlockingBuffer<IBindingSet[]> queryBuffer; /** + * An index from the {@link BOp.Annotations#BOP_ID} to the {@link BOp}. + */ + private final Map<Integer, BOp> bopIndex; + + /** + * A collection of the currently executing future for operators for this + * query. + */ + private final ConcurrentHashMap<BOpShard, Future<?>> operatorFutures = new ConcurrentHashMap<BOpShard, Future<?>>(); + + /** + * A lock guarding {@link #runningTaskCount}, {@link #availableChunkCount}, + * {@link #availableChunkCountMap}. + */ + private final ReentrantLock runStateLock = new ReentrantLock(); + + /** + * The #of tasks for this query which have started but not yet halted and + * ZERO (0) if this is not the query coordinator. + * <p> + * This is guarded by the {@link #runningStateLock}. + */ + private long runningTaskCount = 0; + + /** + * The #of chunks for this query of which a running task has made available + * but which have not yet been accepted for processing by another task and + * ZERO (0) if this is not the query coordinator. + * <p> + * This is guarded by the {@link #runningStateLock}. + */ + private long availableChunkCount = 0; + + /** + * A map reporting the #of chunks available for each operator in the + * pipeline (we only report chunks for pipeline operators). The total #of + * chunks available for any given operator in the pipeline is reported by + * {@link #availableChunkCount}. + * <p> + * The movement of the intermediate binding set chunks forms an acyclic + * directed graph. This map is used to track the #of chunks available for + * each bop in the pipeline. When a bop has no more incoming chunks, we send + * an asynchronous message to all nodes on which that bop had executed + * informing the {@link QueryEngine} on that node that it should immediately + * release all resources associated with that bop. + * <p> + * This is guarded by the {@link #runningStateLock}. + * + * FIXME {@link IConstraint}s for {@link PipelineJoin}, distinct elements + * and other filters for {@link IPredicate}s, conditional routing for + * binding sets in the pipeline (to route around an optional join group + * based on an {@link IConstraint}), and then buffer management for s/o. + * + * @todo SCALEOUT: Life cycle management of the operators and the query + * implies both a per-query bop:NodeList map on the query coordinator + * identifying the nodes on which the query has been executed and a + * per-query bop:ResourceList map identifying the resources associated + * with the execution of that bop on that node. In fact, this could be + * the same {@link #resourceMap} except that we would lose type + * information about the nature of the resource so it is better to + * have distinct maps for this purpose. + */ + private final Map<Integer/* bopId */, AtomicLong/* availableChunkCount */> availableChunkCountMap = new LinkedHashMap<Integer, AtomicLong>(); + + /** + * A collection reporting on the #of instances of a given {@link BOp} which + * are concurrently executing. + * <p> + * This is guarded by the {@link #runningStateLock}. + */ + private final Map<Integer/*bopId*/, AtomicLong/*runningCount*/> runningCountMap = new LinkedHashMap<Integer, AtomicLong>(); + + /** + * A collection of the operators which have executed at least once. ... [truncated message content] |
From: <tho...@us...> - 2010-09-01 22:49:05
|
Revision: 3493 http://bigdata.svn.sourceforge.net/bigdata/?rev=3493&view=rev Author: thompsonbry Date: 2010-09-01 22:48:57 +0000 (Wed, 01 Sep 2010) Log Message: ----------- Fixed bug in test harness for verifying solutions. Added a hash-map based distinct solutions operator. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/HashBindingSet.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ap/DistinctElementFilter.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/aggregation/TestDistinctBindingSets.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/RunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/join/TestPipelineJoin.java Added Paths: ----------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/aggregation/DistinctBindingSetOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/ThreadLocalBufferFactory.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/HashBindingSet.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/HashBindingSet.java 2010-09-01 21:16:52 UTC (rev 3492) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/HashBindingSet.java 2010-09-01 22:48:57 UTC (rev 3493) @@ -66,11 +66,32 @@ * * @param src */ - protected HashBindingSet(HashBindingSet src) { + protected HashBindingSet(final HashBindingSet src) { map = new LinkedHashMap<IVariable, IConstant>(src.map); } + + /** + * Copy constructor. + * + * @param src + */ + public HashBindingSet(final IBindingSet src) { + + map = new LinkedHashMap<IVariable, IConstant>(src.size()); + + final Iterator<Map.Entry<IVariable, IConstant>> itr = src.iterator(); + + while (itr.hasNext()) { + + final Map.Entry<IVariable, IConstant> e = itr.next(); + + map.put(e.getKey(), e.getValue()); + + } + + } public boolean isBound(IVariable var) { @@ -119,7 +140,7 @@ public String toString() { - StringBuilder sb = new StringBuilder(); + final StringBuilder sb = new StringBuilder(); sb.append("{ "); Added: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/aggregation/DistinctBindingSetOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/aggregation/DistinctBindingSetOp.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/aggregation/DistinctBindingSetOp.java 2010-09-01 22:48:57 UTC (rev 3493) @@ -0,0 +1,335 @@ +package com.bigdata.bop.aggregation; + +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.FutureTask; + +import com.bigdata.bop.AbstractPipelineOp; +import com.bigdata.bop.ArrayBindingSet; +import com.bigdata.bop.BOp; +import com.bigdata.bop.BOpContext; +import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.IConstant; +import com.bigdata.bop.IVariable; +import com.bigdata.bop.engine.BOpStats; +import com.bigdata.relation.accesspath.IAsynchronousIterator; +import com.bigdata.relation.accesspath.IBlockingBuffer; + +/** + * A pipelined DISTINCT operator based on a hash table. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id: DistinctElementFilter.java 3466 2010-08-27 14:28:04Z + * thompsonbry $ + */ +public class DistinctBindingSetOp extends AbstractPipelineOp<IBindingSet>{ + + /** + * + */ + private static final long serialVersionUID = 1L; + + public interface Annotations extends BOp.Annotations { + + /** + * The initial capacity of the {@link ConcurrentHashMap} used to impose + * the distinct constraint. + * + * @see #DEFAULT_INITIAL_CAPACITY + */ + String INITIAL_CAPACITY = "initialCapacity"; + + int DEFAULT_INITIAL_CAPACITY = 16; + + /** + * The load factor of the {@link ConcurrentHashMap} used to impose + * the distinct constraint. + * + * @see #DEFAULT_LOAD_FACTOR + */ + String LOAD_FACTOR = "loadFactor"; + + float DEFAULT_LOAD_FACTOR = .75f; + + /** + * The concurrency level of the {@link ConcurrentHashMap} used to impose + * the distinct constraint. + * + * @see #DEFAULT_CONCURRENCY_LEVEL + */ + String CONCURRENCY_LEVEL = "concurrencyLevel"; + + int DEFAULT_CONCURRENCY_LEVEL = 16; + + /** + * The variables on which the distinct constraint will be imposed. + * Binding sets with distinct values for the specified variables will be + * passed on. + */ + String VARIABLES = DistinctBindingSetOp.class.getName() + ".variables"; + + } + + /** + * Required deep copy constructor. + */ + public DistinctBindingSetOp(final DistinctBindingSetOp op) { + super(op); + } + + /** + * Required shallow copy constructor. + */ + public DistinctBindingSetOp(final BOp[] args, + final Map<String, Object> annotations) { + + super(args, annotations); + + final IVariable<?>[] vars = getVariables(); + + if (vars == null) + throw new IllegalArgumentException(); + + if (vars.length == 0) + throw new IllegalArgumentException(); + + } + + /** + * @see Annotations#INITIAL_CAPACITY + */ + public int getInitialCapacity() { + + return getProperty(Annotations.INITIAL_CAPACITY, + Annotations.DEFAULT_INITIAL_CAPACITY); + + } + + /** + * @see Annotations#LOAD_FACTOR + */ + public float getLoadFactor() { + + return getProperty(Annotations.LOAD_FACTOR, + Annotations.DEFAULT_LOAD_FACTOR); + + } + + /** + * @see Annotations#CONCURRENCY_LEVEL + */ + public int getConcurrencyLevel() { + + return getProperty(Annotations.CONCURRENCY_LEVEL, + Annotations.DEFAULT_CONCURRENCY_LEVEL); + + } + + /** + * @see Annotations#VARIABLES + */ + public IVariable<?>[] getVariables() { + + return (IVariable<?>[]) annotations.get(Annotations.VARIABLES); + + } + + public FutureTask<Void> eval(final BOpContext<IBindingSet> context) { + + return new FutureTask<Void>(new DistinctTask(this, context)); + + } + + /** + * Wrapper used for the as bound solutions in the {@link ConcurrentHashMap}. + */ + private static class Solution { + private final int hash; + + private final IConstant<?>[] vals; + + public Solution(final IConstant<?>[] vals) { + this.vals = vals; + this.hash = java.util.Arrays.hashCode(vals); + } + + public int hashCode() { + return hash; + } + + public boolean equals(final Object o) { + if (this == o) + return true; + if (!(o instanceof Solution)) { + return false; + } + final Solution t = (Solution) o; + if (vals.length != t.vals.length) + return false; + for (int i = 0; i < vals.length; i++) { + // @todo allow for nulls. + if (vals[i] == t.vals[i]) + continue; + if (vals[i] == null) + return false; + if (!vals[i].equals(t.vals[i])) + return false; + } + return true; + } + } + + /** + * Task executing on the node. + */ + private class DistinctTask implements Callable<Void> { + + private final BOpContext<IBindingSet> context; + + /** + * A concurrent map whose keys are the bindings on the specified + * variables (the keys and the values are the same since the map + * implementation does not allow <code>null</code> values). + */ + private /*final*/ ConcurrentHashMap<Solution, Solution> map; + + /** + * The variables used to impose a distinct constraint. + */ + private final IVariable<?>[] vars; + + DistinctTask(final DistinctBindingSetOp op, + final BOpContext<IBindingSet> context) { + + this.context = context; + + this.vars = op.getVariables(); + + this.map = new ConcurrentHashMap<Solution, Solution>( + getInitialCapacity(), getLoadFactor(), + getConcurrencyLevel()); + + } + + /** + * If the bindings are distinct for the configured variables then return + * those bindings. + * + * @param bset + * The binding set to be filtered. + * + * @return The distinct as bound values -or- <code>null</code> if the + * binding set duplicates a solution which was already accepted. + */ + private IConstant<?>[] accept(final IBindingSet bset) { + + final IConstant<?>[] r = new IConstant<?>[vars.length]; + + for (int i = 0; i < vars.length; i++) { + + if ((r[i] = bset.get(vars[i])) == null) { + + /* + * @todo probably allow nulls, but write a unit test for it. + */ + + throw new RuntimeException("Not bound: " + vars[i]); + + } + + } + + final Solution s = new Solution(r); + + final boolean distinct = map.putIfAbsent(s, s) == null; + + return distinct ? r : null; + + } + + public Void call() throws Exception { + + final BOpStats stats = context.getStats(); + + final IAsynchronousIterator<IBindingSet[]> itr = context + .getSource(); + + final IBlockingBuffer<IBindingSet[]> sink = context.getSink(); + + try { + + while (itr.hasNext()) { + + final IBindingSet[] a = itr.next(); + + stats.chunksIn.increment(); + stats.unitsIn.add(a.length); + + final List<IBindingSet> accepted = new LinkedList<IBindingSet>(); + + int naccepted = 0; + + for (IBindingSet bset : a) { + +// System.err.println("considering: " + bset); + + final IConstant<?>[] vals = accept(bset); + + if (vals != null) { + +// System.err.println("accepted: " +// + Arrays.toString(vals)); + + /* + * @todo This may cause problems since the + * ArrayBindingSet does not allow mutation with + * variables not declared up front. In that case use + * new HashBindingSet( new ArrayBindingSet(...)). + */ + + accepted.add(new ArrayBindingSet(vars, vals)); + + naccepted++; + + } + + } + + if (naccepted > 0) { + + final IBindingSet[] b = accepted + .toArray(new IBindingSet[naccepted]); + +// System.err.println("output: " +// + Arrays.toString(b)); + + sink.add(b); + + stats.unitsOut.add(naccepted); + stats.chunksOut.increment(); + + } + + } + + // done. + return null; + + } finally { + + sink.flush(); + sink.close(); + + // discard the map. + map = null; + + } + + } + + } + +} Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/aggregation/DistinctBindingSetOp.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ap/DistinctElementFilter.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ap/DistinctElementFilter.java 2010-09-01 21:16:52 UTC (rev 3492) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ap/DistinctElementFilter.java 2010-09-01 22:48:57 UTC (rev 3493) @@ -10,8 +10,10 @@ import com.bigdata.bop.IBindingSet; import com.bigdata.bop.IVariable; import com.bigdata.bop.NV; +import com.bigdata.bop.aggregation.DistinctBindingSetOp; import com.bigdata.btree.keys.KeyBuilder; import com.bigdata.rdf.relation.rule.BindingSetSortKeyBuilder; +import com.bigdata.rdf.spo.DistinctSPOIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; import com.bigdata.relation.rule.eval.IJoinNexus; import com.bigdata.relation.rule.eval.ISolution; @@ -23,7 +25,8 @@ * A DISTINCT operator based on a hash table. * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> - * @version $Id$ + * @version $Id: DistinctElementFilter.java 3466 2010-08-27 14:28:04Z + * thompsonbry $ * @param <E> * * @todo could have an implementation backed by a persistent hash map using an @@ -40,7 +43,8 @@ * increase the map concurrency level, etc. * * @todo Reconcile with {@link IChunkConverter}, {@link DistinctFilter} (handles - * solutions) and {@link MergeFilter} (handles comparables). + * solutions) and {@link MergeFilter} (handles comparables), + * {@link DistinctSPOIterator}, {@link DistinctBindingSetOp}, etc. */ public class DistinctElementFilter<E> extends BOpBase @@ -62,14 +66,15 @@ String LOAD_FACTOR = "loadFactor"; String CONCURRENCY_LEVEL = "concurrencyLevel"; - + } - public DistinctElementFilter(final IVariable<?>[] distinctList, final UUID masterUUID) { + public DistinctElementFilter(final IVariable<?>[] distinctList, + final UUID masterUUID) { super(distinctList, NV.asMap(new NV[] { -// new NV(Annotations.QUERY_ID, masterUUID), - // new NV(Annotations.BOP_ID, bopId) + // new NV(Annotations.QUERY_ID, masterUUID), + // new NV(Annotations.BOP_ID, bopId) })); if (masterUUID == null) Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 2010-09-01 21:16:52 UTC (rev 3492) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 2010-09-01 22:48:57 UTC (rev 3493) @@ -35,7 +35,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; @@ -54,7 +53,6 @@ import com.bigdata.bop.IVariable; import com.bigdata.bop.engine.BOpStats; import com.bigdata.bop.engine.Haltable; -import com.bigdata.btree.AbstractBTree; import com.bigdata.btree.BytesUtil; import com.bigdata.btree.keys.IKeyBuilder; import com.bigdata.counters.CAT; @@ -66,7 +64,6 @@ import com.bigdata.relation.accesspath.IAccessPath; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; -import com.bigdata.relation.accesspath.IBuffer; import com.bigdata.relation.rule.IRule; import com.bigdata.relation.rule.IStarJoin; import com.bigdata.relation.rule.IStarJoin.IStarConstraint; @@ -385,193 +382,8 @@ */ final PipelineJoinStats stats; - /** - * A factory pattern for per-thread objects whose life cycle is tied to - * some container. For example, there may be an instance of this pool - * for a {@link JoinTask} or an {@link AbstractBTree}. The pool can be - * torn down when the container is torn down, which prevents its - * thread-local references from escaping. - * - * @author tho...@us... - * @param <T> - * The generic type of the thread-local object. - * - * @todo There should be two implementations of a common interface or - * abstract base class: one based on a private - * {@link ConcurrentHashMap} and the other on striped locks. The - * advantage of the {@link ConcurrentHashMap} is approximately 3x - * higher concurrency. The advantage of striped locks is that you - * can directly manage the #of buffers when when the threads using - * those buffers is unbounded. However, doing so could lead to - * deadlock since two threads can be hashed onto the same buffer - * object. - */ - abstract public class ThreadLocalFactory<T extends IBuffer<E>, E> { + final private ThreadLocalBufferFactory<AbstractUnsynchronizedArrayBuffer<IBindingSet>, IBindingSet> threadLocalBufferFactory = new ThreadLocalBufferFactory<AbstractUnsynchronizedArrayBuffer<IBindingSet>, IBindingSet>() { - /** - * The thread-local queues. - */ - private final ConcurrentHashMap<Thread, T> map; - - /** - * A list of all objects visible to the caller. This is used to - * ensure that any objects allocated by the factory are visited. - * - * <p> - * Note: Since the collection is not thread-safe, synchronization is - * required when adding to the collection and when visiting the - * elements of the collection. - */ - private final LinkedList<T> list = new LinkedList<T>(); - - protected ThreadLocalFactory() { - - this(16/* initialCapacity */, .75f/* loadFactor */, 16/* concurrencyLevel */); - - } - - protected ThreadLocalFactory(final int initialCapacity, - final float loadFactor, final int concurrencyLevel) { - - map = new ConcurrentHashMap<Thread, T>(initialCapacity, - loadFactor, concurrencyLevel); - - } - - /** - * Return the #of thread-local objects. - */ - final public int size() { - - return map.size(); - - } - - /** - * Add the element to the thread-local buffer. - * - * @param e - * An element. - * - * @throws IllegalStateException - * if the factory is asynchronously closed. - */ - public void add(final E e) { - - get().add(e); - - } - - /** - * Return a thread-local buffer - * - * @return The thread-local buffer. - * - * @throws RuntimeException - * if the join is halted. - */ - final private T get() { - final Thread t = Thread.currentThread(); - T tmp = map.get(t); - if (tmp == null) { - if (map.put(t, tmp = initialValue()) != null) { - /* - * Note: Since the key is the thread it is not possible - * for there to be a concurrent put of an entry under - * the same key so we do not have to use putIfAbsent(). - */ - throw new AssertionError(); - } - // Add to list. - synchronized (list) { - list.add(tmp); - } - } - halted(); - return tmp; - } - - /** - * Flush each of the unsynchronized buffers onto their backing - * synchronized buffer. - * - * @throws RuntimeException - * if the join is halted. - */ - public void flush() { - synchronized (list) { - int n = 0; - long m = 0L; - for (T b : list) { - halted(); - // #of elements to be flushed. - final int size = b.size(); - // flush, returning total #of elements written onto this - // buffer. - final long counter = b.flush(); - m += counter; - if (log.isDebugEnabled()) - log.debug("Flushed buffer: size=" + size - + ", counter=" + counter); - } - if (log.isInfoEnabled()) - log.info("Flushed " + n - + " unsynchronized buffers totalling " + m - + " elements"); - } - } - - /** - * Reset each of the synchronized buffers, discarding their buffered - * writes. - * <p> - * Note: This method is used during error processing, therefore it - * DOES NOT check {@link JoinTask#halt}. - */ - public void reset() { - synchronized (list) { - int n = 0; - for (T b : list) { - // #of elements in the buffer before reset(). - final int size = b.size(); - // reset the buffer. - b.reset(); - if (log.isDebugEnabled()) - log.debug("Reset buffer: size=" + size); - } - if (log.isInfoEnabled()) - log.info("Reset " + n + " unsynchronized buffers"); - } - } - - // /** - // * Reset the per-{@link Thread} unsynchronized output buffers - // (used as - // * part of error handling for the {@link JoinTask}). - // */ - // final protected void resetUnsyncBuffers() throws Exception { - // - // final int n = threadLocalBufferFactory.reset(); - // .close(new - // Visitor<AbstractUnsynchronizedArrayBuffer<IBindingSet>>() { - // - // @Override - // public void meet( - // final AbstractUnsynchronizedArrayBuffer<IBindingSet> b) - // throws Exception { - // - // - // } - - /** - * Create and return a new object. - */ - abstract protected T initialValue(); - - } - - final private ThreadLocalFactory<AbstractUnsynchronizedArrayBuffer<IBindingSet>, IBindingSet> threadLocalBufferFactory = new ThreadLocalFactory<AbstractUnsynchronizedArrayBuffer<IBindingSet>, IBindingSet>() { - @Override protected AbstractUnsynchronizedArrayBuffer<IBindingSet> initialValue() { @@ -579,6 +391,14 @@ return newUnsyncOutputBuffer(); } + + @Override + protected void halted() { + + JoinTask.this.halted(); + + } + }; public String toString() { Added: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/ThreadLocalBufferFactory.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/ThreadLocalBufferFactory.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/ThreadLocalBufferFactory.java 2010-09-01 22:48:57 UTC (rev 3493) @@ -0,0 +1,232 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Sep 1, 2010 + */ + +package com.bigdata.bop.join; + +import java.util.LinkedList; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.log4j.Logger; + +import com.bigdata.bop.engine.Haltable; +import com.bigdata.btree.AbstractBTree; +import com.bigdata.relation.accesspath.IBuffer; +import com.bigdata.relation.rule.eval.pipeline.JoinTask; + +/** + * A factory pattern for per-thread objects whose life cycle is tied to some + * container. For example, there may be an instance of this pool for a + * {@link JoinTask} or an {@link AbstractBTree}. The pool can be torn down when + * the container is torn down, which prevents its thread-local references from + * escaping. + * + * @author tho...@us... + * @version $Id$ + * @param <T> + * The generic type of the thread-local object. + * + * @todo There should be two implementations of a common interface or abstract + * base class: one based on a private {@link ConcurrentHashMap} and the + * other on striped locks. The advantage of the {@link ConcurrentHashMap} + * is approximately 3x higher concurrency. The advantage of striped locks + * is that you can directly manage the #of buffers when when the threads + * using those buffers is unbounded. However, doing so could lead to + * deadlock since two threads can be hashed onto the same buffer object. + * + * @todo refactor into our concurrency package? + */ +abstract public class ThreadLocalBufferFactory<T extends IBuffer<E>, E> { + + static private final Logger log = Logger + .getLogger(ThreadLocalBufferFactory.class); + + /** + * The thread-local queues. + */ + private final ConcurrentHashMap<Thread, T> map; + + /** + * A list of all objects visible to the caller. This is used to ensure that + * any objects allocated by the factory are visited. + * + * <p> + * Note: Since the collection is not thread-safe, synchronization is + * required when adding to the collection and when visiting the elements of + * the collection. + */ + private final LinkedList<T> list = new LinkedList<T>(); + + protected ThreadLocalBufferFactory() { + + this(16/* initialCapacity */, .75f/* loadFactor */, 16/* concurrencyLevel */); + + } + + protected ThreadLocalBufferFactory(final int initialCapacity, + final float loadFactor, final int concurrencyLevel) { + + map = new ConcurrentHashMap<Thread, T>(initialCapacity, loadFactor, + concurrencyLevel); + + } + + /** + * Return the #of thread-local objects. + */ + final public int size() { + + return map.size(); + + } + + /** + * Add the element to the thread-local buffer. + * + * @param e + * An element. + * + * @throws IllegalStateException + * if the factory is asynchronously closed. + */ + public void add(final E e) { + + get().add(e); + + } + + /** + * Return a thread-local buffer + * + * @return The thread-local buffer. + * + * @throws RuntimeException + * if the join is halted. + */ + final public T get() { + final Thread t = Thread.currentThread(); + T tmp = map.get(t); + if (tmp == null) { + if (map.put(t, tmp = initialValue()) != null) { + /* + * Note: Since the key is the thread it is not possible for + * there to be a concurrent put of an entry under the same key + * so we do not have to use putIfAbsent(). + */ + throw new AssertionError(); + } + // Add to list. + synchronized (list) { + list.add(tmp); + } + } + halted(); + return tmp; + } + + /** + * Flush each of the unsynchronized buffers onto their backing synchronized + * buffer. + * + * @throws RuntimeException + * if the join is halted. + */ + public void flush() { + synchronized (list) { + int n = 0; + long m = 0L; + for (T b : list) { + halted(); + // #of elements to be flushed. + final int size = b.size(); + // flush, returning total #of elements written onto this + // buffer. + final long counter = b.flush(); + m += counter; + if (log.isDebugEnabled()) + log.debug("Flushed buffer: size=" + size + ", counter=" + + counter); + } + if (log.isInfoEnabled()) + log.info("Flushed " + n + " unsynchronized buffers totalling " + + m + " elements"); + } + } + + /** + * Reset each of the synchronized buffers, discarding their buffered writes. + * <p> + * Note: This method is used during error processing, therefore it DOES NOT + * check {@link JoinTask#halt}. + */ + public void reset() { + synchronized (list) { + int n = 0; + for (T b : list) { + // #of elements in the buffer before reset(). + final int size = b.size(); + // reset the buffer. + b.reset(); + if (log.isDebugEnabled()) + log.debug("Reset buffer: size=" + size); + } + if (log.isInfoEnabled()) + log.info("Reset " + n + " unsynchronized buffers"); + } + } + + // /** + // * Reset the per-{@link Thread} unsynchronized output buffers + // (used as + // * part of error handling for the {@link JoinTask}). + // */ + // final protected void resetUnsyncBuffers() throws Exception { + // + // final int n = threadLocalBufferFactory.reset(); + // .close(new + // Visitor<AbstractUnsynchronizedArrayBuffer<IBindingSet>>() { + // + // @Override + // public void meet( + // final AbstractUnsynchronizedArrayBuffer<IBindingSet> b) + // throws Exception { + // + // + // } + + /** + * Create and return a new object. + */ + abstract protected T initialValue(); + + /** + * Test to see if the process has been halted. + * + * @see Haltable#halted() + */ + abstract protected void halted(); + +} Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/ThreadLocalBufferFactory.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/aggregation/TestDistinctBindingSets.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/aggregation/TestDistinctBindingSets.java 2010-09-01 21:16:52 UTC (rev 3492) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/aggregation/TestDistinctBindingSets.java 2010-09-01 22:48:57 UTC (rev 3493) @@ -27,12 +27,35 @@ package com.bigdata.bop.aggregation; -import com.bigdata.bop.ap.DistinctElementFilter; +import java.util.LinkedList; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.FutureTask; import junit.framework.TestCase2; +import com.bigdata.bop.ArrayBindingSet; +import com.bigdata.bop.BOp; +import com.bigdata.bop.BOpContext; +import com.bigdata.bop.Constant; +import com.bigdata.bop.HashBindingSet; +import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.IConstant; +import com.bigdata.bop.IVariable; +import com.bigdata.bop.NV; +import com.bigdata.bop.Var; +import com.bigdata.bop.engine.BOpStats; +import com.bigdata.bop.engine.TestQueryEngine; +import com.bigdata.journal.BufferMode; +import com.bigdata.journal.ITx; +import com.bigdata.journal.Journal; +import com.bigdata.relation.accesspath.IAsynchronousIterator; +import com.bigdata.relation.accesspath.IBlockingBuffer; +import com.bigdata.relation.accesspath.ThickAsynchronousIterator; + /** - * Unit tests for {@link DistinctElementFilter}. + * Unit tests for {@link DistinctBindingSetOp}. * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id$ @@ -52,14 +75,157 @@ super(name); } + @Override + public Properties getProperties() { + + final Properties p = new Properties(super.getProperties()); + + p.setProperty(Journal.Options.BUFFER_MODE, BufferMode.Transient + .toString()); + + return p; + + } + + Journal jnl = null; + + List<IBindingSet> data = null; + + public void setUp() throws Exception { + + jnl = new Journal(getProperties()); + + setUpData(); + + } + /** - * @todo write unit tests for distinct based on purely local evaluation. + * Setup the data. + */ + private void setUpData() { + + final Var<?> x = Var.var("x"); + final Var<?> y = Var.var("y"); + + data = new LinkedList<IBindingSet>(); + IBindingSet bset = null; + { + bset = new HashBindingSet(); + bset.set(x, new Constant<String>("John")); + bset.set(y, new Constant<String>("Mary")); + data.add(bset); + } + { + bset = new HashBindingSet(); + bset.set(x, new Constant<String>("Mary")); + bset.set(y, new Constant<String>("Paul")); + data.add(bset); + } + { + bset = new HashBindingSet(); + bset.set(x, new Constant<String>("Mary")); + bset.set(y, new Constant<String>("Jane")); + data.add(bset); + } + { + bset = new HashBindingSet(); + bset.set(x, new Constant<String>("Paul")); + bset.set(y, new Constant<String>("Leon")); + data.add(bset); + } + { + bset = new HashBindingSet(); + bset.set(x, new Constant<String>("Paul")); + bset.set(y, new Constant<String>("John")); + data.add(bset); + } + { + bset = new HashBindingSet(); + bset.set(x, new Constant<String>("Leon")); + bset.set(y, new Constant<String>("Paul")); + data.add(bset); + } + + } + + public void tearDown() throws Exception { + + if (jnl != null) { + jnl.destroy(); + jnl = null; + } + + // clear reference. + data = null; + + } + + /** + * Unit test for distinct. * - * @todo write unit tests for distinct based on a hash partitioned DISTINCT - * filter. + * @throws ExecutionException + * @throws InterruptedException */ - public void test_something() { - fail("write tests"); + public void test_distinct() throws InterruptedException, ExecutionException { + + final Var<?> x = Var.var("x"); +// final Var<?> y = Var.var("y"); + + final int distinctId = 1; + + final DistinctBindingSetOp query = new DistinctBindingSetOp(new BOp[]{}, + NV.asMap(new NV[]{// + new NV(DistinctBindingSetOp.Annotations.BOP_ID,distinctId),// + new NV(DistinctBindingSetOp.Annotations.VARIABLES,new IVariable[]{x}),// + })); + + // the expected solutions + final IBindingSet[] expected = new IBindingSet[] {// + new ArrayBindingSet(// + new IVariable[] { x },// + new IConstant[] { new Constant<String>("John") }// + ),// + new ArrayBindingSet(// + new IVariable[] { x },// + new IConstant[] { new Constant<String>("Mary") }// + ), new ArrayBindingSet(// + new IVariable[] { x },// + new IConstant[] { new Constant<String>("Paul") }// + ), new ArrayBindingSet(// + new IVariable[] { x },// + new IConstant[] { new Constant<String>("Leon") }// + ), }; + + final BOpStats stats = query.newStats(); + + final IAsynchronousIterator<IBindingSet[]> source = new ThickAsynchronousIterator<IBindingSet[]>( + new IBindingSet[][] { data.toArray(new IBindingSet[0]) }); + + final IBlockingBuffer<IBindingSet[]> sink = query.newBuffer(); + + final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( + null/* fed */, jnl/* indexManager */, + ITx.READ_COMMITTED/* readTimestamp */, + ITx.UNISOLATED/* writeTimestamp */, -1/* partitionId */, stats, + source, sink, null/* sink2 */); + + // get task. + final FutureTask<Void> ft = query.eval(context); + + // execute task. + jnl.getExecutorService().execute(ft); + + TestQueryEngine.assertSolutions(expected, sink.iterator()); + + assertTrue(ft.isDone()); + assertFalse(ft.isCancelled()); + ft.get(); // verify nothing thrown. + + assertEquals(1L, stats.chunksIn.get()); + assertEquals(6L, stats.unitsIn.get()); + assertEquals(4L, stats.unitsOut.get()); + assertEquals(1L, stats.chunksOut.get()); + } } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/RunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/RunningQuery.java 2010-09-01 21:16:52 UTC (rev 3492) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/RunningQuery.java 2010-09-01 22:48:57 UTC (rev 3493) @@ -56,6 +56,7 @@ import com.bigdata.bop.IConstraint; import com.bigdata.bop.IPredicate; import com.bigdata.bop.NoSuchBOpException; +import com.bigdata.bop.aggregation.DistinctBindingSetOp; import com.bigdata.bop.ap.Predicate; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; @@ -187,10 +188,12 @@ * <p> * This is guarded by the {@link #runningStateLock}. * - * FIXME {@link IConstraint}s for {@link PipelineJoin}, distinct elements - * and other filters for {@link IPredicate}s, conditional routing for - * binding sets in the pipeline (to route around an optional join group - * based on an {@link IConstraint}), and then buffer management for s/o. + * FIXME {@link IConstraint}s for {@link PipelineJoin}, non-distributed + * filters for {@link IPredicate}s, distinct element filter for + * {@link IPredicate} which is capable of distributed operations, + * conditional routing for binding sets in the pipeline (to route around an + * optional join group based on an {@link IConstraint}), SPARQL to BOP + * translation, and then buffer management for s/o. * * @todo SCALEOUT: Life cycle management of the operators and the query * implies both a per-query bop:NodeList map on the query coordinator Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-09-01 21:16:52 UTC (rev 3492) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-09-01 22:48:57 UTC (rev 3493) @@ -153,14 +153,20 @@ public void tearDown() throws Exception { - if (queryEngine != null) + if (queryEngine != null) { queryEngine.shutdownNow(); + queryEngine = null; + } - if (bufferService != null) + if (bufferService != null) { bufferService.shutdownNow(); + bufferService = null; + } - if (jnl != null) + if (jnl != null) { jnl.destroy(); + jnl = null; + } } @@ -573,7 +579,7 @@ * @param expected * @param itr */ - protected void assertSolutions(final IBindingSet[] expected, + static public void assertSolutions(final IBindingSet[] expected, final IAsynchronousIterator<IBindingSet[]> itr) { try { int n = 0; @@ -588,9 +594,10 @@ fail("n=" + n + ", expected=" + expected[n] + ", actual=" + e[i]); } + n++; } - n++; } + assertEquals("Wrong number of solutions", expected.length, n); } finally { itr.close(); } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/join/TestPipelineJoin.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/join/TestPipelineJoin.java 2010-09-01 21:16:52 UTC (rev 3492) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/join/TestPipelineJoin.java 2010-09-01 22:48:57 UTC (rev 3493) @@ -50,6 +50,7 @@ import com.bigdata.bop.ap.E; import com.bigdata.bop.ap.Predicate; import com.bigdata.bop.ap.R; +import com.bigdata.bop.engine.TestQueryEngine; import com.bigdata.bop.join.PipelineJoin.PipelineJoinStats; import com.bigdata.journal.BufferMode; import com.bigdata.journal.ITx; @@ -143,8 +144,13 @@ public void tearDown() throws Exception { - if (jnl != null) + if (jnl != null) { + jnl.destroy(); + + jnl = null; + + } } @@ -252,20 +258,22 @@ // execute task. jnl.getExecutorService().execute(ft); - final IAsynchronousIterator<IBindingSet[]> itr = sink.iterator(); - try { - int n = 0; - while (itr.hasNext()) { - final IBindingSet[] chunk = itr.next(); - if (log.isInfoEnabled()) - log.info(n + " : chunkSize=" + chunk.length); - for (int i = 0; i < chunk.length; i++) { - assertTrue(expected[n++].equals(chunk[i])); - } - } - } finally { - itr.close(); - } + TestQueryEngine.assertSolutions(expected, sink.iterator()); +// final IAsynchronousIterator<IBindingSet[]> itr = sink.iterator(); +// try { +// int n = 0; +// while (itr.hasNext()) { +// final IBindingSet[] chunk = itr.next(); +// if (log.isInfoEnabled()) +// log.info(n + " : chunkSize=" + chunk.length); +// for (int i = 0; i < chunk.length; i++) { +// assertTrue(expected[n++].equals(chunk[i])); +// } +// } +// assertEquals(n, expected.length); +// } finally { +// itr.close(); +// } // join task assertEquals(1L, stats.chunksIn.get()); @@ -366,6 +374,7 @@ // assertTrue(expected[n++].equals(chunk[i])); // } // } +// assertEquals(n, expected.length); // } finally { // itr.close(); // } This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2010-09-03 00:27:56
|
Revision: 3500 http://bigdata.svn.sourceforge.net/bigdata/?rev=3500&view=rev Author: thompsonbry Date: 2010-09-03 00:27:45 +0000 (Fri, 03 Sep 2010) Log Message: ----------- Reorganized the operators and some utility classes a bit. Added an a conditional routing operator and a test suite for it. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/QuoteOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestAll.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/aggregation/TestAll.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/RunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestAll.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestPipelineUtility.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/join/TestPipelineJoin.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/util/concurrent/TestAll.java Added Paths: ----------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/ConditionalRoutingOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/CopyBindingSetOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/Union.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/accesspath/ThreadLocalBufferFactory.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/accesspath/UnsyncLocalOutputBuffer.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/util/concurrent/Haltable.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/bset/ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/bset/TestAll.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/bset/TestConditionalRoutingOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/bset/TestCopyBindingSets.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/bset/TestUnionBindingSets.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/util/concurrent/TestHaltable.java Removed Paths: ------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineStartOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/aggregation/Union.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bop-notes.txt branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/ThreadLocalBufferFactory.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/UnsyncLocalOutputBuffer.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/aggregation/TestUnionBindingSets.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/Haltable.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestHaltable.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java 2010-09-02 22:24:22 UTC (rev 3499) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java 2010-09-03 00:27:45 UTC (rev 3500) @@ -162,7 +162,7 @@ * @throws NullPointerException * if the argument is <code>null</code>. */ - protected BOpBase(final BOpBase op) { + public BOpBase(final BOpBase op) { // deep copy the arguments. args = deepCopy(op.args); // deep copy the annotations. @@ -173,7 +173,7 @@ * @param args * The arguments to the operator. */ - protected BOpBase(final BOp[] args) { + public BOpBase(final BOp[] args) { this(args, null/* annotations */); @@ -185,7 +185,7 @@ * @param annotations * The annotations for the operator (optional). */ - protected BOpBase(final BOp[] args, + public BOpBase(final BOp[] args, final Map<String, Object> annotations) { if (args == null) Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java 2010-09-02 22:24:22 UTC (rev 3499) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java 2010-09-03 00:27:45 UTC (rev 3500) @@ -41,9 +41,14 @@ * * @param <E> * The generic type of the objects processed by the operator. - * + * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id$ + * + * @todo It is too confusion to have an interface hierarchy which is separate + * from the class hierarchy for the operators. Therefore roll this + * interface into {@link AbstractPipelineOp} and then rename that class to + * {@link PipelineOp} */ public interface PipelineOp<E> extends BOp { Deleted: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineStartOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineStartOp.java 2010-09-02 22:24:22 UTC (rev 3499) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineStartOp.java 2010-09-03 00:27:45 UTC (rev 3500) @@ -1,117 +0,0 @@ -/** - -Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. - -Contact: - SYSTAP, LLC - 4501 Tower Road - Greensboro, NC 27410 - lic...@bi... - -This program is free software; you can redistribute it and/or modify -it under the terms of the GNU General Public License as published by -the Free Software Foundation; version 2 of the License. - -This program is distributed in the hope that it will be useful, -but WITHOUT ANY WARRANTY; without even the implied warranty of -MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -GNU General Public License for more details. - -You should have received a copy of the GNU General Public License -along with this program; if not, write to the Free Software -Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA -*/ -/* - * Created on Aug 25, 2010 - */ - -package com.bigdata.bop; - -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.FutureTask; - -import com.bigdata.bop.engine.BOpStats; -import com.bigdata.relation.accesspath.IAsynchronousIterator; -import com.bigdata.relation.accesspath.IBlockingBuffer; - -/** - * This operator copies its source to its sink. It is used to feed the first - * join in the pipeline. The operator should have no children but may be - * decorated with annotations as necessary. - * - * @author <a href="mailto:tho...@us...">Bryan Thompson</a> - * @version $Id$ - */ -public class PipelineStartOp extends BindingSetPipelineOp { - - /** - * - */ - private static final long serialVersionUID = 1L; - - /** - * Deep copy constructor. - * - * @param op - */ - public PipelineStartOp(PipelineStartOp op) { - super(op); - } - - /** - * Shallow copy constructor. - * - * @param args - * @param annotations - */ - public PipelineStartOp(BOp[] args, Map<String, Object> annotations) { - super(args, annotations); - } - - public FutureTask<Void> eval(final BOpContext<IBindingSet> context) { - - return new FutureTask<Void>(new CopyTask(context)); - - } - - /** - * Copy the source to the sink. - */ - static private class CopyTask implements Callable<Void> { - - private final BOpStats stats; - - private final IAsynchronousIterator<IBindingSet[]> source; - - private final IBlockingBuffer<IBindingSet[]> sink; - - CopyTask(final BOpContext<IBindingSet> context) { - - stats = context.getStats(); - - this.source = context.getSource(); - - this.sink = context.getSink(); - - } - - public Void call() throws Exception { - try { - while (source.hasNext()) { - final IBindingSet[] chunk = source.next(); - stats.chunksIn.increment(); - stats.unitsIn.add(chunk.length); - sink.add(chunk); - stats.chunksOut.increment(); - stats.unitsOut.add(chunk.length); - } - return null; - } finally { - sink.close(); - } - } - - } - -} Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/QuoteOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/QuoteOp.java 2010-09-02 22:24:22 UTC (rev 3499) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/QuoteOp.java 2010-09-03 00:27:45 UTC (rev 3500) @@ -38,10 +38,10 @@ * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id$ * - * @todo If we do nothing then {@link QuoteOp} will already prevent the - * evaluation of its child operand by the expediency of not defining its - * own evaluation semantics. Alternatively, we could add - * <code>eval():Op</code> using an appropriate evaluation interface. + * @todo I think that we can avoid quoting operators by using annotations (for + * some cases) and through explicit interaction between operators for + * others (such as between a join and a predicate). If that proves to be + * true then this class will be dropped. */ public class QuoteOp extends BOpBase { Deleted: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/aggregation/Union.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/aggregation/Union.java 2010-09-02 22:24:22 UTC (rev 3499) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/aggregation/Union.java 2010-09-03 00:27:45 UTC (rev 3500) @@ -1,135 +0,0 @@ -/** - -Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. - -Contact: - SYSTAP, LLC - 4501 Tower Road - Greensboro, NC 27410 - lic...@bi... - -This program is free software; you can redistribute it and/or modify -it under the terms of the GNU General Public License as published by -the Free Software Foundation; version 2 of the License. - -This program is distributed in the hope that it will be useful, -but WITHOUT ANY WARRANTY; without even the implied warranty of -MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -GNU General Public License for more details. - -You should have received a copy of the GNU General Public License -along with this program; if not, write to the Free Software -Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA -*/ -/* - * Created on Aug 18, 2010 - */ - -package com.bigdata.bop.aggregation; - -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.FutureTask; - -import com.bigdata.bop.BOpContext; -import com.bigdata.bop.BindingSetPipelineOp; -import com.bigdata.bop.IBindingSet; -import com.bigdata.bop.engine.Haltable; -import com.bigdata.bop.join.PipelineJoin; -import com.bigdata.rdf.rules.TMUtility; -import com.bigdata.relation.RelationFusedView; - -/** - * The union of two or more {@link BindingSetPipelineOp} operators. - * - * @author <a href="mailto:tho...@us...">Bryan Thompson</a> - * @version $Id$ - * - * @todo I have some basic questions about the ability to use a UNION of two - * predicates in scale-out. I think that this might be more accurately - * modeled as the UNION of two joins. That is, rather than: - * - * <pre> - * JOIN( ..., - * UNION( foo.spo(A,loves,B), - * bar.spo(A,loves,B) ) - * ) - * </pre> - * - * using - * - * <pre> - * UNION( JOIN( ..., foo.spo(A,loves,B) ), - * JOIN( ..., bar.spo(A,loves,B) ) - * ) - * </pre> - * - * which would be a binding set union rather than an element union. - * - * @todo The union of access paths was historically handled by - * {@link RelationFusedView}. That class should be removed once queries - * are rewritten to use the union of joins. - * - * @todo The {@link TMUtility} will have to be updated to use this operator - * rather than specifying multiple source "names" for the relation of the - * predicate. - * - * @todo The FastClosureRuleTask will also need to be updated to use a - * {@link Union} over the joins rather than a {@link RelationFusedView}. - */ -public class Union extends BindingSetPipelineOp { - - /** - * - */ - private static final long serialVersionUID = 1L; - - /** - * @param args - * Two or more operators whose union is desired. - * @param annotations - */ - public Union(final BindingSetPipelineOp[] args, - final Map<String, Object> annotations) { - - super(args, annotations); - - if (args.length < 2) - throw new IllegalArgumentException(); - - } - - public FutureTask<Void> eval(final BOpContext<IBindingSet> context) { - - return new FutureTask<Void>(new UnionTask(this, context)); - - } - - /** - * Pipeline union impl. - * - * FIXME All this does is copy its inputs to its outputs. Since we only run - * one chunk of input at a time, it seems that the easiest way to implement - * a union is to have the operators in the union just target the same sink. - */ - private static class UnionTask extends Haltable<Void> implements Callable<Void> { - - public UnionTask(// - final Union op,// - final BOpContext<IBindingSet> context - ) { - - if (op == null) - throw new IllegalArgumentException(); - if (context == null) - throw new IllegalArgumentException(); - } - - public Void call() throws Exception { - // TODO Auto-generated method stub - throw new UnsupportedOperationException(); - } - - } - -} Deleted: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bop-notes.txt =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bop-notes.txt 2010-09-02 22:24:22 UTC (rev 3499) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bop-notes.txt 2010-09-03 00:27:45 UTC (rev 3500) @@ -1,540 +0,0 @@ -- Add IElement interface with Object:get(int index) to pull off the - fields from the element by index position. Use this to make - RDFJoinNexus#bind() and copyValues() generic. We can just do a cast - to IElement, but we could also change the generic type constraint on - IRelation from <E> to IRelation<E extends IElement>. But we can - just force the cast for now and not have to update all of those - generic constraints. - -- Review annotation names and defaults. Make sure that the annotation - names are all in appropriate namespaces. The namespaces should - probably be the interface or class of the operator which defines - that annotation. - -- RejectAnythingSameAsSelf appears to be assuming termIds rather than - IVs. - -- Get rid of the concept of a relation view (two or more relations - named by a given predicate) in favor of the UNION of the predicates, - which is basically a UNION of their access paths. - -- Expanders will eventually need to be replaced by either shard-wise - expanders (where possible) or query time materialization of various - inferences, e.g., using magic sets or other query rewrite - techniques. - -- IRelation#getAccessPath(IIndexManager,IPredicate). Raise this onto - onto IRelation. It is the shard aware version. There is also a - version getAccessPath(IPredicate) without the IIndexManager - parameter which is used for local indices and for RMI based access - to a scale-out index. - -- IRelation: DDL Support - - Iterator<IKeyOrder<E>> getKeyOrders(); - - IKeyOrder<E> getKeyOrder(IPredicate<E> p); - - IKeyOrder<E> getPrimaryKeyOrder(); - - IIndex getIndex(IKeyOrder) - -- IKeyOrder: DDL Support (reconcile with ITupleSerializer) - - byte[] getFromKey(IKeyBuilder keyBuilder, IPredicate<ISPO> predicate); - - byte[] getToKey(IKeyBuilder keyBuilder, IPredicate<ISPO> predicate); - - byte[] encodeKey(IKeyBuilder keyBuilder, E e); - - E decodeKey(byte[] key); - -- Elements: Add self-description of data to "elements". Harmonize with - binding sets. - - Iterator<Column> getFields(); - -- Column: - - getIndex(); - - getName(); - - getDatatype(); - - getForeignKeys(); - ... - -- Implement lexicon joins. - -- Implement spatial joins. - -- Nested optionals (conditional routing of joins). - -- Make the free text index a "resource" rather than a relation? - -- Use blocking queues with poison objects and chunking or modify - BlockingBuffer to raise up the LinkedBlockingDeque into our code - base. - -- Support tree rewrites using a pattern language. - - - http://functionaljava.org/, Functional Java. - - - tuprolog: http://alice.unibo.it/xwiki/bin/view/Tuprolog/, LGPL, Java. - - - http://scala.sygneca.com/code/mini-kanren, Example of constraint - based programming in Scala. http://scala.sygneca.com/. BSD - license. http://scala.sygneca.com/code/compressed-executable-jar. - - - PrologCafe: http://kaminari.istc.kobe-u.ac.jp/PrologCafe/, Java. - License is not specified. Authors are in Japan. Appears to be - two people. - - - YAP : http://www.dcc.fc.up.pt/~vsc/Yap/, Perl Artistic license, C. - - - XSB : http://xsb.sourceforge.net/, LGPL, C. - -BOp - - Serializable - - Cloneable - - Unmodifiable - - arity():int - - args():List<BOp> - - annotations():Map<String,Object> -.BOpList: Used to represent lists of operators. E.g., IN(var,{graphIds}). - - values():List<BOp> -.IConstantOrVariable - - isConstant() - - isVariable() - - get() // iff isConstant() - - getName() // iff isVariable() -..IConstant -..IVariable -.IOpN -..IPredicate(name,arg...)[shardId:int;optional:boolean;constraint:IConstraint[],expander] - -- Distributed query execution pattern: - - The historical pipeline join propagated evaluation from left to - right. This needs to be revisited in now that we are dealing with - operator trees. Operator trees lend themselves naturally to top-down - evaluation. While I think that we can do top-down evaluation of the - operator tree for scaleup, the distributed query execution logic is - more complicated and top-down evaluation is not compatible with - distributed evaluation because joins must run for different shards - based on the partition identifier associated with each bindingSet[] - they receive. - - What we been doing is pushing binding sets along propagating joins - at specific shards onto nodes together with those binding sets. - This was a left-to-right evaluation strategy when the IRule was just - an ordered array of tails on which we needed to read. However, now - that we have an operator tree it would correspond to a bottom up - evaluation of a left-deep tree where the right operands were always - access path reads. That makes my head hurt just a bit when I - consider that the bottom up evaluation would also be "partial" as - binding sets appear. - - For a given incoming IBindingSet[] chunk we will continue to do - exactly what we have been doing, but the surrounding logic needs to - be replaced. Starting a "join" (at least for piplined scale-out) - needs to merely register a join task factory that will handle - binding sets as they arrive. - - I am thinking that the way to handle this is to send the query - identifier, join operation identifier, and partition identifier - along with the client's proxy object, the senders operator - identifier, and the senders's service identifier. If the node (a - data service) which receives that RMI message has not seen the query - yet it uses RMI against the client's proxy to fetch the query and - then "starts" the query on that node. Starting a query would merely - allow execution of the operators described in the query plan on the - node once they had sufficient data to run. The current pipeline - join is a chunk-wise nested index join. It runs one pass each time - it has a chunk of binding sets for some shard. Query termination - would be controlled by the client. It would instruct all nodes - known to be running the query to halt execution for that query. - Even if a node misses that message, it will find out when it tries - to send intermediate results to another node that the query was - cancelled. - - Per shard locks, tasks, and threads. - - When running within a single Journal, the query plan is executed by - one task which holds all of the necessary locks. Those locks are - acquired up front by an inspection of the query plan to determine - which indices are needed [actually, it may just be using the - unisolated read/write index and otherwise historical views w/o - locks.] - - There are issues relating to execution of the joins under the - concurrency manager, both in order to have access to the correct - IIndexManager and in order to managing reads and writes against the - unisolated indices by acquiring the appropriate locks. The way - things work right now the upstream join tests a cache for the - downstream join task for a given shard. If there is a cache miss, - it sends a factory task which uses a singleton pattern to start a - join task executing with the appropriate locks under the concurrency - manager and then returns the proxy for that join task to the caller. - This guarantees that each join task has the correct locks, but it - does so at the expense of running one thread per join task. It will - be difficult to get around this one task per shard per join - constraint without going directly to the lock manager with the shard - lock requests. [Or just submitting a per binding set chunk task to - the ConcurrencyManager, which might not be that bad if the chunks - are chunky.] - - ==> Given this, maybe it would be easiest to adapt the current join - execution to allow optional gotos by paying close attention to the - termination condition for the query? We could then refactor to - support BOPs within the same general control logic. A DISTINCT - filter could be yet another custom RMI thing layered directly into - the join logic. - - Rather than defining an eval() method for each operator, we have - standoff interpretation of the pipeline operators (whether for - binding sets, elements, or solutions). The query plan could - encapsulate the local versus distributed execution with annotations - on the operators rather than interposing operators and those - annotations would be used to wrap the sink with one which marshells - the outputs onto NIO buffers. - - - Pipelined chunk-wise nested index join. This is the existing - join algorithm. For each binding set chunk received on a node to - be joined with a given shard, we execute that chunk wise join and - emit the intermediate results. [When the join is optional, we - have an optional target and we send the binding sets which do not - join to that optional target.] - - - multi-block io pipelined join. This is a version of the piplined - chunk-wise nested index join which accumulates much larger chunks - (mega chunks) of binding sets (potentially all intermediate - results) and then computes the join of that using the - operator-at-a-time approach for that mega chunk. The tradeoff - between this join and the pure operator at a time join is that we - can keep the intermediate results off the disk using this - approach but we may have to read the shard multiple times. - - - operator-at-a-time shard wise multi-block-io join. This join - proceeds an operator at a time. Once the producer is done, it - computes the entire join using the intermediate results from the - prior join and a single multi-block IO pass over the shard view. - - A tuple read from the shard joins if there exists a binding set - which is consistent with that tuple. For example, given: - - :- ..., POS(A loves B), SPO(B loves C). - - and the following intermediate results from the POS shard: - - B0:[A=John, B=Mary, ...] - B1:[A=Mary, B=Paul, ...] - B2:[A=Paul, B=Leon, ...] - B3:[A=Leon, B=Paul, ...] - - and the following tuples read from the SPO shard: - - T0:(John loves Mary) - T1:(Mary loves Paul) - T2:(Paul loves Leon) - T3:(Leon loves Paul) - - then we have the following joins: - - (T2, B3) // T2:(Paul loves Leon) with B3:[A=Leon, B=Paul, ...]. - (T3, B2) // T3:(Leon loves Leon) with T2:[A=Paul, B=Leon, ...]. - - There are several ways to manipulate the intermediate results to - setup the join: - - Merge join: Merge sort the binding sets based on the relevant - bound values (A,B) and do an ordered scan of the binding sets and - the shard, emitting results which join. - - Hash join: Create a temporary hash table (backed by disk) whose - keys are the relevant bound values for (A,B) and whose values are - either: (a) the associated binding sets (there can be more than - one per (A,B) and all must be preserved; or (b) the #of offsets - and the offsets into an array or a persistent store of the - associated binding sets. - - Based on our need for an N-way merge sort, I would think that the - merge join will offer the most benefit initially and is likely to - be better organized for execution in memory (it has locality at - more memory tiers). - -- Scaleup query execution pattern: - - Queries which are not distributed can use top-down evaluation of the - left deep pipeline joins. - - -- BOp execution. - - (***) Implement all BOps for which there is existing functionality. - - PipelineJoin: Join binding set producer with local access path - using asBound Predicate. This should be **TRIVIAL** and can - be unit tested against mock objects for the left and right - hand operands. (The vast majority of the logic in JoinTask - dealt with coordinating the pipeline operations. It should - even be trivial to put in "gotos" for failed optional joins. - Defer the logic for mapShards until the core joins are - running.) - - Predicate: read on as bound access path (IChunked(Ordered)Iterator). - - Work through evaluation for BOps, perhaps using BOpUtility, - and definately using unit tests. We should be able to unit - test correct termination, abnormal termination, etc. against - the EDS [move to test suite.] - - How are the rule execution statistics going to be passed - around? How will we get visibility into the running rules and - their current execution statistics (especially for long - running rules)? How can an operator cancel long running - rules? - - Implement optional / conditional binding set routing to - ancestors in the BOp tree (parent and eventual parent). - - Implement map shards with RMI then convert to buffers. Figure - out how to label BOps as senders/receivers. - - Implement map nodes. The ordered list of nodes should be - reused for each MapNodes operator. - - All pipeline operators can specify the pipeline annotations - (chunkSize, chunksOfChunks, etc). - - (***) Harmonize IElementFilter, IConstraint, BOp, IChunkConverter, - PipelineEval. - - Work through how to moving the binding sets and related stuff - around, including when running it into a native heap buffer - and when just draining a queue, blocking buffer's iterator, a - bulk filter (DISTINCT, EXISTS, !EXISTS), bulk completion, etc. - - Asynchronous production of binding sets for vectored pipeline - evaluation. Evaluation should also sets future on buffer as - side effect or collects Future's of the operator tree. - - - Future<Void> eval(IJoinNexus,buffer<BindingSet[]>); - - Evaluation of a predicate, obtaining an access path. The - caller can then wrap the result and do range counts, high - level iterators with filters, low level tuple iterators, - sample the index / view, etc. - - IPredicate<E>: - - - Future<IAccessPath<E>> eval(IJoinNexus,IBindingSet); - - IConstant<T>: - - - T eval(); // return the constant's value. - - IVariable<T>: - - - T eval(IBindingSet); // return the bound value (vs variable name?) - - Striterator patterns return the wrapped iterator. The wrapper - iterator is then applied an (eventual) parent bop. This - pattern can be used with the CTC Striterator, - IChunked(Ordered)Iterator, and probably ITupleIterator (with - the FilterConstructor, which could just be some bops). - - - wrapperItr eval(itr) - - - IElementFilter is element-at-a-time filtering of materialized tuples. - - - IConstraint is IBindingSet-at-a-time filtering. - - - BOp.PipelineEval is IBindingSet at a time evaluation, but it is - designed for chunked pipelineing of the binding sets. - - - IChunkConverter is chunk at a time evaluation and has the sense of - returning a chunk for each chunk consumed. That could be used to - return the bit map from the DISTINCT operator, which is something - that is otherwise missing from the BOp.PipelineEval interface. - - - We need a "batch-oriented" constraint for IN (due to compilation - of the set into an efficient representation prior to execution) - and DISTINCT (it has to batch binding sets in order to amortize - the cost of generating the sort keys and/or chunking up the - results). - - - Reconcile Distinct, DistinctSPOIterator, etc. - - - The IN filters should probably be turned into JOINs against an in - memory IBindingSet[] source except in those cases where we will - have locality in the index for the variable in the IN constraint. - At present, IN is only used by the MatchRule. However, it could - also be used for named graphs and default graphs. There are - specialized filters for those purposes SPARQL data sets right now - {InGraphBinarySearchFilter, InGraphHashSetFilter}, but I think - that this is all pretty general purpose stuff. - -IElementFilter: element at a time filtering (does not scale-out). -.ElementFilter: Unused (discard). -.SameVariableConstraint: precompiles some private state (ok iff immutable). -.SolutionFilter: applies filter to the visited elements; related to rule's head... -.SPOFilter: returns false if not an ISPO. - - canAccept(Object):boolean -..DoNotAddFilter -..ExplicitSPOFilter -..InferredSPOFilter -..InGraphBinarySearchFilter: duplicates IN filter? -..InGraphHashSetFilter: duplicates IN filter? - -IChunkConverter<E,F>: bulk conversion in/out (scales-out since RMI can be chunky). -.BulkCompleteConverter -.BulkFilterConverter -.DistinctFilter: Reconcile with Distinct, ISortKeyBuilder, IN, InGraphHashSetFilter, etc. -.HitConverter - -- Evaluation types: - - IRelation(aka namespace) - IDatabase(aka namespace / AbstractTripleStore) - IIndex, ILocalBTreeView, BTree, IndexSegment, FusedView, - IMap? (aka hash map, DHT) - ITable?, IBat? - File (flat file in/out), - - Bloomfilter - - E[], BlockingBuffer<E[]>.iterator() - - IBindingSet[], - IChunkedIterator<IBindingSet[]>, - BlockingBuffer<IBindingSet[]>.iterator(), - - ISolution[], etc. - - - Life cycle management of resources local to the operator - execution. In some cases, resources must be eventually released, - much like "finally{}". This has to be managed in a distributed - environment where there is no "stack" to be unwound. - - - Explicit management of query priority, buffers, timeout, etc. - - - Visibility into long running queries. - - - Left-deep operator trees for pipelined execution. - - - newInstance(IJoinNexus) : Future<T>. Tasks will run on - ForkJoinPools vs Executors depending on whether they support light - weight asynchronous operations all the way down or need to use a - thread per disk IO. However, if the disk IO thread pool is global - to a DataService and we use queues for disk IO requests, then we - can always use the ForkJoinPool for operators (queuing a request - is consistent with fork/join, right?). - -- Add IOp library. Some operators will be specific to standalone (a - ConcurrentHashMap based distinct) or scale-out (e.g., a DHT based - distinct). - - - RuntimeQueryOptimization(JoinGraph) - Execute the join graph using - interleaved query optimization and query execution. - - Evaluate for both selective and unselective joins. Note that - sampling can result in all relevant tuples being materialized, at - which point chain sampling will perform the actual join and - materialize the real intermediate result set. - - - Predicate. This corresponds to an access path with the current - bindings. - - - SCAN(fromKey,toKey) (local is trivial) - - - SCAN(partitionId,fromKey,toKey) (distributed requires the - partitionId and the local IIndexManager must be set on the - DataService where the operation will execute). - - - Sample(Predicate,limit):(E[]). Sample tuples from a relation which - satisify the predicate. Returns the sampled elements. - - - SampleOp(IOp,limit):(E[],est). Sample output from an operation, - halting if the limit is satisfied. Returns the sample and the - estimated cardinality of the operation. - - - MapNodes(f,BS[],DS[]). Hash partition mapping of some binding sets - using a function across a set of nodes (typically data service - nodes, but that is not required). - - - MapShards(BS[],tail[i+1]). Maps binding sets across shards on - which we need to read for the next join predicate. The operator - is associated with a logical port and a maximum buffer allocation. - - @todo If we want to assign indices to variables for binding sets - then that needs to be done with reference to a total order over - the "rule". With the generalization to an operator tree, the fact - that we strip out variables from the binding set when they are no - longer in use, and the possibility of permutations over the as yet - unevaluated parts of the operator, that mapping needs to be quite - explicit. Perhaps it could be part of the {@link IJoinNexus} or - raised into the root node of the operator tree. - - - Receive()[queryId, port]. Receive binding sets a logical port for - a given query. The receiver is given a maximum buffer allocation. - If the buffers are full, it uses flow control to halt the sender - (NACKs the request to send a buffer to the receiver). Can be used - with MapNodes or MapShards. [Can be tested within a single JVM.] - - - IN(var,E[]). Pipeline join operator binds the variable to each of - the elements in the array in turn for each binding set presented - to the operator. [This is useful for RDF dataset constructs.] - - - JOIN(IPredicate). Pipeline join operator. It accepts binding - sets on one side and joins then against the (local) access path - for the specified predicate. - - - StarJoin(...). - - - ? HashJoin? Join two arguments, each of which evaluates to a - collection of binding sets. This probably can't be pipelined - unless one of the arguments is relatively small and hence can be - fully materialized. Look at how to handle cases where both - arguments have large result sets. - - - ? Execute a correlated subquery and join of the results from that - subquery against with the each binding set presented to the - subquery? - - - DISTINCT (local, concurrent hash map). - - - DISTINCT (distributed hash table). - - - SORT (local, Arrays.sort(), radix sort). - - - SORT (distributed, N-way merge sort). - - - CONSTRUCT (create an element from a binding set). - - - INSERT. Insert elements into a relation (local, sharded uses - buffers to move the constructed elements; behavior can use either - unisolated writes, eventually consistent unisolated writes, or - full transaction isolation). - - - REMOVE. Removes elements from a relation (sharded uses buffers to - move the constructed elements). - - Note: Handle remove of elements matching a predicate elements by - first executing the predicate to select the elements and then - removing the elements using the same kinds of mechansims which are - used for insert. - -============================================================ - -Much of the complexity of the current approach owes itself to having to run a separate task for each join for each shard in order to have the appropriate lock when running against the unisolated shard view. This also means that the join task is running inside of the concurrency manager and hence has the local view of the shard. - -The main, and perhaps the only, reason why we run unisolated rules is during closure, when we query against the unisolated indices and then write the entailments back on the unisolated indices. - -Supporting closure has always been complicated. This complexity is mostly handled by ProgramTask#executeMutation() and AbstractTripleStore#newJoinNexusFactory() which play games with the timestamps used to read and write on the database, with commit points designed to create visibility for tuples written by a mutation rule, and with the automated advance of the read timestamp for the query in each closure pass in order to make newly committed tuples visible to subsequent rounds of closure. For scale-out, we do shard-wise auto commits so we always have a commit point which makes each write visible and the read timestamp is actually a read-only transaction which prevents the historical data we need during a closure round from being released as we are driving updates onto the federation. For the RWStore, we are having a similar problem (in the HA branch since that is where we are working on the RWStore) where historically allocated records were being released as writes drove updates on the indices. Again, we "solved" the problem for the RWStore using a commit point followed by a read-only transaction reading on that commit point to hold onto the view on which the next closure round needs to read (this uncovered a problem with the RWStore and transaction service interaction which Martyn is currently working to resolve through a combination of shadow allocators and deferred deletes which are processed once the release time is advanced by the transaction service). - -The WORM does not have some of these problems with closure because we never delete history, so we do not need to create a commit point and a read-behind transaction. However, the WORM would have problems with concurrent access to the unisolated indices except that we hack that problem through the transparent use of the UnisolatedReadWriteIndex, which allows multiple threads to access the same unisolated index view using a read/write lock pattern (concurrent readers are allowed, but there is only one writer and it has exclusive access when it is running). This works out because we never run closure operations against the WORM through the concurrency manager. If we did, we would have to create a commit point after each mutation and use a read-behind transaction to prevent concurrent access to the unisolated index. - -The main advantage that I can see of the current complexity is that it allows us to do load+closure as a single operation on the WORM, resulting in a single commit point. This makes that operation ACID without having to use full read/write transactions. This is how we gain the ACID contract for the standalone Journal in the SAIL for the WORM. Of course, the SAIL does not have that contract for the RWStore because we have to do the commit and read-behind transaction in order to have visibility and avoid concurrent access to the unisolated index (by reading behind on the last commit point). - -I think that the reality is even one step more complicated. When doing truth maintenance (incremental closure), we bring the temporary graph to a fixed point (the rules write on the temp store) and then apply the delta in a single write to the database. That suggests that incremental truth maintenance would continue to be ACID, but that database-at-once-closure would be round-wise ACID. - -So, I would like to suggest that we break ACID for database-at-once-closure and always follow the pattern of (1) do a commit before each round of closure; and (2) create a read-behind transaction to prevent the release of that commit point as we drive writes onto the indices. If we follow this pattern then we can write on the unisolated indices without conflict and read on the historical views without conflict. Since there will be a commit point before each mutation rule runs (which corresponds to a closure round), database-at-once-closure will be atomic within a round, but will not be a single atomic operation. Per above, I think that we would retain the ACID property for incremental truth maintenance against a WORM or RW mode Journal. - ----- - -The advantage of this proposal (commit before each mutation rule and run query against a read-behind transaction) is that this could enormously simplify how we execute joins. - -Right now, we use a factory pattern to create a join task on each node for each shard for which that node receives binding sets for a query. The main reason for doing this is to gain the appropriate lock for the unisolated index. If we never run a query against the unisolated index then we can go around the concurrency manager and run a single "query manager" task for all joins for all shards for all queries. This has some great benefits which I will go into below. - -That "query manager" task would be responsible for accepting buffers containing elements or binding sets from other nodes and scheduling consumption of those data based on various critieria (order of arrival, priority, buffer resource requirements, timeout, etc.). This manager task could use a fork join pool to execute light weight operations (NIO, formulation of access paths from binding sets, mapping of binding sets onto shards, joining a chunk already read from an access path against a binding set, etc). Operations which touch the disk need to run in their own thread (until we get Java 7 async file IO, which is already available in a preview library). We could handle that by queuing those operations against a fixed size thread pool for reads. - -This is a radical change in how we handle distributed query execution, but I think that it could have a huge payoff by reducing the complexity of the join logic, making it significantly easier to execute different kinds of join operations, reducing the overhead for acquiring locks for the unisolated index views, reducing the #of threads consumed by joins (from one per shard per join per query to a fixed pool of N threads for reads), etc. It would centralize the management of resources on each node and make it possible for us to handle things like join termination by simply purging data from the query manager task for the terminated join. Added: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/ConditionalRoutingOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/ConditionalRoutingOp.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/ConditionalRoutingOp.java 2010-09-03 00:27:45 UTC (rev 3500) @@ -0,0 +1,217 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Aug 25, 2010 + */ + +package com.bigdata.bop.bset; + +import java.util.Arrays; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.FutureTask; + +import com.bigdata.bop.AbstractPipelineOp; +import com.bigdata.bop.BOp; +import com.bigdata.bop.BOpContext; +import com.bigdata.bop.BindingSetPipelineOp; +import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.IConstraint; +import com.bigdata.bop.engine.BOpStats; +import com.bigdata.relation.accesspath.IAsynchronousIterator; +import com.bigdata.relation.accesspath.IBlockingBuffer; + +/** + * An operator for conditional routing of binding sets in a pipeline. The + * operator will copy binding sets either to the default sink (if a condition is + * satisfied) and to the alternate sink otherwise. + * <p> + * Conditional routing can be useful where a different data flow is required + * based on the type of an object (for example a term identifier versus an + * inline term in the RDF database) or where there is a need to jump around a + * join group based on some condition. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +public class ConditionalRoutingOp extends BindingSetPipelineOp { + + /** + * + */ + private static final long serialVersionUID = 1L; + + public interface Annotations extends AbstractPipelineOp.Annotations { + + /** + * An {@link IConstraint} which specifies the condition. When the + * condition is satisfied the binding set is routed to the default sink. + * When the condition is not satisfied, the binding set is routed to the + * alternative sink. + */ + String CONDITION = ConditionalRoutingOp.class.getName() + ".condition"; + + } + + /** + * Deep copy constructor. + * + * @param op + */ + public ConditionalRoutingOp(ConditionalRoutingOp op) { + super(op); + } + + /** + * Shallow copy constructor. + * + * @param args + * @param annotations + */ + public ConditionalRoutingOp(BOp[] args, Map<String, Object> annotations) { + super(args, annotations); + } + + /** + * @see Annotations#CONDITION + */ + public IConstraint getCondition() { + + return (IConstraint) getProperty(Annotations.CONDITION); + + } + + public FutureTask<Void> eval(final BOpContext<IBindingSet> context) { + + return new FutureTask<Void>(new ConditionalRouteTask(this, context)); + + } + + /** + * Copy the source to the sink or the alternative sink depending on the + * condition. + */ + static private class ConditionalRouteTask implements Callable<Void> { + + private final BOpStats stats; + + private final IConstraint condition; + + private final IAsynchronousIterator<IBindingSet[]> source; + + private final IBlockingBuffer<IBindingSet[]> sink; + + private final IBlockingBuffer<IBindingSet[]> sink2; + + ConditionalRouteTask(final ConditionalRoutingOp op, + final BOpContext<IBindingSet> context) { + + this.stats = context.getStats(); + + this.condition = op.getCondition(); + + if (condition == null) + throw new IllegalArgumentException(); + + this.source = context.getSource(); + + this.sink = context.getSink(); + + this.sink2 = context.getSink2(); + + if (sink2 == null) + throw new IllegalArgumentException(); + + if (sink == sink2) + throw new IllegalArgumentException(); + + } + + public Void call() throws Exception { + try { + while (source.hasNext()) { + + final IBindingSet[] chunk = source.next(); + + stats.chunksIn.increment(); + stats.unitsIn.add(chunk.length); + + final IBindingSet[] def = new IBindingSet[chunk.length]; + final IBindingSet[] alt = new IBindingSet[chunk.length]; + + int ndef = 0, nalt = 0; + + for(int i=0; i<chunk.length; i++) { + + final IBindingSet bset = chunk[i]; + + if (condition.accept(bset)) { + + def[ndef++] = bset; + + } else { + + alt[nalt++] = bset; + + } + + } + + if (ndef > 0) { + if (ndef == def.length) + sink.add(def); + else + sink.add(Arrays.copyOf(def, ndef)); + stats.chunksOut.increment(); + stats.unitsOut.add(ndef); + } + + if (nalt > 0) { + if (nalt == alt.length) + sink2.add(alt); + else + sink2.add(Arrays.copyOf(alt, nalt)); + stats.chunksOut.increment(); + stats.unitsOut.add(nalt); + } + + } + + sink.flush(); + sink2.flush(); + + return null; + + } finally { + + sink.close(); + sink2.close(); + + } + + } + + } + +} Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/ConditionalRoutingOp.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Copied: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/CopyBindingSetOp.java (from rev 3495, branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineStartOp.java) =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/CopyBindingSetOp.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/CopyBindingSetOp.java 2010-09-03 00:27:45 UTC (rev 3500) @@ -0,0 +1,124 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Aug 25, 2010 + */ + +package com.bigdata.bop.bset; + +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.FutureTask; + +import com.bigdata.bop.BOp; +import com.bigdata.bop.BOpContext; +import com.bigdata.bop.BindingSetPipelineOp; +import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.engine.BOpStats; +import com.bigdata.relation.accesspath.IAsynchronousIterator; +import com.bigdata.relation.accesspath.IBlockingBuffer; + +/** + * This operator copies its source to its sink. It is used to feed the first + * join in the pipeline. The operator should have no children but may be + * decorated with annotations as necessary. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + * + * @todo unit tests. + */ +public class CopyBindingSetOp extends BindingSetPipelineOp { + + /** + * + */ + private static final long serialVersionUID = 1L; + + /** + * Deep copy constructor. + * + * @param op + */ + public CopyBindingSetOp(CopyBindingSetOp op) { + super(op); + } + + /** + * Shallow copy constructor. + * + * @param args + * @param annotations + */ + public CopyBindingSetOp(BOp[] args, Map<String, Object> annotations) { + super(args, annotations); + } + + public FutureTask<Void> eval(final BOpContext<IBindingSet> context) { + + return new FutureTask<Void>(new CopyTask(context)); + + } + + /** + * Copy the source to the sink. + */ + static private class CopyTask implements Callable<Void> { + + private final BOpStats stats; + + private final IAsynchronousIterator<IBindingSet[]> source; + + private final IBlockingBuffer<IBindingSet[]> sink; + + CopyTask(final BOpContext<IBindingSet> context) { + + stats = context.getStats(); + + this.source = context.getSource(); + + this.sink = context.getSink(); + + } + + public Void call() throws Exception { + try { + while (source.hasNext()) { + final IBindingSet[] chunk = source.next(); + stats.chunksIn.increment(); + stats.unitsIn.add(chunk.length); + sink.add(chunk); + stats.chunksOut.increment(); + stats.unitsOut.add(chunk.length); + } + sink.flush(); + return null; + } finally { + sink.close(); + } + } + + } + +} Copied: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/Union.java (from rev 3495, branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/aggregation/Union.java) =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/Union.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/Union.java 2010-09-03 00:27:45 UTC (rev 3500) @@ -0,0 +1,135 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Aug 18, 2010 + */ + +package com.bigdata.bop.bset; + +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.FutureTask; + +import com.bigdata.bop.BOpContext; +import com.bigdata.bop.BindingSetPipelineOp; +import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.join.PipelineJoin; +import com.bigdata.rdf.rules.TMUtility; +import com.bigdata.relation.RelationFusedView; +import com.bigdata.util.concurrent.Haltable; + +/** + * The union of two or more {@link BindingSetPipelineOp} operators. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + * + * @todo I have some basic questions about the ability to use a UNION of two + * predicates in scale-out. I think that this might be more accurately + * modeled as the UNION of two joins. That is, rather than: + * + * <pre> + * JOIN( ..., + * UNION( foo.spo(A,loves,B), + * bar.spo(A,loves,B) ) + * ) + * </pre> + * + * using + * + * <pre> + * UNION( JOIN( ..., foo.spo(A,loves,B) ), + * JOIN( ..., bar.spo(A,loves,B) ) + * ) + * </pre> + * + * which would be a binding set union rather than an element union. + * + * @todo The union of access paths was historically handled by + * {@link RelationFusedView}. That class should be removed once queries + * are rewritten to use the union of joins. + * + * @todo The {@link TMUtility} will have to be updated to use this operator + * rather than specifying multiple source "names" for the relation of the + * predicate. + * + * @todo The FastClosureRuleTask will also need to be updated to use a + * {@link Union} over the joins rather than a {@link RelationFusedView}. + */ +public class Union extends BindingSetPipelineOp { + + /** + * + */ + private static final long serialVersionUID = 1L; + + /** + * @param args + * Two or more operators whose union is desired. + * @param annotations + */ + public Union(final BindingSetPipelineOp[] args, + final Map<String, Object> annotations) { + + super(args, annotations); + + if (args.length < 2) + throw new IllegalArgumentException(); + + } + + public FutureTask<Void> eval(final BOpContext<IBindingSet> context) { + + return new FutureTask<Void>(new UnionTask(this, context)); + + } + + /** + * Pipeline union impl. + * + * FIXME All this does is copy its inputs to its outputs. Since we only run + * one chunk of input at a time, it seems that the easiest way to implement + * a union is to have the operators in the union just target the same sink. + */ + private static class UnionTask extends Haltable<Void> implements Callable<Void> { + + public UnionTask(// + final Union op,// + final BOpContext<IBindingSet> context + ) { + + if (op == null) + throw new IllegalArgumentException(); + if (context == null) + throw new IllegalArgumentException(); + } + + public Void call() throws Exception { + // TODO Auto-generated method stub + throw new UnsupportedOperationException(); + } + + } + +} Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 2010-09-02 22:24:22 UTC (rev 3499) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 20... [truncated message content] |
From: <tho...@us...> - 2010-09-05 20:17:51
|
Revision: 3510 http://bigdata.svn.sourceforge.net/bigdata/?rev=3510&view=rev Author: thompsonbry Date: 2010-09-05 20:17:44 +0000 (Sun, 05 Sep 2010) Log Message: ----------- Introduced an interface for running queries and refactored the BOpContext to use that interface. This makes it possible to test more things with mock objects and also provides a hook from the BOpContext back to the RunningQuery. That hook is necessary when we need to halt a running query (for an offset/limit slice). Added Rule2BOpUtility which will handle the initial conversion from a Rule or Program into a bop tree. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/bset/TestConditionalRoutingOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/QueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/RunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/join/TestPipelineJoin.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestDistinctBindingSets.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java Added Paths: ----------- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/IRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/MockRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/Rule2BOpUtility.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java 2010-09-05 18:16:01 UTC (rev 3509) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java 2010-09-05 20:17:44 UTC (rev 3510) @@ -27,20 +27,18 @@ */ package com.bigdata.bop; -import java.util.Iterator; - import org.apache.log4j.Logger; import com.bigdata.bop.engine.BOpStats; +import com.bigdata.bop.engine.IRunningQuery; import com.bigdata.bop.engine.QueryEngine; +import com.bigdata.bop.engine.RunningQuery; import com.bigdata.bop.solutions.SliceOp; import com.bigdata.btree.IIndex; import com.bigdata.btree.ILocalBTreeView; import com.bigdata.btree.IRangeQuery; import com.bigdata.journal.IIndexManager; -import com.bigdata.journal.ITx; import com.bigdata.journal.TimestampUtility; -import com.bigdata.mdi.PartitionLocator; import com.bigdata.relation.IRelation; import com.bigdata.relation.accesspath.AccessPath; import com.bigdata.relation.accesspath.IAccessPath; @@ -49,11 +47,10 @@ import com.bigdata.relation.locator.IResourceLocator; import com.bigdata.relation.rule.IRule; import com.bigdata.relation.rule.eval.IJoinNexus; -import com.bigdata.service.AbstractScaleOutFederation; import com.bigdata.service.DataService; import com.bigdata.service.IBigdataFederation; -import com.bigdata.service.ndx.IClientIndex; import com.bigdata.striterator.IKeyOrder; +import com.ibm.icu.impl.ByteBuffer; /** * The evaluation context for the operator (NOT serializable). @@ -65,14 +62,16 @@ static private final Logger log = Logger.getLogger(BOpContext.class); - private final IBigdataFederation<?> fed; + private final IRunningQuery runningQuery; + +// private final IBigdataFederation<?> fed; +// +// private final IIndexManager indexManager; +// +// private final long readTimestamp; +// +// private final long writeTimestamp; - private final IIndexManager indexManager; - - private final long readTimestamp; - - private final long writeTimestamp; - private final int partitionId; private final BOpStats stats; @@ -84,13 +83,25 @@ private final IBlockingBuffer<E[]> sink2; /** + * The interface for a running query. + * <p> + * Note: In scale-out each node will have a distinct {@link IRunningQuery} + * object and the query controller will have access to additional state, + * such as the aggregation of the {@link BOpStats} for the query on all + * nodes. + */ + public IRunningQuery getRunningQuery() { + return runningQuery; + } + + /** * The {@link IBigdataFederation} IFF the operator is being evaluated on an * {@link IBigdataFederation}. When evaluating operations against an * {@link IBigdataFederation}, this reference provides access to the * scale-out view of the indices and to other bigdata services. */ public IBigdataFederation<?> getFederation() { - return fed; + return runningQuery.getFederation(); } /** @@ -100,7 +111,7 @@ * {@link ILocalBTreeView}. */ public final IIndexManager getIndexManager() { - return indexManager; + return runningQuery.getIndexManager(); } /** @@ -108,7 +119,7 @@ * reading. */ public final long getReadTimestamp() { - return readTimestamp; + return runningQuery.getReadTimestamp(); } /** @@ -116,7 +127,7 @@ * writing. */ public final long getWriteTimestamp() { - return writeTimestamp; + return runningQuery.getWriteTimestamp(); } /** @@ -137,6 +148,22 @@ /** * Where to read the data to be consumed by the operator. + * + * @todo Since joins now run from locally materialized data in all cases the + * API could be simplified somewhat given that we know that there will + * be a single "source" chunk of binding sets. Also, the reason for + * the {@link IAsynchronousIterator} here is that a downstream join + * could error (or satisfy a slice) and halt the upstream joins. That + * is being coordinated through the {@link RunningQuery} now. + * <p> + * It is not yet clear what the right API is for the source. The + * iterator model might be just fine, but might not need to be + * asynchronous and does not need to be closeable. + * <p> + * Perhaps the right thing is to expose an object with a richer API + * for obtaining various kinds of iterators or even access to the + * direct {@link ByteBuffer}s backing the data (for high volume joins, + * exernal merge sorts, etc). */ public final IAsynchronousIterator<E[]> getSource() { return source; @@ -194,18 +221,6 @@ * failed joins outside of the join group. * * @throws IllegalArgumentException - * if the <i>indexManager</i> is <code>null</code> - * @throws IllegalArgumentException - * if the <i>indexManager</i> is is not a <em>local</em> index - * manager. - * @throws IllegalArgumentException - * if the <i>readTimestamp</i> is {@link ITx#UNISOLATED} - * (queries may not read on the unisolated indices). - * @throws IllegalArgumentException - * if the <i>writeTimestamp</i> is neither - * {@link ITx#UNISOLATED} nor a read-write transaction - * identifier. - * @throws IllegalArgumentException * if the <i>stats</i> is <code>null</code> * @throws IllegalArgumentException * if the <i>source</i> is <code>null</code> (use an empty @@ -213,37 +228,47 @@ * @throws IllegalArgumentException * if the <i>sink</i> is <code>null</code> */ - public BOpContext(final IBigdataFederation<?> fed, - final IIndexManager indexManager, final long readTimestamp, - final long writeTimestamp, final int partitionId, +// * @throws IllegalArgumentException +// * if the <i>indexManager</i> is <code>null</code> +// * @throws IllegalArgumentException +// * if the <i>indexManager</i> is is not a <em>local</em> index +// * manager. +// * @throws IllegalArgumentException +// * if the <i>readTimestamp</i> is {@link ITx#UNISOLATED} +// * (queries may not read on the unisolated indices). +// * @throws IllegalArgumentException +// * if the <i>writeTimestamp</i> is neither +// * {@link ITx#UNISOLATED} nor a read-write transaction +// * identifier. + public BOpContext(final IRunningQuery runningQuery,final int partitionId, final BOpStats stats, final IAsynchronousIterator<E[]> source, final IBlockingBuffer<E[]> sink, final IBlockingBuffer<E[]> sink2) { - if (indexManager == null) - throw new IllegalArgumentException(); - if (indexManager instanceof IBigdataFederation<?>) { - /* - * This is disallowed because the predicate specifies an index - * partition and expects to have access to the local index objects - * for that index partition. - */ - throw new IllegalArgumentException( - "Expecting a local index manager, not: " - + indexManager.getClass().toString()); - } - if (readTimestamp == ITx.UNISOLATED) - throw new IllegalArgumentException(); - if (TimestampUtility.isReadOnly(writeTimestamp)) - throw new IllegalArgumentException(); + this.runningQuery = runningQuery; +// if (indexManager == null) +// throw new IllegalArgumentException(); +// if (indexManager instanceof IBigdataFederation<?>) { +// /* +// * This is disallowed because predicates always read on local index +// * objects, even in scale-out. +// */ +// throw new IllegalArgumentException( +// "Expecting a local index manager, not: " +// + indexManager.getClass().toString()); +// } +// if (readTimestamp == ITx.UNISOLATED) +// throw new IllegalArgumentException(); +// if (TimestampUtility.isReadOnly(writeTimestamp)) +// throw new IllegalArgumentException(); if (stats == null) throw new IllegalArgumentException(); if (source == null) throw new IllegalArgumentException(); if (sink == null) throw new IllegalArgumentException(); - this.fed = fed; // may be null - this.indexManager = indexManager; - this.readTimestamp = readTimestamp; - this.writeTimestamp = writeTimestamp; +// this.fed = fed; // may be null +// this.indexManager = indexManager; +// this.readTimestamp = readTimestamp; +// this.writeTimestamp = writeTimestamp; this.partitionId = partitionId; this.stats = stats; this.source = source; @@ -344,24 +369,28 @@ if (partitionId == -1) { - if(indexManager instanceof IBigdataFederation<?>) + if (getFederation() != null) { + // This is scale-out so the partition identifier is required. throw new UnsupportedOperationException(); + } // The index is not partitioned. - ndx = (ILocalBTreeView) indexManager.getIndex(namespace + "." + ndx = (ILocalBTreeView) getIndexManager().getIndex(namespace + "." + keyOrder.getIndexName(), getWriteTimestamp()); } else { - if(!(indexManager instanceof IBigdataFederation<?>)) + if (getFederation() == null) { + // This is not scale-out so index partitions are not supported. throw new UnsupportedOperationException(); + } // The name of the desired index partition. final String name = DataService.getIndexPartitionName(namespace + "." + keyOrder.getIndexName(), partitionId); // MUST be a local index view. - ndx = (ILocalBTreeView) indexManager.getIndex(name, + ndx = (ILocalBTreeView) getIndexManager().getIndex(name, getWriteTimestamp()); } @@ -427,6 +456,10 @@ PipelineOp.Annotations.FULLY_BUFFERED_READ_THRESHOLD, PipelineOp.Annotations.DEFAULT_FULLY_BUFFERED_READ_THRESHOLD); + final IIndexManager indexManager = getIndexManager(); + + final long readTimestamp = getReadTimestamp(); + if (predicate.getPartitionId() != -1) { /* @@ -669,6 +702,8 @@ * with that thrown cause. */ public void halt() { + + runningQuery.halt(); } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 2010-09-05 18:16:01 UTC (rev 3509) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 2010-09-05 20:17:44 UTC (rev 3510) @@ -101,9 +101,6 @@ */ private static final long serialVersionUID = 1L; - /** - * @todo Declare SLICE annotation and support SLICE in the {@link JoinTask}. - */ public interface Annotations extends BindingSetPipelineOp.Annotations { /** @@ -478,11 +475,6 @@ this.optional = joinOp.isOptional(); this.variablesToKeep = joinOp.variablesToKeep(); this.context = context; - /* - * FIXME Carefully review which index manager (local versus fed) is - * being used to resolve the relation. Also note that we used to - * cache the resourceLocator. - */ this.relation = context.getReadRelation(right); this.source = context.getSource(); this.sink = context.getSink(); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/bset/TestConditionalRoutingOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/bset/TestConditionalRoutingOp.java 2010-09-05 18:16:01 UTC (rev 3509) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/bset/TestConditionalRoutingOp.java 2010-09-05 20:17:44 UTC (rev 3510) @@ -29,7 +29,6 @@ import java.util.LinkedList; import java.util.List; -import java.util.Properties; import java.util.concurrent.ExecutionException; import java.util.concurrent.FutureTask; @@ -47,11 +46,10 @@ import com.bigdata.bop.Var; import com.bigdata.bop.constraint.EQConstant; import com.bigdata.bop.engine.BOpStats; +import com.bigdata.bop.engine.MockRunningQuery; import com.bigdata.bop.engine.TestQueryEngine; import com.bigdata.bop.solutions.DistinctBindingSetOp; -import com.bigdata.journal.BufferMode; import com.bigdata.journal.ITx; -import com.bigdata.journal.Journal; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; import com.bigdata.relation.accesspath.ThickAsynchronousIterator; @@ -79,25 +77,25 @@ super(name); } - @Override - public Properties getProperties() { +// @Override +// public Properties getProperties() { +// +// final Properties p = new Properties(super.getProperties()); +// +// p.setProperty(Journal.Options.BUFFER_MODE, BufferMode.Transient +// .toString()); +// +// return p; +// +// } - final Properties p = new Properties(super.getProperties()); +// Journal jnl = null; - p.setProperty(Journal.Options.BUFFER_MODE, BufferMode.Transient - .toString()); - - return p; - - } - - Journal jnl = null; - List<IBindingSet> data = null; public void setUp() throws Exception { - jnl = new Journal(getProperties()); +// jnl = new Journal(getProperties()); setUpData(); @@ -147,10 +145,10 @@ public void tearDown() throws Exception { - if (jnl != null) { - jnl.destroy(); - jnl = null; - } +// if (jnl != null) { +// jnl.destroy(); +// jnl = null; +// } // clear reference. data = null; @@ -212,16 +210,18 @@ final IBlockingBuffer<IBindingSet[]> sink2 = query.newBuffer(); final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( - null/* fed */, jnl/* indexManager */, + new MockRunningQuery( + null/* fed */, null/* indexManager */, ITx.READ_COMMITTED/* readTimestamp */, - ITx.UNISOLATED/* writeTimestamp */, -1/* partitionId */, stats, + ITx.UNISOLATED/* writeTimestamp */), -1/* partitionId */, stats, source, sink, sink2); // get task. final FutureTask<Void> ft = query.eval(context); // execute task. - jnl.getExecutorService().execute(ft); +// jnl.getExecutorService().execute(ft); + ft.run(); TestQueryEngine.assertSameSolutions(expected, sink.iterator()); TestQueryEngine.assertSameSolutions(expected2, sink2.iterator()); Added: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/IRunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/IRunningQuery.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/IRunningQuery.java 2010-09-05 20:17:44 UTC (rev 3510) @@ -0,0 +1,75 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA + */ +/* + * Created on Sep 5, 2010 + */ + +package com.bigdata.bop.engine; + +import com.bigdata.btree.ILocalBTreeView; +import com.bigdata.journal.IIndexManager; +import com.bigdata.service.IBigdataFederation; + +/** + * Interface exposing a limited set of the state of an executing query. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +public interface IRunningQuery { + + /** + * The {@link IBigdataFederation} IFF the operator is being evaluated on an + * {@link IBigdataFederation}. When evaluating operations against an + * {@link IBigdataFederation}, this reference provides access to the + * scale-out view of the indices and to other bigdata services. + */ + IBigdataFederation<?> getFederation(); + + /** + * The <strong>local</strong> {@link IIndexManager}. Query evaluation occurs + * against the local indices. In scale-out, query evaluation proceeds shard + * wise and this {@link IIndexManager} MUST be able to read on the + * {@link ILocalBTreeView}. + */ + IIndexManager getIndexManager(); + + /** + * The timestamp or transaction identifier against which the query is + * reading. + */ + long getReadTimestamp(); + + /** + * The timestamp or transaction identifier against which the query is + * writing. + */ + long getWriteTimestamp(); + + /** + * Terminate query evaluation + */ + void halt(); + +} Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/IRunningQuery.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Added: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/MockRunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/MockRunningQuery.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/MockRunningQuery.java 2010-09-05 20:17:44 UTC (rev 3510) @@ -0,0 +1,96 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Sep 5, 2010 + */ + +package com.bigdata.bop.engine; + +import org.apache.log4j.Logger; + +import com.bigdata.journal.IIndexManager; +import com.bigdata.service.IBigdataFederation; + +/** + * Mock object. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +public class MockRunningQuery implements IRunningQuery { + + private static final Logger log = Logger.getLogger(MockRunningQuery.class); + + private final IBigdataFederation<?> fed; + + private final IIndexManager indexManager; + + private final long readTimestamp; + + private final long writeTimestamp; + + /** + * Note: This constructor DOES NOT check its arguments so unit tests may be + * written with the minimum dependencies + * + * @param fed + * @param indexManager + * @param readTimestamp + * @param writeTimestamp + */ + public MockRunningQuery(final IBigdataFederation<?> fed, + final IIndexManager indexManager, final long readTimestamp, + final long writeTimestamp) { + + this.fed = fed; + this.indexManager = indexManager; + this.readTimestamp = readTimestamp; + this.writeTimestamp = writeTimestamp; + + } + + public IBigdataFederation<?> getFederation() { + return fed; + } + + public IIndexManager getIndexManager() { + return indexManager; + } + + public long getReadTimestamp() { + return readTimestamp; + } + + public long getWriteTimestamp() { + return writeTimestamp; + } + + /** + * NOP (you have to test things like slices with a full integration). + */ + public void halt() { + log.warn("Mock object does not implement halt()"); + } + +} Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/MockRunningQuery.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/QueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/QueryEngine.java 2010-09-05 18:16:01 UTC (rev 3509) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/QueryEngine.java 2010-09-05 20:17:44 UTC (rev 3510) @@ -41,10 +41,12 @@ import org.apache.log4j.Logger; +import com.bigdata.bop.BOp; import com.bigdata.bop.BindingSetPipelineOp; -import com.bigdata.bop.BOp; import com.bigdata.bop.IBindingSet; import com.bigdata.journal.IIndexManager; +import com.bigdata.journal.ITx; +import com.bigdata.journal.TimestampUtility; import com.bigdata.relation.accesspath.IBlockingBuffer; import com.bigdata.service.IBigdataFederation; @@ -580,6 +582,13 @@ * evaluating the query. * * @throws Exception + * @throws IllegalArgumentException + * if the <i>readTimestamp</i> is {@link ITx#UNISOLATED} + * (queries may not read on the unisolated indices). + * @throws IllegalArgumentException + * if the <i>writeTimestamp</i> is neither + * {@link ITx#UNISOLATED} nor a read-write transaction + * identifier. * * @todo Consider elevating the read/write timestamps into the query plan as * annotations. Closure would then rewrite the query plan for each @@ -597,6 +606,10 @@ if (query == null) throw new IllegalArgumentException(); + if (readTimestamp == ITx.UNISOLATED) + throw new IllegalArgumentException(); + if (TimestampUtility.isReadOnly(writeTimestamp)) + throw new IllegalArgumentException(); final long timeout = query.getProperty(BOp.Annotations.TIMEOUT, BOp.Annotations.DEFAULT_TIMEOUT); Added: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/Rule2BOpUtility.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/Rule2BOpUtility.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/Rule2BOpUtility.java 2010-09-05 20:17:44 UTC (rev 3510) @@ -0,0 +1,93 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Sep 5, 2010 + */ + +package com.bigdata.bop.engine; + +import com.bigdata.bop.BOp; +import com.bigdata.rdf.sail.BigdataSail; +import com.bigdata.relation.rule.IProgram; +import com.bigdata.relation.rule.IRule; +import com.bigdata.relation.rule.IStep; +import com.bigdata.relation.rule.Program; +import com.bigdata.relation.rule.Rule; + +/** + * Utility class converts {@link IRule}s to {@link BOp}s. + * <p> + * Note: This is a stopgap measure designed to allow us to evaluate SPARQL + * queries and verify the standalone {@link QueryEngine} while we develop a + * direct translation from Sesame's SPARQL operator tree onto {@link BOp}s and + * work on the scale-out query buffer transfer mechanisms. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +public class Rule2BOpUtility { + + /** + * Convert an {@link IStep} into an operator tree. This should handle + * {@link IRule}s and {@link IProgram}s as they are currently implemented + * and used by the {@link BigdataSail}. + * + * @param step + * The step. + * + * @return + */ + public static BOp convert(final IStep step) { + + throw new UnsupportedOperationException(); + + } + + /** + * Convert a rule into an operator tree. + * + * @param rule + * + * @return + */ + public static BOp convert(final Rule rule) { + + throw new UnsupportedOperationException(); + + } + + /** + * Convert a program into an operator tree. + * + * @param program + * + * @return + */ + public static BOp convert(final Program program) { + + throw new UnsupportedOperationException(); + + } + +} Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/Rule2BOpUtility.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/RunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/RunningQuery.java 2010-09-05 18:16:01 UTC (rev 3509) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/RunningQuery.java 2010-09-05 20:17:44 UTC (rev 3510) @@ -59,6 +59,8 @@ import com.bigdata.bop.NoSuchBOpException; import com.bigdata.bop.ap.Predicate; import com.bigdata.bop.bset.Union; +import com.bigdata.bop.solutions.SliceOp; +import com.bigdata.journal.IIndexManager; import com.bigdata.rdf.internal.IV; import com.bigdata.rdf.spo.SPORelation; import com.bigdata.relation.IMutableRelation; @@ -69,8 +71,9 @@ import com.bigdata.relation.accesspath.IElementFilter; import com.bigdata.relation.rule.IRule; import com.bigdata.relation.rule.Program; -import com.bigdata.relation.rule.eval.pipeline.DistributedJoinMasterTask; +import com.bigdata.relation.rule.eval.pipeline.DistributedJoinTask; import com.bigdata.resources.ResourceManager; +import com.bigdata.service.IBigdataFederation; import com.bigdata.service.ndx.IAsynchronousWriteBufferFactory; import com.bigdata.striterator.ChunkedArrayIterator; import com.bigdata.striterator.IChunkedOrderedIterator; @@ -82,7 +85,7 @@ * * @todo HA aspects of running queries? Checkpoints for long running queries? */ -public class RunningQuery implements Future<Map<Integer,BOpStats>> { +public class RunningQuery implements Future<Map<Integer,BOpStats>>, IRunningQuery { private final static transient Logger log = Logger .getLogger(RunningQuery.class); @@ -887,10 +890,8 @@ final IBlockingBuffer<IBindingSet[]> altSink = altSinkId == null ? null : op.newBuffer(); // context - final BOpContext context = new BOpContext(queryEngine.getFederation(), - queryEngine.getLocalIndexManager(), readTimestamp, - writeTimestamp, chunk.partitionId, op.newStats(), chunk.source, - sink, altSink); + final BOpContext context = new BOpContext(this, chunk.partitionId, op + .newStats(), chunk.source, sink, altSink); // FutureTask for operator execution (not running yet). final FutureTask<Void> f = op.eval(context); // Hook the FutureTask. @@ -976,9 +977,21 @@ * various methods in order to clean up the state of a completed query. */ + public void halt() { + + cancel(true/* mayInterruptIfRunning */); + + } + /** - * @todo Cancelled queries must reject or drop new chunks, etc. Queries must - * release all of their resources when they are done(). + * @todo Cancelled queries must reject or drop new chunks, etc. + * <p> + * Queries must release all of their resources when they are done(). + * <p> + * Queries MUST NOT cause the solutions to be discarded before the + * client can consume them. This means that we have to carefully + * integrate/product {@link SliceOp} or just wrap the query buffer to + * impose the slice (simpler). */ final public boolean cancel(final boolean mayInterruptIfRunning) { // halt the query. @@ -1028,4 +1041,20 @@ } + public IBigdataFederation<?> getFederation() { + return queryEngine.getFederation(); + } + + public IIndexManager getIndexManager() { + return queryEngine.getLocalIndexManager(); + } + + public long getReadTimestamp() { + return readTimestamp; + } + + public long getWriteTimestamp() { + return writeTimestamp; + } + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-09-05 18:16:01 UTC (rev 3509) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-09-05 20:17:44 UTC (rev 3510) @@ -33,10 +33,10 @@ import junit.framework.TestCase2; -import com.bigdata.bop.BindingSetPipelineOp; import com.bigdata.bop.ArrayBindingSet; import com.bigdata.bop.BOp; import com.bigdata.bop.BOpContext; +import com.bigdata.bop.BindingSetPipelineOp; import com.bigdata.bop.Constant; import com.bigdata.bop.HashBindingSet; import com.bigdata.bop.IBindingSet; @@ -59,6 +59,7 @@ import com.bigdata.service.jini.JiniFederation; import com.bigdata.striterator.ChunkedArrayIterator; import com.bigdata.striterator.ICloseableIterator; +import com.ibm.icu.impl.ByteBuffer; /** * Test suite for the {@link QueryEngine} against a local database instance. @@ -389,6 +390,12 @@ * the {@link PipelineDelayOp} can be used to impose sufficient * latency on the pipeline that the test can close the query buffer * iterator first]. + * <p> + * This must also be tested in scale-out to make sure that the data + * backing the solutions is not discarded before the caller can use + * those data. [This could be handled by materializing binding set + * objects out of a {@link ByteBuffer} rather than using a live decode + * of the data in that {@link ByteBuffer}.] */ public void test_query_closeIterator() { @@ -397,6 +404,26 @@ } /** + * @todo Test ability to impose a limit/offset slice on a query. + * <p> + * Note: While the logic for visiting only the solutions selected by + * the slice can be tested against a mock object, the integration by + * which a slice halts a query when it is satisfied has to be tested + * against a {@link QueryEngine}. + * <p> + * This must also be tested in scale-out to make sure that the data + * backing the solutions is not discarded before the caller can use + * those data. [This could be handled by materializing binding set + * objects out of a {@link ByteBuffer} rather than using a live decode + * of the data in that {@link ByteBuffer}.] + */ + public void test_query_slice() { + + fail("write test"); + + } + + /** * @todo Test the ability run a query reading on an access path using a * element filter (other than DISTINCT). */ Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/join/TestPipelineJoin.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/join/TestPipelineJoin.java 2010-09-05 18:16:01 UTC (rev 3509) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/join/TestPipelineJoin.java 2010-09-05 20:17:44 UTC (rev 3510) @@ -51,6 +51,7 @@ import com.bigdata.bop.ap.R; import com.bigdata.bop.bset.CopyBindingSetOp; import com.bigdata.bop.constraint.INBinarySearch; +import com.bigdata.bop.engine.MockRunningQuery; import com.bigdata.bop.engine.TestQueryEngine; import com.bigdata.bop.join.PipelineJoin.PipelineJoinStats; import com.bigdata.journal.BufferMode; @@ -215,9 +216,9 @@ final IBlockingBuffer<IBindingSet[]> sink = query.newBuffer(); final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( - null/* fed */, jnl/* indexManager */, + new MockRunningQuery(null/* fed */, jnl/* indexManager */, ITx.READ_COMMITTED/* readTimestamp */, - ITx.UNISOLATED/* writeTimestamp */, -1/* partitionId */, stats, + ITx.UNISOLATED/* writeTimestamp */), -1/* partitionId */, stats, source, sink, null/* sink2 */); // get task. @@ -312,9 +313,9 @@ final IBlockingBuffer<IBindingSet[]> sink = query.newBuffer(); final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( - null/* fed */, jnl/* indexManager */, + new MockRunningQuery(null/* fed */, jnl/* indexManager */, ITx.READ_COMMITTED/* readTimestamp */, - ITx.UNISOLATED/* writeTimestamp */, -1/* partitionId */, stats, + ITx.UNISOLATED/* writeTimestamp */), -1/* partitionId */, stats, source, sink, null/* sink2 */); // get task. @@ -435,9 +436,9 @@ final IBlockingBuffer<IBindingSet[]> sink = query.newBuffer(); final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( - null/* fed */, jnl/* indexManager */, + new MockRunningQuery(null/* fed */, jnl/* indexManager */, ITx.READ_COMMITTED/* readTimestamp */, - ITx.UNISOLATED/* writeTimestamp */, -1/* partitionId */, stats, + ITx.UNISOLATED/* writeTimestamp */), -1/* partitionId */, stats, source, sink, null/* sink2 */); // get task. @@ -553,9 +554,9 @@ final PipelineJoinStats stats = query.newStats(); final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( - null/* fed */, jnl/* indexManager */, + new MockRunningQuery(null/* fed */, jnl/* indexManager */, ITx.READ_COMMITTED/* readTimestamp */, - ITx.UNISOLATED/* writeTimestamp */, -1/* partitionId */, stats, + ITx.UNISOLATED/* writeTimestamp */), -1/* partitionId */, stats, source, sink, null/* sink2 */); // get task. @@ -675,9 +676,9 @@ final PipelineJoinStats stats = query.newStats(); final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( - null/* fed */, jnl/* indexManager */, + new MockRunningQuery(null/* fed */, jnl/* indexManager */, ITx.READ_COMMITTED/* readTimestamp */, - ITx.UNISOLATED/* writeTimestamp */, -1/* partitionId */, stats, + ITx.UNISOLATED/* writeTimestamp */), -1/* partitionId */, stats, source, sink, sink2); // get task. Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestDistinctBindingSets.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestDistinctBindingSets.java 2010-09-05 18:16:01 UTC (rev 3509) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestDistinctBindingSets.java 2010-09-05 20:17:44 UTC (rev 3510) @@ -29,7 +29,6 @@ import java.util.LinkedList; import java.util.List; -import java.util.Properties; import java.util.concurrent.ExecutionException; import java.util.concurrent.FutureTask; @@ -46,11 +45,9 @@ import com.bigdata.bop.NV; import com.bigdata.bop.Var; import com.bigdata.bop.engine.BOpStats; +import com.bigdata.bop.engine.MockRunningQuery; import com.bigdata.bop.engine.TestQueryEngine; -import com.bigdata.bop.solutions.DistinctBindingSetOp; -import com.bigdata.journal.BufferMode; import com.bigdata.journal.ITx; -import com.bigdata.journal.Journal; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; import com.bigdata.relation.accesspath.ThickAsynchronousIterator; @@ -78,25 +75,25 @@ super(name); } - @Override - public Properties getProperties() { +// @Override +// public Properties getProperties() { +// +// final Properties p = new Properties(super.getProperties()); +// +// p.setProperty(Journal.Options.BUFFER_MODE, BufferMode.Transient +// .toString()); +// +// return p; +// +// } - final Properties p = new Properties(super.getProperties()); +// Journal jnl = null; - p.setProperty(Journal.Options.BUFFER_MODE, BufferMode.Transient - .toString()); - - return p; - - } - - Journal jnl = null; - List<IBindingSet> data = null; public void setUp() throws Exception { - jnl = new Journal(getProperties()); +// jnl = new Journal(getProperties()); setUpData(); @@ -153,11 +150,11 @@ public void tearDown() throws Exception { - if (jnl != null) { - jnl.destroy(); - jnl = null; - } - +// if (jnl != null) { +// jnl.destroy(); +// jnl = null; +// } +// // clear reference. data = null; @@ -208,16 +205,17 @@ final IBlockingBuffer<IBindingSet[]> sink = query.newBuffer(); final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( - null/* fed */, jnl/* indexManager */, + new MockRunningQuery(null/* fed */, null/* indexManager */, ITx.READ_COMMITTED/* readTimestamp */, - ITx.UNISOLATED/* writeTimestamp */, -1/* partitionId */, stats, + ITx.UNISOLATED/* writeTimestamp */), -1/* partitionId */, stats, source, sink, null/* sink2 */); // get task. final FutureTask<Void> ft = query.eval(context); // execute task. - jnl.getExecutorService().execute(ft); +// jnl.getExecutorService().execute(ft); + ft.run(); TestQueryEngine.assertSameSolutions(expected, sink.iterator()); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java 2010-09-05 18:16:01 UTC (rev 3509) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java 2010-09-05 20:17:44 UTC (rev 3510) @@ -28,7 +28,6 @@ package com.bigdata.bop.solutions; import java.util.ArrayList; -import java.util.Properties; import java.util.concurrent.ExecutionException; import java.util.concurrent.FutureTask; @@ -45,11 +44,9 @@ import com.bigdata.bop.NV; import com.bigdata.bop.Var; import com.bigdata.bop.engine.BOpStats; +import com.bigdata.bop.engine.MockRunningQuery; import com.bigdata.bop.engine.TestQueryEngine; -import com.bigdata.bop.solutions.SliceOp; -import com.bigdata.journal.BufferMode; import com.bigdata.journal.ITx; -import com.bigdata.journal.Journal; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; import com.bigdata.relation.accesspath.ThickAsynchronousIterator; @@ -75,25 +72,25 @@ super(name); } - @Override - public Properties getProperties() { +// @Override +// public Properties getProperties() { +// +// final Properties p = new Properties(super.getProperties()); +// +// p.setProperty(Journal.Options.BUFFER_MODE, BufferMode.Transient +// .toString()); +// +// return p; +// +// } +// +// Journal jnl = null; - final Properties p = new Properties(super.getProperties()); - - p.setProperty(Journal.Options.BUFFER_MODE, BufferMode.Transient - .toString()); - - return p; - - } - - Journal jnl = null; - ArrayList<IBindingSet> data; public void setUp() throws Exception { - jnl = new Journal(getProperties()); +// jnl = new Journal(getProperties()); setUpData(); @@ -150,10 +147,10 @@ public void tearDown() throws Exception { - if (jnl != null) { - jnl.destroy(); - jnl = null; - } +// if (jnl != null) { +// jnl.destroy(); +// jnl = null; +// } // clear reference. data = null; @@ -208,16 +205,17 @@ final IBlockingBuffer<IBindingSet[]> sink = query.newBuffer(); final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( - null/* fed */, jnl/* indexManager */, + new MockRunningQuery(null/* fed */, null/* indexManager */, ITx.READ_COMMITTED/* readTimestamp */, - ITx.UNISOLATED/* writeTimestamp */, -1/* partitionId */, stats, + ITx.UNISOLATED/* writeTimestamp */), -1/* partitionId */, stats, source, sink, null/* sink2 */); // get task. final FutureTask<Void> ft = query.eval(context); // execute task. - jnl.getExecutorService().execute(ft); +// jnl.getExecutorService().execute(ft); + ft.run(); TestQueryEngine.assertSameSolutions(expected, sink.iterator()); This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2010-09-06 20:45:47
|
Revision: 3511 http://bigdata.svn.sourceforge.net/bigdata/?rev=3511&view=rev Author: thompsonbry Date: 2010-09-06 20:45:37 +0000 (Mon, 06 Sep 2010) Log Message: ----------- Modified the IBindingSet API to define the contract for hashCode(). This was necessary in order for DISTINCT on binding sets to work correctly. Reorganized the QueryEngine classes out of the test suite. Added the basic framework for the FederatedQueryEngine test suite. Added a "de-chunk" pattern to unchunk iterators. Reconciled the BufferService with the ResourceService, including their test suites. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ArrayBindingSet.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/EmptyBindingSet.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/HashBindingSet.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/IBindingSet.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/IPipelineOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/rule/eval/AbstractStepTask.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/rule/eval/ProgramTask.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/resources/ResourceManager.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/resources/StoreManager.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/service/DataService.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/service/ResourceService.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/striterator/GenericStriterator.java branches/QUADS_QUERY_BRANCH/bigdata/src/resources/logging/log4j.properties branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestBindingSet.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestAll.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestAll.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/service/TestAll.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/striterator/TestAll.java Added Paths: ----------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BSBundle.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/HaltOpMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/PipelineUtility.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/Rule2BOpUtility.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/StartOpMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/DelegateIndexManager.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/MapBindingSetsOverShardsBuffer.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/service/ManagedResourceService.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/striterator/Dechunkerator.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/service/TestAll_DynamicSharding.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/service/TestAll_ResourceService.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/service/TestCase3.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/service/TestReceiveBuffer.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/service/TestReceiveFile.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/striterator/TestDechunkerator.java Removed Paths: ------------- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/BOpShard.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/BOpStats.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/BindingSetChunk.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/BufferService.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/FederatedQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/HaltOpMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/IQueryClient.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/IQueryPeer.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/IRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/ManagedBufferService.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/PipelineUtility.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/QueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/QueryEngine.properties branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/ReceiveBindingSets.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/Rule2BOpUtility.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/RunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/StartOpMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestCase3.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestReceiveBuffer.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestReceiveFile.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/MapBindingSetsOverShardsBuffer.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestSendReceiveBuffers.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/service/TestResourceService.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ArrayBindingSet.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ArrayBindingSet.java 2010-09-05 20:17:44 UTC (rev 3510) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ArrayBindingSet.java 2010-09-06 20:45:37 UTC (rev 3511) @@ -225,6 +225,9 @@ } + // clear the hash code. + hash = 0; + assert nbound == 0; } @@ -261,6 +264,9 @@ } + // clear the hash code. + hash = 0; + nbound--; break; @@ -316,7 +322,10 @@ if (vars[i] == var) { vals[i] = val; - + + // clear the hash code. + hash = 0; + return; } @@ -327,6 +336,9 @@ vals[nbound] = val; + // clear the hash code. + hash = 0; + nbound++; } @@ -408,11 +420,16 @@ } - public boolean equals(final IBindingSet o) { + public boolean equals(final Object t) { - if (o == this) + if (this == t) return true; + if(!(t instanceof IBindingSet)) + return false; + + final IBindingSet o = (IBindingSet)t; + if (nbound != o.size()) return false; @@ -430,4 +447,27 @@ } + public int hashCode() { + + if (hash == 0) { + + int result = 0; + + for (int i = 0; i < nbound; i++) { + + if (vals[i] == null) + continue; + + result ^= vals[i].hashCode(); + + } + + hash = result; + + } + return hash; + + } + private int hash; + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/EmptyBindingSet.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/EmptyBindingSet.java 2010-09-05 20:17:44 UTC (rev 3510) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/EmptyBindingSet.java 2010-09-06 20:45:37 UTC (rev 3511) @@ -96,18 +96,32 @@ return 0; } - public boolean equals(IBindingSet o) { + public boolean equals(final Object t) { - if (this == o) + if (this == t) return true; - + + if (!(t instanceof IBindingSet)) + return false; + + final IBindingSet o = (IBindingSet) t; + if (o.size() == 0) return true; - + return false; } + /** + * The hash code of an empty binding set is always zero. + */ + public int hashCode() { + + return 0; + + } + public IConstant get(IVariable var) { if (var == null) Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/HashBindingSet.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/HashBindingSet.java 2010-09-05 20:17:44 UTC (rev 3510) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/HashBindingSet.java 2010-09-06 20:45:37 UTC (rev 3511) @@ -146,6 +146,9 @@ throw new IllegalArgumentException(); map.put(var,val); + + // clear the hash code. + hash = 0; } @@ -156,12 +159,18 @@ map.remove(var); + // clear the hash code. + hash = 0; + } public void clearAll() { map.clear(); - + + // clear the hash code. + hash = 0; + } public String toString() { @@ -233,11 +242,14 @@ final HashBindingSet bs = new HashBindingSet(); - for (IVariable var : variablesToKeep) { + for (IVariable<?> var : variablesToKeep) { - IConstant val = map.get(var); + final IConstant<?> val = map.get(var); + if (val != null) { + bs.map.put(var, val); + } } @@ -246,11 +258,16 @@ } - public boolean equals(final IBindingSet o) { + public boolean equals(final Object t) { - if (o == this) + if (this == t) return true; + if(!(t instanceof IBindingSet)) + return false; + + final IBindingSet o = (IBindingSet) t; + if (size() != o.size()) return false; @@ -260,9 +277,9 @@ final Map.Entry<IVariable,IConstant> entry = itr.next(); - final IVariable var = entry.getKey(); + final IVariable<?> var = entry.getKey(); - final IConstant val = entry.getValue(); + final IConstant<?> val = entry.getValue(); // if (!o.isBound(vars[i])) // return false; @@ -276,4 +293,27 @@ } + public int hashCode() { + + if (hash == 0) { + + int result = 0; + + for(IConstant<?> c : map.values()) { + + if (c == null) + continue; + + result ^= c.hashCode(); + + } + + hash = result; + + } + return hash; + + } + private int hash; + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/IBindingSet.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/IBindingSet.java 2010-09-05 20:17:44 UTC (rev 3510) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/IBindingSet.java 2010-09-06 20:45:37 UTC (rev 3511) @@ -167,6 +167,20 @@ * @param o * Another binding set. */ - public boolean equals(IBindingSet o); + public boolean equals(Object o); + + /** + * The hash code of a binding is defined as the bit-wise XOR of the hash + * codes of the {@link IConstant}s for its bound variables. Unbound + * variables are ignored when computing the hash code. Binding sets are + * unordered collections, therefore the calculated hash code intentionally + * does not dependent on the order in which the bindings are iterated over. + * The hash code reflects the current state of the bindings and must be + * recomputed if the bindings are changed. + * + * @todo the test suites should be enhanced to verify the contract for + * {@link IBindingSet#hashCode()} + */ + public int hashCode(); } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/IPipelineOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/IPipelineOp.java 2010-09-05 20:17:44 UTC (rev 3510) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/IPipelineOp.java 2010-09-06 20:45:37 UTC (rev 3511) @@ -27,6 +27,7 @@ package com.bigdata.bop; +import java.util.concurrent.Callable; import java.util.concurrent.FutureTask; import com.bigdata.bop.engine.BOpStats; @@ -69,6 +70,11 @@ * * @return The {@link FutureTask} which will compute the operator's * evaluation. + * + * @todo Modify to return a {@link Callable}s for now since we must run each + * task in its own thread until Java7 gives us fork/join pools and + * asynchronous file I/O. For the fork/join model we will probably + * return the ForkJoinTask. */ FutureTask<Void> eval(BOpContext<E> context); Copied: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java (from rev 3508, branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/BOpStats.java) =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java 2010-09-06 20:45:37 UTC (rev 3511) @@ -0,0 +1,134 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Aug 26, 2010 + */ + +package com.bigdata.bop.engine; + +import java.io.Serializable; + +import com.bigdata.bop.BOp; +import com.bigdata.counters.CAT; + +/** + * Statistics associated with the evaluation of a {@link BOp}. These statistics + * are per {@link BOp}. The top-level {@link BOp} will reflect the throughput + * for the entire pipeline. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + * + * @todo Add time per bop. This can not be directly aggregated into wall time + * since there are concurrent processes. However, this will be useful + * since we tend to process materialized chunks with the new + * {@link QueryEngine} such that the operator evaluation time now more or + * less directly corresponds to the time it takes to act on local data, + * producing local outputs. The {@link QueryEngine} itself now handles the + * transportation of data between the nodes so that time can be factored + * out of the local aspects of query execution. + */ +public class BOpStats implements Serializable { + + /** + * + */ + private static final long serialVersionUID = 1L; + +// /** +// * The timestamp (milliseconds) associated with the start of execution for +// * the join dimension. This is not aggregated. It should only be used to +// * compute the elapsed time for the operator. +// */ +// private final long startTime; + +// /** +// * The index partition for which these statistics were collected or -1 +// * if the statistics are aggregated across index partitions. +// */ +// public final int partitionId; + + /** + * #of chunks in. + */ + final public CAT chunksIn = new CAT(); + + /** + * #of units sets in (tuples, elements, binding sets, etc). + */ + final public CAT unitsIn = new CAT(); + + /** + * #of chunks out. + */ + final public CAT chunksOut = new CAT(); + + /** + * #of units sets in (tuples, elements, binding sets, etc). + */ + final public CAT unitsOut = new CAT(); + + /** + * Constructor. + */ + public BOpStats() { + + } + + /** + * Combine the statistics (addition). + * + * @param o + * Another statistics object. + */ + public void add(final BOpStats o) { + chunksIn.add(o.chunksIn.get()); + unitsIn.add(o.unitsIn.get()); + unitsOut.add(o.unitsOut.get()); + chunksOut.add(o.chunksIn.get()); + } + + public String toString() { + final StringBuilder sb = new StringBuilder(); + sb.append(getClass().getName()); + sb.append("{chunksIn=" + chunksIn.estimate_get()); + sb.append(",unitsIn=" + unitsIn.estimate_get()); + sb.append(",chunksOut=" + chunksOut.estimate_get()); + sb.append(",unitsOut=" + unitsOut.estimate_get()); + toString(sb); + sb.append("}"); + return sb.toString(); + } + + /** + * Extension hook for {@link #toString()}. + * + * @param sb + * Where to write the additional state. + */ + protected void toString(final StringBuilder sb) { + + } + +} Copied: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BSBundle.java (from rev 3489, branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/BOpShard.java) =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BSBundle.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BSBundle.java 2010-09-06 20:45:37 UTC (rev 3511) @@ -0,0 +1,75 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Sep 1, 2010 + */ + +package com.bigdata.bop.engine; + +/** + * An immutable class capturing the evaluation context of an operator against a + * shard. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +public class BSBundle { + + public final int bopId; + + public final int shardId; + + public BSBundle(final int bopId, final int shardId) { + + this.bopId = bopId; + + this.shardId = shardId; + + } + + /** + * {@inheritDoc} + * + * @todo verify that this is a decent hash function. + */ + public int hashCode() { + + return (bopId * 31) + shardId; + + } + + public boolean equals(final Object o) { + + if (this == o) + return true; + + if (!(o instanceof BSBundle)) + return false; + + return bopId == ((BSBundle) o).bopId + && shardId == ((BSBundle) o).shardId; + + } + +} Copied: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java (from rev 3472, branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/BindingSetChunk.java) =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java 2010-09-06 20:45:37 UTC (rev 3511) @@ -0,0 +1,51 @@ +package com.bigdata.bop.engine; + +import com.bigdata.bop.BOp; +import com.bigdata.bop.IBindingSet; +import com.bigdata.relation.accesspath.IAsynchronousIterator; + +/** + * A chunk of intermediate results which are ready to be consumed by some + * {@link BOp} in a specific query. + */ +public class BindingSetChunk { + + /** + * The query identifier. + */ + final long queryId; + + /** + * The target {@link BOp}. + */ + final int bopId; + + /** + * The index partition which is being targeted for that {@link BOp}. + */ + final int partitionId; + + /** + * The binding sets to be consumed by that {@link BOp}. + */ + final IAsynchronousIterator<IBindingSet[]> source; + + public BindingSetChunk(final long queryId, final int bopId, + final int partitionId, + final IAsynchronousIterator<IBindingSet[]> source) { + if (source == null) + throw new IllegalArgumentException(); + this.queryId = queryId; + this.bopId = bopId; + this.partitionId = partitionId; + this.source = source; + } + + public String toString() { + + return getClass().getName() + "{queryId=" + queryId + ",bopId=" + bopId + + ",partitionId=" + partitionId + "}"; + + } + +} Copied: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/HaltOpMessage.java (from rev 3489, branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/HaltOpMessage.java) =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/HaltOpMessage.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/HaltOpMessage.java 2010-09-06 20:45:37 UTC (rev 3511) @@ -0,0 +1,135 @@ +package com.bigdata.bop.engine; + +import java.io.Serializable; +import java.util.UUID; + +/** + * A message sent to the {@link IQueryClient} when an operator is done executing + * for some chunk of inputs. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +public class HaltOpMessage implements Serializable { + + /** + * + */ + private static final long serialVersionUID = 1L; + + /** The identifier of the query. */ + final long queryId; + + /** The identifier of the operator. */ + final int bopId; + + /** + * The index partition identifier against which the operator was + * executing. + */ + final int partitionId; + + /** + * The identifier of the service on which the operator was executing. + */ + final UUID serviceId; + + /** + * * The cause and <code>null</code> if the operator halted normally. + */ + final Throwable cause; + + /** + * The operator identifier for the primary sink -or- <code>null</code> + * if there is no primary sink (for example, if this is the last + * operator in the pipeline). + */ + final Integer sinkId; + + /** + * The number of the {@link BindingSetChunk}s that were output for the + * primary sink. (This information is used for the atomic termination + * decision.) + * <p> + * For a given downstream operator this is ONE (1) for scale-up. For + * scale-out, this is one per index partition over which the + * intermediate results were mapped. + */ + final int sinkChunksOut; + + /** + * The operator identifier for the alternative sink -or- + * <code>null</code> if there is no alternative sink. + */ + final Integer altSinkId; + + /** + * The number of the {@link BindingSetChunk}s that were output for the + * alternative sink. (This information is used for the atomic + * termination decision.) + * <p> + * For a given downstream operator this is ONE (1) for scale-up. For + * scale-out, this is one per index partition over which the + * intermediate results were mapped. It is zero if there was no + * alternative sink for the operator. + */ + final int altSinkChunksOut; + + /** + * The statistics for the execution of the bop against the partition on + * the service. + */ + final BOpStats taskStats; + + /** + * @param queryId + * The query identifier. + * @param bopId + * The operator whose execution phase has terminated for a + * specific index partition and input chunk. + * @param partitionId + * The index partition against which the operator was + * executed. + * @param serviceId + * The node which executed the operator. + * @param cause + * <code>null</code> unless execution halted abnormally. + * @param chunksOut + * A map reporting the #of binding set chunks which were + * output for each downstream operator for which at least one + * chunk of output was produced. + * @param taskStats + * The statistics for the execution of that bop on that shard + * and service. + */ + public HaltOpMessage( + // + final long queryId, final int bopId, final int partitionId, + final UUID serviceId, Throwable cause, // + final Integer sinkId, final int sinkChunksOut,// + final Integer altSinkId, final int altSinkChunksOut,// + final BOpStats taskStats) { + + if (altSinkId != null && sinkId == null) { + // The primary sink must be defined if the altSink is defined. + throw new IllegalArgumentException(); + } + + if (sinkId != null && altSinkId != null + && sinkId.intValue() == altSinkId.intValue()) { + // The primary and alternative sink may not be the same operator. + throw new IllegalArgumentException(); + } + + this.queryId = queryId; + this.bopId = bopId; + this.partitionId = partitionId; + this.serviceId = serviceId; + this.cause = cause; + this.sinkId = sinkId; + this.sinkChunksOut = sinkChunksOut; + this.altSinkId = altSinkId; + this.altSinkChunksOut = altSinkChunksOut; + this.taskStats = taskStats; + } +} \ No newline at end of file Copied: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java (from rev 3489, branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/IQueryClient.java) =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java 2010-09-06 20:45:37 UTC (rev 3511) @@ -0,0 +1,105 @@ +package com.bigdata.bop.engine; + +import java.rmi.Remote; +import java.rmi.RemoteException; + +import com.bigdata.bop.BOp; + +/** + * Interface for a client executing queries. + */ +public interface IQueryClient extends IQueryPeer, Remote { + + /* + * @todo Could return a data structure which encapsulates the query results + * and could allow multiple results from a query, e.g., one per step in a + * program. + */ + +// /** +// * Evaluate a query which materializes elements, such as an +// * {@link IPredicate}. +// * +// * @param queryId +// * The unique identifier for the query. +// * @param timestamp +// * The timestamp or transaction against which the query will run. +// * @param query +// * The query to evaluate. +// * @param source +// * The initial binding sets to get the query going (this is +// * typically an iterator visiting a single empty binding set). +// * +// * @return An iterator visiting the elements materialized by the query. +// * +// * @throws Exception +// */ +// public IChunkedIterator<?> eval(long queryId, long timestamp, BOp query) +// throws Exception; + +// /** +// * Evaluate a query which visits {@link IBindingSet}s, such as a join. +// * +// * @param queryId +// * The unique identifier for the query. +// * @param timestamp +// * The timestamp or transaction against which the query will run. +// * @param query +// * The query to evaluate. +// * @param source +// * The initial binding sets to get the query going (this is +// * typically an iterator visiting a single empty binding set). +// * +// * @return An iterator visiting {@link IBindingSet}s which result from +// * evaluating the query. +// * +// * @throws Exception +// */ +// public IChunkedIterator<IBindingSet> eval(long queryId, long timestamp, +// BOp query, IAsynchronousIterator<IBindingSet[]> source) +// throws Exception; + + /** + * Return the query. + * + * @param queryId + * The query identifier. + * @return The query. + * + * @throws RemoteException + */ + public BOp getQuery(long queryId) throws RemoteException; + + /** + * Notify the client that execution has started for some query, operator, + * node, and index partition. + */ + public void startOp(StartOpMessage msg) + throws RemoteException; + + /** + * Notify the client that execution has halted for some query, operator, + * node, shard, and source binding set chunk(s). If execution halted + * abnormally, then the cause is sent as well. + */ + public void haltOp(HaltOpMessage msg) throws RemoteException; + +// /** +// * Notify the query controller that a chunk of intermediate results is +// * available for the query. +// * +// * @param queryId +// * The query identifier. +// */ +// public void addChunk(long queryId) throws RemoteException; +// +// /** +// * Notify the query controller that a chunk of intermediate results was +// * taken for processing by the query. +// * +// * @param queryId +// * The query identifier. +// */ +// public void takeChunk(long queryId) throws RemoteException; + +} Copied: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java (from rev 3466, branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/IQueryPeer.java) =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java 2010-09-06 20:45:37 UTC (rev 3511) @@ -0,0 +1,36 @@ +package com.bigdata.bop.engine; + +import java.net.InetSocketAddress; +import java.rmi.Remote; +import java.rmi.RemoteException; + +import com.bigdata.bop.BOp; + +/** + * Interface for a node participating in the exchange of NIO buffers to + * support query execution. + */ +public interface IQueryPeer extends Remote { + + /** + * Notify a service that a buffer having data for some {@link BOp} in some + * running query is available. The receiver may request the data when they + * are ready. If the query is cancelled, then the sender will drop the + * buffer. + * + * @param clientProxy + * proxy used to communicate with the client running the query. + * @param serviceAddr + * address which may be used to demand the data. + * @param queryId + * the unique query identifier. + * @param bopId + * the identifier for the target {@link BOp}. + * + * @return <code>true</code> unless the receiver knows that the query has + * already been cancelled. + */ + void bufferReady(IQueryClient clientProxy, InetSocketAddress serviceAddr, + long queryId, int bopId) throws RemoteException; + +} Copied: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java (from rev 3510, branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/IRunningQuery.java) =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java 2010-09-06 20:45:37 UTC (rev 3511) @@ -0,0 +1,75 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA + */ +/* + * Created on Sep 5, 2010 + */ + +package com.bigdata.bop.engine; + +import com.bigdata.btree.ILocalBTreeView; +import com.bigdata.journal.IIndexManager; +import com.bigdata.service.IBigdataFederation; + +/** + * Interface exposing a limited set of the state of an executing query. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +public interface IRunningQuery { + + /** + * The {@link IBigdataFederation} IFF the operator is being evaluated on an + * {@link IBigdataFederation}. When evaluating operations against an + * {@link IBigdataFederation}, this reference provides access to the + * scale-out view of the indices and to other bigdata services. + */ + IBigdataFederation<?> getFederation(); + + /** + * The <strong>local</strong> {@link IIndexManager}. Query evaluation occurs + * against the local indices. In scale-out, query evaluation proceeds shard + * wise and this {@link IIndexManager} MUST be able to read on the + * {@link ILocalBTreeView}. + */ + IIndexManager getIndexManager(); + + /** + * The timestamp or transaction identifier against which the query is + * reading. + */ + long getReadTimestamp(); + + /** + * The timestamp or transaction identifier against which the query is + * writing. + */ + long getWriteTimestamp(); + + /** + * Terminate query evaluation + */ + void halt(); + +} Copied: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/PipelineUtility.java (from rev 3489, branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/PipelineUtility.java) =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/PipelineUtility.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/PipelineUtility.java 2010-09-06 20:45:37 UTC (rev 3511) @@ -0,0 +1,156 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Sep 1, 2010 + */ + +package com.bigdata.bop.engine; + +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.log4j.Logger; + +import com.bigdata.bop.BOp; +import com.bigdata.bop.BOpUtility; +import com.bigdata.bop.NoSuchBOpException; + +/** + * Utility methods relevant to pipelined operator evaluation. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +public class PipelineUtility { + + private static final Logger log = Logger.getLogger(PipelineUtility.class); + + /** + * Return <code>true</code> iff the <i>runningCountMap</i> AND + * <i>availableChunkMap</i> map are ZERO (0) for both the given operator and + * for all operators which proceed the given operator in the tree structure + * of its operands. + * <p> + * Note: The movement of the intermediate binding set chunks forms an + * acyclic directed graph. We can decide whether or not a {@link BOp} in the + * query plan can be triggered by the current activity pattern by inspecting + * the {@link BOp} and its operands recursively. If neither the {@link BOp} + * nor any of its operands (recursively) has non-zero activity then the + * {@link BOp} can not be triggered and this method will return + * <code>true</code>. + * + * @param bopId + * The identifier for an operator which appears in the query + * plan. + * @param queryPlan + * The query plan. + * @param queryIndex + * An index for the query plan as constructed by + * {@link BOpUtility#getIndex(BOp)}. + * @param runningCountMap + * A map reporting the #of instances of each operator which are + * currently being evaluated (distinct evaluations are performed + * for each chunk and shard). + * @param availableChunkCountMap + * A map reporting the #of chunks available for each operator in + * the pipeline (we only report chunks for pipeline operators). + * + * @return <code>true</code> iff the {@link BOp} can not be triggered given + * the query plan and the activity map. + * + * @throws IllegalArgumentException + * if any argument is <code>null</code>. + * @throws NoSuchBOpException + * if <i>bopId</i> is not found in the query index. + */ + static public boolean isDone(final int bopId, final BOp queryPlan, + final Map<Integer, BOp> queryIndex, + final Map<Integer, AtomicLong> runningCountMap, + final Map<Integer, AtomicLong> availableChunkCountMap) { + + if (queryPlan == null) + throw new IllegalArgumentException(); + if (queryIndex == null) + throw new IllegalArgumentException(); + if (availableChunkCountMap == null) + throw new IllegalArgumentException(); + + final BOp op = queryIndex.get(bopId); + + if (op == null) + throw new NoSuchBOpException(bopId); + + final Iterator<BOp> itr = BOpUtility.preOrderIterator(op); + + while (itr.hasNext()) { + + final BOp t = itr.next(); + + final Integer id = (Integer) t.getProperty(BOp.Annotations.BOP_ID); + + if (id == null) + continue; + { + + final AtomicLong runningCount = runningCountMap.get(id); + + if (runningCount != null && runningCount.get() != 0) { + + if (log.isInfoEnabled()) + log.info("Operator can be triggered: op=" + op + + ", possible trigger=" + t + " is running."); + + return false; + + } + + } + + { + + final AtomicLong availableChunkCount = availableChunkCountMap + .get(id); + + if (availableChunkCount != null + && availableChunkCount.get() != 0) { + + if (log.isInfoEnabled()) + log.info("Operator can be triggered: op=" + op + + ", possible trigger=" + t + " has " + + availableChunkCount + " chunks available."); + + return false; + + } + + } + + } + + return true; + + } + +} Copied: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java (from rev 3510, branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/QueryEngine.java) =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java 2010-09-06 20:45:37 UTC (rev 3511) @@ -0,0 +1,800 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Aug 21, 2010 + */ + +package com.bigdata.bop.engine; + +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.rmi.RemoteException; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Future; +import java.util.concurrent.FutureTask; +import java.util.concurrent.PriorityBlockingQueue; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.log4j.Logger; + +import alice.tuprolog.Prolog; + +import com.bigdata.bop.BOp; +import com.bigdata.bop.BindingSetPipelineOp; +import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.IPredicate; +import com.bigdata.bop.bset.Union; +import com.bigdata.btree.BTree; +import com.bigdata.btree.IndexSegment; +import com.bigdata.btree.view.FusedView; +import com.bigdata.journal.IIndexManager; +import com.bigdata.journal.ITx; +import com.bigdata.journal.TimestampUtility; +import com.bigdata.rdf.internal.IV; +import com.bigdata.rdf.spo.SPORelation; +import com.bigdata.relation.IMutableRelation; +import com.bigdata.relation.IRelation; +import com.bigdata.relation.accesspath.IBlockingBuffer; +import com.bigdata.relation.accesspath.IElementFilter; +import com.bigdata.relation.rule.IRule; +import com.bigdata.relation.rule.Program; +import com.bigdata.relation.rule.eval.pipeline.DistributedJoinTask; +import com.bigdata.resources.IndexManager; +import com.bigdata.service.IBigdataFederation; +import com.bigdata.service.IDataService; +import com.bigdata.service.ndx.IAsynchronousWriteBufferFactory; +import com.bigdata.striterator.ChunkedArrayIterator; +import com.bigdata.striterator.IChunkedOrderedIterator; + +/** + * A class managing execution of concurrent queries against a local + * {@link IIndexManager}. + * <p> + * <h2>Design notes</h2> + * <p> + * Much of the complexity of the current approach owes itself to having to run a + * separate task for each join for each shard in order to have the appropriate + * lock when running against the unisolated shard view. This also means that the + * join task is running inside of the concurrency manager and hence has the + * local view of the shard. + * <p> + * The main, and perhaps the only, reason why we run unisolated rules is during + * closure, when we query against the unisolated indices and then write the + * entailments back on the unisolated indices. + * <p> + * Supporting closure has always been complicated. This complexity is mostly + * handled by ProgramTask#executeMutation() and + * AbstractTripleStore#newJoinNexusFactory() which play games with the + * timestamps used to read and write on the database, with commit points + * designed to create visibility for tuples written by a mutation rule, and with + * the automated advance of the read timestamp for the query in each closure + * pass in order to make newly committed tuples visible to subsequent rounds of + * closure. For scale-out, we do shard-wise auto commits so we always have a + * commit point which makes each write visible and the read timestamp is + * actually a read-only transaction which prevents the historical data we need + * during a closure round from being released as we are driving updates onto the + * federation. For the RWStore, we are having a similar problem (in the HA + * branch since that is where we are working on the RWStore) where historically + * allocated records were being released as writes drove updates on the indices. + * Again, we "solved" the problem for the RWStore using a commit point followed + * by a read-only transaction reading on that commit point to hold onto the view + * on which the next closure round needs to read (this uncovered a problem with + * the RWStore and transaction service interaction which Martyn is currently + * working to resolve through a combination of shadow allocators and deferred + * deletes which are processed once the release time is advanced by the + * transaction service). + * <p> + * The WORM does not have some of these problems with closure because we never + * delete history, so we do not need to create a commit point and a read-behind + * transaction. However, the WORM would have problems with concurrent access to + * the unisolated indices except that we hack that problem through the + * transparent use of the UnisolatedReadWriteIndex, which allows multiple + * threads to access the same unisolated index view using a read/write lock + * pattern (concurrent readers are allowed, but there is only one writer and it + * has exclusive access when it is running). This works out because we never run + * closure operations against the WORM through the concurrency manager. If we + * did, we would have to create a commit point after each mutation and use a + * read-behind transaction to prevent concurrent access to the unisolated index. + * <p> + * The main advantage that I can see of the current complexity is that it allows + * us to do load+closure as a single operation on the WORM, resulting in a + * single commit point. This makes that operation ACID without having to use + * full read/write transactions. This is how we gain the ACID contract for the + * standalone Journal in the SAIL for the WORM. Of course, the SAIL does not + * have that contract for the RWStore because we have to do the commit and + * read-behind transaction in order to have visibility and avoid concurrent + * access to the unisolated index (by reading behind on the last commit point). + * <p> + * I think that the reality is even one step more complicated. When doing truth + * maintenance (incremental closure), we bring the temporary graph to a fixed + * point (the rules write on the temp store) and then apply the delta in a + * single write to the database. That suggests that incremental truth + * maintenance would continue to be ACID, but that database-at-once-closure + * would be round-wise ACID. + * <p> + * So, I would like to suggest that we break ACID for database-at-once-closure + * and always follow the pattern of (1) do a commit before each round of + * closure; and (2) create a read-behind transaction to prevent the release of + * that commit point as we drive writes onto the indices. If we follow this + * pattern then we can write on the unisolated indices without conflict and read + * on the historical views without conflict. Since there will be a commit point + * before each mutation rule runs (which corresponds to a closure round), + * database-at-once-closure will be atomic within a round, but will not be a + * single atomic operation. Per above, I think that we would retain the ACID + * property for incremental truth maintenance against a WORM or RW mode Journal. + * + * <p> + * ---- + * </p> + * + * The advantage of this proposal (commit before each mutation rule and run + * query against a read-behind transaction) is that this could enormously + * simplify how we execute joins. + * <p> + * Right now, we use a factory pattern to create a join task on each node for + * each shard for which that node receives binding sets for a query. The main + * reason for doing this is to gain the appropriate lock for the unisolated + * index. If we never run a query against the unisolated index then we can go + * around the concurrency manager and run a single "query manager" task for all + * joins for all shards for all queries. This has some great benefits which I + * will go into below. + * <p> + * That "query manager" task would be responsible for accepting buffers + * containing elements or binding sets from other nodes and scheduling + * consumption of those data based on various criteria (order of arrival, + * priority, buffer resource requirements, timeout, etc.). This manager task + * could use a fork join pool to execute light weight operations (NIO, + * formulation of access paths from binding sets, mapping of binding sets onto + * shards, joining a chunk already read from an access path against a binding + * set, etc). Operations which touch the disk need to run in their own thread + * (until we get Java 7 async file IO, which is already available in a preview + * library). We could handle that by queuing those operations against a fixed + * size thread pool for reads. + * <p> + * This is a radical change in how we handle distributed query execution, but I + * think that it could have a huge payoff by reducing the complexity of the join + * logic, making it significantly easier to execute different kinds of join + * operations, reducing the overhead for acquiring locks for the unisolated + * index views, reducing the #of threads consumed by joins (from one per shard + * per join per query to a fixed pool of N threads for reads), etc. It would + * centralize the management of resources on each node and make it possible for + * us to handle things like join termination by simply purging data from the + * query manager task for the terminated join. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + * + * + * FIXME Unit tests for non-distinct {@link IElementFilter}s on an + * {@link IPredicate}, unit tests for distinct element filter on an + * {@link IPredicate} which is capable of distributed operations. Do not use + * distinct where not required (SPOC, only one graph, etc). + * <p> + * It seems like the right way to approach this is by unifying the stackable CTC + * striterator pattern with the chunked iterator pattern and passing the query + * engine (or the bop context) into the iterator construction process (or simply + * requesting that the query engine construct the iterator stack). + * <p> + * In terms of harmonization, it is difficult to say which way would work + * better. In the short term we could simply allow both and mask the differences + * in how we construct the filters, but the conversion to/from striterators and + * chunked iterators seems to waste a bit of effort. + * <p> + * The trickiest part of all of this is to allow a distributed filter pattern + * where the filter gets created on a set of nodes identified by the operator + * and the elements move among those nodes using the query engine's buffers. + * <p> + * To actually implement the distributed distinct filter we need to stack the + * following: + * + * <pre> + * - ITupleIterator + * - Resolve ITuple to Element (e.g., SPOC). + * - Layer on optional IElementFilter associated with the IPredicate. + * - Layer on SameVariableConstraint iff required (done by AccessPath) + * - Resolve SPO to SPO, stripping off the context position. + * - Chunk SPOs (SPO[], IKeyOrder), where the key order is from the access path. + * - Filter SPO[] using DHT constructed on specified nodes of the cluster. + * The SPO[] chunks should be packaged into NIO buffers and shipped to those + * nodes. The results should be shipped back as a bit vectors packaged into + * a NIO buffers. + * - Dechunk SPO[] to SPO since that is the current expectation for the filter + * stack. + * - The result then gets wrapped as a {@link IChunkedOrderedIterator} by + * the AccessPath using a {@link ChunkedArrayIterator}. + * </pre> + * + * This stack is a bit complex(!). But it is certainly easy enough to generate + * the necessary bits programmatically. + * + * FIXME Handling the {@link Union} of binding sets. Consider whether the chunk + * combiner logic from the {@link DistributedJoinTask} could be reused. + * + * FIXME INSERT and DELETE which will construct elements using + * {@link IRelation#newElement(java.util.List, IBindingSet)} from a binding set + * and then use {@link IMutableRelation#insert(IChunkedOrderedIterator)} and + * {@link IMutableRelation#delete(IChunkedOrderedIterator)}. For s/o, we first + * need to move the bits into the right places so it makes sense to unpack the + * processing of the loop over the elements and move the data around, writing on + * each index as necessary. There could be eventually consistent approaches to + * this as well. For justifications we need to update some additional indices, + * in which case we are stuck going through {@link IRelation} rather than + * routing data directly or using the {@link IAsynchronousWriteBufferFactory}. + * For example, we could handle routing and writing in s/o as follows: + * + * <pre> + * INSERT(relation,bindingSets) + * + * expands to + * + * SEQUENCE( + * SELECT(s,p,o), // drop bindings that we do not need + * PARALLEL( + * INSERT_INDEX(spo), // construct (s,p,o) elements and insert + * INSERT_INDEX(pos), // construct (p,o,s) elements and insert + * INSERT_INDEX(osp), // construct (o,s,p) elements and insert + * )) + * + * </pre> + * + * The output of the SELECT operator would be automatically mapped against the + * shards on which the next operators need to write. Since there is a nested + * PARALLEL operator, the mapping will be against the shards of each of the + * given indices. (A simpler operator would invoke + * {@link SPORelation#insert(IChunkedOrderedIterator)}. Handling justifications + * requires that we also formulate the justification chain from the pattern of + * variable bindings in the rule). + * + * FIXME Handle {@link Program}s. There are three flavors, which should probably + * be broken into three operators: sequence(ops), set(ops), and closure(op). The + * 'set' version would be parallelized, or at least have an annotation for + * parallel evaluation. These things belong in the same broad category as the + * join graph since they are operators which control the evaluation of other + * operators (the current pipeline join also has that characteristic which it + * uses to do the nested index subqueries). + * + * FIXME SPARQL to BOP translation + * <p> + * The initial pass should translate from {@link IRule} to {@link BOp}s so we + * can immediately begin running SPARQL queries against the {@link QueryEngine}. + * A second pass should explore a rules base translation from the openrdf SPARQL + * operator tree into {@link BOp}s, perhaps using an embedded {@link Prolog} + * engine. What follows is a partial list of special considerations for that + * translation: + * <ul> + * <li>Distinct can be trivially enforced for default graph queries against the + * SPOC index.</li> + * <li>Local distinct should wait until there is more than one tuple from the + * index since a single tuple does not need to be made distinct using a hash + * map.</li> + * <li>Low volume distributed queries should use solution modifiers which + * evaluate on the query controller node rather than using distributed sort, + * distinct, slice, or aggregation operators.</li> + * <li></li> + * <li></li> + * <li></li> + * <li>High volume queries should use special operators (different + * implementations of joins, use an external merge sort, etc).</li> + * </ul> + * + * FIXME SPARQL Coverage: Add native support for all SPARQL operators. A lot of + * this can be picked up from Sesame. Some things, such as isIRI() can be done + * natively against the {@link IV}. Likewise, there is already a set of + * comparison methods for {@link IV}s which are inlined values. Add support for + * <ul> + * <li></li> + * <li></li> + * <li></li> + * <li></li> + * <li></li> + * <li></li> + * </ul> + * + * @todo Expander patterns will continue to exist until we handle the standalone + * backchainers in a different manner for scale-out so add support for + * those for now. + * + * @todo There is a going to be a relationship to recycling of intermediates + * (for individual {@link BOp}s or {@link BOp} tree fragments) and a + * distributed query cache which handles invalidation (for updates) and + * {@link BOp} aware reuse of result sets available in the cache. This + * sort of thing will have to be coordinated among the cache nodes. + */ +public class QueryEngine implements IQueryPeer, IQueryClient { + + pr... [truncated message content] |
From: <tho...@us...> - 2010-09-08 13:16:42
|
Revision: 3515 http://bigdata.svn.sourceforge.net/bigdata/?rev=3515&view=rev Author: thompsonbry Date: 2010-09-08 13:16:30 +0000 (Wed, 08 Sep 2010) Log Message: ----------- Modified the ResourceService to use InetSocketAddress and NicUtil for configuration purposes. This change also touched the StoreManager and the MoveTask. Added a BOpEvaluationContext which specifies whether an operator must be mapped across shards, nodes, evaluated on the query controller, or if it may be evaluated without sending its inputs anywhere. This is only for scale-out. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/IPipelineOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/eval/JoinGraph.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/MapBindingSetsOverShardsBuffer.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/mutation/InsertOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ndx/AbstractSampleIndex.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SortOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/resources/MoveTask.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/resources/StoreManager.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/service/ManagedResourceService.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/service/ResourceService.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/service/TestReceiveBuffer.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/service/TestReceiveFile.java Added Paths: ----------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpEvaluationContext.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -135,6 +135,14 @@ BOp clone(); /** + * Return the evaluation context for the operator. The default is + * {@link BOpEvaluationContext#ANY}. Operators which must be mapped against + * shards, mapped against nodes, or evaluated on the query controller must + * override this method. + */ + public BOpEvaluationContext getEvaluationContext(); + + /** * Interface declaring well known annotations. */ public interface Annotations { Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -299,6 +299,7 @@ } + @SuppressWarnings("unchecked") public <T> T getProperty(final String name) { return (T) annotations.get(name); @@ -307,6 +308,7 @@ public <T> T getRequiredProperty(final String name) { + @SuppressWarnings("unchecked") final T tmp = (T) annotations.get(name); if (tmp == null) @@ -336,4 +338,10 @@ } + public BOpEvaluationContext getEvaluationContext() { + + return BOpEvaluationContext.ANY; + + } + } Added: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpEvaluationContext.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpEvaluationContext.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpEvaluationContext.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -0,0 +1,48 @@ +package com.bigdata.bop; + +import com.bigdata.bop.bset.ConditionalRoutingOp; +import com.bigdata.bop.join.PipelineJoin; +import com.bigdata.bop.solutions.DistinctBindingSetOp; +import com.bigdata.bop.solutions.SliceOp; + +/** + * Type safe enumeration indicates where an operator may be evaluated. Operators + * fall into several distinct categories based on whether or not their inputs + * need to be made available on specific nodes ({@link #HASHED} or + * {@link #SHARDED}), whether they can be evaluated anywhere their inputs may + * exist ({@link #ANY}), or whether they must be evaluated at the query + * controller ({@link #CONTROLLER}). + * <p> + * Note: All operators are evaluated locally when running against a standalone + * database. + */ +public enum BOpEvaluationContext { + + /** + * The operator may be evaluated anywhere, including piecewise evaluation on + * any node of the cluster where its inputs are available. This is used for + * operators which do not need to concentrate or coordinate their inputs + * such as {@link ConditionalRoutingOp}. + */ + ANY, + /** + * The input to the operator must be mapped across nodes using a hash + * partitioning schema and the operator must be evaluated on each hash + * partition. This is used for operators such as + * {@link DistinctBindingSetOp}. + */ + HASHED, + /** + * The input to the operator must be mapped across the shards on which the + * operator must read or write and the operator must be evaluated shard wise + * on the services having access to each shard. For example, + * {@link PipelineJoin}. + */ + SHARDED, + /** + * The operator must be evaluated on the query controller. For example, + * {@link SliceOp} may not be evaluated piecewise. + */ + CONTROLLER; + +} \ No newline at end of file Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpEvaluationContext.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/IPipelineOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/IPipelineOp.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/IPipelineOp.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -50,10 +50,14 @@ * class depending on the operator). */ BOpStats newStats(); - + /** * Instantiate a buffer suitable as a sink for this operator. The buffer * will be provisioned based on the operator annotations. + * <p> + * Note: if the operation swallows binding sets from the pipeline (such as + * operators which write on the database) then the operator MAY return an + * immutable empty buffer. * * @return The buffer. */ Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -6,9 +6,9 @@ import com.bigdata.bop.BOp; /** - * Interface for a client executing queries. + * Interface for a client executing queries (the query controller). */ -public interface IQueryClient extends IQueryPeer, Remote { +public interface IQueryClient extends IQueryPeer { /* * @todo Could return a data structure which encapsulates the query results Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -28,7 +28,6 @@ package com.bigdata.bop.engine; import java.net.InetSocketAddress; -import java.nio.ByteBuffer; import java.rmi.RemoteException; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; @@ -410,6 +409,11 @@ /** * The currently executing queries. + * + * @todo DEADLINE: There should be a data structure representing + * {@link RunningQuery} having deadlines so we can + * {@link RunningQuery#cancel(boolean)} queries when their deadline + * expires. */ final ConcurrentHashMap<Long/* queryId */, RunningQuery> runningQueries = new ConcurrentHashMap<Long, RunningQuery>(); @@ -480,20 +484,31 @@ * <p> * Handle priority for unselective queries based on the order in which * they are submitted? - * + * * @todo The approach taken by the {@link QueryEngine} executes one task per * pipeline bop per chunk. Outside of how the tasks are scheduled, * this corresponds closely to the historical pipeline query - * evaluation. The other difference is that there is less opportunity - * for concatenation of chunks. However, chunk concatenation could be - * performed here if we (a) mark the BindingSetChunk with a flag to - * indicate when it has been accepted; and (b) rip through the - * incoming chunks for the query for the target bop and combine them - * to feed the task. Chunks which have already been assigned would be - * dropped when take() discovers them above. [The chunk combination - * could also be done when we output the chunk if the sink has not - * been taken, e.g., by combining the chunk into the same target - * ByteBuffer, or when we add the chunk to the RunningQuery.] + * evaluation. + * <p> + * Chunk concatenation could be performed here if we (a) mark the + * {@link BindingSetChunk} with a flag to indicate when it has been + * accepted; and (b) rip through the incoming chunks for the query for + * the target bop and combine them to feed the task. Chunks which have + * already been assigned would be dropped when take() discovers them. + * [The chunk combination could also be done when we output the chunk + * if the sink has not been taken, e.g., by combining the chunk into + * the same target ByteBuffer, or when we add the chunk to the + * RunningQuery.] + * + * @todo SCALEOUT: High volume query operators must demand that their inputs + * are materialized before they can begin evaluation. Scaleout + * therefore requires a separate queue which looks at the metadata + * concerning chunks available on remote nodes for an operator which + * will run on this node and then demands the data either when the + * predecessors in the pipeline are done (operator at once evaluation) + * or when sufficient data are available to run the operator (mega + * chunk pipelining). Once the data are locally materialized, the + * operator may be queued for evaluation. */ private class QueryEngineTask implements Runnable { public void run() { @@ -778,17 +793,7 @@ } /** - * - * @todo if the top bop is an operation which writes on the database then it - * should swallow the binding sets from the pipeline and we should be - * able to pass along a <code>null</code> query buffer. - * - * @todo SCALEOUT: Return a proxy for the query buffer either here or when - * the query is sent along to another node for evaluation? - * <p> - * Actually, it would be nice if we could reuse the same NIO transfer - * of {@link ByteBuffer}s to move the final results back to the client - * rather than using a proxy object for the query buffer. + * Return a buffer onto which the solutions will be written. */ protected IBlockingBuffer<IBindingSet[]> newQueryBuffer( final BindingSetPipelineOp query) { Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -55,7 +55,6 @@ import com.bigdata.bop.IBindingSet; import com.bigdata.bop.NoSuchBOpException; import com.bigdata.bop.ap.Predicate; -import com.bigdata.bop.solutions.SliceOp; import com.bigdata.journal.IIndexManager; import com.bigdata.relation.accesspath.BlockingBuffer; import com.bigdata.relation.accesspath.IAsynchronousIterator; @@ -123,7 +122,11 @@ final private boolean controller; /** - * The client executing this query. + * The client executing this query (aka the query controller). + * <p> + * Note: The proxy is primarily for light weight RMI messages used to + * coordinate the distributed query evaluation. Ideally, all large objects + * will be transfered among the nodes of the cluster using NIO buffers. */ final private IQueryClient clientProxy; @@ -142,11 +145,11 @@ /** * An index from the {@link BOp.Annotations#BOP_ID} to the {@link BOp}. */ - private final Map<Integer, BOp> bopIndex; + protected final Map<Integer, BOp> bopIndex; /** - * A collection of the currently executing future for operators for this - * query. + * A collection of {@link Future}s for currently executing operators for + * this query. */ private final ConcurrentHashMap<BSBundle, Future<?>> operatorFutures = new ConcurrentHashMap<BSBundle, Future<?>>(); @@ -245,11 +248,6 @@ * Note: This is package private so it will be visible to the * {@link QueryEngine}. * - * @todo SCALEOUT: We need to model the chunks available before they are - * materialized locally such that (a) they can be materialized on - * demand (flow control); and (b) we can run the operator when there - * are sufficient chunks available without taking on too much data. - * * @todo It is likely that we can convert to the use of * {@link BlockingQueue} instead of {@link BlockingBuffer} in the * operators and then handle the logic for combining chunks inside of @@ -273,6 +271,19 @@ return queryEngine; } + + /** + * The client executing this query (aka the query controller). + * <p> + * Note: The proxy is primarily for light weight RMI messages used to + * coordinate the distributed query evaluation. Ideally, all large objects + * will be transfered among the nodes of the cluster using NIO buffers. + */ + public IQueryClient getQueryController() { + + return clientProxy; + + } /** * The unique identifier for this query. @@ -296,7 +307,7 @@ synchronized (queryRef) { - if (queryRef == null) { + if (queryRef.get() == null) { try { @@ -378,14 +389,16 @@ * rather than formatting it onto a {@link ByteBuffer}. * * @param sinkId + * The identifier of the target operator. * @param sink + * The intermediate results to be passed to that target operator. * * @return The #of chunks made available for consumption by the sink. This * will always be ONE (1) for scale-up. For scale-out, there will be * one chunk per index partition over which the intermediate results * were mapped. */ - protected int add(final int sinkId, + protected <E> int add(final int sinkId, final IBlockingBuffer<IBindingSet[]> sink) { /* @@ -394,23 +407,11 @@ final BindingSetChunk chunk = new BindingSetChunk(queryId, sinkId, -1/* partitionId */, sink.iterator()); - addChunkToQueryEngine(chunk); + queryEngine.add(chunk); return 1; } - - /** - * Adds a chunk to the local {@link QueryEngine}. - * - * @param chunk - * The chunk. - */ - protected void addChunkToQueryEngine(final BindingSetChunk chunk) { - - queryEngine.add(chunk); - - } /** * Make a chunk of binding sets available for consumption by the query. @@ -541,16 +542,6 @@ * * @throws UnsupportedOperationException * If this node is not the query coordinator. - * - * @todo Clone the {@link BOpStats} before reporting to avoid concurrent - * modification? - * - * @todo SCALEOUT: Do not release buffers backing the binding set chunks - * generated by an operator or the outputs of the final operator (the - * query results) until the sink has accepted those outputs. This - * means that we must not release the output buffers when the bop - * finishes but when its consumer finishes draining the {@link BOp}s - * outputs. */ public void haltOp(final HaltOpMessage msg) { if (!controller) @@ -768,11 +759,6 @@ final Runnable r = new Runnable() { public void run() { final UUID serviceId = queryEngine.getServiceId(); - /* - * @todo SCALEOUT: Combine chunks available on the queue for the - * current bop. This is not exactly "fan in" since multiple - * chunks could be available in scaleup as well. - */ int fanIn = 1; int sinkChunksOut = 0; int altSinkChunksOut = 0; @@ -822,17 +808,6 @@ * Return an iterator which will drain the solutions from the query. The * query will be cancelled if the iterator is * {@link ICloseableIterator#close() closed}. - * - * @return - * - * @todo Not all queries produce binding sets. For example, mutation - * operations. We could return the mutation count for mutation - * operators, which could be reported by {@link BOpStats} for that - * operator (unitsOut). - * - * @todo SCALEOUT: Track chunks consumed by the client so we do not release - * the backing {@link ByteBuffer} before the client is done draining - * the iterator. */ public IAsynchronousIterator<IBindingSet[]> iterator() { @@ -854,14 +829,17 @@ } /** - * @todo Cancelled queries must reject or drop new chunks, etc. - * <p> - * Queries must release all of their resources when they are done(). - * <p> - * Queries MUST NOT cause the solutions to be discarded before the - * client can consume them. This means that we have to carefully - * integrate {@link SliceOp} or just wrap the query buffer to impose - * the slice (simpler). + * {@inheritDoc} + * <p> + * Cancelled queries : + * <ul> + * <li>must reject new chunks</li> + * <li>must cancel any running operators</li> + * <li>must not begin to evaluate operators</li> + * <li>must release all of their resources</li> + * <li>must not cause the solutions to be discarded before the client can + * consume them.</li> + * </ul> */ final public boolean cancel(final boolean mayInterruptIfRunning) { // halt the query. Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/eval/JoinGraph.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/eval/JoinGraph.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/eval/JoinGraph.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -36,6 +36,7 @@ import com.bigdata.bop.BOp; import com.bigdata.bop.BOpContext; +import com.bigdata.bop.BOpEvaluationContext; import com.bigdata.bop.BindingSetPipelineOp; import com.bigdata.bop.IBindingSet; import com.bigdata.bop.IPredicate; @@ -262,4 +263,14 @@ } + /** + * This operator must be evaluated on the query controller. + */ + @Override + public BOpEvaluationContext getEvaluationContext() { + + return BOpEvaluationContext.CONTROLLER; + + } + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -38,7 +38,11 @@ import com.bigdata.bop.engine.QueryEngine; import com.bigdata.bop.engine.RunningQuery; import com.bigdata.bop.join.PipelineJoin; +import com.bigdata.bop.solutions.SliceOp; import com.bigdata.journal.IIndexManager; +import com.bigdata.relation.accesspath.IAsynchronousIterator; +import com.bigdata.relation.accesspath.IBlockingBuffer; +import com.bigdata.relation.accesspath.IBuffer; import com.bigdata.service.DataService; import com.bigdata.service.IBigdataFederation; import com.bigdata.service.ManagedResourceService; @@ -110,10 +114,10 @@ * join in a query plan. */ private final IBigdataFederation<?> fed; - + /** - * A service used to expose {@link ByteBuffer}s and managed index resources - * for transfer to remote services in support of distributed query + * The service used to expose {@link ByteBuffer}s and managed index + * resources for transfer to remote services in support of distributed query * evaluation. */ private final ManagedResourceService resourceService; @@ -176,6 +180,17 @@ } + /** + * The service used to expose {@link ByteBuffer}s and managed index + * resources for transfer to remote services in support of distributed query + * evaluation. + */ + public ManagedResourceService getResourceService() { + + return resourceService; + + } + @Override public void bufferReady(IQueryClient clientProxy, InetSocketAddress serviceAddr, long queryId, int bopId) { @@ -201,4 +216,34 @@ } + /** + * {@inheritDoc} + * + * @todo Historically, this has been a proxy object for an {@link IBuffer} + * on the {@link IQueryClient query controller}. However, it would be + * nice if we could reuse the same NIO transfer of {@link ByteBuffer}s + * to move the final results back to the client rather than using a + * proxy object for the query buffer. + * <p> + * In scale-out we must track chunks consumed by the client so we do + * not release the backing {@link ByteBuffer} on which the solutions + * are marshalled before the client is done draining the iterator. If + * the solutions are generated on the peers, then the peers must + * retain the data until the client has consumed them or have + * transferred the solutions to itself. + * <p> + * The places where this can show up as a problem are {@link SliceOp}, + * when a query deadline is reached, and when a query terminates + * normally. Also pay attention when the client closes the + * {@link IAsynchronousIterator} from which it is draining solutions + * early. + */ + @Override + protected IBlockingBuffer<IBindingSet[]> newQueryBuffer( + final BindingSetPipelineOp query) { + + return query.newBuffer(); + + } + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -31,11 +31,16 @@ import com.bigdata.bop.BOp; import com.bigdata.bop.IBindingSet; -import com.bigdata.bop.engine.BindingSetChunk; +import com.bigdata.bop.IPredicate; import com.bigdata.bop.engine.IQueryClient; import com.bigdata.bop.engine.RunningQuery; +import com.bigdata.mdi.PartitionLocator; +import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; +import com.bigdata.relation.accesspath.IBuffer; import com.bigdata.service.IBigdataFederation; +import com.bigdata.service.jini.master.IAsynchronousClientTask; +import com.bigdata.striterator.IKeyOrder; /** * Extends {@link RunningQuery} to provide additional state and logic required @@ -43,8 +48,14 @@ * . * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> - * @version $Id$ + * @version $Id: FederatedRunningQuery.java 3511 2010-09-06 20:45:37Z + * thompsonbry $ * + * @todo SCALEOUT: We need to model the chunks available before they are + * materialized locally such that (a) they can be materialized on demand + * (flow control); and (b) we can run the operator when there are + * sufficient chunks available without taking on too much data. + * * @todo SCALEOUT: Life cycle management of the operators and the query implies * both a per-query bop:NodeList map on the query coordinator identifying * the nodes on which the query has been executed and a per-query @@ -76,57 +87,137 @@ } /** - * Create a {@link BindingSetChunk} from a sink and add it to the queue. - * <p> - * Note: If we are running standalone, then we leave the data on the heap - * rather than formatting it onto a {@link ByteBuffer}. + * {@inheritDoc} * - * @param sinkId - * @param sink - * * @return The #of chunks made available for consumption by the sink. This * will always be ONE (1) for scale-up. For scale-out, there will be * one chunk per index partition over which the intermediate results * were mapped. * - * @todo <p> - * For selective queries in s/o, first format the data onto a list of - * byte[]s, one per target shard/node. Then, using a lock, obtain a - * ByteBuffer if there is none associated with the query yet. - * Otherwise, using the same lock, obtain a slice onto that ByteBuffer - * and put as much of the byte[] as will fit, continuing onto a newly - * recruited ByteBuffer if necessary. Release the lock and notify the - * target of the ByteBuffer slice (buffer#, off, len). Consider - * pushing the data proactively for selective queries. - * <p> - * For unselective queries in s/o, proceed as above but we need to get - * the data off the heap and onto the {@link ByteBuffer}s quickly - * (incrementally) and we want the consumers to impose flow control on - * the producers to bound the memory demand (this needs to be - * coordinated carefully to avoid deadlocks). Typically, large result - * sets should result in multiple passes over the consumer's shard - * rather than writing the intermediate results onto the disk. + * FIXME SCALEOUT: This is where we need to map the binding sets + * over the shards for the target operator. Once they are mapped, + * write the binding sets onto an NIO buffer for the target node and + * then send an RMI message to the node telling it that there is a + * chunk available for the given (queryId,bopId,partitionId). + * <p> + * For selective queries in s/o, first format the data onto a list + * of byte[]s, one per target shard/node. Then, using a lock, obtain + * a ByteBuffer if there is none associated with the query yet. + * Otherwise, using the same lock, obtain a slice onto that + * ByteBuffer and put as much of the byte[] as will fit, continuing + * onto a newly recruited ByteBuffer if necessary. Release the lock + * and notify the target of the ByteBuffer slice (buffer#, off, + * len). Consider pushing the data proactively for selective + * queries. + * <p> + * For unselective queries in s/o, proceed as above but we need to + * get the data off the heap and onto the {@link ByteBuffer}s + * quickly (incrementally) and we want the consumers to impose flow + * control on the producers to bound the memory demand (this needs + * to be coordinated carefully to avoid deadlocks). Typically, large + * result sets should result in multiple passes over the consumer's + * shard rather than writing the intermediate results onto the disk. * - * FIXME SCALEOUT: This is where we need to map the binding sets over - * the shards for the target operator. Once they are mapped, write the - * binding sets onto an NIO buffer for the target node and then send - * an RMI message to the node telling it that there is a chunk - * available for the given (queryId,bopId,partitionId). - */ + * */ @Override - protected int add(final int sinkId, + protected <E> int add(final int sinkId, final IBlockingBuffer<IBindingSet[]> sink) { - /* - * Note: The partitionId will always be -1 in scale-up. - */ - final BindingSetChunk chunk = new BindingSetChunk(getQueryId(), sinkId, - -1/* partitionId */, sink.iterator()); + if (sink == null) + throw new IllegalArgumentException(); - addChunkToQueryEngine(chunk); + final BOp bop = bopIndex.get(sinkId); - return 1; + if (bop == null) + throw new IllegalArgumentException(); + switch (bop.getEvaluationContext()) { + case ANY: + return super.add(sinkId, sink); + case HASHED: { + /* + * FIXME The sink self describes the nodes over which the + * binding sets will be mapped and the hash function to be applied + * so we look up those metadata and apply them to distributed the + * binding sets across the nodes. + */ + throw new UnsupportedOperationException(); + } + case SHARDED: { + /* + * FIXME The sink must read or write on a shard so we map the + * binding sets across the access path for the sink. + * + * @todo For a pipeline join, the predicate is the right hand + * operator of the sink. This might be true for INSERT and DELETE + * operators as well. + * + * @todo IKeyOrder tells us which index will be used and should be + * set on the predicate by the join optimizer. + * + * @todo Use the read or write timestamp depending on whether the + * operator performs mutation [this must be part of the operator + * metadata.] + * + * @todo Set the capacity of the the "map" buffer to the size of the + * data contained in the sink (in fact, we should just process the + * sink data in place). + */ + final IPredicate<E> pred = null; // @todo + final IKeyOrder<E> keyOrder = null; // @todo + final long timestamp = getReadTimestamp(); // @todo + final int capacity = 1000;// @todo + final MapBindingSetsOverShardsBuffer<IBindingSet, E> mapper = new MapBindingSetsOverShardsBuffer<IBindingSet, E>( + getFederation(), pred, keyOrder, timestamp, capacity) { + + @Override + IBuffer<IBindingSet> newBuffer(PartitionLocator locator) { + // TODO Auto-generated method stub + return null; + } + + }; + /* + * Map the binding sets over shards. + * + * FIXME The buffers created above need to become associated with + * this query as resources of the query. Once we are done mapping + * the binding sets over the shards, the target node for each buffer + * needs to be set an RMI message to let it know that there is a + * chunk available for it for the target operator. + */ + { + final IAsynchronousIterator<IBindingSet[]> itr = sink + .iterator(); + try { + while (itr.hasNext()) { + final IBindingSet[] chunk = itr.next(); + for (IBindingSet bset : chunk) { + mapper.add(bset); + } + } + } finally { + itr.close(); + sink.close(); + } + } + + throw new UnsupportedOperationException(); + } + case CONTROLLER: { + + final IQueryClient clientProxy = getQueryController(); + +// getQueryEngine().getResourceService().port; +// +// clientProxy.bufferReady(clientProxy, serviceAddr, getQueryId(), sinkId); + + throw new UnsupportedOperationException(); + } + default: + throw new AssertionError(bop.getEvaluationContext()); + } + } } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/MapBindingSetsOverShardsBuffer.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/MapBindingSetsOverShardsBuffer.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/MapBindingSetsOverShardsBuffer.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -84,9 +84,9 @@ extends AbstractUnsynchronizedArrayBuffer<E> { /** - * The predicate from which we generate the asBound predicates. This + * The predicate from which we generate the asBound binding sets. This * predicate and the {@link IKeyOrder} together determine the required - * access path. + * access path. */ private final IPredicate<F> pred; @@ -123,7 +123,12 @@ * @param fed * The federation. * @param pred - * The target predicate. + * The predicate associated with the target operator. The + * predicate identifies which variables and/or constants form the + * key for the access path and hence selects the shards on which + * the target operator must read or write. For example, when the + * target operator is a JOIN, this is the {@link IPredicate} + * associated with the right hand operator of the join. * @param keyOrder * Identifies the access path for the target predicate. * @param timestamp @@ -136,9 +141,10 @@ * The capacity of this buffer. */ public MapBindingSetsOverShardsBuffer( - final IBigdataFederation<?> fed, - final IPredicate<F> pred, final IKeyOrder<F> keyOrder, - final long timestamp, + final IBigdataFederation<?> fed,// + final IPredicate<F> pred, // + final IKeyOrder<F> keyOrder,// + final long timestamp,// final int capacity) { super(capacity); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -45,6 +45,7 @@ import com.bigdata.bop.BOp; import com.bigdata.bop.BOpContext; +import com.bigdata.bop.BOpEvaluationContext; import com.bigdata.bop.BindingSetPipelineOp; import com.bigdata.bop.IBindingSet; import com.bigdata.bop.IConstraint; @@ -1774,4 +1775,14 @@ }// class JoinTask + /** + * This is a shard wise operator. + */ + @Override + public BOpEvaluationContext getEvaluationContext() { + + return BOpEvaluationContext.SHARDED; + + } + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/mutation/InsertOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/mutation/InsertOp.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/mutation/InsertOp.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -35,8 +35,10 @@ import com.bigdata.bop.BOp; import com.bigdata.bop.BOpContext; +import com.bigdata.bop.BOpEvaluationContext; import com.bigdata.bop.BindingSetPipelineOp; import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.IPredicate; import com.bigdata.bop.IVariableOrConstant; import com.bigdata.bop.engine.BOpStats; import com.bigdata.btree.ILocalBTreeView; @@ -70,6 +72,9 @@ * An ordered {@link IVariableOrConstant}[]. Elements will be created * using the binding sets which flow through the operator and * {@link IRelation#newElement(java.util.List, IBindingSet)}. + * + * @todo This should be an {@link IPredicate} and should be the right + * hand operand just like for a JOIN. */ String SELECTED = InsertOp.class.getName() + ".selected"; @@ -267,5 +272,15 @@ // .getClass()); // // a[i] = e; - + + /** + * This is a shard wise operator. + */ + @Override + public BOpEvaluationContext getEvaluationContext() { + + return BOpEvaluationContext.SHARDED; + + } + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ndx/AbstractSampleIndex.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ndx/AbstractSampleIndex.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ndx/AbstractSampleIndex.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -28,6 +28,7 @@ package com.bigdata.bop.ndx; +import com.bigdata.bop.BOpEvaluationContext; import com.bigdata.bop.PipelineOp; import com.bigdata.bop.BOp; import com.bigdata.bop.IBindingSet; @@ -103,4 +104,14 @@ } + /** + * This is a shard wise operator. + */ + @Override + public BOpEvaluationContext getEvaluationContext() { + + return BOpEvaluationContext.SHARDED; + + } + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -33,6 +33,7 @@ import com.bigdata.bop.BOp; import com.bigdata.bop.BOpContext; +import com.bigdata.bop.BOpEvaluationContext; import com.bigdata.bop.BindingSetPipelineOp; import com.bigdata.bop.IBindingSet; import com.bigdata.bop.PipelineOp; @@ -272,4 +273,14 @@ } + /** + * This operator must be evaluated on the query controller. + */ + @Override + public BOpEvaluationContext getEvaluationContext() { + + return BOpEvaluationContext.CONTROLLER; + + } + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SortOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SortOp.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SortOp.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -30,6 +30,7 @@ import java.util.Map; import com.bigdata.bop.BOp; +import com.bigdata.bop.BOpEvaluationContext; import com.bigdata.bop.BindingSetPipelineOp; import com.bigdata.bop.IBindingSet; import com.bigdata.bop.PipelineOp; @@ -83,4 +84,16 @@ } + /** + * This operator must be evaluated on the query controller. + * + * @todo Define a distributed (external) merge sort operator. + */ + @Override + public BOpEvaluationContext getEvaluationContext() { + + return BOpEvaluationContext.CONTROLLER; + + } + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/resources/MoveTask.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/resources/MoveTask.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/resources/MoveTask.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -31,7 +31,7 @@ import java.io.File; import java.io.IOException; import java.io.Serializable; -import java.net.InetAddress; +import java.net.InetSocketAddress; import java.util.Map; import java.util.UUID; import java.util.concurrent.ExecutionException; @@ -63,7 +63,6 @@ import com.bigdata.service.IMetadataService; import com.bigdata.service.MetadataService; import com.bigdata.service.ResourceService; -import com.bigdata.util.config.NicUtil; /** * Task moves an index partition to another {@link IDataService}. @@ -433,7 +432,7 @@ private final UUID targetDataServiceUUID; private final int targetIndexPartitionId; private final Event parentEvent; - private final InetAddress thisInetAddr; +// private final InetAddress thisInetAddr; /** * @@ -481,11 +480,11 @@ this.targetIndexPartitionId = targetIndexPartitionId; this.parentEvent = parentEvent; - try { - this.thisInetAddr = InetAddress.getByName(NicUtil.getIpAddress("default.nic", "default", false)); - } catch(Throwable t) { - throw new IllegalArgumentException(t.getMessage(), t); - } +// try { +// this.thisInetAddr = InetAddress.getByName(NicUtil.getIpAddress("default.nic", "default", false)); +// } catch(Throwable t) { +// throw new IllegalArgumentException(t.getMessage(), t); +// } } /** @@ -596,9 +595,8 @@ targetIndexPartitionId,// historicalWritesBuildResult.segmentMetadata,// bufferedWritesBuildResult.segmentMetadata,// - thisInetAddr, resourceManager - .getResourceServicePort()// + .getResourceService().getAddr()// )).get(); } catch (ExecutionException ex) { @@ -913,8 +911,8 @@ final private int targetIndexPartitionId; final private SegmentMetadata historyIndexSegmentMetadata; final private SegmentMetadata bufferedWritesIndexSegmentMetadata; - final private InetAddress addr; - final private int port; + final private InetSocketAddress addr; +// final private int port; /** * @param sourceIndexMetadata @@ -927,14 +925,14 @@ * Describes the {@link IndexSegmentStore} containing the * historical data for the source index partition. * @param bufferedWritesIndexSegmentMetadata - * Desribes the {@link IndexSegmentStore} containing the + * Describes the {@link IndexSegmentStore} containing the * buffered writes from the live journal for the source index * partition. * @param addr - * The {@link InetAddress} of the source data service. - * @param port - * The port at which the source data service has exposed its - * {@link ResourceService} + * The {@link InetSocketAddress} of the + * {@link ResourceService} running on the source data service + * (the one from which the resources will be copied during + * the move). */ ReceiveIndexPartitionTask(// final IndexMetadata sourceIndexMetadata,// @@ -942,8 +940,7 @@ final int targetIndexPartitionId,// final SegmentMetadata historyIndexSegmentMetadata,// final SegmentMetadata bufferedWritesIndexSegmentMetadata,// - final InetAddress addr, - final int port + final InetSocketAddress addr ) { this.sourceIndexMetadata = sourceIndexMetadata; @@ -952,7 +949,6 @@ this.historyIndexSegmentMetadata = historyIndexSegmentMetadata; this.bufferedWritesIndexSegmentMetadata = bufferedWritesIndexSegmentMetadata; this.addr = addr; - this.port = port; } // private transient DataService dataService; @@ -1000,8 +996,7 @@ targetIndexPartitionId,// historyIndexSegmentMetadata,// bufferedWritesIndexSegmentMetadata,// - addr,// - port// + addr// )).get(); // update the index partition receive counter. @@ -1058,8 +1053,8 @@ final private SegmentMetadata sourceBufferedWritesSegmentMetadata; final private Event parentEvent; final private String summary; - final InetAddress addr; - final int port; + final InetSocketAddress addr; +// final int port; /** * @param resourceManager @@ -1080,10 +1075,9 @@ * buffered writes from the live journal for the source index * partition. * @param addr - * The {@link InetAddress} of the source data service. - * @param port - * The port at which the source data service has exposed its - * {@link ResourceService} + * The {@link InetSocketAddress} of the + * {@link ResourceService} of the source data service (the + * one from which the resources will be copied). */ InnerReceiveIndexPartitionTask(final ResourceManager resourceManager, final String targetIndexName, @@ -1092,8 +1086,7 @@ final int targetIndexPartitionId, final SegmentMetadata historyIndexSegmentMetadata, final SegmentMetadata bufferedWritesIndexSegmentMetadata, - final InetAddress addr, - final int port + final InetSocketAddress addr ) { super(resourceManager.getConcurrencyManager(), ITx.UNISOLATED, @@ -1124,7 +1117,6 @@ this.sourceHistorySegmentMetadata = historyIndexSegmentMetadata; this.sourceBufferedWritesSegmentMetadata = bufferedWritesIndexSegmentMetadata; this.addr = addr; - this.port = port; this.summary = OverflowActionEnum.Move + "(" + sourceIndexName + "->" + targetIndexName + ")"; @@ -1267,7 +1259,7 @@ try { // read the resource, writing onto that file. - new ResourceService.ReadResourceTask(addr, port, + new ResourceService.ReadResourceTask(addr, sourceSegmentMetadata.getUUID(), file).call(); } catch (Throwable t) { Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/resources/StoreManager.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/resources/StoreManager.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/resources/StoreManager.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -104,7 +104,6 @@ import com.bigdata.service.IBigdataFederation; import com.bigdata.service.ManagedResourceService; import com.bigdata.service.MetadataService; -import com.bigdata.service.ResourceService; import com.bigdata.sparse.SparseRowStore; import com.bigdata.util.concurrent.DaemonThreadFactory; import com.ibm.icu.impl.ByteBuffer; @@ -657,24 +656,24 @@ } - /** - * The port at which you can connect to the {@link ResourceService}. This - * service provides remote access to resources hosted by the owning - * {@link DataService}. This is used for moving resources to other data - * services in the federation, including supporting service failover. - * - * @return The port used to connect to that service. - * - * @todo this could also be used for remote backup. however, note that you - * can not read the live journal using this object. - */ - public int getResourceServicePort() { - - assertRunning(); - - return resourceService.port; - - } +// /** +// * The port at which you can connect to the {@link ResourceService}. This +// * service provides remote access to resources hosted by the owning +// * {@link DataService}. This is used for moving resources to other data +// * services in the federation, including supporting service failover. +// * +// * @return The port used to connect to that service. +// * +// * @todo this could also be used for remote backup. however, note that you +// * can not read the live journal using this object. +// */ +// public int getResourceServicePort() { +// +// assertRunning(); +// +// return resourceService.port; +// +// } /** * @see Options#IGNORE_BAD_FILES @@ -1569,7 +1568,7 @@ resourceService = new ManagedResourceService() { @Override - protected File getResource(UUID uuid) throws Exception { + protected File getResource(final UUID uuid) throws Exception { if (!isRunning()) { Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/service/ManagedResourceService.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/service/ManagedResourceService.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/service/ManagedResourceService.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -64,12 +64,17 @@ private final ConcurrentHashMap<UUID, ByteBuffer> buffers = new ConcurrentHashMap<UUID, ByteBuffer>(); /** + * Constructor uses the default nic, any free port, and the default request + * service pool size. + * * @throws IOException */ public ManagedResourceService() throws IOException { + super(); } /** + * * @param port * @throws IOException */ @@ -82,8 +87,8 @@ * @param requestServicePoolSize * @throws IOException */ - public ManagedResourceService(int port, int requestServicePoolSize) - throws IOException { + public ManagedResourceService(final int port, + final int requestServicePoolSize) throws IOException { super(port, requestServicePoolSize); } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/service/ResourceService.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/service/ResourceService.java 2010-09-07 20:14:45 UTC (rev 3514) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/service/ResourceService.java 2010-09-08 13:16:30 UTC (rev 3515) @@ -39,6 +39,7 @@ import java.io.InputStream; import java.io.OutputStream; import java.net.InetAddress; +import java.net.InetSocketAddress; import java.net.ServerSocket; import java.net.Socket; import java.nio.ByteBuffer; @@ -70,11 +71,12 @@ import com.bigdata.rawstore.Bytes; import com.bigdata.util.concurrent.DaemonThreadFactory; import com.bigdata.util.concurrent.ShutdownHelper; +import com.bigdata.util.config.NicUtil; /** - * A class which permits buffers identified by a {@link UUID} to be read by a - * remote service. This class runs one thread to accept connections and thread - * pool to send data. + * A service which permits resources (managed files or buffers) identified by a + * {@link UUID} to be read by a remote service. This class runs one thread to + * accept connections and thread pool to send data. * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id$ @@ -92,6 +94,12 @@ * overhead with establishing those connections and problems with * immediate reconnect under heavy load. * + * @todo Should have start() method to defer initialization of thread pools and + * the {@link ServerSocket} until outside of the constructor. Either + * handle visibility appropriately for those fields or move them and the + * shutdown protocol into an inner class which is only initialized when + * the service is running. + * * @todo Verify that multiple transfers may proceed in parallel to the from the * same source to the same receiver. This is important in order for low * latency queries to remain lively when we are moving a shard from one @@ -135,9 +143,15 @@ protected static final Logger log = Logger.getLogger(ResourceService.class); /** - * The port on which the service is accepting connections. + * The Internet socket address at which the service is accepting + * connections. */ - public final int port; + private final InetSocketAddress addr; + +// /** +// * The port on which the service is accepting connections. +// */ +// public final int port; /** * The server socket. @@ -150,163 +164,22 @@ private volatile boolean open = false; /** - * Performance counters for the {@link BufferService}. - * - * @todo could also monitor the accept and request thread pools. The latter - * is the more interesting from a workload perspective. + * Performance counters for this service. */ - static public class Counters { + public final Counters counters = new Counters(); - /** - * #of requests. - */ - public final CAT requestCount = new CAT(); - - /** - * #of requests which are denied. - */ - public final CAT denyCount = new CAT(); - - /** - * #of requests for which the resource was not found. - */ - public final CAT notFoundCount = new CAT(); - - /** - * #of requests which end in an internal error. - */ - public final CAT internalErrorCount = new CAT(); - - /** - * #of errors for responses where we attempt to write the requested data - * on the socket. - */ - public final CAT writeErrorCount = new CAT(); - - /** - * #of responses where we attempt to write the data on the socket. - */ - public final CAT nwrites = new CAT(); - - /** - * #of data bytes sent. - */ - public final CAT bytesWritten = new CAT(); - - /** - * The largest response written so far. - */ - public long maxWriteSize; - - /** - * A lock used to make updates to {@link #maxWriteSize} atomic. - */ - final private Object maxWriteSizeLock = new Object(); - - /** - * #of nanoseconds sending data (this will double count time for data - * that are served concurrently to different receivers). - */ - public final CAT elapsedWriteNanos = new CAT(); - - synchronized public CounterSet getCounters() { - - if (root == null) { - - root = new CounterSet(); - - /* - * #of requests and their status outcome counters. - */ - { - final CounterSet tmp = root.makePath("status"); - - tmp.addCounter("Request Count", new Instrument<Long>() { - public void sample() { - setValue(requestCount.get()); - } - }); - - tmp.addCounter("Deny Count", new Instrument<Long>() { - public void sample() { - setValue(denyCount.get()); - } - }); - - tmp.addCounter("Not Found Count", new Instrument<Long>() { - public void sample() { - setValue(notFoundCount.get()); - } - }); - - tmp.addCounter("Internal Error Count", new Instrument<Long>() { - public void sample() { - setValue(internalErrorCount.get()); - } - }); - - } - - /* - * writes (A write is a response where we try to write the file - * on the socket). - */ - { - - final CounterSet tmp = root.makePath("writes"); - tmp.addCounter("nwrites", new Instrument<Long>() { - public void sample() { - setValue(nwrites.get()); - } - }); - - tmp.addCounter("bytesWritten", new Instrument<Long>() { - public void sample() { - ... [truncated message content] |
From: <tho...@us...> - 2010-09-09 17:17:30
|
Revision: 3526 http://bigdata.svn.sourceforge.net/bigdata/?rev=3526&view=rev Author: thompsonbry Date: 2010-09-09 17:17:21 +0000 (Thu, 09 Sep 2010) Log Message: ----------- Working through support for moving bindingSet chunks around in scale-out and life cycle management of buffers in scale-out. I've raised the read/write timestamp into operator annotations. This might turn into a single "timestamp" operator and a BOp#isMutationOp() method to mark operators which write data rather than reading data. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BSBundle.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/StartOpMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/service/ResourceService.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java Added Paths: ----------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IChunkMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryDecl.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryDecl.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/AllocationContextKey.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/QueryContext.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ServiceContext.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ShardContext.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ThickChunkMessage.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java 2010-09-09 13:50:53 UTC (rev 3525) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java 2010-09-09 17:17:21 UTC (rev 3526) @@ -147,13 +147,6 @@ */ public interface Annotations { -// /** -// * A cross reference to the query identifier. This is required on -// * operators which associate distributed state with a query. [We can -// * probably get this from the evaluation context.] -// */ -// String QUERY_REF = "queryRef"; - /** * The unique identifier within a query for a specific {@link BOp}. The * {@link #QUERY_ID} and the {@link #BOP_ID} together provide a unique @@ -161,11 +154,19 @@ * query. */ String BOP_ID = "bopId"; - + /** * The timeout for the operator evaluation (milliseconds). * * @see #DEFAULT_TIMEOUT + * + * @todo Probably support both deadlines and timeouts. A deadline + * expresses when the query must be done while a timeout expresses + * how long it may run. A deadline may be imposed as soon as the + * query plan is formulated and could even be communicated from a + * remote client (e.g., as an httpd header). A timeout will always + * be interpreted with respect to the time when the query began to + * execute. */ String TIMEOUT = "timeout"; @@ -175,12 +176,30 @@ long DEFAULT_TIMEOUT = Long.MAX_VALUE; /** + * The timestamp (or transaction identifier) associated with a read from + * the database. + * + * @todo Combine the read and write timestamps as a single + * <code>TX</code> value and require this on any operator which + * reads or writes on the database. + */ + String READ_TIMESTAMP = BOp.class.getName() + ".readTimestamp"; + + /** + * The timestamp (or transaction identifier) associated with a write on + * the database. + */ + String WRITE_TIMESTAMP = BOp.class.getName() + ".writeTimestamp"; + + /** * For hash partitioned operators, this is the set of the member nodes * for the operator. * <p> * This annotation is required for such operators since the set of known * nodes of a given type (such as all data services) can otherwise * change at runtime. + * + * @todo Move onto an interface parallel to {@link IShardwisePipelineOp} */ String MEMBER_SERVICES = "memberServices"; Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java 2010-09-09 13:50:53 UTC (rev 3525) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java 2010-09-09 17:17:21 UTC (rev 3526) @@ -30,6 +30,7 @@ import org.apache.log4j.Logger; import com.bigdata.bop.engine.BOpStats; +import com.bigdata.bop.engine.IChunkMessage; import com.bigdata.bop.engine.IRunningQuery; import com.bigdata.bop.engine.QueryEngine; import com.bigdata.bop.engine.RunningQuery; @@ -227,6 +228,10 @@ * source if the source will be ignored). * @throws IllegalArgumentException * if the <i>sink</i> is <code>null</code> + * + * @todo modify to accept {@link IChunkMessage} or an interface available + * from getChunk() on {@link IChunkMessage} which provides us with + * flexible mechanisms for accessing the chunk data. */ // * @throws IllegalArgumentException // * if the <i>indexManager</i> is <code>null</code> Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BSBundle.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BSBundle.java 2010-09-09 13:50:53 UTC (rev 3525) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BSBundle.java 2010-09-09 17:17:21 UTC (rev 3526) @@ -50,8 +50,6 @@ /** * {@inheritDoc} - * - * @todo verify that this is a decent hash function. */ public int hashCode() { Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java 2010-09-09 13:50:53 UTC (rev 3525) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java 2010-09-09 17:17:21 UTC (rev 3526) @@ -1,44 +1,87 @@ package com.bigdata.bop.engine; +import java.io.Serializable; + import com.bigdata.bop.BOp; import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.fed.FederatedRunningQuery; import com.bigdata.relation.accesspath.IAsynchronousIterator; /** - * A chunk of intermediate results which are ready to be consumed by some - * {@link BOp} in a specific query. + * An non-{@link Serializable} chunk of intermediate results which are ready to + * be consumed by some {@link BOp} in a specific query (this is only used in + * query evaluation for the standalone database). */ -public class BindingSetChunk { +public class BindingSetChunk implements IChunkMessage { + /** The query controller. */ + private final IQueryClient clientProxy; + /** * The query identifier. */ - final long queryId; + private final long queryId; /** * The target {@link BOp}. */ - final int bopId; + private final int bopId; /** * The index partition which is being targeted for that {@link BOp}. */ - final int partitionId; + private final int partitionId; /** * The binding sets to be consumed by that {@link BOp}. */ - final IAsynchronousIterator<IBindingSet[]> source; + private IAsynchronousIterator<IBindingSet[]> source; - public BindingSetChunk(final long queryId, final int bopId, - final int partitionId, + public IQueryClient getQueryController() { + return clientProxy; + } + + public long getQueryId() { + return queryId; + } + + public int getBOpId() { + return bopId; + } + + public int getPartitionId() { + return partitionId; + } + + public boolean isMaterialized() { + return true; + } + + /** + * + * @todo constructor to accept the BlockingBuffer instead as part of + * {@link IChunkMessage} harmonization (or an "IChunk" API). + */ + public BindingSetChunk(final IQueryClient clientProxy, final long queryId, + final int bopId, final int partitionId, final IAsynchronousIterator<IBindingSet[]> source) { + + if (clientProxy == null) + throw new IllegalArgumentException(); + if (source == null) throw new IllegalArgumentException(); + + this.clientProxy = clientProxy; + this.queryId = queryId; + this.bopId = bopId; + this.partitionId = partitionId; + this.source = source; + } public String toString() { @@ -47,5 +90,13 @@ + ",partitionId=" + partitionId + "}"; } + + public void materialize(FederatedRunningQuery runningQuery) { + // NOP + } + public IAsynchronousIterator<IBindingSet[]> iterator() { + return source; + } + } Added: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IChunkMessage.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IChunkMessage.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IChunkMessage.java 2010-09-09 17:17:21 UTC (rev 3526) @@ -0,0 +1,93 @@ +package com.bigdata.bop.engine; + +import java.nio.ByteBuffer; +import java.util.concurrent.BlockingQueue; + +import com.bigdata.bop.BOp; +import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.fed.FederatedRunningQuery; +import com.bigdata.relation.accesspath.BlockingBuffer; +import com.bigdata.relation.accesspath.IAsynchronousIterator; +import com.bigdata.service.ResourceService; + +/** + * A message describing a chunk of intermediate results which are available for + * processing. There are several implementations of this interface supporting + * same-JVM messages, thick RMI messages, and RMI messages where the payload is + * materialized using NIO transfers from the {@link ResourceService}. + */ +public interface IChunkMessage { + + /** The proxy for the query controller. */ + IQueryClient getQueryController(); + + /** The query identifier. */ + long getQueryId(); + + /** The identifier for the target {@link BOp}. */ + int getBOpId(); + + /** The identifier for the target index partition. */ + int getPartitionId(); + + /* + * @todo Report the #of bytes available with this message. However, first + * figure out if that if the #of bytes in this {@link OutputChunk} or across + * all {@link OutputChunk}s available for the target service and sink. + */ + // @todo move to concrete subclass or allow ZERO if data are in memory (no RMI). +// /** The #of bytes of data which are available for that operator. */ +// int getBytesAvailable(); + + /** + * Return <code>true</code> if the chunk is materialized on the receiver. + */ + boolean isMaterialized(); + + /** + * Materialize the chunk on the receiver. + * + * @param runningQuery + * The running query. + */ + void materialize(FederatedRunningQuery runningQuery); + + /** + * Visit the binding sets in the chunk. + * + * @todo we do not need to use {@link IAsynchronousIterator} any more. This + * could be much more flexible and should be harmonized to support + * high volume operators, GPU operators, etc. probably the right thing + * to do is introduce another interface here with a getChunk():IChunk + * where IChunk let's you access the chunks data in different ways + * (and chunks can be both {@link IBindingSet}[]s and element[]s so we + * might need to raise that into the interfaces and/or generics as + * well). + * + * @todo It is likely that we can convert to the use of + * {@link BlockingQueue} instead of {@link BlockingBuffer} in the + * operators and then handle the logic for combining chunks inside of + * the {@link QueryEngine}. E.g., by scanning this list for chunks for + * the same bopId and combining them logically into a single chunk. + * <p> + * For scale-out, chunk combination will naturally occur when the node + * on which the operator will run requests the {@link ByteBuffer}s + * from the source nodes. Those will get wrapped up logically into a + * source for processing. For selective operators, those chunks can be + * combined before we execute the operator. For unselective operators, + * we are going to run over all the data anyway. + */ + IAsynchronousIterator<IBindingSet[]> iterator(); + + // /** + // * The Internet address and port of a {@link ResourceService} from which + // * the receiver may demand the data. + // */ + // InetSocketAddress getServiceAddr(); + // + // /** + // * The set of resources on the sender which comprise the data. + // */ + // Iterator<UUID> getChunkIds(); + +} Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IChunkMessage.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java 2010-09-09 13:50:53 UTC (rev 3525) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java 2010-09-09 17:17:21 UTC (rev 3526) @@ -1,76 +1,24 @@ package com.bigdata.bop.engine; -import java.rmi.Remote; import java.rmi.RemoteException; -import com.bigdata.bop.BOp; - /** * Interface for a client executing queries (the query controller). */ public interface IQueryClient extends IQueryPeer { - /* - * @todo Could return a data structure which encapsulates the query results - * and could allow multiple results from a query, e.g., one per step in a - * program. - */ - // /** -// * Evaluate a query which materializes elements, such as an -// * {@link IPredicate}. +// * Return the query. // * // * @param queryId -// * The unique identifier for the query. -// * @param timestamp -// * The timestamp or transaction against which the query will run. -// * @param query -// * The query to evaluate. -// * @param source -// * The initial binding sets to get the query going (this is -// * typically an iterator visiting a single empty binding set). +// * The query identifier. +// * @return The query. // * -// * @return An iterator visiting the elements materialized by the query. -// * -// * @throws Exception +// * @throws RemoteException // */ -// public IChunkedIterator<?> eval(long queryId, long timestamp, BOp query) -// throws Exception; +// public BOp getQuery(long queryId) throws RemoteException; -// /** -// * Evaluate a query which visits {@link IBindingSet}s, such as a join. -// * -// * @param queryId -// * The unique identifier for the query. -// * @param timestamp -// * The timestamp or transaction against which the query will run. -// * @param query -// * The query to evaluate. -// * @param source -// * The initial binding sets to get the query going (this is -// * typically an iterator visiting a single empty binding set). -// * -// * @return An iterator visiting {@link IBindingSet}s which result from -// * evaluating the query. -// * -// * @throws Exception -// */ -// public IChunkedIterator<IBindingSet> eval(long queryId, long timestamp, -// BOp query, IAsynchronousIterator<IBindingSet[]> source) -// throws Exception; - /** - * Return the query. - * - * @param queryId - * The query identifier. - * @return The query. - * - * @throws RemoteException - */ - public BOp getQuery(long queryId) throws RemoteException; - - /** * Notify the client that execution has started for some query, operator, * node, and index partition. */ @@ -84,22 +32,4 @@ */ public void haltOp(HaltOpMessage msg) throws RemoteException; -// /** -// * Notify the query controller that a chunk of intermediate results is -// * available for the query. -// * -// * @param queryId -// * The query identifier. -// */ -// public void addChunk(long queryId) throws RemoteException; -// -// /** -// * Notify the query controller that a chunk of intermediate results was -// * taken for processing by the query. -// * -// * @param queryId -// * The query identifier. -// */ -// public void takeChunk(long queryId) throws RemoteException; - } Added: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryDecl.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryDecl.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryDecl.java 2010-09-09 17:17:21 UTC (rev 3526) @@ -0,0 +1,25 @@ +package com.bigdata.bop.engine; + +import com.bigdata.bop.BindingSetPipelineOp; + +/** + * A query declaration. + */ +public interface IQueryDecl { + + /** + * The proxy for the query controller. + */ + IQueryClient getQueryController(); + + /** + * The query identifier. + */ + long getQueryId(); + + /** + * The query. + */ + BindingSetPipelineOp getQuery(); + +} Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryDecl.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java 2010-09-09 13:50:53 UTC (rev 3525) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java 2010-09-09 17:17:21 UTC (rev 3526) @@ -1,6 +1,5 @@ package com.bigdata.bop.engine; -import java.net.InetSocketAddress; import java.rmi.Remote; import java.rmi.RemoteException; import java.util.UUID; @@ -23,26 +22,30 @@ UUID getServiceUUID() throws RemoteException; /** + * Declare a query to a peer. This message is sent to the peer before any + * other message for that query and declares the query and the query + * controller with which the peer must communicate during query evaluation. + * + * @param queryDecl + * The query declaration. + * + * @throws UnsupportedOperationException + * unless running in scale-out. + */ + void declareQuery(IQueryDecl queryDecl); + + /** * Notify a service that a buffer having data for some {@link BOp} in some * running query is available. The receiver may request the data when they * are ready. If the query is cancelled, then the sender will drop the * buffer. * - * @param clientProxy - * proxy used to communicate with the client running the query. - * @param serviceAddr - * address which may be used to demand the data. - * @param queryId - * the unique query identifier. - * @param bopId - * the identifier for the target {@link BOp}. + * @param msg + * The message. * - * @return <code>true</code> unless the receiver knows that the query has - * already been cancelled. + * @throws UnsupportedOperationException + * unless running in scale-out. */ -// * @param nbytes -// * The #of bytes of data which are available for that operator. - void bufferReady(IQueryClient clientProxy, InetSocketAddress serviceAddr, - long queryId, int bopId/*, int nbytes*/) throws RemoteException; + void bufferReady(IChunkMessage msg) throws RemoteException; } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java 2010-09-09 13:50:53 UTC (rev 3525) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java 2010-09-09 17:17:21 UTC (rev 3526) @@ -27,6 +27,7 @@ package com.bigdata.bop.engine; +import com.bigdata.bop.BOp; import com.bigdata.btree.ILocalBTreeView; import com.bigdata.journal.IIndexManager; import com.bigdata.service.IBigdataFederation; @@ -58,12 +59,18 @@ /** * The timestamp or transaction identifier against which the query is * reading. + * + * @todo may be moved into the individual operator. See + * {@link BOp.Annotations#READ_TIMESTAMP} */ long getReadTimestamp(); /** * The timestamp or transaction identifier against which the query is * writing. + * + * @todo may be moved into the individual operator. See + * {@link BOp.Annotations#WRITE_TIMESTAMP} */ long getWriteTimestamp(); Added: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryDecl.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryDecl.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryDecl.java 2010-09-09 17:17:21 UTC (rev 3526) @@ -0,0 +1,82 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Sep 9, 2010 + */ + +package com.bigdata.bop.engine; + +import java.io.Serializable; + +import com.bigdata.bop.BindingSetPipelineOp; + +/** + * Default implementation. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +public class QueryDecl implements IQueryDecl, Serializable { + + /** + * + */ + private static final long serialVersionUID = 1L; + + private final long queryId; + + private final IQueryClient clientProxy; + + private final BindingSetPipelineOp query; + + public QueryDecl(final IQueryClient clientProxy, final long queryId, + final BindingSetPipelineOp query) { + + if (clientProxy == null) + throw new IllegalArgumentException(); + + if (query == null) + throw new IllegalArgumentException(); + + this.clientProxy = clientProxy; + + this.queryId = queryId; + + this.query = query; + + } + + public BindingSetPipelineOp getQuery() { + return query; + } + + public IQueryClient getQueryController() { + return clientProxy; + } + + public long getQueryId() { + return queryId; + } + +} Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryDecl.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java 2010-09-09 13:50:53 UTC (rev 3525) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java 2010-09-09 17:17:21 UTC (rev 3526) @@ -27,7 +27,6 @@ package com.bigdata.bop.engine; -import java.net.InetSocketAddress; import java.rmi.RemoteException; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; @@ -50,8 +49,6 @@ import com.bigdata.btree.IndexSegment; import com.bigdata.btree.view.FusedView; import com.bigdata.journal.IIndexManager; -import com.bigdata.journal.ITx; -import com.bigdata.journal.TimestampUtility; import com.bigdata.rdf.internal.IV; import com.bigdata.rdf.spo.SPORelation; import com.bigdata.relation.IMutableRelation; @@ -328,6 +325,11 @@ .getLogger(QueryEngine.class); /** + * Error message used if a query is not running. + */ + protected static final transient String ERR_QUERY_NOT_RUNNING = "Query is not running:"; + + /** * Access to the indices. * <p> * Note: You MUST NOT use unisolated indices without obtaining the necessary @@ -477,6 +479,19 @@ private volatile boolean shutdown = false; /** + * Return if the query engine is running. + * + * @throws IllegalStateException + * if the query engine is shutting down. + */ + protected void assertRunning() { + + if (shutdown) + throw new IllegalStateException("Shutting down."); + + } + + /** * Runnable submits chunks available for evaluation against running queries. * * @todo Handle priority for selective queries based on the time remaining @@ -519,7 +534,7 @@ final long queryId = q.getQueryId(); if (q.isCancelled()) continue; - final BindingSetChunk chunk = q.chunksIn.poll(); + final IChunkMessage chunk = q.chunksIn.poll(); if (chunk == null) { // not expected, but can't do anything without a chunk. if (log.isDebugEnabled()) @@ -528,7 +543,7 @@ } if (log.isTraceEnabled()) log.trace("Accepted chunk: queryId=" + queryId - + ", bopId=" + chunk.bopId); + + ", bopId=" + chunk.getBOpId()); try { // create task. final FutureTask<?> ft = q.newChunkTask(chunk); @@ -558,19 +573,27 @@ * * @param chunk * A chunk of intermediate results. + * + * @throws IllegalArgumentException + * if the chunk is <code>null</code>. + * @throws IllegalStateException + * if the chunk is not materialized. */ - void add(final BindingSetChunk chunk) { + void acceptChunk(final IChunkMessage chunk) { if (chunk == null) throw new IllegalArgumentException(); - final RunningQuery q = runningQueries.get(chunk.queryId); + if (!chunk.isMaterialized()) + throw new IllegalStateException(); + + final RunningQuery q = runningQueries.get(chunk.getQueryId()); if(q == null) throw new IllegalStateException(); // add chunk to the query's input queue on this node. - q.add(chunk); + q.acceptChunk(chunk); // add query to the engine's task queue. priorityQueue.add(q); @@ -657,33 +680,35 @@ * IQueryPeer */ - public void bufferReady(IQueryClient clientProxy, - InetSocketAddress serviceAddr, long queryId, int bopId) { - // NOP + public void declareQuery(final IQueryDecl queryDecl) { + + throw new UnsupportedOperationException(); + } + public void bufferReady(IChunkMessage msg) { + + throw new UnsupportedOperationException(); + + } + /* * IQueryClient */ - /** - * @todo Define the behavior for these methods if the queryId is not found - * whether because the caller has the wrong value or because the query - * has terminated. - */ - public BOp getQuery(final long queryId) throws RemoteException { - - final RunningQuery q = runningQueries.get(queryId); - - if (q != null) { - - return q.getQuery(); - - } - - return null; - - } +// public BOp getQuery(final long queryId) throws RemoteException { +// +// final RunningQuery q = runningQueries.get(queryId); +// +// if (q != null) { +// +// return q.getQuery(); +// +// } +// +// return null; +// +// } public void startOp(final StartOpMessage msg) throws RemoteException { @@ -715,85 +740,110 @@ * * @param queryId * The unique identifier for the query. - * @param readTimestamp - * The timestamp or transaction against which the query will run. - * @param writeTimestamp - * The timestamp or transaction against which the query will - * write. * @param query * The query to evaluate. * * @return An iterator visiting {@link IBindingSet}s which result from * evaluating the query. * - * @throws IllegalArgumentException - * if the <i>readTimestamp</i> is {@link ITx#UNISOLATED} - * (queries may not read on the unisolated indices). - * @throws IllegalArgumentException - * if the <i>writeTimestamp</i> is neither - * {@link ITx#UNISOLATED} nor a read-write transaction - * identifier. * @throws IllegalStateException * if the {@link QueryEngine} has been {@link #shutdown()}. * @throws Exception - * - * @todo Consider elevating the read/write timestamps into the query plan as - * annotations. Closure would then rewrite the query plan for each - * pass, replacing the readTimestamp with the new read-behind - * timestamp. [This is related to how we will handle sequences of - * steps, parallel steps, and closure of steps.] */ - public RunningQuery eval(final long queryId, final long readTimestamp, - final long writeTimestamp, final BindingSetPipelineOp query) - throws Exception { + public RunningQuery eval(final long queryId, + final BindingSetPipelineOp query) throws Exception { if (query == null) throw new IllegalArgumentException(); - - if (readTimestamp == ITx.UNISOLATED) - throw new IllegalArgumentException(); - - if (TimestampUtility.isReadOnly(writeTimestamp)) - throw new IllegalArgumentException(); + final RunningQuery runningQuery = newRunningQuery(this, queryId, +// System.currentTimeMillis()/* begin */, + true/* controller */, this/* clientProxy */, query); + + assertRunning(); + final long timeout = query.getProperty(BOp.Annotations.TIMEOUT, BOp.Annotations.DEFAULT_TIMEOUT); - final RunningQuery runningQuery = newRunningQuery(this, queryId, - readTimestamp, writeTimestamp, - System.currentTimeMillis()/* begin */, timeout, - true/* controller */, this/* clientProxy */, query); + if (timeout < 0) + throw new IllegalArgumentException(BOp.Annotations.TIMEOUT); - if (shutdown) { + if (timeout != Long.MAX_VALUE) { - throw new IllegalStateException("Shutting down."); + // Compute the deadline (may overflow if timeout is very large). + final long deadline = System.currentTimeMillis() + timeout; + if (deadline > 0) { + /* + * Impose a deadline on the query. + */ + runningQuery.setDeadline(deadline); + + } + } - runningQueries.put(queryId, runningQuery); + putRunningQuery(queryId, runningQuery); return runningQuery; } /** + * Return the {@link RunningQuery} associated with that query identifier. + * + * @param queryId + * The query identifier. + * + * @return The {@link RunningQuery} -or- <code>null</code> if there is no + * query associated with that query identifier. + */ + protected RunningQuery getRunningQuery(final long queryId) { + + return runningQueries.get(queryId); + + } + + /** + * Places the {@link RunningQuery} object into the internal map. + * + * @param queryId + * The query identifier. + * @param runningQuery + * The {@link RunningQuery}. + */ + protected void putRunningQuery(final long queryId, + final RunningQuery runningQuery) { + + if (runningQuery == null) + throw new IllegalArgumentException(); + + runningQueries.put(queryId, runningQuery); + + } + + /** * Factory for {@link RunningQuery}s. */ protected RunningQuery newRunningQuery(final QueryEngine queryEngine, - final long queryId, final long readTimestamp, - final long writeTimestamp, final long begin, final long timeout, - final boolean controller, final IQueryClient clientProxy, - final BindingSetPipelineOp query) { + final long queryId, final boolean controller, + final IQueryClient clientProxy, final BindingSetPipelineOp query) { - return new RunningQuery(this, queryId, readTimestamp, writeTimestamp, - System.currentTimeMillis()/* begin */, timeout, - true/* controller */, this/* clientProxy */, query, - newQueryBuffer(query)); + return new RunningQuery(this, queryId, true/* controller */, + this/* clientProxy */, query, newQueryBuffer(query)); } /** * Return a buffer onto which the solutions will be written. + * + * @todo This method is probably in the wrong place. We should use whatever + * is associated with the top-level {@link BOp} in the query and then + * rely on the NIO mechanisms to move the data around as necessary. + * + * @todo Could return a data structure which encapsulates the query results + * and could allow multiple results from a query, e.g., one per step + * in a program. */ protected IBlockingBuffer<IBindingSet[]> newQueryBuffer( final BindingSetPipelineOp query) { Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-09 13:50:53 UTC (rev 3525) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-09 17:17:21 UTC (rev 3526) @@ -27,7 +27,6 @@ */ package com.bigdata.bop.engine; -import java.nio.ByteBuffer; import java.rmi.RemoteException; import java.util.LinkedHashMap; import java.util.LinkedHashSet; @@ -43,7 +42,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantLock; import org.apache.log4j.Logger; @@ -54,9 +52,9 @@ import com.bigdata.bop.BindingSetPipelineOp; import com.bigdata.bop.IBindingSet; import com.bigdata.bop.NoSuchBOpException; -import com.bigdata.bop.ap.Predicate; import com.bigdata.journal.IIndexManager; -import com.bigdata.relation.accesspath.BlockingBuffer; +import com.bigdata.journal.ITx; +import com.bigdata.journal.TimestampUtility; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; import com.bigdata.service.IBigdataFederation; @@ -103,10 +101,15 @@ */ final private long writeTimestamp; +// /** +// * The timestamp when the query was accepted by this node (ms). +// */ +// final private long begin; /** - * The timestamp when the query was accepted by this node (ms). + * The query deadline. The value is the system clock time in milliseconds + * when the query is due and {@link Long#MAX_VALUE} if there is no deadline. */ - final private long begin; + final private AtomicLong deadline = new AtomicLong(Long.MAX_VALUE); /** * How long the query is allowed to run (elapsed milliseconds) -or- @@ -129,15 +132,16 @@ */ final private IQueryClient clientProxy; - /** The query iff materialized on this node. */ - final private AtomicReference<BOp> queryRef; +// /** The query iff materialized on this node. */ +// final private AtomicReference<BOp> queryRef; + /** The query. */ + final private BOp query; /** * The buffer used for the overall output of the query pipeline. * - * @todo How does the pipeline get attached to this buffer? Via a special - * operator? Or do we just target the coordinating {@link QueryEngine} - * as the sink of the last operator so we can use NIO transfers? + * FIXME SCALEOUT: This should only exist on the query controller. Other + * nodes will send {@link IChunkMessage}s to the query controller. */ final private IBlockingBuffer<IBindingSet[]> queryBuffer; @@ -208,26 +212,49 @@ private final Set<Integer/*bopId*/> startedSet = new LinkedHashSet<Integer>(); /** - * The chunks available for immediate processing. + * The chunks available for immediate processing (they must have been + * materialized). * <p> * Note: This is package private so it will be visible to the * {@link QueryEngine}. + */ + final/* private */BlockingQueue<IChunkMessage> chunksIn = new LinkedBlockingDeque<IChunkMessage>(); + + /** + * Set the query deadline. The query will be cancelled when the deadline is + * passed. If the deadline is passed, the query is immediately cancelled. * - * @todo It is likely that we can convert to the use of - * {@link BlockingQueue} instead of {@link BlockingBuffer} in the - * operators and then handle the logic for combining chunks inside of - * the {@link QueryEngine}. E.g., by scanning this list for chunks for - * the same bopId and combining them logically into a single chunk. - * <p> - * For scale-out, chunk combination will naturally occur when the node - * on which the operator will run requests the {@link ByteBuffer}s - * from the source nodes. Those will get wrapped up logically into a - * source for processing. For selective operators, those chunks can be - * combined before we execute the operator. For unselective operators, - * we are going to run over all the data anyway. + * @param deadline + * The deadline. + * @throws IllegalArgumentException + * if the deadline is non-positive. + * @throws IllegalStateException + * if the deadline was already set. + * @throws UnsupportedOperationException + * unless node is the query controller. */ - final /*private*/ BlockingQueue<BindingSetChunk> chunksIn = new LinkedBlockingDeque<BindingSetChunk>(); + public void setDeadline(final long deadline) { + if(!controller) + throw new UnsupportedOperationException(); + + if (deadline <= 0) + throw new IllegalArgumentException(); + + // set the deadline. + if (!this.deadline + .compareAndSet(Long.MAX_VALUE/* expect */, deadline/* update */)) { + // the deadline is already set. + throw new IllegalStateException(); + } + + if (deadline < System.currentTimeMillis()) { + // deadline has already expired. + cancel(true/* mayInterruptIfRunning */); + } + + } + /** * The class executing the query on this node. */ @@ -259,41 +286,45 @@ } - /** - * Return the operator tree for this query. If query processing is - * distributed and the query has not been materialized on this node, then it - * is materialized now. - * - * @return The query. - */ public BOp getQuery() { + return query; + } + +// /** +// * Return the operator tree for this query. If query processing is +// * distributed and the query has not been materialized on this node, then it +// * is materialized now. +// * +// * @return The query. +// */ +// public BOp getQuery() { +// +// if (queryRef.get() == null) { +// +// synchronized (queryRef) { +// +// if (queryRef.get() == null) { +// +// try { +// +// queryRef.set(clientProxy.getQuery(queryId)); +// +// } catch (RemoteException e) { +// +// throw new RuntimeException(e); +// +// } +// +// } +// +// } +// +// } +// +// return queryRef.get(); +// +// } - if (queryRef.get() == null) { - - synchronized (queryRef) { - - if (queryRef.get() == null) { - - try { - - queryRef.set(clientProxy.getQuery(queryId)); - - } catch (RemoteException e) { - - throw new RuntimeException(e); - - } - - } - - } - - } - - return queryRef.get(); - - } - /** * Return <code>true</code> iff this is the query controller. */ @@ -305,11 +336,8 @@ /** * Return the current statistics for the query and <code>null</code> unless - * this is the query controller. - * - * @todo When the query is done, there will be one entry in this map for - * each operator in the pipeline. Non-pipeline operators such as - * {@link Predicate}s do not currently make it into this map. + * this is the query controller. For {@link BindingSetPipelineOp} operator + * which is evaluated there will be a single entry in this map. */ public Map<Integer/*bopId*/,BOpStats> getStats() { @@ -323,35 +351,100 @@ * @param begin * @param clientProxy * @param query - * The query (optional). + * + * @throws IllegalArgumentException + * if any argument is <code>null</code>. + * @throws IllegalArgumentException + * if the <i>readTimestamp</i> is {@link ITx#UNISOLATED} + * (queries may not read on the unisolated indices). + * @throws IllegalArgumentException + * if the <i>writeTimestamp</i> is neither + * {@link ITx#UNISOLATED} nor a read-write transaction + * identifier. + * + * @todo is queryBuffer required? should it be allocated from the top bop? */ public RunningQuery(final QueryEngine queryEngine, final long queryId, - final long readTimestamp, final long writeTimestamp, - final long begin, final long timeout, final boolean controller, +// final long begin, + final boolean controller, final IQueryClient clientProxy, final BOp query, final IBlockingBuffer<IBindingSet[]> queryBuffer) { + + if (queryEngine == null) + throw new IllegalArgumentException(); + + if (clientProxy == null) + throw new IllegalArgumentException(); + + if (query == null) + throw new IllegalArgumentException(); + this.queryEngine = queryEngine; this.queryId = queryId; - this.readTimestamp = readTimestamp; - this.writeTimestamp = writeTimestamp; - this.begin = begin; - this.timeout = timeout; +// this.begin = begin; this.controller = controller; this.clientProxy = clientProxy; - this.queryRef = new AtomicReference<BOp>(query); - if (controller && query == null) - throw new IllegalArgumentException(); + this.query = query; this.queryBuffer = queryBuffer; this.bopIndex = BOpUtility.getIndex(query); this.statsMap = controller ? new ConcurrentHashMap<Integer, BOpStats>() : null; + /* + * @todo when making a per-bop annotation, queries must obtain a tx for + * each timestamp up front on the controller and rewrite the bop to hold + * the tx until it is done. + * + * @todo This is related to how we handle sequences of steps, parallel + * steps, closure of steps, and join graphs. Those operations need to be + * evaluated on the controller. We will have to model the relationship + * between the subquery and the query in order to terminate the subquery + * when the query halts and to terminate the query if the subquery + * fails. + * + * @todo Closure operations must rewrite the query to update the + * annotations. Each pass in a closure needs to be its own "subquery" + * and will need to have a distinct queryId. + */ + final Long readTimestamp = query + .getProperty(BOp.Annotations.READ_TIMESTAMP); + + // @todo remove default when elevating to per-writable bop annotation. + final long writeTimestamp = query.getProperty( + BOp.Annotations.WRITE_TIMESTAMP, ITx.UNISOLATED); + + if (readTimestamp == null) + throw new IllegalArgumentException(); + + if (readTimestamp.longValue() == ITx.UNISOLATED) + throw new IllegalArgumentException(); + + if (TimestampUtility.isReadOnly(writeTimestamp)) + throw new IllegalArgumentException(); + + this.readTimestamp = readTimestamp; + + this.writeTimestamp = writeTimestamp; + + this.timeout = query.getProperty(BOp.Annotations.TIMEOUT, + BOp.Annotations.DEFAULT_TIMEOUT); + + if (timeout < 0) + throw new IllegalArgumentException(); + } /** - * Create a {@link BindingSetChunk} from a sink and add it to the queue. + * Take a chunk generated by some pass over an operator and make it + * available to the target operator. How this is done depends on whether the + * query is running against a standalone database or the scale-out database. * <p> - * Note: If we are running standalone, then we leave the data on the heap - * rather than formatting it onto a {@link ByteBuffer}. + * Note: The return value is used as part of the termination criteria for + * the query. + * <p> + * The default implementation supports a standalone database. The generated + * chunk is left on the Java heap and handed off synchronously using + * {@link QueryEngine#add(IChunkMessage)}. That method will queue the chunk + * for asynchronous processing. * * @param sinkId * The identifier of the target operator. @@ -363,39 +456,42 @@ * one chunk per index partition over which the intermediate results * were mapped. */ - protected <E> int add(final int sinkId, + protected <E> int handleOutputChunk(final int sinkId, final IBlockingBuffer<IBindingSet[]> sink) { /* * Note: The partitionId will always be -1 in scale-up. */ - final BindingSetChunk chunk = new BindingSetChunk(queryId, sinkId, - -1/* partitionId */, sink.iterator()); + final BindingSetChunk chunk = new BindingSetChunk(clientProxy, queryId, + sinkId, -1/* partitionId */, sink.iterator()); - queryEngine.add(chunk); + queryEngine.acceptChunk(chunk); return 1; - } + } /** * Make a chunk of binding sets available for consumption by the query. * <p> * Note: this is invoked by {@link QueryEngine#add(BindingSetChunk)}. * - * @param chunk + * @param msg * The chunk. */ - void add(final BindingSetChunk chunk) { + protected void acceptChunk(final IChunkMessage msg) { - if (chunk == null) + if (msg == null) throw new IllegalArgumentException(); + if (!msg.isMaterialized()) + throw new IllegalStateException(); + // verify still running. future.halted(); // add chunk to be consumed. - chunksIn.add(chunk); + chunksIn.add(msg); if (log.isDebugEnabled()) log.debug("queryId=" + queryId + ", chunksIn.size()=" @@ -409,30 +505,31 @@ * * @todo this should reject multiple invocations for a given query instance. */ - public void startQuery(final BindingSetChunk chunk) { + public void startQuery(final IChunkMessage chunk) { if (!controller) throw new UnsupportedOperationException(); if (chunk == null) throw new IllegalArgumentException(); - if (chunk.queryId != queryId) // @todo equals() if queryId is UUID. + if (chunk.getQueryId() != queryId) // @todo equals() if queryId is UUID. throw new IllegalArgumentException(); + final int bopId = chunk.getBOpId(); runStateLock.lock(); try { lifeCycleSetUpQuery(); availableChunkCount++; { - AtomicLong n = availableChunkCountMap.get(chunk.bopId); + AtomicLong n = availableChunkCountMap.get(bopId); if (n == null) - availableChunkCountMap.put(chunk.bopId, n = new AtomicLong()); + availableChunkCountMap.put(bopId, n = new AtomicLong()); n.incrementAndGet(); } if (log.isInfoEnabled()) log.info("queryId=" + queryId + ",runningTaskCount=" + runningTaskCount + ",availableChunks=" + availableChunkCount); - System.err.println("startQ : bopId=" + chunk.bopId + ",running=" + System.err.println("startQ : bopId=" + bopId + ",running=" + runningTaskCount + ",available=" + availableChunkCount); - queryEngine.add(chunk); + queryEngine.acceptChunk(chunk); } finally { runStateLock.unlock(); } @@ -484,13 +581,9 @@ System.err.println("startOp: bopId=" + msg.bopId + ",running=" + runningTaskCount + ",available=" + availableChunkCount + ",fanIn=" + msg.nchunks); - final long elapsed = System.currentTimeMillis() - begin; - if (log.isTraceEnabled()) - log.trace("bopId=" + msg.bopId + ",partitionId=" + msg.partitionId - + ",serviceId=" + msg.serviceId + " : runningTaskCount=" - + runningTaskCount + ", availableChunkCount=" - + availableChunkCount + ", elapsed=" + elapsed); - if (elapsed > timeout) { + if (deadline.get() < System.currentTimeMillis()) { + if (log.isTraceEnabled()) + log.trace("queryId: deadline expired."); future.halt(new TimeoutException()); cancel(true/* mayInterruptIfRunning */); } @@ -563,13 +656,13 @@ + runningTaskCount; assert availableChunkCount >= 0 : "availableChunkCount=" + availableChunkCount; - final long elapsed = System.currentTimeMillis() - begin; +// final long elapsed = System.currentTimeMillis() - begin; if (log.isTraceEnabled()) log.trace("bopId=" + msg.bopId + ",partitionId=" + msg.partitionId + ",serviceId=" + queryEngine.getServiceUUID() + ", nchunks=" + fanOut + " : runningTaskCount=" + runningTaskCount + ", availableChunkCount=" - + availableChunkCount + ", elapsed=" + elapsed); + + availableChunkCount);// + ", elapsed=" + elapsed); // test termination criteria if (msg.cause != null) { // operator failed on this chunk. @@ -582,8 +675,9 @@ // success (all done). future.halt(getStats()); cancel(true/* mayInterruptIfRunning */); - } else if (elapsed > timeout) { - // timeout + } else if (deadline.get() < System.currentTimeMillis()) { + if (log.isTraceEnabled()) + log.trace("queryId: deadline expired."); future.halt(new TimeoutException()); cancel(true/* mayInterruptIfRunning */); } @@ -614,8 +708,8 @@ if (!runStateLock.isHeldByCurrentThread()) throw new IllegalMonitorStateException(); - return PipelineUtility.isDone(bopId, queryRef.get(), bopIndex, - runningCountMap, availableChunkCountMap); + return PipelineUtility.isDone(bopId, query, bopIndex, runningCountMap, + availableChunkCountMap); } @@ -681,14 +775,16 @@ * A chunk to be consumed. */ @SuppressWarnings("unchecked") - protected FutureTask<Void> newChunkTask(final BindingSetChunk chunk) { + protected FutureTask<Void> newChunkTask(final IChunkMessage chunk) { /* * Look up the BOp in the index, create the BOpContext for that BOp, and * return the value returned by BOp.eval(context). */ - final BOp bop = bopIndex.get(chunk.bopId); + final int bopId = chunk.getBOpId(); + final int partitionId = chunk.getPartitionId(); + final BOp bop = bopIndex.get(bopId); if (bop == null) { - throw new NoSuchBOpException(chunk.bopId); + throw new NoSuchBOpException(bopId); } if (!(bop instanceof BindingSetPipelineOp)) { /* @@ -701,7 +797,7 @@ // self final BindingSetPipelineOp op = ((BindingSetPipelineOp) bop); // parent (null if this is the root of the operator tree). - final BOp p = BOpUtility.getParent(queryRef.get(), op); + final BOp p = BOpUtility.getParent(query, op); // sink (null unless parent is defined) final Integer sinkId = p == null ? null : (Integer) p .getProperty(BindingSetPipelineOp.Annotations.BOP_ID); @@ -716,8 +812,8 @@ final IBlockingBuffer<IBindingSet[]> altSink = altSinkId == null ? null : op.newBuffer(); // context - final BOpContext context = new BOpContext(this, chunk.partitionId, op - .newStats(), chunk.source, sink, altSink); + final BOpContext context = new BOpContext(this, partitionId, op + .newStats(), chunk.iterator(), sink, altSink); // FutureTask for operator execution (not running yet). final FutureTask<Void> f = op.eval(context); // Hook the FutureTask. @@ -729,29 +825,29 @@ int altSinkChunksOut = 0; try { clientProxy.startOp(new StartOpMessage(queryId, - chunk.bopId, chunk.partitionId, serviceId, fanIn)); + bopId, partitionId, serviceId, fanIn)); if (log.isDebugEnabled()) log.debug("Running chunk: queryId=" + queryId - + ", bopId=" + chunk.bopId + ", bop=" + bop); + + ", bopId=" + bopId + ", bop=" + bop); f.run(); // run f.get(); // verify success if (sink != queryBuffer && !sink.isEmpty()) { // handle output chunk. - sinkChunksOut += add(sinkId, sink); + sinkChunksOut += handleOutputChunk(sinkId, sink); } if (altSink != queryBuffer && altSink != null && !altSink.isEmpty()) { // handle alt sink output chunk. - altSinkChunksOut += add(altSinkId, altSink); + altSinkChunksOut += handleOutputChunk(altSinkId, altSink); } - clientProxy.haltOp(new HaltOpMessage(queryId, chunk.bopId, - chunk.partitionId, serviceId, null/* cause */, + clientProxy.haltOp(new HaltOpMessage(queryId, bopId, + partitionId, serviceId, null/* cause */, sinkId, sinkChunksOut, altSinkId, altSinkChunksOut, context.getStats())); } catch (Throwable t) { try { clientProxy.haltOp(new HaltOpMessage(queryId, - chunk.bopId, chunk.partitionId, serviceId, + bopId, partitionId, serviceId, t/* cause */, sinkId, sinkChunksOut, altSinkId, altSinkChunksOut, context.getStats())); } catch (RemoteE... [truncated message content] |
From: <tho...@us...> - 2010-09-10 20:47:35
|
Revision: 3531 http://bigdata.svn.sourceforge.net/bigdata/?rev=3531&view=rev Author: thompsonbry Date: 2010-09-10 20:47:27 +0000 (Fri, 10 Sep 2010) Log Message: ----------- Move the timestamp and mutation onto the BOp. More work on moving binding sets around. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IChunkMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ThickChunkMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/mutation/InsertOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ndx/SampleLocalShard.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/bset/TestConditionalRoutingOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/MockRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/join/TestPipelineJoin.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestDistinctBindingSets.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java Added Paths: ----------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ChunkMessageWithNIOPayload.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java 2010-09-10 19:44:37 UTC (rev 3530) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java 2010-09-10 20:47:27 UTC (rev 3531) @@ -140,9 +140,25 @@ * shards, mapped against nodes, or evaluated on the query controller must * override this method. */ - public BOpEvaluationContext getEvaluationContext(); + BOpEvaluationContext getEvaluationContext(); + + /** + * Return <code>true</code> iff this operator is an access path which writes + * on the database. + * + * @see Annotations#MUTATION + */ + boolean isMutation(); /** + * The timestamp or transaction identifier on which the operator will read + * or write. + * + * @see Annotations#TIMESTAMP + */ + long getTimestamp(); + + /** * Interface declaring well known annotations. */ public interface Annotations { @@ -176,20 +192,31 @@ long DEFAULT_TIMEOUT = Long.MAX_VALUE; /** - * The timestamp (or transaction identifier) associated with a read from - * the database. + * Boolean property whose value is <code>true</code> iff this operator + * writes on a database. + * <p> + * Most operators operate solely on streams of elements or binding sets. + * Some operators read or write on the database using an access path, + * which is typically described by an {@link IPredicate}. This property + * MUST be <code>true</code> when access path is used to write on the + * database. + * <p> + * Operators which read or write on the database must declare the + * {@link Annotations#TIMESTAMP} associated with that operation. * - * @todo Combine the read and write timestamps as a single - * <code>TX</code> value and require this on any operator which - * reads or writes on the database. + * @see #TIMESTAMP */ - String READ_TIMESTAMP = BOp.class.getName() + ".readTimestamp"; - + String MUTATION = BOp.class.getName() + ".mutation"; + + boolean DEFAULT_MUTATION = false; + /** - * The timestamp (or transaction identifier) associated with a write on - * the database. + * The timestamp (or transaction identifier) used by this operator if it + * reads or writes on the database. + * + * @see #MUTATION */ - String WRITE_TIMESTAMP = BOp.class.getName() + ".writeTimestamp"; + String TIMESTAMP = BOp.class.getName() + ".timestamp"; /** * For hash partitioned operators, this is the set of the member nodes @@ -202,7 +229,7 @@ * @todo Move onto an interface parallel to {@link IShardwisePipelineOp} */ String MEMBER_SERVICES = "memberServices"; - + } - + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java 2010-09-10 19:44:37 UTC (rev 3530) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java 2010-09-10 20:47:27 UTC (rev 3531) @@ -338,10 +338,28 @@ } + /** + * The default implementation returns {@link BOpEvaluationContext#ANY} and + * must be overridden by operators which have a different {@link BOpEvaluationContext}. + * <p> + * {@inheritDoc} + */ public BOpEvaluationContext getEvaluationContext() { return BOpEvaluationContext.ANY; } - + + public final boolean isMutation() { + + return getProperty(Annotations.MUTATION, Annotations.DEFAULT_MUTATION); + + } + + public final long getTimestamp() { + + return getRequiredProperty(Annotations.TIMESTAMP); + + } + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java 2010-09-10 19:44:37 UTC (rev 3530) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java 2010-09-10 20:47:27 UTC (rev 3531) @@ -115,23 +115,27 @@ return runningQuery.getIndexManager(); } - /** - * The timestamp or transaction identifier against which the query is - * reading. - */ - public final long getReadTimestamp() { - return runningQuery.getReadTimestamp(); - } +// /** +// * The timestamp or transaction identifier against which the query is +// * reading. +// * +// * @deprecated by {@link BOp.Annotations#TIMESTAMP} +// */ +// public final long getReadTimestamp() { +// return runningQuery.getReadTimestamp(); +// } +// +// /** +// * The timestamp or transaction identifier against which the query is +// * writing. +// * +// * @deprecated by {@link BOp.Annotations#TIMESTAMP} +// */ +// public final long getWriteTimestamp() { +// return runningQuery.getWriteTimestamp(); +// } /** - * The timestamp or transaction identifier against which the query is - * writing. - */ - public final long getWriteTimestamp() { - return runningQuery.getWriteTimestamp(); - } - - /** * The index partition identifier -or- <code>-1</code> if the index is not * sharded. */ @@ -305,105 +309,49 @@ * order to support mutation operator we will also have to pass in the * {@link #writeTimestamp} or differentiate this in the method name. */ - public IRelation getReadRelation(final IPredicate<?> pred) { + public IRelation getRelation(final IPredicate<?> pred) { /* - * @todo Cache the resource locator? - * - * @todo This should be using the federation as the index manager when - * locating a resource for scale-out, right? But s/o reads must use the - * local index manager when actually obtaining the index view for the - * relation. + * Note: This uses the federation as the index manager when locating a + * resource for scale-out. However, s/o reads must use the local index + * manager when actually obtaining the index view for the relation. */ - return (IRelation) getIndexManager().getResourceLocator().locate( - pred.getOnlyRelationName(), getReadTimestamp()); + final IIndexManager tmp = getFederation() == null ? getIndexManager() + : getFederation(); + + final long timestamp = pred + .getRequiredProperty(BOp.Annotations.TIMESTAMP); - } + return (IRelation<?>) tmp.getResourceLocator().locate( + pred.getOnlyRelationName(), timestamp); - /** - * Return a writable view of the relation. - * - * @param namespace - * The namespace of the relation. - * - * @return A writable view of the relation. - */ - public IRelation getWriteRelation(final String namespace) { - - /* - * @todo Cache the resource locator? - * - * @todo This should be using the federation as the index manager when - * locating a resource for scale-out, right? But s/o writes must use - * the local index manager when actually obtaining the index view for - * the relation. - */ - return (IRelation) getIndexManager().getResourceLocator().locate( - namespace, getWriteTimestamp()); - } - /** - * Return an mutable view of the specified index. - * - * @param <T> - * The generic type of the elements in the relation. - * @param relation - * The relation. - * @param keyOrder - * The key order for that index. - * @param partitionId - * The partition identifier and <code>-1</code> unless running - * against an {@link IBigdataFederation}. - * - * @return The mutable view of the index. - * - * @throws UnsupportedOperationException - * if there is an attempt to read on an index partition when the - * database is not an {@link IBigdataFederation} or when the - * database is an {@link IBigdataFederation} unless the index - * partition was specified. - */ - public <T> ILocalBTreeView getMutableLocalIndexView( - final IRelation<T> relation, final IKeyOrder<T> keyOrder, - final int partitionId) { +// /** +// * Return a writable view of the relation. +// * +// * @param namespace +// * The namespace of the relation. +// * +// * @return A writable view of the relation. +// * +// * @deprecated by getRelation() +// */ +// public IRelation getWriteRelation(final String namespace) { +// +// /* +// * @todo Cache the resource locator? +// * +// * @todo This should be using the federation as the index manager when +// * locating a resource for scale-out, right? But s/o writes must use +// * the local index manager when actually obtaining the index view for +// * the relation. +// */ +// return (IRelation) getIndexManager().getResourceLocator().locate( +// namespace, getWriteTimestamp()); +// +// } - final String namespace = relation.getNamespace(); - - final ILocalBTreeView ndx; - - if (partitionId == -1) { - - if (getFederation() != null) { - // This is scale-out so the partition identifier is required. - throw new UnsupportedOperationException(); - } - - // The index is not partitioned. - ndx = (ILocalBTreeView) getIndexManager().getIndex(namespace + "." - + keyOrder.getIndexName(), getWriteTimestamp()); - - } else { - - if (getFederation() == null) { - // This is not scale-out so index partitions are not supported. - throw new UnsupportedOperationException(); - } - - // The name of the desired index partition. - final String name = DataService.getIndexPartitionName(namespace - + "." + keyOrder.getIndexName(), partitionId); - - // MUST be a local index view. - ndx = (ILocalBTreeView) getIndexManager().getIndex(name, - getWriteTimestamp()); - - } - - return ndx; - - } - /** * Obtain an access path reading from relation for the specified predicate * (from the tail of some rule). @@ -443,10 +391,13 @@ final int partitionId = predicate.getPartitionId(); + final long timestamp = predicate + .getRequiredProperty(BOp.Annotations.TIMESTAMP); + final int flags = predicate.getProperty( PipelineOp.Annotations.FLAGS, PipelineOp.Annotations.DEFAULT_FLAGS) - | (TimestampUtility.isReadOnly(getReadTimestamp()) ? IRangeQuery.READONLY + | (TimestampUtility.isReadOnly(timestamp) ? IRangeQuery.READONLY : 0); final int chunkOfChunksCapacity = predicate.getProperty( @@ -463,8 +414,6 @@ final IIndexManager indexManager = getIndexManager(); - final long readTimestamp = getReadTimestamp(); - if (predicate.getPartitionId() != -1) { /* @@ -497,9 +446,9 @@ // MUST be a local index view. final ILocalBTreeView ndx = (ILocalBTreeView) indexManager - .getIndex(name, readTimestamp); + .getIndex(name, timestamp); - return new AccessPath(relation, indexManager, readTimestamp, + return new AccessPath(relation, indexManager, timestamp, predicate, keyOrder, ndx, flags, chunkOfChunksCapacity, chunkCapacity, fullyBufferedReadThreshold).init(); @@ -522,13 +471,13 @@ throw new IllegalArgumentException("no index? relation=" + relation.getNamespace() + ", timestamp=" - + readTimestamp + ", keyOrder=" + keyOrder + ", pred=" + + timestamp + ", keyOrder=" + keyOrder + ", pred=" + predicate + ", indexManager=" + getIndexManager()); } accessPath = new AccessPath((IRelation) relation, indexManager, - readTimestamp, (IPredicate) predicate, + timestamp, (IPredicate) predicate, (IKeyOrder) keyOrder, ndx, flags, chunkOfChunksCapacity, chunkCapacity, fullyBufferedReadThreshold).init(); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java 2010-09-10 19:44:37 UTC (rev 3530) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java 2010-09-10 20:47:27 UTC (rev 3531) @@ -15,7 +15,7 @@ public class BindingSetChunk implements IChunkMessage { /** The query controller. */ - private final IQueryClient clientProxy; + private final IQueryClient queryController; /** * The query identifier. @@ -38,7 +38,7 @@ private IAsynchronousIterator<IBindingSet[]> source; public IQueryClient getQueryController() { - return clientProxy; + return queryController; } public long getQueryId() { @@ -57,22 +57,17 @@ return true; } - /** - * - * @todo constructor to accept the BlockingBuffer instead as part of - * {@link IChunkMessage} harmonization (or an "IChunk" API). - */ - public BindingSetChunk(final IQueryClient clientProxy, final long queryId, - final int bopId, final int partitionId, + public BindingSetChunk(final IQueryClient queryController, + final long queryId, final int bopId, final int partitionId, final IAsynchronousIterator<IBindingSet[]> source) { - if (clientProxy == null) + if (queryController == null) throw new IllegalArgumentException(); if (source == null) throw new IllegalArgumentException(); - this.clientProxy = clientProxy; + this.queryController = queryController; this.queryId = queryId; Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IChunkMessage.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IChunkMessage.java 2010-09-10 19:44:37 UTC (rev 3530) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IChunkMessage.java 2010-09-10 20:47:27 UTC (rev 3531) @@ -30,15 +30,6 @@ /** The identifier for the target index partition. */ int getPartitionId(); - /* - * @todo Report the #of bytes available with this message. However, first - * figure out if that if the #of bytes in this {@link OutputChunk} or across - * all {@link OutputChunk}s available for the target service and sink. - */ - // @todo move to concrete subclass or allow ZERO if data are in memory (no RMI). -// /** The #of bytes of data which are available for that operator. */ -// int getBytesAvailable(); - /** * Return <code>true</code> if the chunk is materialized on the receiver. */ @@ -79,15 +70,4 @@ */ IAsynchronousIterator<IBindingSet[]> iterator(); - // /** - // * The Internet address and port of a {@link ResourceService} from which - // * the receiver may demand the data. - // */ - // InetSocketAddress getServiceAddr(); - // - // /** - // * The set of resources on the sender which comprise the data. - // */ - // Iterator<UUID> getChunkIds(); - } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java 2010-09-10 19:44:37 UTC (rev 3530) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java 2010-09-10 20:47:27 UTC (rev 3531) @@ -56,25 +56,25 @@ */ IIndexManager getIndexManager(); - /** - * The timestamp or transaction identifier against which the query is - * reading. - * - * @todo may be moved into the individual operator. See - * {@link BOp.Annotations#READ_TIMESTAMP} - */ - long getReadTimestamp(); +// /** +// * The timestamp or transaction identifier against which the query is +// * reading. +// * +// * @deprecated move into the individual operator. See +// * {@link BOp.Annotations#TIMESTAMP} +// */ +// long getReadTimestamp(); +// +// /** +// * The timestamp or transaction identifier against which the query is +// * writing. +// * +// * @deprecated moved into the individual operator. See +// * {@link BOp.Annotations#TIMESTAMP} +// */ +// long getWriteTimestamp(); /** - * The timestamp or transaction identifier against which the query is - * writing. - * - * @todo may be moved into the individual operator. See - * {@link BOp.Annotations#WRITE_TIMESTAMP} - */ - long getWriteTimestamp(); - - /** * Terminate query evaluation */ void halt(); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-10 19:44:37 UTC (rev 3530) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-10 20:47:27 UTC (rev 3531) @@ -89,18 +89,18 @@ /** The unique identifier for this query. */ final private long queryId; - /** - * The timestamp or transaction identifier against which the query is - * reading. - */ - final private long readTimestamp; +// /** +// * The timestamp or transaction identifier against which the query is +// * reading. +// */ +// final private long readTimestamp; +// +// /** +// * The timestamp or transaction identifier against which the query is +// * writing. +// */ +// final private long writeTimestamp; - /** - * The timestamp or transaction identifier against which the query is - * writing. - */ - final private long writeTimestamp; - // /** // * The timestamp when the query was accepted by this node (ms). // */ @@ -389,41 +389,42 @@ this.bopIndex = BOpUtility.getIndex(query); this.statsMap = controller ? new ConcurrentHashMap<Integer, BOpStats>() : null; - /* - * @todo when making a per-bop annotation, queries must obtain a tx for - * each timestamp up front on the controller and rewrite the bop to hold - * the tx until it is done. - * - * @todo This is related to how we handle sequences of steps, parallel - * steps, closure of steps, and join graphs. Those operations need to be - * evaluated on the controller. We will have to model the relationship - * between the subquery and the query in order to terminate the subquery - * when the query halts and to terminate the query if the subquery - * fails. - * - * @todo Closure operations must rewrite the query to update the - * annotations. Each pass in a closure needs to be its own "subquery" - * and will need to have a distinct queryId. - */ - final Long readTimestamp = query - .getProperty(BOp.Annotations.READ_TIMESTAMP); - - // @todo remove default when elevating to per-writable bop annotation. - final long writeTimestamp = query.getProperty( - BOp.Annotations.WRITE_TIMESTAMP, ITx.UNISOLATED); - - if (readTimestamp == null) - throw new IllegalArgumentException(); - - if (readTimestamp.longValue() == ITx.UNISOLATED) - throw new IllegalArgumentException(); - - if (TimestampUtility.isReadOnly(writeTimestamp)) - throw new IllegalArgumentException(); - - this.readTimestamp = readTimestamp; - this.writeTimestamp = writeTimestamp; +// /* +// * @todo when making a per-bop annotation, queries must obtain a tx for +// * each timestamp up front on the controller and rewrite the bop to hold +// * the tx until it is done. +// * +// * @todo This is related to how we handle sequences of steps, parallel +// * steps, closure of steps, and join graphs. Those operations need to be +// * evaluated on the controller. We will have to model the relationship +// * between the subquery and the query in order to terminate the subquery +// * when the query halts and to terminate the query if the subquery +// * fails. +// * +// * @todo Closure operations must rewrite the query to update the +// * annotations. Each pass in a closure needs to be its own "subquery" +// * and will need to have a distinct queryId. +// */ +// final Long timestamp = query +// .getProperty(BOp.Annotations.TIMESTAMP); +// +// // @todo remove default when elevating to per-writable bop annotation. +// final long writeTimestamp = query.getProperty( +// BOp.Annotations.WRITE_TIMESTAMP, ITx.UNISOLATED); +// +// if (readTimestamp == null) +// throw new IllegalArgumentException(); +// +// if (readTimestamp.longValue() == ITx.UNISOLATED) +// throw new IllegalArgumentException(); +// +// if (TimestampUtility.isReadOnly(writeTimestamp)) +// throw new IllegalArgumentException(); +// +// this.readTimestamp = readTimestamp; +// +// this.writeTimestamp = writeTimestamp; this.timeout = query.getProperty(BOp.Annotations.TIMEOUT, BOp.Annotations.DEFAULT_TIMEOUT); @@ -960,16 +961,16 @@ } - public long getReadTimestamp() { - - return readTimestamp; - - } +// public long getReadTimestamp() { +// +// return readTimestamp; +// +// } +// +// public long getWriteTimestamp() { +// +// return writeTimestamp; +// +// } - public long getWriteTimestamp() { - - return writeTimestamp; - - } - } Added: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ChunkMessageWithNIOPayload.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ChunkMessageWithNIOPayload.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ChunkMessageWithNIOPayload.java 2010-09-10 20:47:27 UTC (rev 3531) @@ -0,0 +1,244 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Sep 10, 2010 + */ + +package com.bigdata.bop.fed; + +import java.io.Serializable; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.UUID; + +import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.engine.IChunkMessage; +import com.bigdata.bop.engine.IQueryClient; +import com.bigdata.io.DirectBufferPoolAllocator.IAllocation; +import com.bigdata.io.DirectBufferPoolAllocator.IAllocationContext; +import com.bigdata.relation.accesspath.IAsynchronousIterator; +import com.bigdata.service.ResourceService; + +/** + * An {@link IChunkMessage} where the payload is made available to the receiving + * service using an NIO transfer against the sender's {@link ResourceService}. + * This is suitable for moving large blocks of data during query evaluation. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +public class ChunkMessageWithNIOPayload implements IChunkMessage, Serializable { + + /** + * + */ + private static final long serialVersionUID = 1L; + + /** + * Metadata about an allocation to be retrieved from the sender's + * {@link ResourceService}. + */ + private final class A implements Serializable { + + /** + * + */ + private static final long serialVersionUID = 1L; + + /** + * The identifier of the resource on the sender's + * {@link ResourceService}. + */ + private final UUID bufferId; + + /** + * The size of that resource in bytes. + */ + private final int nbytes; + + /** + * + * @param bufferId + * The identifier of the resource on the sender's + * {@link ResourceService}. + * @param nbytes + * The size of that resource in bytes. + */ + public A(final UUID bufferId, final int nbytes) { + this.bufferId = bufferId; + this.nbytes = nbytes; + } + } + + final private IQueryClient queryController; + + final private long queryId; + + final private int bopId; + + final private int partitionId; + + final private int nbytes; + + /** + * Note: Even when we send one message per chunk, we can still have a list + * of {@link IAllocation}s if the chunk did not get formatted onto a single + * {@link IAllocation}. + */ + final private A[] allocations; + + /** + * The Internet address and port where the receiver can fetch the payload + * using the sender's {@link ResourceService}. + */ + final private InetSocketAddress addr; + + public IQueryClient getQueryController() { + return queryController; + } + + public long getQueryId() { + return queryId; + } + + public int getBOpId() { + return bopId; + } + + public int getPartitionId() { + return partitionId; + } + + /** The #of bytes of data which are available for that operator. */ + public int getBytesAvailable() { + return nbytes; + } + + /** + * The Internet address and port of a {@link ResourceService} from which the + * receiver may demand the data. + */ + public InetSocketAddress getServiceAddr() { + return addr; + } + + /** + * + * @param queryController + * @param queryId + * @param sinkId + * @param partitionId + * @param allocations + * The ordered list of {@link IAllocation}s comprising the chunk. + * @param addr + * The Internet address and port where the receiver can fetch the + * payload using the sender's {@link ResourceService}. + */ + public ChunkMessageWithNIOPayload(final IQueryClient queryController, + final long queryId, final int sinkId, final int partitionId, + final List<IAllocation> allocations, final InetSocketAddress addr) { + + if (queryController == null) + throw new IllegalArgumentException(); + + if (allocations == null) + throw new IllegalArgumentException(); + + if (addr == null) + throw new IllegalArgumentException(); + + this.queryController = queryController; + this.queryId = queryId; + this.bopId = sinkId; + this.partitionId = partitionId; + final int n = allocations.size(); + this.allocations = new A[n]; + int i = 0; + int nbytes = 0; + final Iterator<IAllocation> itr = allocations.iterator(); + while (itr.hasNext()) { + final IAllocation alloc = itr.next(); + final int len = alloc.getSlice().capacity(); + this.allocations[i++] = new A(alloc.getId(), len); + nbytes += len; + } + this.nbytes = nbytes; + this.addr = addr; + + } + + public boolean isMaterialized() { + return materialized; + } + private volatile boolean materialized = false; + + /** + * + * FIXME unit tests for materializing and visiting the chunk. + */ + synchronized public void materialize(FederatedRunningQuery runningQuery) { + + if (materialized) + return; + + final AllocationContextKey key = new ShardContext(queryId, bopId, + partitionId); + + final IAllocationContext allocationContext = runningQuery + .getAllocationContext(key); + + final ResourceService resourceService = runningQuery.getQueryEngine() + .getResourceService(); + +// for (A a : allocations) { +// +// /* +// * FIXME harmonize an IAllocation[] with a ByteBuffer for the {@link +// * ResourceService}. The problem is that an object to be sent across +// * the wire may span multiple ByteBuffers. +// */ +// final ByteBuffer tmp = allocationContext.alloc(a.nbytes); +// +// new ResourceService.ReadBufferTask(addr, a.bufferId, tmp); +// +// } + + throw new UnsupportedOperationException(); + + } + + public IAsynchronousIterator<IBindingSet[]> iterator() { + + if (!isMaterialized()) + throw new UnsupportedOperationException(); + + // TODO Auto-generated method stub + throw new UnsupportedOperationException(); + + } + +} Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ChunkMessageWithNIOPayload.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java 2010-09-10 19:44:37 UTC (rev 3530) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java 2010-09-10 20:47:27 UTC (rev 3531) @@ -299,5 +299,5 @@ * Cache for {@link #getQueryPeer(UUID)}. */ private final ConcurrentHashMap<UUID, IQueryPeer> proxyMap = new ConcurrentHashMap<UUID, IQueryPeer>(); - + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java 2010-09-10 19:44:37 UTC (rev 3530) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java 2010-09-10 20:47:27 UTC (rev 3531) @@ -27,7 +27,6 @@ package com.bigdata.bop.fed; -import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.rmi.RemoteException; import java.util.Arrays; @@ -47,7 +46,6 @@ import com.bigdata.bop.engine.IChunkMessage; import com.bigdata.bop.engine.IQueryClient; import com.bigdata.bop.engine.IQueryPeer; -import com.bigdata.bop.engine.QueryEngine; import com.bigdata.bop.engine.RunningQuery; import com.bigdata.io.DirectBufferPoolAllocator; import com.bigdata.io.SerializerUtil; @@ -242,22 +240,81 @@ } /** + * Overridden to make this visible to the {@link FederatedQueryEngine}. + * <p> + * {@inheritDoc} + */ + @Override + protected void acceptChunk(final IChunkMessage msg) { + + super.acceptChunk(msg); + + } + + /** + * Resolve the proxy for an {@link IQueryPeer}. This is special cased for + * both <i>this</i> service (the actual reference is returned) and the query + * controller (we use an alternative path to discover the query controller + * since it might not be registered against a lookup service if it is not a + * data service). + * + * @param serviceUUID + * The service identifier for the peer. + * + * @return The proxy for the service or <code>null</code> if the service + * could not be discovered. + */ + protected IQueryPeer getQueryPeer(final UUID serviceUUID) { + + if (serviceUUID == null) + throw new IllegalArgumentException(); + + final IQueryPeer queryPeer; + + if(serviceUUID.equals(getQueryEngine().getServiceUUID())) { + + // Return a hard reference to the query engine (NOT a proxy). + return getQueryEngine(); + + } else if (serviceUUID.equals(queryControllerUUID)) { + + // The target is the query controller. + queryPeer = getQueryController(); + + } else { + + // The target is some data service. + queryPeer = getQueryEngine().getQueryPeer(serviceUUID); + + } + + return queryPeer; + + } + + /** * Return the {@link IAllocationContext} for the given key. * * @param key * The key. * * @return The allocation context. + * + * @todo use typesafe enum for the types of allocation contexts? */ - private IAllocationContext getAllocationContext( + public IAllocationContext getAllocationContext( final AllocationContextKey key) { final IAllocationContext ctx = getQueryEngine().getResourceService() .getAllocator().getAllocationContext(key); // note the allocation contexts associated with this running query. - allocationContexts.putIfAbsent(key, ctx); + final IAllocationContext ctx2 = allocationContexts + .putIfAbsent(key, ctx); + if (ctx2 != null) + return ctx2; + return ctx; } @@ -309,10 +366,6 @@ * Note: IKeyOrder tells us which index will be used and should be * set on the predicate by the join optimizer. * - * @todo Use the read or write timestamp depending on whether the - * operator performs mutation [this must be part of the operator - * metadata.] - * * @todo Set the capacity of the the "map" buffer to the size of the * data contained in the sink (in fact, we should just process the * sink data in place). @@ -321,7 +374,7 @@ @SuppressWarnings("unchecked") final IPredicate<E> pred = ((IShardwisePipelineOp) bop).getPredicate(); final IKeyOrder<E> keyOrder = pred.getKeyOrder(); - final long timestamp = getReadTimestamp(); // @todo read vs write timestamp. + final long timestamp = pred.getTimestamp(); final int capacity = 1000;// @todo final int capacity2 = 1000;// @todo final MapBindingSetsOverShardsBuffer<IBindingSet, E> mapper = new MapBindingSetsOverShardsBuffer<IBindingSet, E>( @@ -373,11 +426,11 @@ final IBuffer<IBindingSet> shardSink = e.getValue(); - // FIXME harmonize IBuffer<IBindingSet> vs IBuffer<IBindingSet[]> -// sendOutputChunkReadyMessage(newOutputChunk(locator -// .getDataServiceUUID(), sinkId, allocationContext, -// shardSink)); +// // FIXME harmonize IBuffer<IBindingSet> vs IBuffer<IBindingSet[]> +// sendChunkMessage(locator.getDataServiceUUID(), sinkId, locator +// .getPartitionId(), allocationContext, shardSink); throw new UnsupportedOperationException(); + } return nchunksout; @@ -394,8 +447,8 @@ final IAllocationContext allocationContext = getAllocationContext(new QueryContext( getQueryId())); - sendOutputChunkReadyMessage(newOutputChunk(queryControllerUUID, - sinkId, allocationContext, sink)); + sendChunkMessage(queryControllerUUID, sinkId, -1/* partitionId */, + allocationContext, sink); /* * Chunks send to the query controller do not keep the query @@ -411,7 +464,9 @@ } /** - * Create an {@link OutputChunk} from some intermediate results. + * Create and send an {@link IChunkMessage} from some intermediate results. + * Various optimizations are employed depending on the amount of data to be + * moved and whether or not the target is this service. * * @param serviceUUID * The {@link UUID} of the {@link IQueryPeer} who is the @@ -420,15 +475,46 @@ * The identifier of the target {@link BOp}. * @param allocationContext * The allocation context within which the {@link ByteBuffer}s - * will be managed for this {@link OutputChunk}. + * will be managed for this {@link ChunkMessageWithNIOPayload}. * @param source * The binding sets to be formatted onto a buffer. * - * @return The {@link OutputChunk}. + * @return The {@link ChunkMessageWithNIOPayload}. + * + * @todo This is basically a factory for creating {@link IChunkMessage}s. + * That factory pattern in combined with the logic to send the message + * so we can do within JVM handoffs. We could break these things apart + * using {@link IChunkMessage#isMaterialized()} to detect inline + * cases. That would let us send out the messages in parallel, which + * could help to cut latency when an operator has a large fan out (in + * scale-out when mapping over shards or nodes). + * + * @todo We probably need to use the {@link DirectBufferPoolAllocator} to + * receive the chunks within the {@link ManagedResourceService} as + * well. + * + * @todo Release the allocations associated with each output chunk once it + * is received by the remote service. + * <p> + * When the query terminates all output chunks targeting any node + * EXCEPT the query controller should be immediately dropped. + * <p> + * If there is an error during query evaluation, then the output + * chunks for the query controller should be immediately dropped. + * <p> + * If the iterator draining the results on the query controller is + * closed, then the output chunks for the query controller should be + * immediately dropped. + * + * @todo There are a few things where the resource must be made available to + * more than one operator evaluation phase. The best examples are + * temporary graphs for parallel closure and large collections of + * graphIds for SPARQL "NAMED FROM DATA SET" extensions. */ - protected OutputChunk newOutputChunk( + protected void sendChunkMessage( final UUID serviceUUID, final int sinkId, + final int partitionId, final IAllocationContext allocationContext, final IBlockingBuffer<IBindingSet[]> source) { @@ -441,6 +527,97 @@ if (source == null) throw new IllegalArgumentException(); + if (source.isEmpty()) + throw new RuntimeException(); + + // The peer to be notified. + final IQueryPeer peerProxy = getQueryPeer(serviceUUID); + + if (peerProxy == null) + throw new RuntimeException("Not found: serviceId=" + serviceUUID); + + // true iff the target is this service (no proxy, no RMI). + final boolean thisService = peerProxy == getQueryEngine(); + + if(thisService) { + /* + * Leave the chunk as Java objects and drop it directly onto the + * query engine. + */ + + final IChunkMessage msg = new BindingSetChunk(getQueryController(), + getQueryId(), sinkId, partitionId, source.iterator()); + + getQueryEngine().bufferReady(msg); + + return; + + } + + /* + * We will be notifying another service (RMI) that a chunk is available. + * + * Note: Depending on how much data it involved, we may move it with the + * RMI message or out of band using NIO. This decision effects how we + * serialize the chunk. + */ + final IChunkMessage msg; + if (source.size() < 100) { + + /* + * FIXME Send payload inline with the RMI message. + */ + +// final byte[] data = SerializerUtil.serialize(obj); +// +// // @todo harmonize serialization and compression and ctors. +// msg = new ThickChunkMessage(getQueryController(), getQueryId(), +// sinkId, partitionId, data); + throw new UnsupportedOperationException(); + + } else + { + + /* + * Marshall the data onto direct ByteBuffer(s) and send a thin + * message by RMI. The receiver will retrieve the data using NIO + * against the ResourceService. + * + * @todo harmonize serialization and compression and ctors. + */ + final List<IAllocation> allocations = moveToNIOBuffers( + allocationContext, source); + + msg = new ChunkMessageWithNIOPayload(getQueryController(), + getQueryId(), sinkId, partitionId, allocations, + getQueryEngine().getResourceService().getAddr()); + + } + + try { + + peerProxy.bufferReady(msg); + + } catch (RemoteException e) { + + throw new RuntimeException(e); + + } + + } + + /** + * Chunk-wise serialization of the data onto allocations. + * @param allocationContext + * @param source + * @return + * + * @todo should be on message per chunk, right? + */ + private List<IAllocation> moveToNIOBuffers( + final IAllocationContext allocationContext, + final IBlockingBuffer<IBindingSet[]> source) { + int nbytes = 0; final List<IAllocation> allocations = new LinkedList<IAllocation>(); @@ -476,135 +653,14 @@ } + return allocations; + } finally { itr.close(); } - return new OutputChunk(getQueryId(), serviceUUID, sinkId, nbytes, - allocations); - } - - protected IQueryPeer getQueryPeer(final UUID serviceUUID) { - if (serviceUUID == null) - throw new IllegalArgumentException(); - - final IQueryPeer queryPeer; - - if (serviceUUID.equals(queryControllerUUID)) { - - // The target is the query controller. - queryPeer = getQueryController(); - - } else { - - // The target is some data service. - queryPeer = getQueryEngine().getQueryPeer(serviceUUID); - - } - - return queryPeer; - - } - - /** - * Overridden to make this visible to the {@link FederatedQueryEngine}. - * <p> - * {@inheritDoc} - */ - @Override - protected void acceptChunk(final IChunkMessage msg) { - - super.acceptChunk(msg); - - } - - /** - * Notify a remote {@link IQueryPeer} that data is available for it. - * - * @todo If the target for the {@link OutputChunk} is this node then just - * drop it onto the {@link QueryEngine}. - * - * FIXME Fast path with inline RMI based transfer for small sets of - * data using a 'think' {@link IChunkMessage}. - */ - protected void sendOutputChunkReadyMessage(final OutputChunk outputChunk) { - - try { - - // The peer to be notified. - final IQueryPeer peerProxy = getQueryPeer(outputChunk.serviceId); - - // The Internet address and port where the peer can read the data - // from this node. - final InetSocketAddress serviceAddr = getQueryEngine() - .getResourceService().getAddr(); - - // FIXME invoke peerProxy.bufferReady(msg) here! -// peerProxy.bufferReady(getQueryController(), serviceAddr, -// getQueryId(), outputChunk.sinkId); - peerProxy.bufferReady(null/*FIXME msg.*/); - - } catch (RemoteException e) { - - throw new RuntimeException(e); - - } - - } - - /** - * A chunk of outputs. - * - * @todo We probably need to use the {@link DirectBufferPoolAllocator} to - * receive the chunks within the {@link ManagedResourceService} as - * well. - * - * @todo Release the allocations associated with each output chunk once it - * is received by the remote service. - * <p> - * When the query terminates all output chunks targeting any node - * EXCEPT the query controller should be immediately dropped. - * <p> - * If there is an error during query evaluation, then the output - * chunks for the query controller should be immediately dropped. - * <p> - * If the iterator draining the results on the query controller is - * closed, then the output chunks for the query controller should be - * immediately dropped. - * - * @todo There are a few things where the resource must be made available to - * more than one operator evaluation phase. The best examples are - * temporary graphs for parallel closure and large collections of - * graphIds for SPARQL "NAMED FROM DATA SET" extensions. - */ - private static class OutputChunk { - - final long queryId; - - final UUID serviceId; - - final int sinkId; - - final int nbytes; - - final List<IAllocation> allocations; - - public OutputChunk(final long queryId, final UUID serviceId, - final int sinkId, final int nbytes, - final List<IAllocation> allocations) { - - this.queryId = queryId; - this.serviceId = serviceId; - this.sinkId = sinkId; - this.nbytes = nbytes; - this.allocations = allocations; - - } - - } - } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ThickChunkMessage.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ThickChunkMessage.java 2010-09-10 19:44:37 UTC (rev 3530) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ThickChunkMessage.java 2010-09-10 20:47:27 UTC (rev 3531) @@ -50,7 +50,7 @@ */ private static final long serialVersionUID = 1L; - final private IQueryClient clientProxy; + final private IQueryClient queryController; final private long queryId; @@ -61,7 +61,7 @@ final private byte[] data; public IQueryClient getQueryController() { - return clientProxy; + return queryController; } public long getQueryId() { @@ -84,11 +84,19 @@ return data.length; } - public ThickChunkMessage(final IQueryClient clientProxy, + /** + * + * @param queryController + * @param queryId + * @param bopId + * @param partitionId + * @param data + */ + public ThickChunkMessage(final IQueryClient queryController, final long queryId, final int bopId, final int partitionId, final byte[] data) { - if (clientProxy == null) + if (queryController == null) throw new IllegalArgumentException(); if (data == null) @@ -98,7 +106,7 @@ if (data.length == 0) throw new IllegalArgumentException(); - this.clientProxy = clientProxy; + this.queryController = queryController; this.queryId = queryId; Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 2010-09-10 19:44:37 UTC (rev 3530) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 2010-09-10 20:47:27 UTC (rev 3531) @@ -492,7 +492,7 @@ this.optional = joinOp.isOptional(); this.variablesToKeep = joinOp.variablesToKeep(); this.context = context; - this.relation = context.getReadRelation(right); + this.relation = context.getRelation(right); this.source = context.getSource(); this.sink = context.getSink(); this.sink2 = context.getSink2(); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/mutation/InsertOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/mutation/InsertOp.java 2010-09-10 19:44:37 UTC (rev 3530) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/mutation/InsertOp.java 2010-09-10 20:47:27 UTC (rev 3531) @@ -41,10 +41,15 @@ import com.bigdata.bop.engine.BOpStats; import com.bigdata.btree.ILocalBTreeView; import com.bigdata.btree.ITupleSerializer; +import com.bigdata.btree.UnisolatedReadWriteIndex; import com.bigdata.btree.keys.IKeyBuilder; +import com.bigdata.journal.IIndexManager; +import com.bigdata.journal.ITx; import com.bigdata.relation.IRelation; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; +import com.bigdata.service.DataService; +import com.bigdata.service.IBigdataFederation; import com.bigdata.striterator.IKeyOrder; /** @@ -181,7 +186,7 @@ predicate = op.getPredicate(); - relation = context.getWriteRelation(op.getRelation()); + relation = context.getRelation(predicate); keyOrder = op.getKeyOrder(); @@ -198,14 +203,8 @@ */ public Void call() throws Exception { - /* - * @todo validate for s/o. Since this goes through a common code - * path, what we really need to test is getMutableLocalIndexView(). - * The rest of the insert operation can be tested against a local - * Journal. - */ - final ILocalBTreeView ndx = context.getMutableLocalIndexView( - relation, keyOrder, context.getPartitionId()); + final ILocalBTreeView ndx = getMutableLocalIndexView(relation, + keyOrder, context.getPartitionId()); final IKeyBuilder keyBuilder = ndx.getIndexMetadata() .getKeyBuilder(); @@ -260,6 +259,95 @@ } + /** + * Return an mutable view of the specified index. + * + * @param <T> + * The generic type of the elements in the relation. + * @param relation + * The relation. + * @param keyOrder + * The key order for that index. + * @param partitionId + * The partition identifier and <code>-1</code> unless + * running against an {@link IBigdataFederation}. + * + * @return The mutable view of the index. + * + * @throws UnsupportedOperationException + * if there is an attempt to read on an index partition when + * the database is not an {@link IBigdataFederation} or when + * the database is an {@link IBigdataFederation} unless the + * index partition was specified. + * + * @todo validate for standalone. probably needs to be wrapped as an + * {@link UnisolatedReadWriteIndex} whcih migtht be done by how we + * get the relation view. + * + * @todo validate for s/o. Since this goes through a common code path, + * what we really need to test is getMutableLocalIndexView(). The + * rest of the insert operation can be tested against a local + * Journal. + * + * FIXME This must obtain the appropriate lock for the mutable + * index in scale-out. + */ + public <T> ILocalBTreeView getMutableLocalIndexView( + final IRelation<T> relation, final IKeyOrder<T> keyOrder, + final int partitionId) { + + if(true) { + /* + * FIXME Concurrency control and locks. Maybe submit as an + * AbstractTask? + */ + throw new UnsupportedOperationException(); + } + + final IBigdataFederation<?> fed = context.getFederation(); + final IIndexManager indexManager = context.getIndexManager(); + final long writeTimestamp = predicate.getTimestamp(); + + final String namespace = relation.getNamespace(); + + final ILocalBTreeView ndx; + + if (partitionId == -1) { + + if (fed != null) { + // This is scale-out so the partition identifier is required. + throw new UnsupportedOperationException(); + } + + // The index is not partitioned. + ndx = (ILocalBTreeView) indexManager.getIndex(namespace + "." + + keyOrder.getIndexName(), writeTimestamp); + + } else { + + if (fed == null) { + /* + * This is not scale-out so index partitions are not + * supported. + */ + throw new UnsupportedOperationException(); + } + + // The name of the desired index partition. + final String name = DataService.getIndexPartitionName(namespace + + "." + keyOrder.getIndexName(), partitionId); + + // MUST be a local index view. + ndx = (ILocalBTreeView) indexManager.getIndex(name, + writeTimestamp); + + } + + return ndx; + + } + + } @@ -280,5 +368,5 @@ return BOpEvaluationContext.SHARDED; } - + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ndx/SampleLocalShard.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ndx/SampleLocalShard.java 2010-09-10 19:44:37 UTC (rev 3530) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ndx/SampleLocalShard.java 2010-09-10 20:47:27 UTC (rev 3531) @@ -64,7 +64,7 @@ final IPredicate<E> pred = pred(); - final IRelation<E> view = context.getReadRelation(pred); + final IRelation<E> view = context.getRelation(pred); final IAccessPath<E> accessPath = view.getAccessPath(pred); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/bset/TestConditionalRoutingOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/bset/TestConditionalRoutingOp.java 2010-09-10 19:44:37 UTC (rev 3530) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/bset/TestConditionalRoutingOp.java 2010-09-10 20:47:27 UTC (rev 3531) @@ -49,7 +49,6 @@ import com.bigdata.bop.engine.MockRunningQuery; import com.bigdata.bop.engine.TestQueryEngine; import com.bigdata.bop.solutions.DistinctBindingSetOp; -import com.bigdata.journal.ITx; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; import com.bigdata.relation.accesspath.ThickAsynchronousIterator; @@ -210,11 +209,8 @@ final IBlockingBuffer<IBindingSet[]> sink2 =... [truncated message content] |
From: <tho...@us...> - 2010-09-11 22:53:49
|
Revision: 3532 http://bigdata.svn.sourceforge.net/bigdata/?rev=3532&view=rev Author: thompsonbry Date: 2010-09-11 22:53:40 +0000 (Sat, 11 Sep 2010) Log Message: ----------- Further reorganization of the federated query engine and its use of buffers. There are now thick (payload including with RMI) and thin (RMI message with payload via NIO over the resource service) messages for moving chunks around during distributed query processing. There are bare bones unit tests for these as well. (The ResourceServer does not actually use NIO yet, but it can be optimized later. Also, we are not yet applying compression suitable for binding sets, but again that can be an optimization.) Now that RMI messages and payload transfers are more or less in place, I am going to work through some unit tests of distributed query evaluation. To do that I still need to reconcile the concept of a "query buffer" where the final solutions are written with the new model for moving data around. I think that the query buffer will no longer be privledged (it used to be a proxy object for a buffer on the client). Instead, scale-out will require an operator at the top of the query plan whose evaluation context is the query controller. The mere presence of an operator which copies its inputs to its outputs whose evaluation context is the query controller is sufficient to do the trick. For practical purposes, this can be a SliceOp, since that already must run in the query controller context. If an offset/limit are not specified, then they can be set to 0L and MAX_LONG on the SliceOp which has the effect of turning it into a NOP (unless you are visiting an unbelievable #of results). Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IChunkMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ThickChunkMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/io/DirectBufferPoolAllocator.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/service/ManagedResourceService.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/striterator/Dechunkerator.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestAll.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/mutation/TestDelete.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/mutation/TestInsert.java Added Paths: ----------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/NIOChunkMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestNIOChunkMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestThickChunkMessage.java Removed Paths: ------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ChunkMessageWithNIOPayload.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java 2010-09-10 20:47:27 UTC (rev 3531) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java 2010-09-11 22:53:40 UTC (rev 3532) @@ -12,7 +12,7 @@ * be consumed by some {@link BOp} in a specific query (this is only used in * query evaluation for the standalone database). */ -public class BindingSetChunk implements IChunkMessage { +public class BindingSetChunk<E> implements IChunkMessage<E> { /** The query controller. */ private final IQueryClient queryController; @@ -35,7 +35,7 @@ /** * The binding sets to be consumed by that {@link BOp}. */ - private IAsynchronousIterator<IBindingSet[]> source; + private IAsynchronousIterator<E[]> source; public IQueryClient getQueryController() { return queryController; @@ -59,7 +59,7 @@ public BindingSetChunk(final IQueryClient queryController, final long queryId, final int bopId, final int partitionId, - final IAsynchronousIterator<IBindingSet[]> source) { + final IAsynchronousIterator<E[]> source) { if (queryController == null) throw new IllegalArgumentException(); @@ -89,8 +89,12 @@ public void materialize(FederatedRunningQuery runningQuery) { // NOP } + + public void release() { + // NOP + } - public IAsynchronousIterator<IBindingSet[]> iterator() { + public IAsynchronousIterator<E[]> iterator() { return source; } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IChunkMessage.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IChunkMessage.java 2010-09-10 20:47:27 UTC (rev 3531) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IChunkMessage.java 2010-09-11 22:53:40 UTC (rev 3532) @@ -15,8 +15,18 @@ * processing. There are several implementations of this interface supporting * same-JVM messages, thick RMI messages, and RMI messages where the payload is * materialized using NIO transfers from the {@link ResourceService}. + * + * @param <E> + * The generic type of the elements in the chunk (binding sets, + * elements from a relation, etc). + * + * @todo Compressed representations of binding sets with the ability to read + * them in place or materialize them onto the java heap. The + * representation should be amenable to processing in C since we want to + * use them on GPUs as well. See {@link IChunkMessage} and perhaps + * {@link IRaba}. */ -public interface IChunkMessage { +public interface IChunkMessage<E> { /** The proxy for the query controller. */ IQueryClient getQueryController(); @@ -44,6 +54,11 @@ void materialize(FederatedRunningQuery runningQuery); /** + * Discard the materialized data. + */ + void release(); + + /** * Visit the binding sets in the chunk. * * @todo we do not need to use {@link IAsynchronousIterator} any more. This @@ -67,7 +82,10 @@ * source for processing. For selective operators, those chunks can be * combined before we execute the operator. For unselective operators, * we are going to run over all the data anyway. + * + * @throws IllegalStateException + * if the payload is not materialized. */ - IAsynchronousIterator<IBindingSet[]> iterator(); + IAsynchronousIterator<E[]> iterator(); } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java 2010-09-10 20:47:27 UTC (rev 3531) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java 2010-09-11 22:53:40 UTC (rev 3532) @@ -5,6 +5,7 @@ import java.util.UUID; import com.bigdata.bop.BOp; +import com.bigdata.bop.IBindingSet; import com.bigdata.service.IService; /** @@ -46,6 +47,6 @@ * @throws UnsupportedOperationException * unless running in scale-out. */ - void bufferReady(IChunkMessage msg) throws RemoteException; + void bufferReady(IChunkMessage<IBindingSet> msg) throws RemoteException; } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java 2010-09-10 20:47:27 UTC (rev 3531) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java 2010-09-11 22:53:40 UTC (rev 3532) @@ -45,6 +45,7 @@ import com.bigdata.bop.IBindingSet; import com.bigdata.bop.IPredicate; import com.bigdata.bop.bset.Union; +import com.bigdata.bop.fed.FederatedQueryEngine; import com.bigdata.btree.BTree; import com.bigdata.btree.IndexSegment; import com.bigdata.btree.view.FusedView; @@ -411,11 +412,6 @@ /** * The currently executing queries. - * - * @todo DEADLINE: There should be a data structure representing - * {@link RunningQuery} having deadlines so we can - * {@link RunningQuery#cancel(boolean)} queries when their deadline - * expires. */ final ConcurrentHashMap<Long/* queryId */, RunningQuery> runningQueries = new ConcurrentHashMap<Long, RunningQuery>(); @@ -514,54 +510,39 @@ * if the sink has not been taken, e.g., by combining the chunk into * the same target ByteBuffer, or when we add the chunk to the * RunningQuery.] - * - * @todo SCALEOUT: High volume query operators must demand that their inputs - * are materialized before they can begin evaluation. Scaleout - * therefore requires a separate queue which looks at the metadata - * concerning chunks available on remote nodes for an operator which - * will run on this node and then demands the data either when the - * predecessors in the pipeline are done (operator at once evaluation) - * or when sufficient data are available to run the operator (mega - * chunk pipelining). Once the data are locally materialized, the - * operator may be queued for evaluation. */ private class QueryEngineTask implements Runnable { public void run() { - try { - System.err.println("QueryEngine running: "+this); - while (true) { + System.err.println("QueryEngine running: " + this); + while (true) { + try { final RunningQuery q = priorityQueue.take(); final long queryId = q.getQueryId(); if (q.isCancelled()) continue; - final IChunkMessage chunk = q.chunksIn.poll(); - if (chunk == null) { - // not expected, but can't do anything without a chunk. - if (log.isDebugEnabled()) - log.debug("Dropping chunk: queryId=" + queryId); - continue; - } + final IChunkMessage<IBindingSet> chunk = q.chunksIn.poll(); if (log.isTraceEnabled()) log.trace("Accepted chunk: queryId=" + queryId + ", bopId=" + chunk.getBOpId()); + // create task. try { - // create task. final FutureTask<?> ft = q.newChunkTask(chunk); // execute task. localIndexManager.getExecutorService().execute(ft); } catch (RejectedExecutionException ex) { - // shutdown of the pool (should be an unbounded pool). + // shutdown of the pool (should be an unbounded + // pool). log.warn("Dropping chunk: queryId=" + queryId); continue; - } catch (Throwable ex) { - // log and continue - log.error(ex, ex); - continue; } + } catch (InterruptedException e) { + log.warn("Interrupted."); + return; + } catch (Throwable ex) { + // log and continue + log.error(ex, ex); + continue; } - } catch (InterruptedException e) { - log.warn("Interrupted."); - return; } } } // QueryEngineTask @@ -579,7 +560,7 @@ * @throws IllegalStateException * if the chunk is not materialized. */ - void acceptChunk(final IChunkMessage chunk) { + void acceptChunk(final IChunkMessage<IBindingSet> chunk) { if (chunk == null) throw new IllegalArgumentException(); @@ -625,9 +606,25 @@ } + // hook for subclasses. + didShutdown(); + + // stop the query engine. + final Future<?> f = engineFuture.get(); + if (f != null) + f.cancel(true/* mayInterruptIfRunning */); + } /** + * Hook is notified by {@link #shutdown()} when all running queries have + * terminated. + */ + protected void didShutdown() { + + } + + /** * Do not accept new queries and halt any running binding set chunk tasks. */ public void shutdownNow() { @@ -686,7 +683,7 @@ } - public void bufferReady(IChunkMessage msg) { + public void bufferReady(IChunkMessage<IBindingSet> msg) { throw new UnsupportedOperationException(); @@ -844,6 +841,10 @@ * @todo Could return a data structure which encapsulates the query results * and could allow multiple results from a query, e.g., one per step * in a program. + * + * @deprecated This is going away. + * + * @see FederatedQueryEngine#newQueryBuffer(BindingSetPipelineOp) */ protected IBlockingBuffer<IBindingSet[]> newQueryBuffer( final BindingSetPipelineOp query) { Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-10 20:47:27 UTC (rev 3531) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-11 22:53:40 UTC (rev 3532) @@ -48,13 +48,15 @@ import com.bigdata.bop.BOp; import com.bigdata.bop.BOpContext; +import com.bigdata.bop.BOpEvaluationContext; import com.bigdata.bop.BOpUtility; import com.bigdata.bop.BindingSetPipelineOp; import com.bigdata.bop.IBindingSet; import com.bigdata.bop.NoSuchBOpException; +import com.bigdata.bop.bset.CopyBindingSetOp; +import com.bigdata.bop.solutions.SliceOp; import com.bigdata.journal.IIndexManager; import com.bigdata.journal.ITx; -import com.bigdata.journal.TimestampUtility; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; import com.bigdata.service.IBigdataFederation; @@ -90,24 +92,15 @@ final private long queryId; // /** -// * The timestamp or transaction identifier against which the query is -// * reading. -// */ -// final private long readTimestamp; -// -// /** -// * The timestamp or transaction identifier against which the query is -// * writing. -// */ -// final private long writeTimestamp; - -// /** // * The timestamp when the query was accepted by this node (ms). // */ // final private long begin; + /** * The query deadline. The value is the system clock time in milliseconds * when the query is due and {@link Long#MAX_VALUE} if there is no deadline. + * In order to have a guarantee of a consistent clock, the deadline is + * interpreted by the query controller. */ final private AtomicLong deadline = new AtomicLong(Long.MAX_VALUE); @@ -132,8 +125,6 @@ */ final private IQueryClient clientProxy; -// /** The query iff materialized on this node. */ -// final private AtomicReference<BOp> queryRef; /** The query. */ final private BOp query; @@ -141,7 +132,12 @@ * The buffer used for the overall output of the query pipeline. * * FIXME SCALEOUT: This should only exist on the query controller. Other - * nodes will send {@link IChunkMessage}s to the query controller. + * nodes will send {@link IChunkMessage}s to the query controller. s/o will + * use an operator with {@link BOpEvaluationContext#CONTROLLER} in order to + * ensure that the results are transferred to the query controller. When a + * {@link SliceOp} is used, this is redundant. The operator in other cases + * can be a {@link CopyBindingSetOp} whose {@link BOpEvaluationContext} has + * been overridden. */ final private IBlockingBuffer<IBindingSet[]> queryBuffer; @@ -218,7 +214,7 @@ * Note: This is package private so it will be visible to the * {@link QueryEngine}. */ - final/* private */BlockingQueue<IChunkMessage> chunksIn = new LinkedBlockingDeque<IChunkMessage>(); + final/* private */BlockingQueue<IChunkMessage<IBindingSet>> chunksIn = new LinkedBlockingDeque<IChunkMessage<IBindingSet>>(); /** * Set the query deadline. The query will be cancelled when the deadline is @@ -286,45 +282,13 @@ } + /** + * Return the operator tree for this query. + */ public BOp getQuery() { return query; } -// /** -// * Return the operator tree for this query. If query processing is -// * distributed and the query has not been materialized on this node, then it -// * is materialized now. -// * -// * @return The query. -// */ -// public BOp getQuery() { -// -// if (queryRef.get() == null) { -// -// synchronized (queryRef) { -// -// if (queryRef.get() == null) { -// -// try { -// -// queryRef.set(clientProxy.getQuery(queryId)); -// -// } catch (RemoteException e) { -// -// throw new RuntimeException(e); -// -// } -// -// } -// -// } -// -// } -// -// return queryRef.get(); -// -// } - /** * Return <code>true</code> iff this is the query controller. */ @@ -361,8 +325,6 @@ * if the <i>writeTimestamp</i> is neither * {@link ITx#UNISOLATED} nor a read-write transaction * identifier. - * - * @todo is queryBuffer required? should it be allocated from the top bop? */ public RunningQuery(final QueryEngine queryEngine, final long queryId, // final long begin, @@ -390,42 +352,6 @@ this.statsMap = controller ? new ConcurrentHashMap<Integer, BOpStats>() : null; -// /* -// * @todo when making a per-bop annotation, queries must obtain a tx for -// * each timestamp up front on the controller and rewrite the bop to hold -// * the tx until it is done. -// * -// * @todo This is related to how we handle sequences of steps, parallel -// * steps, closure of steps, and join graphs. Those operations need to be -// * evaluated on the controller. We will have to model the relationship -// * between the subquery and the query in order to terminate the subquery -// * when the query halts and to terminate the query if the subquery -// * fails. -// * -// * @todo Closure operations must rewrite the query to update the -// * annotations. Each pass in a closure needs to be its own "subquery" -// * and will need to have a distinct queryId. -// */ -// final Long timestamp = query -// .getProperty(BOp.Annotations.TIMESTAMP); -// -// // @todo remove default when elevating to per-writable bop annotation. -// final long writeTimestamp = query.getProperty( -// BOp.Annotations.WRITE_TIMESTAMP, ITx.UNISOLATED); -// -// if (readTimestamp == null) -// throw new IllegalArgumentException(); -// -// if (readTimestamp.longValue() == ITx.UNISOLATED) -// throw new IllegalArgumentException(); -// -// if (TimestampUtility.isReadOnly(writeTimestamp)) -// throw new IllegalArgumentException(); -// -// this.readTimestamp = readTimestamp; -// -// this.writeTimestamp = writeTimestamp; - this.timeout = query.getProperty(BOp.Annotations.TIMEOUT, BOp.Annotations.DEFAULT_TIMEOUT); @@ -463,8 +389,9 @@ /* * Note: The partitionId will always be -1 in scale-up. */ - final BindingSetChunk chunk = new BindingSetChunk(clientProxy, queryId, - sinkId, -1/* partitionId */, sink.iterator()); + final BindingSetChunk<IBindingSet> chunk = new BindingSetChunk<IBindingSet>( + clientProxy, queryId, sinkId, -1/* partitionId */, sink + .iterator()); queryEngine.acceptChunk(chunk); @@ -475,12 +402,12 @@ /** * Make a chunk of binding sets available for consumption by the query. * <p> - * Note: this is invoked by {@link QueryEngine#add(BindingSetChunk)}. + * Note: this is invoked by {@link QueryEngine#acceptChunk(IChunkMessage)} * * @param msg * The chunk. */ - protected void acceptChunk(final IChunkMessage msg) { + protected void acceptChunk(final IChunkMessage<IBindingSet> msg) { if (msg == null) throw new IllegalArgumentException(); @@ -506,7 +433,7 @@ * * @todo this should reject multiple invocations for a given query instance. */ - public void startQuery(final IChunkMessage chunk) { + public void startQuery(final IChunkMessage<IBindingSet> chunk) { if (!controller) throw new UnsupportedOperationException(); if (chunk == null) @@ -776,7 +703,7 @@ * A chunk to be consumed. */ @SuppressWarnings("unchecked") - protected FutureTask<Void> newChunkTask(final IChunkMessage chunk) { + protected FutureTask<Void> newChunkTask(final IChunkMessage<IBindingSet> chunk) { /* * Look up the BOp in the index, create the BOpContext for that BOp, and * return the value returned by BOp.eval(context). @@ -792,6 +719,9 @@ * @todo evaluation of element[] pipelines needs to use pretty much * the same code, but it needs to be typed for E[] rather than * IBindingSet[]. + * + * @todo evaluation of Monet style BATs would also operate under + * different assumptions, closer to those of an element[]. */ throw new UnsupportedOperationException(bop.getClass().getName()); } @@ -961,16 +891,4 @@ } -// public long getReadTimestamp() { -// -// return readTimestamp; -// -// } -// -// public long getWriteTimestamp() { -// -// return writeTimestamp; -// -// } - } Deleted: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ChunkMessageWithNIOPayload.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ChunkMessageWithNIOPayload.java 2010-09-10 20:47:27 UTC (rev 3531) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ChunkMessageWithNIOPayload.java 2010-09-11 22:53:40 UTC (rev 3532) @@ -1,244 +0,0 @@ -/** - -Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. - -Contact: - SYSTAP, LLC - 4501 Tower Road - Greensboro, NC 27410 - lic...@bi... - -This program is free software; you can redistribute it and/or modify -it under the terms of the GNU General Public License as published by -the Free Software Foundation; version 2 of the License. - -This program is distributed in the hope that it will be useful, -but WITHOUT ANY WARRANTY; without even the implied warranty of -MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -GNU General Public License for more details. - -You should have received a copy of the GNU General Public License -along with this program; if not, write to the Free Software -Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA -*/ -/* - * Created on Sep 10, 2010 - */ - -package com.bigdata.bop.fed; - -import java.io.Serializable; -import java.net.InetSocketAddress; -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.UUID; - -import com.bigdata.bop.IBindingSet; -import com.bigdata.bop.engine.IChunkMessage; -import com.bigdata.bop.engine.IQueryClient; -import com.bigdata.io.DirectBufferPoolAllocator.IAllocation; -import com.bigdata.io.DirectBufferPoolAllocator.IAllocationContext; -import com.bigdata.relation.accesspath.IAsynchronousIterator; -import com.bigdata.service.ResourceService; - -/** - * An {@link IChunkMessage} where the payload is made available to the receiving - * service using an NIO transfer against the sender's {@link ResourceService}. - * This is suitable for moving large blocks of data during query evaluation. - * - * @author <a href="mailto:tho...@us...">Bryan Thompson</a> - * @version $Id$ - */ -public class ChunkMessageWithNIOPayload implements IChunkMessage, Serializable { - - /** - * - */ - private static final long serialVersionUID = 1L; - - /** - * Metadata about an allocation to be retrieved from the sender's - * {@link ResourceService}. - */ - private final class A implements Serializable { - - /** - * - */ - private static final long serialVersionUID = 1L; - - /** - * The identifier of the resource on the sender's - * {@link ResourceService}. - */ - private final UUID bufferId; - - /** - * The size of that resource in bytes. - */ - private final int nbytes; - - /** - * - * @param bufferId - * The identifier of the resource on the sender's - * {@link ResourceService}. - * @param nbytes - * The size of that resource in bytes. - */ - public A(final UUID bufferId, final int nbytes) { - this.bufferId = bufferId; - this.nbytes = nbytes; - } - } - - final private IQueryClient queryController; - - final private long queryId; - - final private int bopId; - - final private int partitionId; - - final private int nbytes; - - /** - * Note: Even when we send one message per chunk, we can still have a list - * of {@link IAllocation}s if the chunk did not get formatted onto a single - * {@link IAllocation}. - */ - final private A[] allocations; - - /** - * The Internet address and port where the receiver can fetch the payload - * using the sender's {@link ResourceService}. - */ - final private InetSocketAddress addr; - - public IQueryClient getQueryController() { - return queryController; - } - - public long getQueryId() { - return queryId; - } - - public int getBOpId() { - return bopId; - } - - public int getPartitionId() { - return partitionId; - } - - /** The #of bytes of data which are available for that operator. */ - public int getBytesAvailable() { - return nbytes; - } - - /** - * The Internet address and port of a {@link ResourceService} from which the - * receiver may demand the data. - */ - public InetSocketAddress getServiceAddr() { - return addr; - } - - /** - * - * @param queryController - * @param queryId - * @param sinkId - * @param partitionId - * @param allocations - * The ordered list of {@link IAllocation}s comprising the chunk. - * @param addr - * The Internet address and port where the receiver can fetch the - * payload using the sender's {@link ResourceService}. - */ - public ChunkMessageWithNIOPayload(final IQueryClient queryController, - final long queryId, final int sinkId, final int partitionId, - final List<IAllocation> allocations, final InetSocketAddress addr) { - - if (queryController == null) - throw new IllegalArgumentException(); - - if (allocations == null) - throw new IllegalArgumentException(); - - if (addr == null) - throw new IllegalArgumentException(); - - this.queryController = queryController; - this.queryId = queryId; - this.bopId = sinkId; - this.partitionId = partitionId; - final int n = allocations.size(); - this.allocations = new A[n]; - int i = 0; - int nbytes = 0; - final Iterator<IAllocation> itr = allocations.iterator(); - while (itr.hasNext()) { - final IAllocation alloc = itr.next(); - final int len = alloc.getSlice().capacity(); - this.allocations[i++] = new A(alloc.getId(), len); - nbytes += len; - } - this.nbytes = nbytes; - this.addr = addr; - - } - - public boolean isMaterialized() { - return materialized; - } - private volatile boolean materialized = false; - - /** - * - * FIXME unit tests for materializing and visiting the chunk. - */ - synchronized public void materialize(FederatedRunningQuery runningQuery) { - - if (materialized) - return; - - final AllocationContextKey key = new ShardContext(queryId, bopId, - partitionId); - - final IAllocationContext allocationContext = runningQuery - .getAllocationContext(key); - - final ResourceService resourceService = runningQuery.getQueryEngine() - .getResourceService(); - -// for (A a : allocations) { -// -// /* -// * FIXME harmonize an IAllocation[] with a ByteBuffer for the {@link -// * ResourceService}. The problem is that an object to be sent across -// * the wire may span multiple ByteBuffers. -// */ -// final ByteBuffer tmp = allocationContext.alloc(a.nbytes); -// -// new ResourceService.ReadBufferTask(addr, a.bufferId, tmp); -// -// } - - throw new UnsupportedOperationException(); - - } - - public IAsynchronousIterator<IBindingSet[]> iterator() { - - if (!isMaterialized()) - throw new UnsupportedOperationException(); - - // TODO Auto-generated method stub - throw new UnsupportedOperationException(); - - } - -} Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java 2010-09-10 20:47:27 UTC (rev 3531) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java 2010-09-11 22:53:40 UTC (rev 3532) @@ -31,7 +31,13 @@ import java.nio.ByteBuffer; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Future; +import java.util.concurrent.FutureTask; +import java.util.concurrent.PriorityBlockingQueue; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.log4j.Logger; + import com.bigdata.bop.BindingSetPipelineOp; import com.bigdata.bop.IBindingSet; import com.bigdata.bop.engine.IChunkMessage; @@ -41,7 +47,6 @@ import com.bigdata.bop.engine.QueryEngine; import com.bigdata.bop.engine.RunningQuery; import com.bigdata.bop.solutions.SliceOp; -import com.bigdata.btree.raba.IRaba; import com.bigdata.journal.IIndexManager; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; @@ -51,6 +56,7 @@ import com.bigdata.service.IDataService; import com.bigdata.service.ManagedResourceService; import com.bigdata.service.ResourceService; +import com.bigdata.util.InnerCause; /** * An {@link IBigdataFederation} aware {@link QueryEngine}. @@ -59,18 +65,13 @@ * @version $Id: FederatedQueryEngine.java 3508 2010-09-05 17:02:34Z thompsonbry * $ * - * @todo buffer management for s/o bindingSet[] movement - * - * @todo buffer management for s/o DHT element[] movement - * - * @todo Compressed representations of binding sets with the ability to read - * them in place or materialize them onto the java heap. The - * representation should be amenable to processing in C since we want to - * use them on GPUs as well. See {@link IChunkMessage} and perhaps - * {@link IRaba}. + * @todo DEFAULT_GRAPH_QUERY: buffer management for s/o DHT element[] movement */ public class FederatedQueryEngine extends QueryEngine { + private final static transient Logger log = Logger + .getLogger(FederatedQueryEngine.class); + /** * The {@link IBigdataFederation} iff running in scale-out. * <p> @@ -88,6 +89,17 @@ private final ManagedResourceService resourceService; /** + * A priority queue of {@link IChunkMessage}s which needs to have their data + * materialized so an operator can consume those data on this node. + */ + final private PriorityBlockingQueue<IChunkMessage<?>> chunkMaterializationQueue = new PriorityBlockingQueue<IChunkMessage<?>>(); + + /** + * The {@link Future} for the task draining the {@link #chunkMaterializationQueue}. + */ + private final AtomicReference<FutureTask<Void>> materializeChunksFuture = new AtomicReference<FutureTask<Void>>(); + + /** * Constructor used on a {@link DataService} (a query engine peer). * * @param dataService @@ -101,7 +113,44 @@ } + @Override + public UUID getServiceUUID() { + + return fed.getServiceUUID(); + + } + + @Override + public IBigdataFederation<?> getFederation() { + + return fed; + + } + /** + * The service used to expose {@link ByteBuffer}s and managed index + * resources for transfer to remote services in support of distributed query + * evaluation. + */ + public ManagedResourceService getResourceService() { + + return resourceService; + + } + + /** + * Overridden to strengthen the return type. + * <p> + * {@inheritDoc} + */ + @Override + protected FederatedRunningQuery getRunningQuery(final long queryId) { + + return (FederatedRunningQuery) super.getRunningQuery(queryId); + + } + + /** * Constructor used on a non-{@link DataService} node to expose a query * controller. Since the query controller is not embedded within a data * service it needs to provide its own {@link ResourceService} and local @@ -131,43 +180,121 @@ } + /** + * {@inheritDoc} + * <p> + * Extended to also initialize a thread which will materialize + * {@link IChunkMessage} for consumption by this node. + * + * @todo ANALYTIC_QUERY: {@link IChunkMessage} are dropped onto a queue and + * materialized in order of arrival. This works fine for low latency + * pipelined query evaluation. + * <p> + * For analytic query, we (a) manage the #of high volume operators + * which run concurrently, presumably based on their demands on + * memory; and (b) model the chunks available before they are + * materialized locally such that (c) they can be materialized on + * demand (flow control); and (d) we can run the operator when there + * are sufficient chunks available without taking on too much data. + * <p> + * This requires a separate queue for executing high volume operators + * and also separate consideration of when chunks available on remote + * nodes should be materialized. + */ @Override - public UUID getServiceUUID() { + public void init() { - return fed.getServiceUUID(); + final FutureTask<Void> ft = new FutureTask<Void>( + new MaterializeChunksTask(), (Void) null); + + if (materializeChunksFuture.compareAndSet(null/* expect */, ft)) { + + getIndexManager().getExecutorService().execute(ft); + + } else { + + throw new IllegalStateException("Already running"); + + } } + /** + * {@inheritDoc} + * <p> + * Extended to stop materializing chunks once all running queries are done. + */ @Override - public IBigdataFederation<?> getFederation() { + protected void didShutdown() { + + // stop materializing chunks. + final Future<?> f = materializeChunksFuture.get(); + if (f != null) + f.cancel(true/* mayInterruptIfRunning */); - return fed; - } - - /** - * The service used to expose {@link ByteBuffer}s and managed index - * resources for transfer to remote services in support of distributed query - * evaluation. - */ - public ManagedResourceService getResourceService() { - return resourceService; - - } - /** - * Overridden to strengthen the return type. + * {@inheritDoc} * <p> - * {@inheritDoc} + * Extended to stop materializing chunks. */ @Override - protected FederatedRunningQuery getRunningQuery(final long queryId) { + public void shutdownNow() { + + // stop materializing chunks. + final Future<?> f = materializeChunksFuture.get(); + if (f != null) + f.cancel(true/* mayInterruptIfRunning */); - return (FederatedRunningQuery) super.getRunningQuery(queryId); + super.shutdownNow(); } + /** + * Runnable materializes chunks and makes them available for further + * processing. + */ + private class MaterializeChunksTask implements Runnable { + public void run() { + while (true) { + try { + final IChunkMessage<?> c = chunkMaterializationQueue.take(); + final long queryId = c.getQueryId(); + final FederatedRunningQuery q = getRunningQuery(queryId); + if (q.isCancelled()) + continue; + final IChunkMessage<?> msg = chunkMaterializationQueue + .poll(); + try { + msg.materialize(q); + /* + * @todo The type warning here is because the rest of + * the API does not know what to do with messages for + * chunks other than IBindingSet[], e.g., IElement[], + * etc. + */ + FederatedQueryEngine.this + .bufferReady((IChunkMessage) msg); + } catch(Throwable t) { + if(InnerCause.isInnerCause(t, InterruptedException.class)) { + log.warn("Interrupted."); + return; + } + throw new RuntimeException(t); + } + } catch (InterruptedException e) { + log.warn("Interrupted."); + return; + } catch (Throwable ex) { + // log and continue + log.error(ex, ex); + continue; + } + } + } + } // MaterializeChunksTask + public void declareQuery(final IQueryDecl queryDecl) { final long queryId = queryDecl.getQueryId(); @@ -179,7 +306,7 @@ } @Override - public void bufferReady(final IChunkMessage msg) { + public void bufferReady(final IChunkMessage<IBindingSet> msg) { if (msg == null) throw new IllegalArgumentException(); @@ -200,12 +327,6 @@ } else { /* - * FIXME SCALEOUT: We need to model the chunks available before they - * are materialized locally such that (a) they can be materialized - * on demand (flow control); and (b) we can run the operator when - * there are sufficient chunks available without taking on too much - * data. [For the sort term, they can be dropped onto a queue and - * materialized in order of arrival.] */ throw new UnsupportedOperationException("FIXME"); @@ -248,6 +369,8 @@ * normally. Also pay attention when the client closes the * {@link IAsynchronousIterator} from which it is draining solutions * early. + * + * @deprecated This is going away. */ @Override protected IBlockingBuffer<IBindingSet[]> newQueryBuffer( Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java 2010-09-10 20:47:27 UTC (rev 3531) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java 2010-09-11 22:53:40 UTC (rev 3532) @@ -29,10 +29,7 @@ import java.nio.ByteBuffer; import java.rmi.RemoteException; -import java.util.Arrays; import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; import java.util.Map; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; @@ -47,9 +44,7 @@ import com.bigdata.bop.engine.IQueryClient; import com.bigdata.bop.engine.IQueryPeer; import com.bigdata.bop.engine.RunningQuery; -import com.bigdata.io.DirectBufferPoolAllocator; -import com.bigdata.io.SerializerUtil; -import com.bigdata.io.DirectBufferPoolAllocator.IAllocation; +import com.bigdata.io.DirectBufferPool; import com.bigdata.io.DirectBufferPoolAllocator.IAllocationContext; import com.bigdata.mdi.PartitionLocator; import com.bigdata.relation.accesspath.BlockingBuffer; @@ -58,7 +53,6 @@ import com.bigdata.relation.accesspath.IBuffer; import com.bigdata.resources.ResourceManager; import com.bigdata.service.IBigdataFederation; -import com.bigdata.service.ManagedResourceService; import com.bigdata.service.ResourceService; import com.bigdata.striterator.IKeyOrder; @@ -245,7 +239,7 @@ * {@inheritDoc} */ @Override - protected void acceptChunk(final IChunkMessage msg) { + protected void acceptChunk(final IChunkMessage<IBindingSet> msg) { super.acceptChunk(msg); @@ -475,11 +469,11 @@ * The identifier of the target {@link BOp}. * @param allocationContext * The allocation context within which the {@link ByteBuffer}s - * will be managed for this {@link ChunkMessageWithNIOPayload}. + * will be managed for this {@link NIOChunkMessage}. * @param source * The binding sets to be formatted onto a buffer. * - * @return The {@link ChunkMessageWithNIOPayload}. + * @return The {@link NIOChunkMessage}. * * @todo This is basically a factory for creating {@link IChunkMessage}s. * That factory pattern in combined with the logic to send the message @@ -489,10 +483,6 @@ * could help to cut latency when an operator has a large fan out (in * scale-out when mapping over shards or nodes). * - * @todo We probably need to use the {@link DirectBufferPoolAllocator} to - * receive the chunks within the {@link ManagedResourceService} as - * well. - * * @todo Release the allocations associated with each output chunk once it * is received by the remote service. * <p> @@ -506,10 +496,32 @@ * closed, then the output chunks for the query controller should be * immediately dropped. * - * @todo There are a few things where the resource must be made available to - * more than one operator evaluation phase. The best examples are - * temporary graphs for parallel closure and large collections of - * graphIds for SPARQL "NAMED FROM DATA SET" extensions. + * @todo There are a few things for which the resource must be made + * available to more than one operator evaluation phase. The best + * examples are temporary graphs for parallel closure and large + * collections of graphIds for SPARQL "NAMED FROM DATA SET" + * extensions. + * + * @todo Rethink the multiplicity relationship between chunks output from an + * operator, chunks output from mapping the operator over shards or + * nodes, RMI messages concerning buffers available for the sink + * operator on the various nodes, and the #of allocations per RMI + * message on both the sender and the receiver. + * <p> + * I am pretty sure that none of these are strongly coupled, e.g., + * they are not 1:1. Some stages can combine chunks. Multiple + * allocations could be required on either the sender or the receiver + * purely due to where the slices fall on the backing direct + * {@link ByteBuffer}s in the {@link DirectBufferPool} and the sender + * and receiver do not need to use the same allocation context or have + * the same projection of slices onto the backing buffers. + * <p> + * The one thing which is critical is that the query controller is + * properly informed of the #of chunks made available to an operator + * and consumed by that operator, that those reports must be in the + * same units, and that the reports must be delivered back to the + * query controller in a manner which does not transiently violate the + * termination conditions of the query. */ protected void sendChunkMessage( final UUID serviceUUID, @@ -540,13 +552,15 @@ final boolean thisService = peerProxy == getQueryEngine(); if(thisService) { + /* * Leave the chunk as Java objects and drop it directly onto the * query engine. */ - final IChunkMessage msg = new BindingSetChunk(getQueryController(), - getQueryId(), sinkId, partitionId, source.iterator()); + final IChunkMessage<IBindingSet> msg = new BindingSetChunk<IBindingSet>( + getQueryController(), getQueryId(), sinkId, partitionId, + source.iterator()); getQueryEngine().bufferReady(msg); @@ -561,37 +575,23 @@ * RMI message or out of band using NIO. This decision effects how we * serialize the chunk. */ - final IChunkMessage msg; + final IChunkMessage<IBindingSet> msg; if (source.size() < 100) { - /* - * FIXME Send payload inline with the RMI message. - */ + msg = new ThickChunkMessage<IBindingSet>(getQueryController(), + getQueryId(), sinkId, partitionId, source); -// final byte[] data = SerializerUtil.serialize(obj); -// -// // @todo harmonize serialization and compression and ctors. -// msg = new ThickChunkMessage(getQueryController(), getQueryId(), -// sinkId, partitionId, data); - throw new UnsupportedOperationException(); + } else { - } else - { - /* * Marshall the data onto direct ByteBuffer(s) and send a thin * message by RMI. The receiver will retrieve the data using NIO * against the ResourceService. - * - * @todo harmonize serialization and compression and ctors. */ - final List<IAllocation> allocations = moveToNIOBuffers( - allocationContext, source); + msg = new NIOChunkMessage<IBindingSet>(getQueryController(), + getQueryId(), sinkId, partitionId, allocationContext, + source, getQueryEngine().getResourceService().getAddr()); - msg = new ChunkMessageWithNIOPayload(getQueryController(), - getQueryId(), sinkId, partitionId, allocations, - getQueryEngine().getResourceService().getAddr()); - } try { @@ -606,61 +606,4 @@ } - /** - * Chunk-wise serialization of the data onto allocations. - * @param allocationContext - * @param source - * @return - * - * @todo should be on message per chunk, right? - */ - private List<IAllocation> moveToNIOBuffers( - final IAllocationContext allocationContext, - final IBlockingBuffer<IBindingSet[]> source) { - - int nbytes = 0; - - final List<IAllocation> allocations = new LinkedList<IAllocation>(); - - final IAsynchronousIterator<IBindingSet[]> itr = source.iterator(); - - try { - - while (itr.hasNext()) { - - // Next chunk to be serialized. - final IBindingSet[] chunk = itr.next(); - - // serialize the chunk of binding sets. - final byte[] data = SerializerUtil.serialize(chunk); - - // track size of the allocations. - nbytes += data.length; - - // allocate enough space for those data. - final IAllocation[] tmp; - try { - tmp = allocationContext.alloc(data.length); - } catch (InterruptedException ex) { - throw new RuntimeException(ex); - } - - // copy the data into the allocations. - DirectBufferPoolAllocator.put(data, tmp); - - // append the new allocations. - allocations.addAll(Arrays.asList(tmp)); - - } - - return allocations; - - } finally { - - itr.close(); - - } - - } - } Copied: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/NIOChunkMessage.java (from rev 3531, branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ChunkMessageWithNIOPayload.java) =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/NIOChunkMessage.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/NIOChunkMessage.java 2010-09-11 22:53:40 UTC (rev 3532) @@ -0,0 +1,502 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Sep 10, 2010 + */ + +package com.bigdata.bop.fed; + +import java.io.Serializable; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import com.bigdata.bop.engine.IChunkMessage; +import com.bigdata.bop.engine.IQueryClient; +import com.bigdata.io.DirectBufferPoolAllocator; +import com.bigdata.io.SerializerUtil; +import com.bigdata.io.DirectBufferPoolAllocator.IAllocation; +import com.bigdata.io.DirectBufferPoolAllocator.IAllocationContext; +import com.bigdata.relation.accesspath.IAsynchronousIterator; +import com.bigdata.relation.accesspath.IBlockingBuffer; +import com.bigdata.service.ManagedResourceService; +import com.bigdata.service.ResourceService; + +/** + * An {@link IChunkMessage} where the payload is made available to the receiving + * service using an NIO transfer against the sender's {@link ResourceService}. + * This is suitable for moving large blocks of data during query evaluation. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +public class NIOChunkMessage<E> implements IChunkMessage<E>, Serializable { + + /** + * + */ + private static final long serialVersionUID = 1L; + + final private IQueryClient queryController; + + final private long queryId; + + final private int bopId; + + final private int partitionId; + + final private int solutionCount; + + final private int nbytes; + + /** + * Note: Even when we send one message per chunk, we can still have a list + * of {@link IAllocation}s if the chunk did not get formatted onto a single + * {@link IAllocation}. + */ + final private A[] allocations; + + /** + * The Internet address and port where the receiver can fetch the payload + * using the sender's {@link ResourceService}. + */ + final private InetSocketAddress addr; + + public IQueryClient getQueryController() { + return queryController; + } + + public long getQueryId() { + return queryId; + } + + public int getBOpId() { + return bopId; + } + + public int getPartitionId() { + return partitionId; + } + + /** + * The #of elements in this chunk. + * + * @todo we could track this in total and in {@link A} on a per-slice basis. + */ + public int getSolutionCount() { + return solutionCount; + } + + /** The #of bytes of data which are available for that operator. */ + public int getBytesAvailable() { + return nbytes; + } + + /** + * The Internet address and port of a {@link ResourceService} from which the + * receiver may demand the data. + */ + public InetSocketAddress getServiceAddr() { + return addr; + } + + public String toString() { + + return getClass().getName() + "{queryId=" + queryId + ",bopId=" + bopId + + ",partitionId=" + partitionId + ", solutionCount=" + + solutionCount + ", bytesAvailable=" + nbytes + ", nslices=" + + allocations.length + ", serviceAddr=" + addr + "}"; + + } + + /** + * + * @param queryController + * @param queryId + * @param sinkId + * @param partitionId + * @param allocations + * The ordered list of {@link IAllocation}s comprising the chunk. + * @param addr + * The Internet address and port where the receiver can fetch the + * payload using the sender's {@link ResourceService}. + */ + public NIOChunkMessage(final IQueryClient queryController, + final long queryId, final int sinkId, final int partitionId, + final IAllocationContext allocationContext, + final IBlockingBuffer<E[]> source, + final InetSocketAddress addr) { + + if (queryController == null) + throw new IllegalArgumentException(); + + if (allocationContext == null) + throw new IllegalArgumentException(); + + if (source == null) + throw new IllegalArgumentException(); + + if (addr == null) + throw new IllegalArgumentException(); + + // format onto NIO buffers. + final AtomicInteger nsolutions = new AtomicInteger(); + final List<IAllocation> allocations = moveToNIOBuffers( + allocationContext, source, nsolutions); + + this.queryController = queryController; + this.queryId = queryId; + this.bopId = sinkId; + this.partitionId = partitionId; + final int n = allocations.size(); + this.allocations = new A[n]; + int i = 0; + int nbytes = 0; + final Iterator<IAllocation> itr = allocations.iterator(); + while (itr.hasNext()) { + final IAllocation alloc = itr.next(); + final int len = alloc.getSlice().capacity(); + this.allocations[i++] = new A(alloc.getId(), len); + nbytes += len; + } + this.solutionCount = nsolutions.get(); + this.nbytes = nbytes; + this.addr = addr; + + } + + /** + * Chunk-wise serialization of the data onto allocations. + * + * @param allocationContext + * @param source + * @return + */ + static private <E> List<IAllocation> moveToNIOBuffers( + final IAllocationContext allocationContext, + final IBlockingBuffer<E[]> source, + final AtomicInteger nsolutions) { + + int nbytes = 0; + + int n = 0; + + final List<IAllocation> allocations = new LinkedList<IAllocation>(); + + final IAsynchronousIterator<E[]> itr = source.iterator(); + + try { + + while (itr.hasNext()) { + + // Next chunk to be serialized. + final E[] chunk = itr.next(); + + // track #of solutions. + n += chunk.length; + + // serialize the chunk of binding sets. + final byte[] data = SerializerUtil.serialize(chunk); + + // track size of the allocations. + nbytes += data.length; + + // allocate enough space for those data. + final IAllocation[] tmp; + try { + tmp = allocationContext.alloc(data.length); + } catch (InterruptedException ex) { + throw new RuntimeException(ex); + } + + // copy the data into the allocations. + DirectBufferPoolAllocator.put(data, tmp); + + for(IAllocation a : tmp) { + + // prepare for reading. + a.getSlice().flip(); + + // append the allocation. + allocations.add(a); + + } + + } + + nsolutions.addAndGet(n); + + return allocations; + + } finally { + + itr.close(); + + } + + } + + /** + * Metadata about an allocation to be retrieved from the sender's + * {@link ResourceService}. + */ + private static final class A implements Serializable { + + /** + * + */ + private static final long serialVersionUID = 1L; + + /** + * The identifier of the resource on the sender's + * {@link ResourceService}. + */ + private final UUID bufferId; + + /** + * The size of that resource in bytes. + */ + private final int nbytes; + + /** + ... [truncated message content] |
From: <tho...@us...> - 2010-09-14 20:48:27
|
Revision: 3554 http://bigdata.svn.sourceforge.net/bigdata/?rev=3554&view=rev Author: thompsonbry Date: 2010-09-14 20:48:21 +0000 (Tue, 14 Sep 2010) Log Message: ----------- Working through RMI invocations on 2-DS cluster. IQueryClient#eval() was added, but it needs to be modified to pass along the initial IChunkMessage which gets query evaluation started and the unit tests need to be updated to reflect that change. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestNIOChunkMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestThickChunkMessage.java Added Paths: ----------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/StartOp.java Added: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/StartOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/StartOp.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/StartOp.java 2010-09-14 20:48:21 UTC (rev 3554) @@ -0,0 +1,31 @@ +package com.bigdata.bop.bset; + +import java.util.Map; + +import com.bigdata.bop.BOp; +import com.bigdata.bop.BOpEvaluationContext; + +/** + * A version of {@link CopyBindingSetOp} which is always evaluated on the query + * controller. + */ +public class StartOp extends CopyBindingSetOp { + + /** + * + */ + private static final long serialVersionUID = 1L; + + public StartOp(StartOp op) { + super(op); + } + + public StartOp(BOp[] args, Map<String, Object> annotations) { + super(args, annotations); + } + + final public BOpEvaluationContext getEvaluationContext() { + return BOpEvaluationContext.CONTROLLER; + } + +} Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java 2010-09-14 20:46:47 UTC (rev 3553) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java 2010-09-14 20:48:21 UTC (rev 3554) @@ -3,6 +3,7 @@ import java.rmi.RemoteException; import com.bigdata.bop.BindingSetPipelineOp; +import com.bigdata.bop.IBindingSet; /** * Interface for a client executing queries (the query controller). @@ -10,6 +11,25 @@ public interface IQueryClient extends IQueryPeer { /** + * Evaluate a query which visits {@link IBindingSet}s, such as a join. This + * node will serve as the controller for the query. + * + * @param queryId + * The unique identifier for the query. + * @param query + * The query to evaluate. + * + * @return An iterator visiting {@link IBindingSet}s which result from + * evaluating the query. + * + * @throws IllegalStateException + * if the {@link QueryEngine} has been {@link #shutdown()}. + * @throws Exception + * @throws RemoteException + */ + RunningQuery eval(long queryId, BindingSetPipelineOp query) throws Exception, RemoteException; + + /** * Return the query. * * @param queryId @@ -19,13 +39,13 @@ * @throws IllegalArgumentException * if there is no such query. */ - public BindingSetPipelineOp getQuery(long queryId) throws RemoteException; + BindingSetPipelineOp getQuery(long queryId) throws RemoteException; /** * Notify the client that execution has started for some query, operator, * node, and index partition. */ - public void startOp(StartOpMessage msg) + void startOp(StartOpMessage msg) throws RemoteException; /** @@ -33,6 +53,6 @@ * node, shard, and source binding set chunk(s). If execution halted * abnormally, then the cause is sent as well. */ - public void haltOp(HaltOpMessage msg) throws RemoteException; + void haltOp(HaltOpMessage msg) throws RemoteException; } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java 2010-09-14 20:46:47 UTC (rev 3553) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java 2010-09-14 20:48:21 UTC (rev 3554) @@ -33,7 +33,7 @@ * @throws UnsupportedOperationException * unless running in scale-out. */ - void declareQuery(IQueryDecl queryDecl); + void declareQuery(IQueryDecl queryDecl) throws RemoteException; /** * Notify a service that a buffer having data for some {@link BOp} in some Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java 2010-09-14 20:46:47 UTC (rev 3553) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java 2010-09-14 20:48:21 UTC (rev 3554) @@ -27,7 +27,6 @@ package com.bigdata.bop.engine; -import com.bigdata.bop.BOp; import com.bigdata.btree.ILocalBTreeView; import com.bigdata.journal.IIndexManager; import com.bigdata.service.IBigdataFederation; Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java 2010-09-14 20:46:47 UTC (rev 3553) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java 2010-09-14 20:48:21 UTC (rev 3554) @@ -734,22 +734,6 @@ } - /** - * Evaluate a query which visits {@link IBindingSet}s, such as a join. This - * node will serve as the controller for the query. - * - * @param queryId - * The unique identifier for the query. - * @param query - * The query to evaluate. - * - * @return An iterator visiting {@link IBindingSet}s which result from - * evaluating the query. - * - * @throws IllegalStateException - * if the {@link QueryEngine} has been {@link #shutdown()}. - * @throws Exception - */ public RunningQuery eval(final long queryId, final BindingSetPipelineOp query) throws Exception { Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java 2010-09-14 20:46:47 UTC (rev 3553) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java 2010-09-14 20:48:21 UTC (rev 3554) @@ -37,7 +37,6 @@ import com.bigdata.bop.ArrayBindingSet; import com.bigdata.bop.BOp; import com.bigdata.bop.BOpContext; -import com.bigdata.bop.BOpEvaluationContext; import com.bigdata.bop.BindingSetPipelineOp; import com.bigdata.bop.Constant; import com.bigdata.bop.HashBindingSet; @@ -50,9 +49,10 @@ import com.bigdata.bop.ap.E; import com.bigdata.bop.ap.Predicate; import com.bigdata.bop.ap.R; -import com.bigdata.bop.bset.CopyBindingSetOp; +import com.bigdata.bop.bset.StartOp; import com.bigdata.bop.engine.BOpStats; import com.bigdata.bop.engine.BindingSetChunk; +import com.bigdata.bop.engine.IQueryClient; import com.bigdata.bop.engine.IQueryPeer; import com.bigdata.bop.engine.PipelineDelayOp; import com.bigdata.bop.engine.QueryEngine; @@ -62,15 +62,15 @@ import com.bigdata.bop.solutions.SliceOp; import com.bigdata.bop.solutions.SortOp; import com.bigdata.btree.keys.KeyBuilder; -import com.bigdata.jini.util.JiniUtil; +import com.bigdata.io.SerializerUtil; import com.bigdata.journal.ITx; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.ThickAsynchronousIterator; -import com.bigdata.service.DataService; import com.bigdata.service.EmbeddedFederation; -import com.bigdata.service.jini.DataServer; +import com.bigdata.service.IBigdataFederation; +import com.bigdata.service.IDataService; +import com.bigdata.service.jini.JiniClient; import com.bigdata.service.jini.JiniFederation; -import com.bigdata.service.jini.util.JiniServicesHelper; import com.bigdata.striterator.ChunkedArrayIterator; import com.bigdata.striterator.Dechunkerator; import com.ibm.icu.impl.ByteBuffer; @@ -128,40 +128,43 @@ } // Namespace for the relation. - static private final String namespace = "ns"; + static private final String namespace = TestFederatedQueryEngine.class.getName(); // The separator key between the index partitions. private byte[] separatorKey; - private FederatedQueryEngine queryEngine; + private IQueryClient queryEngine; + private JiniClient<?> client; - private JiniServicesHelper helper; - -// private JiniClient<?> client; - + private IDataService dataService0; + private IDataService dataService1; + protected void setUp() throws Exception { + + client = new JiniClient(new String[]{"/nas/bigdata/bigdata-0.83.2/dist/bigdata/var/config/jini/bigdataStandalone.config"}); + + final IBigdataFederation<?> fed = client.connect(); - helper = new JiniServicesHelper(); - - // start services. - helper.start(); - -// // expose to subclasses. -// client = helper.client; - + final int maxCount = 2; + UUID[] dataServices = null; + while((dataServices = fed.getDataServiceUUIDs(maxCount)).length < maxCount) { + System.err.println("Waiting for "+maxCount+" data services. There are "+dataServices.length+" discovered."); + Thread.sleep(250/*ms*/); + } + super.setUp(); - + + dataService0 = fed.getDataService(dataServices[0]); + dataService1 = fed.getDataService(dataServices[1]); { - - final DataServer dataServer = helper.dataServer0; - - assertTrue(((DataService) dataServer.getProxy()) - .getResourceManager().awaitRunning()); + // @todo need to wait for the dataService to be running. +// assertTrue(((DataService) dataServer.getProxy()) +// .getResourceManager().awaitRunning()); + // resolve the query engine on one of the data services. - while ((queryEngine = (FederatedQueryEngine) ((DataService) dataServer - .getProxy()).getQueryEngine()) == null) { + while ((queryEngine = (IQueryClient) dataService0.getQueryEngine()) == null) { if (log.isInfoEnabled()) log.info("Waiting for query engine on dataService0"); @@ -175,17 +178,14 @@ } // resolve the query engine on the other data services. - if (helper.dataServer1 != null) { + { - final DataServer dataServer = helper.dataServer1; - IQueryPeer other = null; - assertTrue(((DataService) dataServer.getProxy()) - .getResourceManager().awaitRunning()); +// assertTrue(((DataService) dataServer.getProxy()) +// .getResourceManager().awaitRunning()); - while ((other = ((DataService) dataServer.getProxy()) - .getQueryEngine()) == null) { + while ((other = dataService1.getQueryEngine()) == null) { if (log.isInfoEnabled()) log.info("Waiting for query engine on dataService1"); @@ -207,9 +207,13 @@ // clear reference. separatorKey = null; - helper.destroy(); + client.disconnect(true/*immediateShutdown*/); + client = null; + + dataService0 = null; + dataService1 = null; - helper = null; + queryEngine = null; super.tearDown(); @@ -244,8 +248,8 @@ }; final UUID[] dataServices = new UUID[] {// - JiniUtil.serviceID2UUID(helper.dataServer0.getServiceID()),// - JiniUtil.serviceID2UUID(helper.dataServer1.getServiceID()),// + dataService0.getServiceUUID(),// + dataService1.getServiceUUID(),// }; /* @@ -253,14 +257,19 @@ * using the given separator keys and data services. */ - final R rel = new R(helper.getFederation(), namespace, ITx.UNISOLATED, new Properties()); + final R rel = new R(client.getFederation(), namespace, ITx.UNISOLATED, new Properties()); + if(client.getFederation() + .getResourceLocator().locate(namespace, ITx.UNISOLATED)==null) { + rel.create(separatorKeys, dataServices); /* * Insert data into the appropriate index partitions. */ rel.insert(new ChunkedArrayIterator<E>(a.length, a, null/* keyOrder */)); + + } } @@ -302,18 +311,12 @@ public void test_query_startRun() throws Exception { final int startId = 1; - final BindingSetPipelineOp query = new CopyBindingSetOp(new BOp[] {}, NV + final BindingSetPipelineOp query = new StartOp(new BOp[] {}, NV .asMap(new NV[] {// new NV(Predicate.Annotations.BOP_ID, startId),// // new NV(Predicate.Annotations.READ_TIMESTAMP, ITx.READ_COMMITTED),// - })){ - private static final long serialVersionUID = 1L; + })); - public BOpEvaluationContext getEvaluationContext() { - return BOpEvaluationContext.CONTROLLER; - } - }; - final long queryId = 1L; final RunningQuery runningQuery = queryEngine.eval(queryId, query); @@ -379,7 +382,7 @@ final BindingSetPipelineOp query = new SliceOp(new BOp[]{new PipelineJoin<E>( // left - new CopyBindingSetOp(new BOp[] {}, NV.asMap(new NV[] {// + new StartOp(new BOp[] {}, NV.asMap(new NV[] {// new NV(Predicate.Annotations.BOP_ID, startId),// })), // right @@ -563,7 +566,7 @@ final int predId2 = 5; final int sliceId = 6; - final BindingSetPipelineOp startOp = new CopyBindingSetOp(new BOp[] {}, + final BindingSetPipelineOp startOp = new StartOp(new BOp[] {}, NV.asMap(new NV[] {// new NV(Predicate.Annotations.BOP_ID, startId),// })); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestNIOChunkMessage.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestNIOChunkMessage.java 2010-09-14 20:46:47 UTC (rev 3553) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestNIOChunkMessage.java 2010-09-14 20:48:21 UTC (rev 3554) @@ -46,6 +46,7 @@ import com.bigdata.bop.engine.IChunkMessage; import com.bigdata.bop.engine.IQueryClient; import com.bigdata.bop.engine.IQueryDecl; +import com.bigdata.bop.engine.RunningQuery; import com.bigdata.bop.engine.StartOpMessage; import com.bigdata.io.DirectBufferPoolAllocator.IAllocationContext; import com.bigdata.relation.accesspath.BlockingBuffer; @@ -255,6 +256,12 @@ throws RemoteException { return null; } + + @Override + public RunningQuery eval(long queryId, BindingSetPipelineOp query) + throws Exception, RemoteException { + return null; + } } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestThickChunkMessage.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestThickChunkMessage.java 2010-09-14 20:46:47 UTC (rev 3553) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestThickChunkMessage.java 2010-09-14 20:48:21 UTC (rev 3554) @@ -43,6 +43,7 @@ import com.bigdata.bop.engine.IChunkMessage; import com.bigdata.bop.engine.IQueryClient; import com.bigdata.bop.engine.IQueryDecl; +import com.bigdata.bop.engine.RunningQuery; import com.bigdata.bop.engine.StartOpMessage; import com.bigdata.relation.accesspath.BlockingBuffer; import com.bigdata.relation.accesspath.IBlockingBuffer; @@ -180,6 +181,12 @@ return null; } + @Override + public RunningQuery eval(long queryId, BindingSetPipelineOp query) + throws Exception, RemoteException { + return null; + } + } } This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2010-09-15 10:22:41
|
Revision: 3555 http://bigdata.svn.sourceforge.net/bigdata/?rev=3555&view=rev Author: thompsonbry Date: 2010-09-15 10:22:34 +0000 (Wed, 15 Sep 2010) Log Message: ----------- Modified QueryEngine#eval() to accept the initial binding set chunk. Modified TestFederatedQueryEngine to create a local query controller to which the queries are submitted. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestNIOChunkMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestThickChunkMessage.java Added Paths: ----------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/LocalChunkMessage.java Removed Paths: ------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java Deleted: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java 2010-09-14 20:48:21 UTC (rev 3554) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java 2010-09-15 10:22:34 UTC (rev 3555) @@ -1,108 +0,0 @@ -package com.bigdata.bop.engine; - -import java.io.Serializable; - -import com.bigdata.bop.BOp; -import com.bigdata.bop.fed.FederatedRunningQuery; -import com.bigdata.relation.accesspath.IAsynchronousIterator; - -/** - * An non-{@link Serializable} chunk of intermediate results which are ready to - * be consumed by some {@link BOp} in a specific query (this is only used in - * query evaluation for the standalone database). - */ -public class BindingSetChunk<E> implements IChunkMessage<E> { - - /** The query controller. */ - private final IQueryClient queryController; - - /** - * The query identifier. - */ - private final long queryId; - - /** - * The target {@link BOp}. - */ - private final int bopId; - - /** - * The index partition which is being targeted for that {@link BOp}. - */ - private final int partitionId; - - /** - * The binding sets to be consumed by that {@link BOp}. - */ - private IAsynchronousIterator<E[]> source; - - public IQueryClient getQueryController() { - return queryController; - } - - public long getQueryId() { - return queryId; - } - - public int getBOpId() { - return bopId; - } - - public int getPartitionId() { - return partitionId; - } - - public boolean isMaterialized() { - return true; - } - - public BindingSetChunk(final IQueryClient queryController, - final long queryId, final int bopId, final int partitionId, - final IAsynchronousIterator<E[]> source) { - - if (queryController == null) - throw new IllegalArgumentException(); - - if (source == null) - throw new IllegalArgumentException(); - - this.queryController = queryController; - - this.queryId = queryId; - - this.bopId = bopId; - - this.partitionId = partitionId; - - this.source = source; - - } - - public String toString() { - - return getClass().getName() + "{queryId=" + queryId + ",bopId=" + bopId - + ",partitionId=" + partitionId + "}"; - - } - - public void materialize(FederatedRunningQuery runningQuery) { - // NOP - } - - public void release() { - // NOP - } - - public IChunkAccessor<E> getChunkAccessor() { - return new ChunkAccessor(); - } - - private class ChunkAccessor implements IChunkAccessor<E> { - - public IAsynchronousIterator<E[]> iterator() { - return source; - } - - } - -} Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java 2010-09-14 20:48:21 UTC (rev 3554) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryClient.java 2010-09-15 10:22:34 UTC (rev 3555) @@ -3,7 +3,6 @@ import java.rmi.RemoteException; import com.bigdata.bop.BindingSetPipelineOp; -import com.bigdata.bop.IBindingSet; /** * Interface for a client executing queries (the query controller). @@ -11,25 +10,6 @@ public interface IQueryClient extends IQueryPeer { /** - * Evaluate a query which visits {@link IBindingSet}s, such as a join. This - * node will serve as the controller for the query. - * - * @param queryId - * The unique identifier for the query. - * @param query - * The query to evaluate. - * - * @return An iterator visiting {@link IBindingSet}s which result from - * evaluating the query. - * - * @throws IllegalStateException - * if the {@link QueryEngine} has been {@link #shutdown()}. - * @throws Exception - * @throws RemoteException - */ - RunningQuery eval(long queryId, BindingSetPipelineOp query) throws Exception, RemoteException; - - /** * Return the query. * * @param queryId Copied: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/LocalChunkMessage.java (from rev 3554, branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BindingSetChunk.java) =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/LocalChunkMessage.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/LocalChunkMessage.java 2010-09-15 10:22:34 UTC (rev 3555) @@ -0,0 +1,110 @@ +package com.bigdata.bop.engine; + +import java.io.Serializable; + +import com.bigdata.bop.BOp; +import com.bigdata.bop.fed.FederatedRunningQuery; +import com.bigdata.relation.accesspath.IAsynchronousIterator; + +/** + * An non-{@link Serializable} chunk of intermediate results which are ready to + * be consumed by some {@link BOp} in a specific query (this is only used in + * query evaluation for the standalone database). + * + * @todo test suite + */ +public class LocalChunkMessage<E> implements IChunkMessage<E> { + + /** The query controller. */ + private final IQueryClient queryController; + + /** + * The query identifier. + */ + private final long queryId; + + /** + * The target {@link BOp}. + */ + private final int bopId; + + /** + * The index partition which is being targeted for that {@link BOp}. + */ + private final int partitionId; + + /** + * The binding sets to be consumed by that {@link BOp}. + */ + private IAsynchronousIterator<E[]> source; + + public IQueryClient getQueryController() { + return queryController; + } + + public long getQueryId() { + return queryId; + } + + public int getBOpId() { + return bopId; + } + + public int getPartitionId() { + return partitionId; + } + + public boolean isMaterialized() { + return true; + } + + public LocalChunkMessage(final IQueryClient queryController, + final long queryId, final int bopId, final int partitionId, + final IAsynchronousIterator<E[]> source) { + + if (queryController == null) + throw new IllegalArgumentException(); + + if (source == null) + throw new IllegalArgumentException(); + + this.queryController = queryController; + + this.queryId = queryId; + + this.bopId = bopId; + + this.partitionId = partitionId; + + this.source = source; + + } + + public String toString() { + + return getClass().getName() + "{queryId=" + queryId + ",bopId=" + bopId + + ",partitionId=" + partitionId + "}"; + + } + + public void materialize(FederatedRunningQuery runningQuery) { + // NOP + } + + public void release() { + // NOP + } + + public IChunkAccessor<E> getChunkAccessor() { + return new ChunkAccessor(); + } + + private class ChunkAccessor implements IChunkAccessor<E> { + + public IAsynchronousIterator<E[]> iterator() { + return source; + } + + } + +} Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java 2010-09-14 20:48:21 UTC (rev 3554) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java 2010-09-15 10:22:34 UTC (rev 3555) @@ -45,6 +45,7 @@ import com.bigdata.bop.IBindingSet; import com.bigdata.bop.IPredicate; import com.bigdata.bop.bset.Union; +import com.bigdata.bop.fed.FederatedQueryEngine; import com.bigdata.btree.BTree; import com.bigdata.btree.IndexSegment; import com.bigdata.btree.view.FusedView; @@ -500,7 +501,7 @@ * evaluation. * <p> * Chunk concatenation could be performed here if we (a) mark the - * {@link BindingSetChunk} with a flag to indicate when it has been + * {@link LocalChunkMessage} with a flag to indicate when it has been * accepted; and (b) rip through the incoming chunks for the query for * the target bop and combine them to feed the task. Chunks which have * already been assigned would be dropped when take() discovers them. @@ -734,18 +735,57 @@ } + /** + * Evaluate a query which visits {@link IBindingSet}s, such as a join. This + * node will serve as the controller for the query. + * + * @param queryId + * The unique identifier for the query. + * @param query + * The query to evaluate. + * + * @return An iterator visiting {@link IBindingSet}s which result from + * evaluating the query. + * @param msg + * A message providing access to the initial {@link IBindingSet + * binding set(s)} used to begin query evaluation. + * + * @throws IllegalStateException + * if the {@link QueryEngine} has been {@link #shutdown()}. + * @throws Exception + * @throws RemoteException + * + * FIXME The test suites need to be modified to create a local + * {@link FederatedQueryEngine} object which fronts for an + * {@link IIndexManager} which is local to the client - not on a + * data service at all. This is necessary in order for the unit + * test (or application code) to directly access the + * RunningQuery reference, which is needed to use get() (to wait + * for the query), iterator() (to drain the query), etc. + * <p> + * This will also give us a place to hang query-local resources + * on the client. + * <p> + * This has to be a {@link FederatedQueryEngine} because it + * needs to talk to a federation. There should be nothing DS + * specific about the {@link FederatedQueryEngine}. + */ public RunningQuery eval(final long queryId, - final BindingSetPipelineOp query) throws Exception { + final BindingSetPipelineOp query, + final IChunkMessage<IBindingSet> msg) throws Exception { if (query == null) throw new IllegalArgumentException(); + if (msg == null) + throw new IllegalArgumentException(); + + if (queryId != msg.getQueryId()) // @todo use equals() to compare UUIDs. + throw new IllegalArgumentException(); + final RunningQuery runningQuery = newRunningQuery(this, queryId, -// System.currentTimeMillis()/* begin */, true/* controller */, this/* clientProxy */, query); - assertRunning(); - final long timeout = query.getProperty(BOp.Annotations.TIMEOUT, BOp.Annotations.DEFAULT_TIMEOUT); @@ -767,8 +807,12 @@ } + assertRunning(); + putRunningQuery(queryId, runningQuery); + runningQuery.startQuery(msg); + return runningQuery; } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-14 20:48:21 UTC (rev 3554) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-15 10:22:34 UTC (rev 3555) @@ -366,7 +366,7 @@ /* * Note: The partitionId will always be -1 in scale-up. */ - final BindingSetChunk<IBindingSet> chunk = new BindingSetChunk<IBindingSet>( + final LocalChunkMessage<IBindingSet> chunk = new LocalChunkMessage<IBindingSet>( clientProxy, queryId, sinkId, -1/* partitionId */, sink .iterator()); @@ -646,11 +646,9 @@ /** * Invoked once by the query controller with the initial - * {@link BindingSetChunk} which gets the query moving. - * - * @todo this should reject multiple invocations for a given query instance. + * {@link IChunkMessage} which gets the query moving. */ - public void startQuery(final IChunkMessage<IBindingSet> msg) { + void startQuery(final IChunkMessage<IBindingSet> msg) { if (!controller) throw new UnsupportedOperationException(); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java 2010-09-14 20:48:21 UTC (rev 3554) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java 2010-09-15 10:22:34 UTC (rev 3555) @@ -122,20 +122,6 @@ } /** - * Constructor used on a {@link DataService} (a query engine peer). - * - * @param dataService - * The data service. - */ - public FederatedQueryEngine(final DataService dataService) { - - this(dataService.getFederation(), - new DelegateIndexManager(dataService), dataService - .getResourceManager().getResourceService()); - - } - - /** * Overridden to strengthen the return type. * <p> * {@inheritDoc} @@ -152,6 +138,20 @@ return getClass().getName() + "{serviceUUID=" + getServiceUUID() + "}"; } + + /** + * Constructor used on a {@link DataService} (a query engine peer). + * + * @param dataService + * The data service. + */ + public FederatedQueryEngine(final DataService dataService) { + + this(dataService.getFederation(), + new DelegateIndexManager(dataService), dataService + .getResourceManager().getResourceService()); + + } /** * Constructor used on a non-{@link DataService} node to expose a query Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java 2010-09-14 20:48:21 UTC (rev 3554) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java 2010-09-15 10:22:34 UTC (rev 3555) @@ -42,7 +42,7 @@ import com.bigdata.bop.IBindingSet; import com.bigdata.bop.IPredicate; import com.bigdata.bop.IShardwisePipelineOp; -import com.bigdata.bop.engine.BindingSetChunk; +import com.bigdata.bop.engine.LocalChunkMessage; import com.bigdata.bop.engine.IChunkMessage; import com.bigdata.bop.engine.IQueryClient; import com.bigdata.bop.engine.IQueryPeer; @@ -94,7 +94,7 @@ * A map associating resources with running queries. When a query halts, the * resources listed in its resource map are released. Resources can include * {@link ByteBuffer}s backing either incoming or outgoing - * {@link BindingSetChunk}s, temporary files associated with the query, hash + * {@link LocalChunkMessage}s, temporary files associated with the query, hash * tables, etc. * * @todo This map will eventually need to be moved into {@link RunningQuery} @@ -604,7 +604,7 @@ * query engine. */ - final IChunkMessage<IBindingSet> msg = new BindingSetChunk<IBindingSet>( + final IChunkMessage<IBindingSet> msg = new LocalChunkMessage<IBindingSet>( getQueryController(), getQueryId(), sinkId, partitionId, source.iterator()); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-09-14 20:48:21 UTC (rev 3554) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-09-15 10:22:34 UTC (rev 3555) @@ -198,16 +198,11 @@ })); final long queryId = 1L; - final RunningQuery runningQuery = queryEngine.eval(queryId,query); + final RunningQuery runningQuery = queryEngine.eval(queryId, query, + new LocalChunkMessage<IBindingSet>(queryEngine, queryId, + startId, -1/* partitionId */, + newBindingSetIterator(new HashBindingSet()))); - runningQuery.startQuery(new BindingSetChunk( - queryEngine, - queryId, - startId,// - -1, //partitionId - new ThickAsynchronousIterator<IBindingSet[]>( - new IBindingSet[][] { new IBindingSet[] { new HashBindingSet()} }))); - // Wait until the query is done. final Map<Integer, BOpStats> statsMap = runningQuery.get(); { @@ -290,13 +285,11 @@ ) }; final long queryId = 1L; - final RunningQuery runningQuery = queryEngine.eval(queryId, query); + final RunningQuery runningQuery = queryEngine.eval(queryId, query, + new LocalChunkMessage<IBindingSet>(queryEngine, queryId, + startId, -1 /* partitionId */, + newBindingSetIterator(new HashBindingSet()))); - runningQuery.startQuery(new BindingSetChunk(queryEngine, queryId, - startId,// - -1, // partitionId - newBindingSetIterator(new HashBindingSet()))); - // verify solutions. assertSameSolutions(expected, runningQuery.iterator()); @@ -478,21 +471,22 @@ final BindingSetPipelineOp query = join2Op; - final long queryId = 1L; - final RunningQuery runningQuery = queryEngine.eval(queryId, query); - // start the query. + final long queryId = 1L; + final IChunkMessage<IBindingSet> initialChunkMessage; { - + final IBindingSet initialBindings = new HashBindingSet(); - + initialBindings.set(Var.var("x"), new Constant<String>("Mary")); - runningQuery.startQuery(new BindingSetChunk<IBindingSet>( - queryEngine, queryId, startId,// + initialChunkMessage = new LocalChunkMessage<IBindingSet>(queryEngine, + queryId, startId,// -1, // partitionId - newBindingSetIterator(initialBindings))); + newBindingSetIterator(initialBindings)); } + final RunningQuery runningQuery = queryEngine.eval(queryId, query, + initialChunkMessage); // verify solutions. { Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java 2010-09-14 20:48:21 UTC (rev 3554) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java 2010-09-15 10:22:34 UTC (rev 3555) @@ -27,10 +27,15 @@ package com.bigdata.bop.fed; +import java.io.File; import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; import java.util.Map; import java.util.Properties; import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import junit.framework.TestCase2; @@ -51,9 +56,8 @@ import com.bigdata.bop.ap.R; import com.bigdata.bop.bset.StartOp; import com.bigdata.bop.engine.BOpStats; -import com.bigdata.bop.engine.BindingSetChunk; -import com.bigdata.bop.engine.IQueryClient; -import com.bigdata.bop.engine.IQueryPeer; +import com.bigdata.bop.engine.IChunkMessage; +import com.bigdata.bop.engine.LocalChunkMessage; import com.bigdata.bop.engine.PipelineDelayOp; import com.bigdata.bop.engine.QueryEngine; import com.bigdata.bop.engine.RunningQuery; @@ -62,17 +66,21 @@ import com.bigdata.bop.solutions.SliceOp; import com.bigdata.bop.solutions.SortOp; import com.bigdata.btree.keys.KeyBuilder; -import com.bigdata.io.SerializerUtil; +import com.bigdata.journal.BufferMode; import com.bigdata.journal.ITx; +import com.bigdata.journal.Journal; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.ThickAsynchronousIterator; import com.bigdata.service.EmbeddedFederation; import com.bigdata.service.IBigdataFederation; import com.bigdata.service.IDataService; +import com.bigdata.service.ManagedResourceService; +import com.bigdata.service.ResourceService; import com.bigdata.service.jini.JiniClient; import com.bigdata.service.jini.JiniFederation; import com.bigdata.striterator.ChunkedArrayIterator; import com.bigdata.striterator.Dechunkerator; +import com.bigdata.util.config.NicUtil; import com.ibm.icu.impl.ByteBuffer; /** @@ -133,71 +141,130 @@ // The separator key between the index partitions. private byte[] separatorKey; - private IQueryClient queryEngine; - private JiniClient<?> client; - private IDataService dataService0; - private IDataService dataService1; + /** The local persistence store for the {@link #queryEngine}. */ + private Journal queryEngineStore; + + /** The local {@link ResourceService} for the {@link #queryEngine}. */ + private ManagedResourceService queryEngineResourceService; + + /** The query controller. */ + private FederatedQueryEngine queryEngine; + + private IDataService dataService0; + private IDataService dataService1; + protected void setUp() throws Exception { - client = new JiniClient(new String[]{"/nas/bigdata/bigdata-0.83.2/dist/bigdata/var/config/jini/bigdataStandalone.config"}); - - final IBigdataFederation<?> fed = client.connect(); - - final int maxCount = 2; - UUID[] dataServices = null; - while((dataServices = fed.getDataServiceUUIDs(maxCount)).length < maxCount) { - System.err.println("Waiting for "+maxCount+" data services. There are "+dataServices.length+" discovered."); - Thread.sleep(250/*ms*/); - } - - super.setUp(); + /* + * FIXME This is hardcoded to a specific location in the file system. + * + * Also, the dependency on JiniClient means that we must move this test + * class into the bigdata-jini package. + */ + client = new JiniClient( + new String[] { "/nas/bigdata/bigdata-0.83.2/dist/bigdata/var/config/jini/bigdataStandalone.config" }); - dataService0 = fed.getDataService(dataServices[0]); - dataService1 = fed.getDataService(dataServices[1]); + final IBigdataFederation<?> fed = client.connect(); + + // create index manager for the query controller. { + final Properties p = new Properties(); + p.setProperty(Journal.Options.BUFFER_MODE, BufferMode.Transient + .toString()); + queryEngineStore = new Journal(p); + } + + // create resource service for the query controller. + { + queryEngineResourceService = new ManagedResourceService( + new InetSocketAddress(InetAddress + .getByName(NicUtil.getIpAddress("default.nic", + "default", true/* loopbackOk */)), 0/* port */ + ), 0/* requestServicePoolSize */) { - // @todo need to wait for the dataService to be running. -// assertTrue(((DataService) dataServer.getProxy()) -// .getResourceManager().awaitRunning()); - - // resolve the query engine on one of the data services. - while ((queryEngine = (IQueryClient) dataService0.getQueryEngine()) == null) { - - if (log.isInfoEnabled()) - log.info("Waiting for query engine on dataService0"); - - Thread.sleep(250); - - } - - System.err.println("controller: " + queryEngine); - + @Override + protected File getResource(UUID uuid) throws Exception { + // Will not serve up files. + return null; + } + }; } + + // create the query controller. + queryEngine = new FederatedQueryEngine(fed, queryEngineStore, + queryEngineResourceService); + + /* + * Discover the data services. We need their UUIDs in order to create + * the test relation split across an index partition located on each of + * the two data services. + */ + final int maxCount = 2; + UUID[] dataServices = null; + final long begin = System.currentTimeMillis(); + long elapsed = 0L; + while ((dataServices = fed.getDataServiceUUIDs(maxCount)).length < maxCount + && ((elapsed = System.currentTimeMillis() - begin) < TimeUnit.SECONDS + .toMillis(60))) { + System.err.println("Waiting for " + maxCount + + " data services. There are " + dataServices.length + + " discovered : elapsed=" + elapsed + "ms"); + Thread.sleep(250/* ms */); + } - // resolve the query engine on the other data services. - { + if (dataServices.length < maxCount) + throw new TimeoutException("Discovered " + dataServices.length + + " data services in " + elapsed + "ms but require " + + maxCount); + + super.setUp(); - IQueryPeer other = null; - -// assertTrue(((DataService) dataServer.getProxy()) -// .getResourceManager().awaitRunning()); - - while ((other = dataService1.getQueryEngine()) == null) { - - if (log.isInfoEnabled()) - log.info("Waiting for query engine on dataService1"); - - Thread.sleep(250); - - } +// dataService0 = fed.getDataService(dataServices[0]); +// dataService1 = fed.getDataService(dataServices[1]); +// { +// +// // @todo need to wait for the dataService to be running. +//// assertTrue(((DataService) dataServer.getProxy()) +//// .getResourceManager().awaitRunning()); +// +// // resolve the query engine on one of the data services. +// while ((queryEngine = (IQueryClient) dataService0.getQueryEngine()) == null) { +// +// if (log.isInfoEnabled()) +// log.info("Waiting for query engine on dataService0"); +// +// Thread.sleep(250); +// +// } +// +// System.err.println("controller: " + queryEngine); +// +// } +// +// // resolve the query engine on the other data services. +// { +// +// IQueryPeer other = null; +// +//// assertTrue(((DataService) dataServer.getProxy()) +//// .getResourceManager().awaitRunning()); +// +// while ((other = dataService1.getQueryEngine()) == null) { +// +// if (log.isInfoEnabled()) +// log.info("Waiting for query engine on dataService1"); +// +// Thread.sleep(250); +// +// } +// +// System.err.println("other : " + other); +// +// } - System.err.println("other : " + other); - - } - loadData(); } @@ -213,7 +280,18 @@ dataService0 = null; dataService1 = null; - queryEngine = null; + if (queryEngineResourceService != null) { + queryEngineResourceService.shutdownNow(); + queryEngineResourceService = null; + } + if (queryEngineStore != null) { + queryEngineStore.destroy(); + queryEngineStore = null; + } + if (queryEngine != null) { + queryEngine.shutdownNow(); + queryEngine = null; + } super.tearDown(); @@ -256,19 +334,21 @@ * Create the relation with the primary index key-range partitioned * using the given separator keys and data services. */ - - final R rel = new R(client.getFederation(), namespace, ITx.UNISOLATED, new Properties()); - if(client.getFederation() - .getResourceLocator().locate(namespace, ITx.UNISOLATED)==null) { - - rel.create(separatorKeys, dataServices); + final R rel = new R(client.getFederation(), namespace, ITx.UNISOLATED, + new Properties()); - /* - * Insert data into the appropriate index partitions. - */ - rel.insert(new ChunkedArrayIterator<E>(a.length, a, null/* keyOrder */)); - + if (client.getFederation().getResourceLocator().locate(namespace, + ITx.UNISOLATED) == null) { + + rel.create(separatorKeys, dataServices); + + /* + * Insert data into the appropriate index partitions. + */ + rel + .insert(new ChunkedArrayIterator<E>(a.length, a, null/* keyOrder */)); + } } @@ -314,20 +394,14 @@ final BindingSetPipelineOp query = new StartOp(new BOp[] {}, NV .asMap(new NV[] {// new NV(Predicate.Annotations.BOP_ID, startId),// -// new NV(Predicate.Annotations.READ_TIMESTAMP, ITx.READ_COMMITTED),// })); final long queryId = 1L; - final RunningQuery runningQuery = queryEngine.eval(queryId, query); + final RunningQuery runningQuery = queryEngine.eval(queryId, query, + new LocalChunkMessage<IBindingSet>(queryEngine, queryId, + startId, -1 /* partitionId */, + newBindingSetIterator(new HashBindingSet()))); - runningQuery.startQuery(new BindingSetChunk( - queryEngine, - queryId, - startId,// - -1, //partitionId - new ThickAsynchronousIterator<IBindingSet[]>( - new IBindingSet[][] { new IBindingSet[] { new HashBindingSet()} }))); - // Wait until the query is done. final Map<Integer, BOpStats> statsMap = runningQuery.get(); { @@ -425,13 +499,12 @@ ) }; final long queryId = 1L; - final RunningQuery runningQuery = queryEngine.eval(queryId, query); + final RunningQuery runningQuery = queryEngine.eval(queryId, query, + new LocalChunkMessage<IBindingSet>(queryEngine, queryId, + startId,// + -1, /* partitionId */ + newBindingSetIterator(new HashBindingSet()))); - runningQuery.startQuery(new BindingSetChunk(queryEngine, queryId, - startId,// - -1, // partitionId - newBindingSetIterator(new HashBindingSet()))); - // verify solutions. TestQueryEngine.assertSameSolutionsAnyOrder(expected, new Dechunkerator<IBindingSet>(runningQuery.iterator())); @@ -618,22 +691,23 @@ new NV(Predicate.Annotations.BOP_ID, sliceId),// })); + // start the query. final long queryId = 1L; - final RunningQuery runningQuery = queryEngine.eval(queryId, query); + final IChunkMessage<IBindingSet> initialChunkMessage; + { - // start the query. - { - final IBindingSet initialBindings = new HashBindingSet(); - + initialBindings.set(Var.var("x"), new Constant<String>("Mary")); - runningQuery.startQuery(new BindingSetChunk(queryEngine, queryId, - startId,// + initialChunkMessage = new LocalChunkMessage<IBindingSet>( + queryEngine, queryId, startId,// -1, // partitionId - newBindingSetIterator(initialBindings))); + newBindingSetIterator(initialBindings)); } + final RunningQuery runningQuery = queryEngine.eval(queryId, query, + initialChunkMessage); // verify solutions. { Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestNIOChunkMessage.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestNIOChunkMessage.java 2010-09-14 20:48:21 UTC (rev 3554) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestNIOChunkMessage.java 2010-09-15 10:22:34 UTC (rev 3555) @@ -257,12 +257,6 @@ return null; } - @Override - public RunningQuery eval(long queryId, BindingSetPipelineOp query) - throws Exception, RemoteException { - return null; - } - } private static class MyNIOChunkMessage<E> extends NIOChunkMessage<E> { Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestThickChunkMessage.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestThickChunkMessage.java 2010-09-14 20:48:21 UTC (rev 3554) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestThickChunkMessage.java 2010-09-15 10:22:34 UTC (rev 3555) @@ -180,12 +180,6 @@ throws RemoteException { return null; } - - @Override - public RunningQuery eval(long queryId, BindingSetPipelineOp query) - throws Exception, RemoteException { - return null; - } } This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2010-09-15 19:52:07
|
Revision: 3558 http://bigdata.svn.sourceforge.net/bigdata/?rev=3558&view=rev Author: thompsonbry Date: 2010-09-15 19:52:00 +0000 (Wed, 15 Sep 2010) Log Message: ----------- Added stress tests and tracked down the query termination problem. In fact, it was an assertion on totalAvailableChunkCount in RunState. The total #of available chunks can transiently become negative during query evaluation due to the interleaving of operators for a given query. The assertion has been removed (conditionally disabled). Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/HaltOpMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunState.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/StartOpMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/resources/logging/log4j.properties branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/HaltOpMessage.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/HaltOpMessage.java 2010-09-15 15:54:52 UTC (rev 3557) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/HaltOpMessage.java 2010-09-15 19:52:00 UTC (rev 3558) @@ -72,7 +72,7 @@ * operator. */ final public int altSinkChunksOut; - + /** * The statistics for the execution of the bop against the partition on the * service. @@ -119,4 +119,21 @@ this.taskStats = taskStats; } + public String toString() { + final StringBuilder sb = new StringBuilder(getClass().getName()); + sb.append("{queryId=" + queryId); + sb.append(",bopId=" + bopId); + sb.append(",partitionId=" + partitionId); + sb.append(",serviceId=" + serviceId); + if (cause != null) + sb.append(",cause=" + cause); + sb.append(",sinkId=" + sinkId); + sb.append(",sinkChunksOut=" + sinkChunksOut); + sb.append(",altSinkId=" + altSinkId); + sb.append(",altSinkChunksOut=" + altSinkChunksOut); + sb.append(",stats=" + taskStats); + sb.append("}"); + return sb.toString(); + } + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunState.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunState.java 2010-09-15 15:54:52 UTC (rev 3557) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunState.java 2010-09-15 19:52:00 UTC (rev 3558) @@ -42,7 +42,9 @@ import com.bigdata.bop.BOp; /** - * The run state for a {@link RunningQuery}. + * The run state for a {@link RunningQuery}. This class is NOT thread-safe. + * {@link RunningQuery} uses an internal lock to serialize requests against the + * public methods of this class. * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id$ @@ -61,6 +63,16 @@ } /** + * Note: Due to concurrency, it is possible for an {@link IChunkMessage} to + * be accepted and the corresponding chunk task started, before a + * {@link RunState#startOp(StartOpMessage)} transition has been fully + * processed. This means that the {@link RunState#totalAvailableChunkCount} + * can become transiently negative. This flag disables asserts which would + * otherwise fail on legal transient negatives. + */ + static private boolean availableChunkCountMayBeNegative = true; + + /** * The query. */ private final RunningQuery query; @@ -76,19 +88,13 @@ private long nsteps = 0; /** - * The #of tasks for this query which have started but not yet halted and - * ZERO (0) if this is not the query coordinator. - * <p> - * This is guarded by the {@link #runningStateLock}. + * The #of tasks for this query which have started but not yet halted. */ private long totalRunningTaskCount = 0; /** * The #of chunks for this query of which a running task has made available - * but which have not yet been accepted for processing by another task and - * ZERO (0) if this is not the query coordinator. - * <p> - * This is guarded by the {@link #runningStateLock}. + * but which have not yet been accepted for processing by another task. */ private long totalAvailableChunkCount = 0; @@ -100,27 +106,22 @@ * <p> * The movement of the intermediate binding set chunks forms an acyclic * directed graph. This map is used to track the #of chunks available for - * each bop in the pipeline. When a bop has no more incoming chunks, we send - * an asynchronous message to all nodes on which that bop had executed - * informing the {@link QueryEngine} on that node that it should immediately - * release all resources associated with that bop. - * <p> - * This is guarded by the {@link #runningStateLock}. + * each {@link BOp} in the pipeline. When a {@link BOp} has no more incoming + * chunks, we send an asynchronous message to all nodes on which that + * {@link BOp} had executed informing the {@link QueryEngine} on that node + * that it should immediately release all resources associated with that + * {@link BOp}. */ private final Map<Integer/* bopId */, AtomicLong/* availableChunkCount */> availableChunkCountMap = new LinkedHashMap<Integer, AtomicLong>(); /** * A collection reporting on the #of instances of a given {@link BOp} which * are concurrently executing. - * <p> - * This is guarded by the {@link #runningStateLock}. */ private final Map<Integer/* bopId */, AtomicLong/* runningCount */> runningTaskCountMap = new LinkedHashMap<Integer, AtomicLong>(); /** * A collection of the operators which have executed at least once. - * <p> - * This is guarded by the {@link #runningStateLock}. */ private final Set<Integer/* bopId */> startedSet = new LinkedHashSet<Integer>(); @@ -140,6 +141,9 @@ // query.lifeCycleSetUpQuery(); + if (log.isInfoEnabled()) + log.info(msg.toString()); + final Integer bopId = Integer.valueOf(msg.getBOpId()); totalAvailableChunkCount++; @@ -161,11 +165,6 @@ } - if (log.isInfoEnabled()) - log.info("queryId=" + queryId + ",totalRunningTaskCount=" - + totalRunningTaskCount + ",totalAvailableChunkCount=" - + totalAvailableChunkCount); - if (TableLog.tableLog.isInfoEnabled()) { /* * Note: RunState is only used by the query controller so this will @@ -180,13 +179,14 @@ TableLog.tableLog.info("\n\nqueryId=" + queryId + "\n"); // TableLog.tableLog.info(query.getQuery().toString()+"\n"); TableLog.tableLog.info(getTableHeader()); - TableLog.tableLog.info(getTableRow("startQ", serviceId, - -1/* shardId */, 1/* fanIn */)); + TableLog.tableLog + .info(getTableRow("startQ", serviceId, msg.getBOpId(), + -1/* shardId */, 1/* fanIn */, null/* stats */)); } - System.err.println("startQ : nstep="+nsteps+", bopId=" + bopId - + ",totalRunningTaskCount=" + totalRunningTaskCount - + ",totalAvailableTaskCount=" + totalAvailableChunkCount); +// System.err.println("startQ : nstep="+nsteps+", bopId=" + bopId +// + ",totalRunningTaskCount=" + totalRunningTaskCount +// + ",totalAvailableTaskCount=" + totalAvailableChunkCount); } @@ -206,7 +206,9 @@ totalRunningTaskCount++; assert totalRunningTaskCount >= 1 : "runningTaskCount=" - + totalRunningTaskCount + " :: msg=" + msg; + + totalRunningTaskCount + " :: runState=" + this + ", msg=" + + msg; + final boolean firstTime; { @@ -218,7 +220,7 @@ final long tmp = n.incrementAndGet(); assert tmp >= 0 : "runningTaskCount=" + tmp + " for bopId=" - + msg.bopId + " :: msg=" + msg; + + msg.bopId + " :: runState=" + this + ", msg=" + msg; firstTime = startedSet.add(bopId); // @@ -231,35 +233,38 @@ totalAvailableChunkCount -= msg.nchunks; - assert totalAvailableChunkCount >= 0 : "totalAvailableChunkCount=" - + totalAvailableChunkCount + " :: msg=" + msg; + assert availableChunkCountMayBeNegative || totalAvailableChunkCount >= 0 : "totalAvailableChunkCount=" + + totalAvailableChunkCount + " :: runState=" + this + ", msg=" + + msg; { AtomicLong n = availableChunkCountMap.get(bopId); if (n == null) - throw new AssertionError(); + availableChunkCountMap.put(bopId, n = new AtomicLong()); final long tmp = n.addAndGet(-msg.nchunks); - assert tmp >= 0 : "availableChunkCount=" + tmp + " for bopId=" - + msg.bopId + " :: msg=" + msg; + assert availableChunkCountMayBeNegative || tmp >= 0 : "availableChunkCount=" + tmp + " for bopId=" + + msg.bopId + " :: runState=" + this + ", msg=" + msg; } - System.err.println("startOp: nstep="+nsteps+", bopId=" + bopId - + ",totalRunningTaskCount=" + totalRunningTaskCount - + ",totalAvailableChunkCount=" + totalAvailableChunkCount - + ",fanIn=" + msg.nchunks); +// System.err.println("startOp: nstep=" + nsteps + ", bopId=" + bopId +// + ",totalRunningTaskCount=" + totalRunningTaskCount +// + ",totalAvailableChunkCount=" + totalAvailableChunkCount +// + ",fanIn=" + msg.nchunks); if (TableLog.tableLog.isInfoEnabled()) { - TableLog.tableLog.info(getTableRow("startOp", msg.serviceId, - msg.partitionId, msg.nchunks/* fanIn */)); + TableLog.tableLog + .info(getTableRow("startOp", msg.serviceId, msg.bopId, + msg.partitionId, msg.nchunks/* fanIn */, null/* stats */)); } // check deadline. final long deadline = query.getDeadline(); + if (deadline < System.currentTimeMillis()) { if (log.isTraceEnabled()) @@ -271,6 +276,7 @@ query.cancel(true/* mayInterruptIfRunning */); } + return firstTime; } @@ -291,8 +297,9 @@ totalAvailableChunkCount += fanOut; - assert totalAvailableChunkCount >= 0 : "totalAvailableChunkCount=" - + totalAvailableChunkCount + " :: msg=" + msg; + assert availableChunkCountMayBeNegative || totalAvailableChunkCount >= 0 : "totalAvailableChunkCount=" + + totalAvailableChunkCount + " :: runState=" + this + + ", msg=" + msg; if (msg.sinkId != null) { AtomicLong n = availableChunkCountMap.get(msg.sinkId); @@ -302,8 +309,8 @@ final long tmp = n.addAndGet(msg.sinkChunksOut); - assert tmp >= 0 : "availableChunkCount=" + tmp + " for bopId=" - + msg.sinkId + " :: msg=" + msg; + assert availableChunkCountMayBeNegative || tmp >= 0 : "availableChunkCount=" + tmp + " for bopId=" + + msg.sinkId + " :: runState=" + this + ", msg=" + msg; } @@ -317,8 +324,9 @@ final long tmp = n.addAndGet(msg.altSinkChunksOut); - assert tmp >= 0 : "availableChunkCount=" + tmp + " for bopId=" - + msg.altSinkId + " :: msg=" + msg; + assert availableChunkCountMayBeNegative || tmp >= 0 : "availableChunkCount=" + tmp + " for bopId=" + + msg.altSinkId + " :: runState=" + this + ", msg=" + + msg; } @@ -328,7 +336,8 @@ totalRunningTaskCount--; assert totalRunningTaskCount >= 0 : "runningTaskCount=" - + totalRunningTaskCount + " :: msg=" + msg; + + totalRunningTaskCount + " :: runState=" + this + ", msg=" + + msg; { @@ -340,32 +349,35 @@ final long tmp = n.decrementAndGet(); assert tmp >= 0 : "runningTaskCount=" + tmp + " for bopId=" - + msg.bopId + " :: msg=" + msg; + + msg.bopId + " :: runState=" + this + ", msg=" + msg; } // Figure out if this operator is done. final boolean isDone = isOperatorDone(msg.bopId); - System.err.println("haltOp : nstep=" + nsteps + ", bopId=" + msg.bopId - + ",totalRunningTaskCount=" + totalRunningTaskCount - + ",totalAvailableTaskCount=" + totalAvailableChunkCount - + ",fanOut=" + fanOut); +// System.err.println("haltOp : nstep=" + nsteps + ", bopId=" + msg.bopId +// + ",totalRunningTaskCount=" + totalRunningTaskCount +// + ",totalAvailableTaskCount=" + totalAvailableChunkCount +// + ",fanOut=" + fanOut); if (TableLog.tableLog.isInfoEnabled()) { TableLog.tableLog.info(getTableRow("haltOp", msg.serviceId, - msg.partitionId, fanOut)); + msg.bopId, msg.partitionId, fanOut, msg.taskStats)); } - if (log.isTraceEnabled()) - log.trace("bopId=" + msg.bopId + ",partitionId=" + msg.partitionId - + ",serviceId=" + query.getQueryEngine().getServiceUUID() - + ", nchunks=" + fanOut + " : totalRunningTaskCount=" - + totalRunningTaskCount + ", totalAvailableChunkCount=" - + totalAvailableChunkCount); +// if (log.isTraceEnabled()) +// log.trace("bopId=" + msg.bopId + ",partitionId=" + msg.partitionId +// + ",serviceId=" + query.getQueryEngine().getServiceUUID() +// + ", nchunks=" + fanOut + " : totalRunningTaskCount=" +// + totalRunningTaskCount + ", totalAvailableChunkCount=" +// + totalAvailableChunkCount); - // test termination criteria + /* + * Test termination criteria + */ final long deadline = query.getDeadline(); + if (msg.cause != null) { // operator failed on this chunk. @@ -458,7 +470,7 @@ Arrays.sort(bopIds); // header 2. - sb.append("step\tlabel\tshardId\tfanIO\tavail\trun"); + sb.append("step\tlabel\tbopId\tshardId\tfanIO\tavail\trun"); for (int i = 0; i < bopIds.length; i++) { @@ -470,6 +482,10 @@ sb.append("\tserviceId"); + sb.append("\tbop"); + + sb.append("\tstats"); + sb.append('\n'); return sb.toString(); @@ -485,17 +501,23 @@ * @param label * The state change level (startQ, startOp, haltOp). * @param serviceId - * The node on which the operator is/was executed. + * The node on which the operator will be / was executed. + * @param bopId + * The identifier for the bop which will be / was executed. * @param shardId * The index partition against which the operator was running and * <code>-1</code> if the operator was not evaluated against a * specific index partition. - * @param * @param fanIO * The fanIn (startQ,startOp) or fanOut (haltOp). + * @param stats + * The statistics from the operator evaluation and + * <code>null</code> unless {@link #haltOp(HaltOpMessage)} is + * the invoker. */ private String getTableRow(final String label, final UUID serviceId, - final int shardId, final int fanIO) { + final int bopId, final int shardId, final int fanIO, + final BOpStats stats) { final StringBuilder sb = new StringBuilder(); @@ -503,6 +525,8 @@ sb.append('\t'); sb.append(label); sb.append('\t'); + sb.append(Integer.toString(bopId)); + sb.append('\t'); sb.append(Integer.toString(shardId)); sb.append('\t'); sb.append(Integer.toString(fanIO)); @@ -534,6 +558,15 @@ sb.append('\t'); sb.append(serviceId == null ? "N/A" : serviceId.toString()); + sb.append('\t'); + sb.append(query.bopIndex.get(bopId)); + + if (stats != null) { + // @todo use a multi-column version of stats. + sb.append('\t'); + sb.append(stats.toString()); + } + sb.append('\n'); return sb.toString(); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-15 15:54:52 UTC (rev 3557) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-15 19:52:00 UTC (rev 3558) @@ -776,7 +776,7 @@ if (altSink != null && altSink != queryBuffer && !altSink.isEmpty()) { /* - * Handle alt sink output , sending appropriate chunk + * Handle alt sink output, sending appropriate chunk * message(s). * * Note: This maps output over shards/nodes in s/o. Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/StartOpMessage.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/StartOpMessage.java 2010-09-15 15:54:52 UTC (rev 3557) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/StartOpMessage.java 2010-09-15 19:52:00 UTC (rev 3558) @@ -48,4 +48,10 @@ this.nchunks = nchunks; } + public String toString() { + return getClass().getName() + "{queryId=" + queryId + ",bopId=" + bopId + + ",partitionId=" + partitionId + ",serviceId=" + serviceId + + ",nchunks=" + nchunks + "}"; + } + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/resources/logging/log4j.properties =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/resources/logging/log4j.properties 2010-09-15 15:54:52 UTC (rev 3557) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/resources/logging/log4j.properties 2010-09-15 19:52:00 UTC (rev 3558) @@ -211,8 +211,8 @@ log4j.appender.dest2.layout.ConversionPattern=%-5p: %r %X{hostname} %X{serviceUUID} %X{taskname} %X{timestamp} %X{resources} %t %l: %m%n ## -# Rule execution log. This is a formatted log file (comma delimited). -log4j.logger.com.bigdata.bop.engine.RunState$TableLog=INFO,queryRunStateLog +# BOp run state trace (tab delimited file). Uncomment the next line to enable. +#log4j.logger.com.bigdata.bop.engine.RunState$TableLog=INFO,queryRunStateLog log4j.additivity.com.bigdata.bop.engine.RunState$TableLog=false log4j.appender.queryRunStateLog=org.apache.log4j.FileAppender log4j.appender.queryRunStateLog.Threshold=ALL Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-09-15 15:54:52 UTC (rev 3557) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-09-15 19:52:00 UTC (rev 3558) @@ -38,6 +38,7 @@ import java.util.concurrent.Executor; import java.util.concurrent.FutureTask; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import junit.framework.TestCase2; @@ -639,6 +640,12 @@ protected int doStressTest(final long timeout, final int ntrials, final int poolSize) throws Exception { + // start time in nanos. + final long begin = System.nanoTime(); + + // timeout in nanos. + final long nanos = TimeUnit.MILLISECONDS.toNanos(timeout); + final Executor service = new LatchedExecutor(jnl.getExecutorService(), poolSize); @@ -646,9 +653,12 @@ for (int i = 0; i < ntrials; i++) { + final int trial = i; final FutureTask<Void> ft = new FutureTask<Void>(new Runnable() { public void run() { try { + if (log.isInfoEnabled()) + log.info("trial=" + trial); test_query_join2(); } catch (Exception e) { // wrap exception. @@ -662,26 +672,30 @@ service.execute(ft); } - - Thread.sleep(timeout); - + int nerror = 0; int ncancel = 0; + int ntimeout = 0; int nsuccess = 0; for (FutureTask<Void> ft : futures) { - ft.cancel(true/* mayInterruptIfRunning */); + // remaining nanoseconds. + final long remaining = nanos - (System.nanoTime() - begin); + if (remaining <= 0) + ft.cancel(true/* mayInterruptIfRunning */); try { - ft.get(); + ft.get(remaining, TimeUnit.NANOSECONDS); nsuccess++; } catch (CancellationException ex) { ncancel++; + } catch (TimeoutException ex) { + ntimeout++; } catch (ExecutionException ex) { nerror++; } } final String msg = "nerror=" + nerror + ", ncancel=" + ncancel - + ", nsuccess=" + nsuccess; + + ", ntimeout=" + ntimeout + ", nsuccess=" + nsuccess; if(log.isInfoEnabled()) log.info(msg); This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2010-09-16 10:55:33
|
Revision: 3564 http://bigdata.svn.sourceforge.net/bigdata/?rev=3564&view=rev Author: thompsonbry Date: 2010-09-16 10:55:26 +0000 (Thu, 16 Sep 2010) Log Message: ----------- Working through SliceOp integration for standalone and scale-out. It currently cancels the query, which results in an exception being reported by RunningQuery.get(). That might be Ok, but the unit test needs to be updated and we need to report out the statistics anyway. I am also looking at termination conditions when a message is routed from a query peer to the query controller in scale-out, which is what happens for a SliceOp since it is evaluated at the query controller. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpUtility.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IChunkAccessor.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunState.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/MapBindingSetsOverShardsBuffer.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/NIOChunkMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ThickChunkMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/resources/logging/log4j.properties branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestThickChunkMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java 2010-09-15 23:01:17 UTC (rev 3563) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java 2010-09-16 10:55:26 UTC (rev 3564) @@ -168,7 +168,7 @@ * Perhaps the right thing is to expose an object with a richer API * for obtaining various kinds of iterators or even access to the * direct {@link ByteBuffer}s backing the data (for high volume joins, - * exernal merge sorts, etc). + * external merge sorts, etc). */ public final IAsynchronousIterator<E[]> getSource() { return source; Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpUtility.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpUtility.java 2010-09-15 23:01:17 UTC (rev 3563) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpUtility.java 2010-09-16 10:55:26 UTC (rev 3564) @@ -33,10 +33,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.log4j.Logger; - import com.bigdata.bop.BOp.Annotations; import com.bigdata.bop.engine.BOpStats; import com.bigdata.btree.AbstractNode; @@ -54,7 +51,7 @@ */ public class BOpUtility { - private static final Logger log = Logger.getLogger(BOpUtility.class); +// private static final Logger log = Logger.getLogger(BOpUtility.class); /** * Pre-order recursive visitation of the operator tree (arguments only, no Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IChunkAccessor.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IChunkAccessor.java 2010-09-15 23:01:17 UTC (rev 3563) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IChunkAccessor.java 2010-09-16 10:55:26 UTC (rev 3564) @@ -57,14 +57,14 @@ /** * Visit the binding sets in the chunk. * - * @deprecated We do not need to use {@link IAsynchronousIterator} any more. - * This could be much more flexible and should be harmonized to - * support high volume operators, GPU operators, etc. probably - * the right thing to do is introduce another interface here - * with a getChunk():IChunk where IChunk let's you access the - * chunks data in different ways (and chunks can be both - * {@link IBindingSet}[]s and element[]s so we might need to - * raise that into the interfaces and/or generics as well). + * @todo We do not need to use {@link IAsynchronousIterator} any more. This + * could be much more flexible and should be harmonized to support + * high volume operators, GPU operators, etc. probably the right thing + * to do is introduce another interface here with a getChunk():IChunk + * where IChunk let's you access the chunks data in different ways + * (and chunks can be both {@link IBindingSet}[]s and element[]s so we + * might need to raise that into the interfaces and/or generics as + * well). * * @todo It is likely that we can convert to the use of * {@link BlockingQueue} instead of {@link BlockingBuffer} in the Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunState.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunState.java 2010-09-15 23:01:17 UTC (rev 3563) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunState.java 2010-09-16 10:55:26 UTC (rev 3564) @@ -179,9 +179,9 @@ TableLog.tableLog.info("\n\nqueryId=" + queryId + "\n"); // TableLog.tableLog.info(query.getQuery().toString()+"\n"); TableLog.tableLog.info(getTableHeader()); - TableLog.tableLog - .info(getTableRow("startQ", serviceId, msg.getBOpId(), - -1/* shardId */, 1/* fanIn */, null/* stats */)); + TableLog.tableLog.info(getTableRow("startQ", serviceId, msg + .getBOpId(), -1/* shardId */, 1/* fanIn */, + null/* cause */, null/* stats */)); } // System.err.println("startQ : nstep="+nsteps+", bopId=" + bopId @@ -258,8 +258,9 @@ if (TableLog.tableLog.isInfoEnabled()) { TableLog.tableLog - .info(getTableRow("startOp", msg.serviceId, msg.bopId, - msg.partitionId, msg.nchunks/* fanIn */, null/* stats */)); +.info(getTableRow("startOp", msg.serviceId, + msg.bopId, msg.partitionId, msg.nchunks/* fanIn */, + null/* cause */, null/* stats */)); } // check deadline. @@ -363,7 +364,8 @@ if (TableLog.tableLog.isInfoEnabled()) { TableLog.tableLog.info(getTableRow("haltOp", msg.serviceId, - msg.bopId, msg.partitionId, fanOut, msg.taskStats)); + msg.bopId, msg.partitionId, fanOut, msg.cause, + msg.taskStats)); } // if (log.isTraceEnabled()) @@ -409,6 +411,7 @@ query.cancel(true/* mayInterruptIfRunning */); } + return isDone; } @@ -484,6 +487,8 @@ sb.append("\tbop"); + sb.append("\tcause"); + sb.append("\tstats"); sb.append('\n'); @@ -510,13 +515,18 @@ * specific index partition. * @param fanIO * The fanIn (startQ,startOp) or fanOut (haltOp). + * @param cause + * The {@link Throwable} in a {@link HaltOpMessage} and + * <code>null</code> for other messages or if the + * {@link Throwable} was null. * @param stats * The statistics from the operator evaluation and - * <code>null</code> unless {@link #haltOp(HaltOpMessage)} is - * the invoker. + * <code>null</code> unless {@link #haltOp(HaltOpMessage)} is the + * invoker. */ private String getTableRow(final String label, final UUID serviceId, final int bopId, final int shardId, final int fanIO, + final Throwable cause, final BOpStats stats) { final StringBuilder sb = new StringBuilder(); @@ -558,12 +568,19 @@ sb.append('\t'); sb.append(serviceId == null ? "N/A" : serviceId.toString()); + // the operator. sb.append('\t'); sb.append(query.bopIndex.get(bopId)); + + // the thrown cause. + sb.append('\t'); + if (cause != null) + sb.append(cause.getLocalizedMessage()); + // the statistics. + sb.append('\t'); if (stats != null) { // @todo use a multi-column version of stats. - sb.append('\t'); sb.append(stats.toString()); } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-15 23:01:17 UTC (rev 3563) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-16 10:55:26 UTC (rev 3564) @@ -51,7 +51,6 @@ import com.bigdata.bop.IBindingSet; import com.bigdata.bop.NoSuchBOpException; import com.bigdata.bop.PipelineOp; -import com.bigdata.bop.bset.CopyBindingSetOp; import com.bigdata.bop.solutions.SliceOp; import com.bigdata.journal.IIndexManager; import com.bigdata.journal.ITx; @@ -125,14 +124,12 @@ /** * The buffer used for the overall output of the query pipeline. - * - * FIXME SCALEOUT: This should only exist on the query controller. Other - * nodes will send {@link IChunkMessage}s to the query controller. s/o will - * use an operator with {@link BOpEvaluationContext#CONTROLLER} in order to - * ensure that the results are transferred to the query controller. When a - * {@link SliceOp} is used, this is redundant. The operator in other cases - * can be a {@link CopyBindingSetOp} whose {@link BOpEvaluationContext} has - * been overridden. + * <p> + * Note: In scale out, this only exists on the query controller. In order to + * ensure that the results are transferred to the query controller, the + * top-level operator in the query plan must specify + * {@link BOpEvaluationContext#CONTROLLER}. For example, {@link SliceOp} + * uses this {@link BOpEvaluationContext}. */ final private IBlockingBuffer<IBindingSet[]> queryBuffer; @@ -330,8 +327,15 @@ runState = controller ? new RunState(this) : null; - this.queryBuffer = newQueryBuffer(); + // Note: only exists on the query controller. + this.queryBuffer = controller ? newQueryBuffer() : null; +// System.err +// .println("new RunningQuery:: queryId=" + queryId +// + ", isController=" + controller + ", queryController=" +// + clientProxy + ", queryEngine=" +// + queryEngine.getServiceUUID()); + } /** @@ -619,6 +623,12 @@ /** Alias for the {@link ChunkTask}'s logger. */ private final Logger log = chunkTaskLog; + /** + * The message with the materialized chunk to be consumed by the + * operator. + */ + final IChunkMessage<IBindingSet> msg; + /** The index of the bop which is being evaluated. */ private final int bopId; @@ -682,13 +692,20 @@ * by {@link PipelineOp#eval(BOpContext)} in order to handle the outputs * written on those sinks. * - * @param chunk + * @param msg * A message containing the materialized chunk and metadata * about the operator which will consume that chunk. + * + * @throws IllegalStateException + * unless {@link IChunkMessage#isMaterialized()} is + * <code>true</code>. */ - public ChunkTask(final IChunkMessage<IBindingSet> chunk) { - bopId = chunk.getBOpId(); - partitionId = chunk.getPartitionId(); + public ChunkTask(final IChunkMessage<IBindingSet> msg) { + if(!msg.isMaterialized()) + throw new IllegalStateException(); + this.msg = msg; + bopId = msg.getBOpId(); + partitionId = msg.getPartitionId(); bop = bopIndex.get(bopId); if (bop == null) { throw new NoSuchBOpException(bopId); @@ -740,9 +757,9 @@ altSink = altSinkId == null ? null : op.newBuffer(); - // context + // context : @todo pass in IChunkMessage or IChunkAccessor context = new BOpContext<IBindingSet>(RunningQuery.this, - partitionId, op.newStats(), chunk.getChunkAccessor() + partitionId, op.newStats(), msg.getChunkAccessor() .iterator(), sink, altSink); // FutureTask for operator execution (not running yet). @@ -762,8 +779,7 @@ clientProxy.startOp(new StartOpMessage(queryId, bopId, partitionId, serviceId, fanIn)); if (log.isDebugEnabled()) - log.debug("Running chunk: queryId=" + queryId + ", bopId=" - + bopId + ", bop=" + bop); + log.debug("Running chunk: " + msg); ft.run(); // run ft.get(); // verify success if (sink != null && sink != queryBuffer && !sink.isEmpty()) { @@ -835,14 +851,20 @@ } // run() } // class ChunkTask - + /** * Return an iterator which will drain the solutions from the query. The * query will be cancelled if the iterator is * {@link ICloseableIterator#close() closed}. + * + * @throws UnsupportedOperationException + * if this is not the query controller. */ public IAsynchronousIterator<IBindingSet[]> iterator() { + if(!controller) + throw new UnsupportedOperationException(); + return queryBuffer.iterator(); } @@ -872,11 +894,35 @@ * <li>must not cause the solutions to be discarded before the client can * consume them.</li> * </ul> + * + * FIXME SCALEOUT: Each query engine peer touched by the running query (or + * known to have an operator task running at the time that the query was + * halted) must be notified that the query has been terminated and the + * receiving query engines must interrupt any running tasks which they have + * locally for that query. + * <p> + * Since this involves RMI to the nodes, we should not issue those RMIs + * while holding the {@link #runStateLock} (and this could even deadlock + * with callback from those nodes). Perhaps + * {@link RunState#haltOp(HaltOpMessage)} should throw back the + * {@link HaltOpMessage} or a {@link TimeoutException} if the deadline has + * expired and then let {@link RunningQuery#haltOp(HaltOpMessage)} handle + * the termination of the query, which it can do without holding the lock. + * <p> + * When the controller sends a node a terminate signal for an operator, it + * should not bother to RMI back to the controller (unless this is done for + * the purposes of confirmation, which is available from the RMI return in + * any case). + * + * FIXME SCALEOUT: Life cycle methods for operators must have hooks for the + * operator implementations which are evaluated on the query controller + * (here) but also on the nodes on which the query will run (for hash + * partitioned operators). */ final public boolean cancel(final boolean mayInterruptIfRunning) { // halt the query. boolean cancelled = future.cancel(mayInterruptIfRunning); - // cancel any running operators for this query. + // cancel any running operators for this query on this node. for (Future<?> f : operatorFutures.values()) { if (f.cancel(mayInterruptIfRunning)) cancelled = true; Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java 2010-09-15 23:01:17 UTC (rev 3563) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java 2010-09-16 10:55:26 UTC (rev 3564) @@ -359,18 +359,18 @@ + getServiceUUID() + ", msg=" + msg); } - // request from the query controller. + // request from the query controller (RMI). final BindingSetPipelineOp query = msg.getQueryController() .getQuery(msg.getQueryId()); q = newRunningQuery(FederatedQueryEngine.this, queryId, - isController, msg.getQueryController(), query); + false/* controller */, msg.getQueryController(), query); final RunningQuery tmp = runningQueries.putIfAbsent(queryId, q); if(tmp != null) { - // another thread won this race. + // another thread won this race : @todo memoize, RMI is too expensive. q = (FederatedRunningQuery) tmp; } @@ -424,8 +424,8 @@ final boolean controller, final IQueryClient clientProxy, final BindingSetPipelineOp query) { - return new FederatedRunningQuery(this, queryId, true/* controller */, - this/* clientProxy */, query); + return new FederatedRunningQuery(this, queryId, controller, + clientProxy, query); } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java 2010-09-15 23:01:17 UTC (rev 3563) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java 2010-09-16 10:55:26 UTC (rev 3564) @@ -42,13 +42,14 @@ import com.bigdata.bop.IBindingSet; import com.bigdata.bop.IPredicate; import com.bigdata.bop.IShardwisePipelineOp; -import com.bigdata.bop.engine.LocalChunkMessage; import com.bigdata.bop.engine.IChunkMessage; import com.bigdata.bop.engine.IQueryClient; import com.bigdata.bop.engine.IQueryPeer; +import com.bigdata.bop.engine.LocalChunkMessage; import com.bigdata.bop.engine.RunningQuery; import com.bigdata.io.DirectBufferPool; import com.bigdata.io.DirectBufferPoolAllocator.IAllocationContext; +import com.bigdata.journal.TemporaryStoreFactory; import com.bigdata.mdi.PartitionLocator; import com.bigdata.relation.accesspath.BlockingBuffer; import com.bigdata.relation.accesspath.IAsynchronousIterator; @@ -89,14 +90,24 @@ * this query. */ private final UUID queryControllerUUID; - + /** * A map associating resources with running queries. When a query halts, the * resources listed in its resource map are released. Resources can include * {@link ByteBuffer}s backing either incoming or outgoing - * {@link LocalChunkMessage}s, temporary files associated with the query, hash - * tables, etc. + * {@link LocalChunkMessage}s, temporary files associated with the query, + * hash tables, etc. * + * @todo The {@link IAllocationContext} allows us to automatically release + * native {@link ByteBuffer}s used by the query. Such buffers do not + * need to be part of this map. This means that the only real use for + * the map will be temporary persistent resources, such as graphs or + * hash tables backed by a local file or the intermediate outputs of a + * sort operator. We may be able to manage the local persistent data + * structures using the {@link TemporaryStoreFactory} and manage the + * life cycle of the intermediate results for sort within its operator + * implementation. + * * @todo This map will eventually need to be moved into {@link RunningQuery} * in order to support temporary graphs or other disk-backed resources * associated with the evaluation of a query against a standalone @@ -111,8 +122,9 @@ * * @todo Only use the values in the map for transient objects, such as a * hash table which is not backed by the disk. For {@link ByteBuffer}s - * we want to make the references go through the {@link ResourceService} - * . For files, through the {@link ResourceManager}. + * we want to make the references go through the + * {@link ResourceService} . For files, through the + * {@link ResourceManager}. * * @todo We need to track the resources in use by the query so they can be * released when the query terminates. This includes: buffers; joins @@ -292,7 +304,7 @@ if(serviceUUID.equals(getQueryEngine().getServiceUUID())) { - // Return a hard reference to the query engine (NOT a proxy). + // Return a hard reference to this query engine (NOT a proxy). return getQueryEngine(); } else if (serviceUUID.equals(queryControllerUUID)) { @@ -369,6 +381,9 @@ switch (bop.getEvaluationContext()) { case ANY: { + /* + * This operator may be evaluated anywhere. + */ return super.handleOutputChunk(sinkId, sink); } case HASHED: { @@ -490,11 +505,7 @@ sendChunkMessage(queryControllerUUID, sinkId, -1/* partitionId */, allocationContext, sink); - /* - * Chunks send to the query controller do not keep the query - * running. - */ - return 0; + return 1; } default: @@ -588,7 +599,7 @@ if (source.isEmpty()) throw new RuntimeException(); - // The peer to be notified. + // The peer to whom we send the message. final IQueryPeer peerProxy = getQueryPeer(serviceUUID); if (peerProxy == null) Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/MapBindingSetsOverShardsBuffer.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/MapBindingSetsOverShardsBuffer.java 2010-09-15 23:01:17 UTC (rev 3563) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/MapBindingSetsOverShardsBuffer.java 2010-09-16 10:55:26 UTC (rev 3564) @@ -395,8 +395,10 @@ slice[j] = bset; if (log.isTraceEnabled()) - log.trace("Mapping: keyOrder=" + keyOrder + ",bset=" + bset - + " onto " + split.pmd.getPartitionId()); + log + .trace("Mapping: keyOrder=" + keyOrder + ",bset=" + + bset + " onto partitionId=" + + split.pmd.getPartitionId()); } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/NIOChunkMessage.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/NIOChunkMessage.java 2010-09-15 23:01:17 UTC (rev 3563) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/NIOChunkMessage.java 2010-09-16 10:55:26 UTC (rev 3564) @@ -511,7 +511,7 @@ } public boolean isExhausted() { - return hasNext(); + return !hasNext(); } public E[] next(long timeout, TimeUnit unit) Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ThickChunkMessage.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ThickChunkMessage.java 2010-09-15 23:01:17 UTC (rev 3563) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/ThickChunkMessage.java 2010-09-16 10:55:26 UTC (rev 3564) @@ -235,6 +235,7 @@ } + @SuppressWarnings("unchecked") public boolean hasNext() { if (current != null) @@ -293,7 +294,7 @@ } public boolean isExhausted() { - return hasNext(); + return !hasNext(); } public E[] next(long timeout, TimeUnit unit) Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 2010-09-15 23:01:17 UTC (rev 3563) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 2010-09-16 10:55:26 UTC (rev 3564) @@ -97,7 +97,7 @@ public class PipelineJoin<E> extends BindingSetPipelineOp implements IShardwisePipelineOp<E> { - static private final Logger log = Logger.getLogger(PipelineJoin.class); + static private final transient Logger log = Logger.getLogger(PipelineJoin.class); /** * Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java 2010-09-15 23:01:17 UTC (rev 3563) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java 2010-09-16 10:55:26 UTC (rev 3564) @@ -132,25 +132,25 @@ // // if (!(args[0] instanceof BindingSetPipelineOp)) // throw new IllegalArgumentException(); - + } /** - * @see Annotations#OFFSET + * @see Annotations#OFFSET */ public long getOffset() { - - return (Long) getRequiredProperty(Annotations.OFFSET); - + + return getProperty(Annotations.OFFSET, Annotations.DEFAULT_OFFSET); + } /** - * @see Annotations#LIMIT + * @see Annotations#LIMIT */ public long getLimit() { + + return getProperty(Annotations.LIMIT, Annotations.DEFAULT_LIMIT); - return (Long) getRequiredProperty(Annotations.LIMIT); - } public FutureTask<Void> eval(final BOpContext<IBindingSet> context) { Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/resources/logging/log4j.properties =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/resources/logging/log4j.properties 2010-09-15 23:01:17 UTC (rev 3563) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/resources/logging/log4j.properties 2010-09-16 10:55:26 UTC (rev 3564) @@ -89,7 +89,8 @@ #log4j.logger.com.bigdata.service=ALL #log4j.logger.com.bigdata.bop=ALL -#log4j.logger.com.bigdata.bop.join.PipelineJoin=ALL +log4j.logger.com.bigdata.bop.join.PipelineJoin=ALL + log4j.logger.com.bigdata.bop.engine=ALL log4j.logger.com.bigdata.bop.engine.QueryEngine=ALL log4j.logger.com.bigdata.bop.engine.RunningQuery=ALL @@ -98,6 +99,7 @@ log4j.logger.com.bigdata.bop.fed.FederatedQueryEngine=ALL log4j.logger.com.bigdata.bop.fed.FederatedRunningQuery=ALL log4j.logger.com.bigdata.bop.fed.MapBindingSetsOverShardsBuffer=ALL + #log4j.logger.com.bigdata.relation.rule.eval.RuleLog=INFO #log4j.logger.com.bigdata.relation.rule.eval=INFO #log4j.logger.com.bigdata.relation.rule.eval.RuleState=DEBUG @@ -212,7 +214,7 @@ ## # BOp run state trace (tab delimited file). Uncomment the next line to enable. -#log4j.logger.com.bigdata.bop.engine.RunState$TableLog=INFO,queryRunStateLog +log4j.logger.com.bigdata.bop.engine.RunState$TableLog=INFO,queryRunStateLog log4j.additivity.com.bigdata.bop.engine.RunState$TableLog=false log4j.appender.queryRunStateLog=org.apache.log4j.FileAppender log4j.appender.queryRunStateLog.Threshold=ALL Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-09-15 23:01:17 UTC (rev 3563) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-09-16 10:55:26 UTC (rev 3564) @@ -61,6 +61,7 @@ import com.bigdata.bop.bset.StartOp; import com.bigdata.bop.fed.TestFederatedQueryEngine; import com.bigdata.bop.join.PipelineJoin; +import com.bigdata.bop.solutions.SliceOp; import com.bigdata.journal.BufferMode; import com.bigdata.journal.ITx; import com.bigdata.journal.Journal; @@ -139,12 +140,12 @@ final R rel = new R(store, namespace, ITx.UNISOLATED, new Properties()); rel.create(); - // data to insert. + // data to insert (in key order for convenience). final E[] a = {// - new E("John", "Mary"),// - new E("Mary", "Paul"),// - new E("Paul", "Leon"),// - new E("Leon", "Paul"),// + new E("John", "Mary"),// [0] + new E("Leon", "Paul"),// [1] + new E("Mary", "Paul"),// [2] + new E("Paul", "Leon"),// [3] }; // insert data (the records are not pre-sorted). @@ -270,7 +271,7 @@ final int predId = 3; final BindingSetPipelineOp query = new PipelineJoin<E>( // left - new CopyBindingSetOp(new BOp[] {}, NV.asMap(new NV[] {// + new StartOp(new BOp[] {}, NV.asMap(new NV[] {// new NV(Predicate.Annotations.BOP_ID, startId),// })), // right @@ -373,23 +374,141 @@ } /** - * @todo Test ability to impose a limit/offset slice on a query. - * <p> - * Note: While the logic for visiting only the solutions selected by - * the slice can be tested against a mock object, the integration by - * which a slice halts a query when it is satisfied has to be tested - * against a {@link QueryEngine}. - * <p> - * This must also be tested in scale-out to make sure that the data - * backing the solutions is not discarded before the caller can use - * those data. [This could be handled by materializing binding set - * objects out of a {@link ByteBuffer} rather than using a live decode - * of the data in that {@link ByteBuffer}.] + * Run a join with a slice. The slice is always evaluated on the query + * controller so adding it to the query plan touches a slightly different + * code path from adding another join (joins are evaluated shardwise, at + * least in scale-out). + * <p> + * Note: While the logic for visiting only the solutions selected by the + * slice can be tested against a mock object, the integration by which a + * slice halts a query when it is satisfied has to be tested against a + * {@link QueryEngine}. + * <p> + * This must also be tested in scale-out to make sure that the data backing + * the solutions is not discarded before the caller can use those data. + * [This could be handled by materializing binding set objects out of a + * {@link ByteBuffer} rather than using a live decode of the data in that + * {@link ByteBuffer}.] */ - public void test_query_slice() { + public void test_query_slice() throws Exception { - fail("write test"); + final Var<?> x = Var.var("x"); + final Var<?> y = Var.var("y"); + final int startId = 1; + final int joinId = 2; + final int predId = 3; + final int sliceId = 4; + + final StartOp startOp = new StartOp(new BOp[] {}, NV.asMap(new NV[] {// + new NV(Predicate.Annotations.BOP_ID, startId),// + })); + + final Predicate<E> predOp = new Predicate<E>(new IVariableOrConstant[] { + x, y }, NV.asMap(new NV[] {// + new NV(Predicate.Annotations.RELATION_NAME, + new String[] { namespace }),// + new NV(Predicate.Annotations.PARTITION_ID, Integer + .valueOf(-1)),// + new NV(Predicate.Annotations.OPTIONAL, Boolean.FALSE),// + new NV(Predicate.Annotations.CONSTRAINT, null),// + new NV(Predicate.Annotations.EXPANDER, null),// + new NV(Predicate.Annotations.BOP_ID, predId),// + new NV(Predicate.Annotations.TIMESTAMP, + ITx.READ_COMMITTED),// + })); + + final PipelineJoin<E> joinOp = new PipelineJoin<E>(startOp/* left */, + predOp/* right */, + // join annotations + NV.asMap(new NV[] { // + new NV(Predicate.Annotations.BOP_ID, joinId),// + })// + ); + + final BindingSetPipelineOp query = new SliceOp(new BOp[] { joinOp }, + // slice annotations + NV.asMap(new NV[] { // + new NV(BOp.Annotations.BOP_ID, sliceId),// + new NV(SliceOp.Annotations.OFFSET, 0L),// + new NV(SliceOp.Annotations.LIMIT, 2L),// + })// + ); + + // the expected solutions. + final IBindingSet[] expected = new IBindingSet[] {// + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { new Constant<String>("John"), + new Constant<String>("Mary") }// + ),// + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { new Constant<String>("Leon"), + new Constant<String>("Paul") }// + ) }; + + final UUID queryId = UUID.randomUUID(); + final RunningQuery runningQuery = queryEngine.eval(queryId, query, + new LocalChunkMessage<IBindingSet>(queryEngine, queryId, + startId, -1 /* partitionId */, + newBindingSetIterator(new HashBindingSet()))); + + // verify solutions. + assertSameSolutions(expected, runningQuery.iterator()); + + // Wait until the query is done. + final Map<Integer, BOpStats> statsMap = runningQuery.get(); + { + // validate the stats map. + assertNotNull(statsMap); + assertEquals(3, statsMap.size()); + if (log.isInfoEnabled()) + log.info(statsMap.toString()); + } + + // validate the stats for the start operator. + { + final BOpStats stats = statsMap.get(startId); + assertNotNull(stats); + if (log.isInfoEnabled()) + log.info("start: " + stats.toString()); + + // verify query solution stats details. + assertEquals(1L, stats.chunksIn.get()); + assertEquals(1L, stats.unitsIn.get()); + assertEquals(1L, stats.unitsOut.get()); + assertEquals(1L, stats.chunksOut.get()); + } + + // validate the stats for the join operator. + { + final BOpStats stats = statsMap.get(joinId); + assertNotNull(stats); + if (log.isInfoEnabled()) + log.info("join : " + stats.toString()); + + // verify query solution stats details. + assertEquals(1L, stats.chunksIn.get()); + assertEquals(1L, stats.unitsIn.get()); + assertEquals(4L, stats.unitsOut.get()); + assertEquals(1L, stats.chunksOut.get()); + } + + // validate the stats for the slice operator. + { + final BOpStats stats = statsMap.get(sliceId); + assertNotNull(stats); + if (log.isInfoEnabled()) + log.info("slice: " + stats.toString()); + + // verify query solution stats details. + assertEquals(1L, stats.chunksIn.get()); + assertEquals(4L, stats.unitsIn.get()); + assertEquals(2L, stats.unitsOut.get()); + assertEquals(1L, stats.chunksOut.get()); + } + } /** @@ -788,7 +907,7 @@ if (!actual.hasNext()) { fail(msg - + ": Index exhausted while expecting more object(s)" + + ": Iterator exhausted while expecting more object(s)" + ": index=" + j); } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java 2010-09-15 23:01:17 UTC (rev 3563) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java 2010-09-16 10:55:26 UTC (rev 3564) @@ -509,6 +509,20 @@ assertEquals(1L, stats.chunksOut.get()); // @todo this depends on which index partitions we read on. } + // validate the stats for the slice operator. + { + final BOpStats stats = statsMap.get(sliceId); + assertNotNull(stats); + if (log.isInfoEnabled()) + log.info("slice: "+stats.toString()); + + // verify query solution stats details. + assertEquals(1L, stats.chunksIn.get()); + assertEquals(2L, stats.unitsIn.get()); + assertEquals(2L, stats.unitsOut.get()); + assertEquals(1L, stats.chunksOut.get()); + } + } /** Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestThickChunkMessage.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestThickChunkMessage.java 2010-09-15 23:01:17 UTC (rev 3563) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestThickChunkMessage.java 2010-09-16 10:55:26 UTC (rev 3564) @@ -70,6 +70,52 @@ } /** + * Unit test for a message with a single chunk containing a single empty + * binding set. + */ + public void test_oneChunkWithEmptyBindingSet() { + + final List<IBindingSet> data = new LinkedList<IBindingSet>(); + { + data.add(new HashBindingSet()); + } + + final IQueryClient queryController = new MockQueryController(); + final UUID queryId = UUID.randomUUID(); + final int bopId = 1; + final int partitionId = 2; + final IBlockingBuffer<IBindingSet[]> source = new BlockingBuffer<IBindingSet[]>( + 10); + + // populate the source. + source.add(data.toArray(new IBindingSet[0])); + + // close the source. + source.close(); + + // build the chunk. + final IChunkMessage<IBindingSet> msg = new ThickChunkMessage<IBindingSet>( + queryController, queryId, bopId, partitionId, source); + + assertTrue(queryController == msg.getQueryController()); + + assertEquals(queryId, msg.getQueryId()); + + assertEquals(bopId, msg.getBOpId()); + + assertEquals(partitionId, msg.getPartitionId()); + + // the data is inline with the message. + assertTrue(msg.isMaterialized()); + + // verify the iterator. + assertSameIterator(data.toArray(new IBindingSet[0]), + new Dechunkerator<IBindingSet>(msg.getChunkAccessor() + .iterator())); + + } + + /** * Unit test for a message with a single chunk of binding sets. */ public void test_oneChunk() { Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java 2010-09-15 23:01:17 UTC (rev 3563) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java 2010-09-16 10:55:26 UTC (rev 3564) @@ -178,6 +178,10 @@ new NV(SliceOp.Annotations.LIMIT, 3L),// })); + assertEquals("offset", 1L, query.getOffset()); + + assertEquals("limit", 3L, query.getLimit()); + // the expected solutions final IBindingSet[] expected = new IBindingSet[] {// new ArrayBindingSet(// This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2010-09-16 20:29:58
|
Revision: 3574 http://bigdata.svn.sourceforge.net/bigdata/?rev=3574&view=rev Author: thompsonbry Date: 2010-09-16 20:29:51 +0000 (Thu, 16 Sep 2010) Log Message: ----------- Added a unit test for the query engine in which an IConstraint is applied. Identified a problem with multiple concurrent evaluation of SliceOp. It needs to be modified to use the same state for each invocation and to use CATs (or AtomicLong or chunk-wise locking, or bop invocation wise locking) to prevent concurrency failures (such as letting through too many solutions). Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/EQConstant.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java 2010-09-16 19:43:08 UTC (rev 3573) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java 2010-09-16 20:29:51 UTC (rev 3574) @@ -638,6 +638,14 @@ } + if (log.isTraceEnabled()) { + + log.debug("Accepted by " + + constraint.getClass().getSimpleName() + " : " + + bindingSet); + + } + } return true; Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/EQConstant.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/EQConstant.java 2010-09-16 19:43:08 UTC (rev 3573) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/EQConstant.java 2010-09-16 20:29:51 UTC (rev 3574) @@ -70,15 +70,21 @@ } - public boolean accept(final IBindingSet s) { + public boolean accept(final IBindingSet bset) { + final IVariable<?> var = (IVariable<?>) get(0)/* var */; + // get binding for the variable. - final IConstant<?> tmp = s.get((IVariable<?>) get(0)/* var */); + final IConstant<?> asBound = bset.get(var); - if (tmp == null) + if (asBound == null) return true; // not yet bound. - return tmp.equals(get(1)); + final IConstant<?> cnst = (IConstant<?>) get(1); + + final boolean ret = asBound.equals(cnst); + + return ret; } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 2010-09-16 19:43:08 UTC (rev 3573) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java 2010-09-16 20:29:51 UTC (rev 3574) @@ -1730,10 +1730,14 @@ } if (log.isDebugEnabled()) - log.debug("Accepted element for " + naccepted - + " of " + bindingSets.length - + " possible bindingSet combinations: " - + e.toString() + ", joinOp=" + joinOp); + if (naccepted == 0) { + log.debug("Rejected element: " + e.toString()); + } else { + log.debug("Accepted element for " + naccepted + + " of " + bindingSets.length + + " possible bindingSet combinations: " + + e.toString()); + } } // if something is accepted in the chunk return true. Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-09-16 19:43:08 UTC (rev 3573) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-09-16 20:29:51 UTC (rev 3574) @@ -50,6 +50,7 @@ import com.bigdata.bop.HashBindingSet; import com.bigdata.bop.IBindingSet; import com.bigdata.bop.IConstant; +import com.bigdata.bop.IConstraint; import com.bigdata.bop.IVariable; import com.bigdata.bop.IVariableOrConstant; import com.bigdata.bop.NV; @@ -59,6 +60,7 @@ import com.bigdata.bop.ap.R; import com.bigdata.bop.bset.ConditionalRoutingOp; import com.bigdata.bop.bset.StartOp; +import com.bigdata.bop.constraint.EQConstant; import com.bigdata.bop.fed.TestFederatedQueryEngine; import com.bigdata.bop.join.PipelineJoin; import com.bigdata.bop.solutions.SliceOp; @@ -68,6 +70,7 @@ import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.ThickAsynchronousIterator; import com.bigdata.striterator.ChunkedArrayIterator; +import com.bigdata.striterator.Dechunkerator; import com.bigdata.striterator.ICloseableIterator; import com.bigdata.util.concurrent.LatchedExecutor; import com.ibm.icu.impl.ByteBuffer; @@ -515,6 +518,173 @@ } /** + * A join with an {@link IConstraint}. + */ + public void test_query_join_withConstraint() throws Exception { + + final Var<?> x = Var.var("x"); + final Var<?> y = Var.var("y"); + + final int startId = 1; + final int joinId = 2; + final int predId = 3; + final int sliceId = 4; + + final StartOp startOp = new StartOp(new BOp[] {}, NV.asMap(new NV[] {// + new NV(Predicate.Annotations.BOP_ID, startId),// + })); + + /* + * + * Note: Since the index on which this reads is formed as (column1 + + * column2) the probe key will be [null] if it does not bind the first + * column. Therefore, in order to have the 2nd column constraint we have + * to model it as an IElementFilter on the predicate. + */ + final Predicate<E> predOp = new Predicate<E>(new IVariableOrConstant[] { + x, y}, NV + .asMap(new NV[] {// + new NV(Predicate.Annotations.RELATION_NAME, + new String[] { namespace }),// + new NV(Predicate.Annotations.PARTITION_ID, Integer + .valueOf(-1)),// + new NV(Predicate.Annotations.OPTIONAL, Boolean.FALSE),// + new NV(Predicate.Annotations.CONSTRAINT,null),// + new NV(Predicate.Annotations.EXPANDER, null),// + new NV(Predicate.Annotations.BOP_ID, predId),// + new NV(Predicate.Annotations.TIMESTAMP, + ITx.READ_COMMITTED),// + new NV(Predicate.Annotations.KEY_ORDER, + R.primaryKeyOrder),// + })); + + final PipelineJoin<E> joinOp = new PipelineJoin<E>(startOp/* left */, + predOp/* right */, + // join annotations + NV.asMap(new NV[] {// + new NV(Predicate.Annotations.BOP_ID, joinId),// + // impose constraint on the join. + new NV(PipelineJoin.Annotations.CONSTRAINTS, + new IConstraint[] { new EQConstant(y, + new Constant<String>("Paul")) }),// + })// + ); + + final BindingSetPipelineOp query = new SliceOp(new BOp[] { joinOp }, + // slice annotations + NV.asMap(new NV[] {// + new NV(Predicate.Annotations.BOP_ID, sliceId),// + })// + ); + + // the expected solutions (order is not reliable due to concurrency). + final IBindingSet[] expected = new IBindingSet[] {// +// new ArrayBindingSet(// +// new IVariable[] { x, y },// +// new IConstant[] { new Constant<String>("John"), +// new Constant<String>("Mary") }// +// ), // + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { new Constant<String>("Leon"), + new Constant<String>("Paul") }// + ), // +// new ArrayBindingSet(// +// new IVariable[] { x, y },// +// new IConstant[] { new Constant<String>("Mary"), +// new Constant<String>("John") }// +// ), // + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { new Constant<String>("Mary"), + new Constant<String>("Paul") }// + ), // +// new ArrayBindingSet(// +// new IVariable[] { x, y },// +// new IConstant[] { new Constant<String>("Paul"), +// new Constant<String>("Leon") }// +// ), // + }; +// new E("John", "Mary"),// [0] +// new E("Leon", "Paul"),// [1] +// new E("Mary", "Paul"),// [2] +// new E("Paul", "Leon"),// [3] + + final RunningQuery runningQuery; + { + final IBindingSet initialBindingSet = new HashBindingSet(); + +// initialBindingSet.set(y, new Constant<String>("Paul")); + + final UUID queryId = UUID.randomUUID(); + + runningQuery = queryEngine.eval(queryId, query, + new LocalChunkMessage<IBindingSet>(queryEngine, queryId, + startId,// + -1, /* partitionId */ + newBindingSetIterator(initialBindingSet))); + } + + // verify solutions. + TestQueryEngine.assertSameSolutionsAnyOrder(expected, + new Dechunkerator<IBindingSet>(runningQuery.iterator())); + + // Wait until the query is done. + runningQuery.get(); + final Map<Integer, BOpStats> statsMap = runningQuery.getStats(); + { + // validate the stats map. + assertNotNull(statsMap); + assertEquals(3, statsMap.size()); + if (log.isInfoEnabled()) + log.info(statsMap.toString()); + } + + // validate the stats for the start operator. + { + final BOpStats stats = statsMap.get(startId); + assertNotNull(stats); + if (log.isInfoEnabled()) + log.info("start: "+stats.toString()); + + // verify query solution stats details. + assertEquals(1L, stats.chunksIn.get()); + assertEquals(1L, stats.unitsIn.get()); + assertEquals(1L, stats.unitsOut.get()); + assertEquals(1L, stats.chunksOut.get()); + } + + // validate the stats for the join operator. + { + final BOpStats stats = statsMap.get(joinId); + assertNotNull(stats); + if (log.isInfoEnabled()) + log.info("join : "+stats.toString()); + + // verify query solution stats details. + assertEquals(1L, stats.chunksIn.get()); + assertEquals(1L, stats.unitsIn.get()); + assertEquals(2L, stats.unitsOut.get()); + assertEquals(1L, stats.chunksOut.get()); + } + + // validate the stats for the slice operator. + { + final BOpStats stats = statsMap.get(sliceId); + assertNotNull(stats); + if (log.isInfoEnabled()) + log.info("slice: "+stats.toString()); + + // verify query solution stats details. + assertEquals(1L, stats.chunksIn.get()); + assertEquals(2L, stats.unitsIn.get()); + assertEquals(2L, stats.unitsOut.get()); + assertEquals(1L, stats.chunksOut.get()); + } + + } + + /** * @todo Test the ability run a query reading on an access path using a * element filter (other than DISTINCT). */ Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java 2010-09-16 19:43:08 UTC (rev 3573) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java 2010-09-16 20:29:51 UTC (rev 3574) @@ -44,6 +44,7 @@ import com.bigdata.bop.HashBindingSet; import com.bigdata.bop.IBindingSet; import com.bigdata.bop.IConstant; +import com.bigdata.bop.IConstraint; import com.bigdata.bop.IVariable; import com.bigdata.bop.IVariableOrConstant; import com.bigdata.bop.NV; @@ -52,6 +53,7 @@ import com.bigdata.bop.ap.Predicate; import com.bigdata.bop.ap.R; import com.bigdata.bop.bset.StartOp; +import com.bigdata.bop.constraint.EQConstant; import com.bigdata.bop.engine.BOpStats; import com.bigdata.bop.engine.IChunkMessage; import com.bigdata.bop.engine.LocalChunkMessage; @@ -63,6 +65,7 @@ import com.bigdata.bop.solutions.SliceOp; import com.bigdata.bop.solutions.SortOp; import com.bigdata.btree.keys.KeyBuilder; +import com.bigdata.counters.CAT; import com.bigdata.journal.BufferMode; import com.bigdata.journal.ITx; import com.bigdata.journal.Journal; @@ -331,6 +334,21 @@ } /** + * Return an {@link IAsynchronousIterator} that will read a single, chunk + * containing all of the specified {@link IBindingSet}s. + * + * @param bindingSets + * the binding sets. + */ + protected ThickAsynchronousIterator<IBindingSet[]> newBindingSetIterator( + final IBindingSet[] bindingSets) { + + return new ThickAsynchronousIterator<IBindingSet[]>( + new IBindingSet[][] { bindingSets }); + + } + + /** * Starts and stops the {@link QueryEngine}, but does not validate the * semantics of shutdown() versus shutdownNow() since we need to be * evaluating query mixes in order to verify the semantics of those @@ -404,6 +422,92 @@ } /** + * Unit test uses a {@link StartOp} to copy some binding sets through a + * {@link SliceOp} without involving any joins or access path reads. For + * this test, the binding sets never leave the query controller. + * + * @throws Exception + */ + public void test_query_startThenSlice() throws Exception { + + final int startId = 1; + final int sliceId = 4; + + final StartOp startOp = new StartOp(new BOp[] {}, NV.asMap(new NV[] {// + new NV(Predicate.Annotations.BOP_ID, startId),// + })); + + final BindingSetPipelineOp query = new SliceOp(new BOp[] { startOp }, + // slice annotations + NV.asMap(new NV[] {// + new NV(Predicate.Annotations.BOP_ID, sliceId),// + })// + ); + + // the expected solutions (order is not reliable due to concurrency). + final IBindingSet[] expected = new IBindingSet[] {// + new ArrayBindingSet(// + new IVariable[] { Var.var("value") },// + new IConstant[] { new Constant<String>("Paul") }// + ), // + new ArrayBindingSet(// + new IVariable[] { Var.var("value") },// + new IConstant[] { new Constant<String>("John") }// + ) }; + + final UUID queryId = UUID.randomUUID(); + final RunningQuery runningQuery = queryEngine.eval(queryId, query, + new LocalChunkMessage<IBindingSet>(queryEngine, queryId, + startId,// + -1, /* partitionId */ + newBindingSetIterator(expected))); + + // verify solutions. + TestQueryEngine.assertSameSolutionsAnyOrder(expected, + new Dechunkerator<IBindingSet>(runningQuery.iterator())); + + // Wait until the query is done. + runningQuery.get(); + final Map<Integer, BOpStats> statsMap = runningQuery.getStats(); + { + // validate the stats map. + assertNotNull(statsMap); + assertEquals(2, statsMap.size()); + if (log.isInfoEnabled()) + log.info(statsMap.toString()); + } + + // validate the stats for the start operator. + { + final BOpStats stats = statsMap.get(startId); + assertNotNull(stats); + if (log.isInfoEnabled()) + log.info("start: "+stats.toString()); + + // verify query solution stats details. + assertEquals(1L, stats.chunksIn.get()); + assertEquals((long) expected.length, stats.unitsIn.get()); + assertEquals((long) expected.length, stats.unitsOut.get()); + assertEquals(1L, stats.chunksOut.get()); + } + + // validate the stats for the slice operator. + { + final BOpStats stats = statsMap.get(sliceId); + assertNotNull(stats); + if (log.isInfoEnabled()) + log.info("slice: " + stats.toString()); + + // verify query solution stats details. + assertEquals(1L, stats.chunksIn.get()); + assertEquals((long) expected.length, stats.unitsIn.get()); + assertEquals((long) expected.length, stats.unitsOut.get()); + assertEquals(1L, stats.chunksOut.get()); + } + + } + + /** * Test the ability run a simple join. There are three operators. One feeds * an empty binding set[] into the join, another is the predicate for the * access path on which the join will read (it probes the index once for @@ -448,7 +552,7 @@ ); final BindingSetPipelineOp query = new SliceOp(new BOp[] { joinOp }, - // slice annotations + // slice annotations NV.asMap(new NV[] {// new NV(Predicate.Annotations.BOP_ID, sliceId),// })// @@ -533,16 +637,24 @@ /** * Test the ability run a simple join which is mapped across two index - * partitions. There are three operators. One feeds an empty binding set[] - * in which the 2nd column of the relation is bound into the join, another - * is the predicate for the access path on which the join will read (it will - * read everything since the primary key is on the first column then the - * second column and hence can not be used to select the index partition for - * this access path), and the third is the join itself. + * partitions. * - * @throws Exception + * FIXME This is failing because the {@link SliceOp} is not remembering its + * state across distinct invocations and is cancelling the query as soon as + * it exhausts its input. In order to have correct decision boundaries, + * slice needs to be invoked either once, concurrently if using {@link CAT} + * s, or in a series of presentations otherwise. + * <p> + * The easiest way to fix this is to have {@link SliceOp} specialize the + * {@link BOpStats}s and carry its state there. That will also make it safe + * for concurrent evaluation within the same query, and we will have to + * write a unit test for that. + * <p> + * I am not yet convinced that the problem with the test failure is double + * invocation of {@link SliceOp}. It could also be that we are not invoking + * it the 2nd time. */ - public void test_query_join1_2shards() throws Exception { + public void test_query_join_withConstraint_readsOn2shards() throws Exception { final Var<?> x = Var.var("x"); final Var<?> y = Var.var("y"); @@ -556,15 +668,22 @@ new NV(Predicate.Annotations.BOP_ID, startId),// })); + /* + * + * Note: Since the index on which this reads is formed as (column1 + + * column2) the probe key will be [null] if it does not bind the first + * column. Therefore, in order to have the 2nd column constraint we have + * to model it as an IElementFilter on the predicate. + */ final Predicate<E> predOp = new Predicate<E>(new IVariableOrConstant[] { - x, new Constant<String>("Paul")}, NV + x, y}, NV .asMap(new NV[] {// new NV(Predicate.Annotations.RELATION_NAME, new String[] { namespace }),// new NV(Predicate.Annotations.PARTITION_ID, Integer .valueOf(-1)),// new NV(Predicate.Annotations.OPTIONAL, Boolean.FALSE),// - new NV(Predicate.Annotations.CONSTRAINT, null),// + new NV(Predicate.Annotations.CONSTRAINT,null),// new NV(Predicate.Annotations.EXPANDER, null),// new NV(Predicate.Annotations.BOP_ID, predId),// new NV(Predicate.Annotations.TIMESTAMP, @@ -578,6 +697,10 @@ // join annotations NV.asMap(new NV[] {// new NV(Predicate.Annotations.BOP_ID, joinId),// + // impose constraint on the join. + new NV(PipelineJoin.Annotations.CONSTRAINTS, + new IConstraint[] { new EQConstant(y, + new Constant<String>("Paul")) }),// })// ); @@ -590,31 +713,16 @@ // the expected solutions (order is not reliable due to concurrency). final IBindingSet[] expected = new IBindingSet[] {// -// new ArrayBindingSet(// -// new IVariable[] { x, y },// -// new IConstant[] { new Constant<String>("John"), -// new Constant<String>("Mary") }// -// ), // new ArrayBindingSet(// new IVariable[] { x, y },// - new IConstant[] { new Constant<String>("John"), + new IConstant[] { new Constant<String>("Leon"), new Constant<String>("Paul") }// ), // -// new ArrayBindingSet(// -// new IVariable[] { x, y },// -// new IConstant[] { new Constant<String>("Mary"), -// new Constant<String>("John") }// -// ), // new ArrayBindingSet(// new IVariable[] { x, y },// new IConstant[] { new Constant<String>("Mary"), new Constant<String>("Paul") }// ), // -// new ArrayBindingSet(// -// new IVariable[] { x, y },// -// new IConstant[] { new Constant<String>("Paul"), -// new Constant<String>("Leon") }// -// ), // }; // // partition0 // new E("John", "Mary"),// @@ -628,7 +736,7 @@ { final IBindingSet initialBindingSet = new HashBindingSet(); - initialBindingSet.set(y, new Constant<String>("Paul")); +// initialBindingSet.set(y, new Constant<String>("Paul")); final UUID queryId = UUID.randomUUID(); This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2010-09-16 21:09:48
|
Revision: 3575 http://bigdata.svn.sourceforge.net/bigdata/?rev=3575&view=rev Author: thompsonbry Date: 2010-09-16 21:09:42 +0000 (Thu, 16 Sep 2010) Log Message: ----------- Modified SliceOp to be thread safe. It may be a CAS hotspot, but that can be addressed by batching each chunk through a lock. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java 2010-09-16 20:29:51 UTC (rev 3574) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java 2010-09-16 21:09:42 UTC (rev 3575) @@ -63,12 +63,6 @@ // */ // private final long startTime; -// /** -// * The index partition for which these statistics were collected or -1 -// * if the statistics are aggregated across index partitions. -// */ -// public final int partitionId; - /** * #of chunks in. */ @@ -116,7 +110,7 @@ sb.append(",unitsIn=" + unitsIn.estimate_get()); sb.append(",chunksOut=" + chunksOut.estimate_get()); sb.append(",unitsOut=" + unitsOut.estimate_get()); - toString(sb); + toString(sb); // extension hook sb.append("}"); return sb.toString(); } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java 2010-09-16 20:29:51 UTC (rev 3574) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java 2010-09-16 21:09:42 UTC (rev 3575) @@ -30,6 +30,7 @@ import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.FutureTask; +import java.util.concurrent.atomic.AtomicLong; import org.apache.log4j.Logger; @@ -55,11 +56,18 @@ * <p> * Note: When running on an {@link IBigdataFederation}, this operator must be * imposed on the query controller so it can count the solutions as they flow - * through. + * through - see {@link #getEvaluationContext()}. + * <p> + * Note: {@link SliceOp} is safe for concurrent invocations for the same query. + * Multiple chunks may flow through multiple invocations of the operator so long + * as they use the same {@link BOpStats} object. * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id$ * + * @todo unit test with stress test for concurrent {@link SliceOp} invocations + * against a streaming chunk producer. + * * @todo If this operator is invoked for each chunk output by a query onto the * pipeline then it will over produce unless (A) it is given the same * {@link BOpStats} each time; and (B) it is not invoked for two chunks @@ -156,7 +164,55 @@ return getProperty(Annotations.LIMIT, Annotations.DEFAULT_LIMIT); } + + /** + * Extends {@link BOpStats} to capture the state of the {@link SliceOp}. + */ + public static class SliceStats extends BOpStats { + + /** + * + */ + private static final long serialVersionUID = 1L; + + public final AtomicLong nseen = new AtomicLong(); + + public final AtomicLong naccepted = new AtomicLong(); + + @Override + public void add(final BOpStats o) { + + super.add(o); + + if (o instanceof SliceStats) { + + final SliceStats t = (SliceStats) o; + + nseen.addAndGet(t.nseen.get()); + + naccepted.addAndGet(t.naccepted.get()); + + } + + } + + @Override + protected void toString(final StringBuilder sb) { + + sb.append(",nseed=" + nseen); + + sb.append(",naccepted=" + naccepted); + + } + + } + public SliceStats newStats() { + + return new SliceStats(); + + } + public FutureTask<Void> eval(final BOpContext<IBindingSet> context) { return new FutureTask<Void>(new SliceTask(this, context)); @@ -179,12 +235,14 @@ /** #of solutions to accept. */ private final long limit; - /** #of solutions visited. */ - private long nseen; - - /** #of solutions accepted. */ - private long naccepted; - +// /** #of solutions visited. */ +// private long nseen; +// +// /** #of solutions accepted. */ +// private long naccepted; +// + private final SliceStats stats; + SliceTask(final SliceOp op, final BOpContext<IBindingSet> context) { this.op = op; @@ -201,6 +259,8 @@ if (limit <= 0) throw new IllegalArgumentException(Annotations.LIMIT); + this.stats = (SliceStats) context.getStats(); + } public Void call() throws Exception { @@ -211,14 +271,9 @@ final IAsynchronousIterator<IBindingSet[]> source = context .getSource(); - /* - * @todo This needs to be wrapping to automatically update the #of - * chunks actually output in order to have correct reporting. Review - * all of the other operators for this same issue. - */ final IBlockingBuffer<IBindingSet[]> sink = context.getSink(); - final BOpStats stats = context.getStats(); +// final BOpStats stats = context.getStats(); try { @@ -230,17 +285,20 @@ while (source.hasNext()) { + /* + * @todo batch each chunk through a lock for better + * concurrency (avoids CAS contention). + */ final IBindingSet[] chunk = source.next(); - + stats.chunksIn.increment(); for (int i = 0; i < chunk.length; i++) { stats.unitsIn.increment(); - if (nseen < offset) { + if (stats.nseen.incrementAndGet() <= offset) { // skip solution. - nseen++; if(log.isTraceEnabled()) log.trace(toString()); continue; @@ -258,9 +316,7 @@ // stats.unitsOut.increment(); - naccepted++; - nseen++; - if (naccepted >= limit) { + if (stats.naccepted.incrementAndGet() >= limit) { if (!out.isEmpty()) { out.flush(); // stats.chunksOut.increment(); @@ -306,8 +362,8 @@ public String toString() { return getClass().getName() + "{offset=" + offset + ",limit=" - + limit + ",nseen=" + nseen + ",naccepted=" + naccepted - + "}"; + + limit + ",nseen=" + stats.nseen + ",naccepted=" + + stats.naccepted + "}"; } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java 2010-09-16 20:29:51 UTC (rev 3574) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java 2010-09-16 21:09:42 UTC (rev 3575) @@ -643,7 +643,9 @@ * state across distinct invocations and is cancelling the query as soon as * it exhausts its input. In order to have correct decision boundaries, * slice needs to be invoked either once, concurrently if using {@link CAT} - * s, or in a series of presentations otherwise. + * s, or in a series of presentations otherwise (single-threaded operator or + * internal locking in the operator implementation on its {@link SliceOp} to + * achieve chunk-wise serialization of processing). * <p> * The easiest way to fix this is to have {@link SliceOp} specialize the * {@link BOpStats}s and carry its state there. That will also make it safe This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2010-09-17 10:44:08
|
Revision: 3577 http://bigdata.svn.sourceforge.net/bigdata/?rev=3577&view=rev Author: thompsonbry Date: 2010-09-17 10:44:01 +0000 (Fri, 17 Sep 2010) Log Message: ----------- fixed SliceOp and added a concurrent stress test. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java 2010-09-17 10:19:27 UTC (rev 3576) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java 2010-09-17 10:44:01 UTC (rev 3577) @@ -27,6 +27,7 @@ package com.bigdata.bop.solutions; +import java.math.BigInteger; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.FutureTask; @@ -40,6 +41,7 @@ import com.bigdata.bop.BindingSetPipelineOp; import com.bigdata.bop.IBindingSet; import com.bigdata.bop.engine.BOpStats; +import com.bigdata.bop.engine.QueryEngine; import com.bigdata.bop.engine.RunningQuery; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; @@ -65,20 +67,18 @@ * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id$ * - * @todo unit test with stress test for concurrent {@link SliceOp} invocations - * against a streaming chunk producer. + * @todo Unit test with stress test for concurrent {@link SliceOp} invocations + * against a streaming chunk producer. Make sure that the same + * {@link SliceStats} are used for each concurrent invocation of the same + * query. * - * @todo If this operator is invoked for each chunk output by a query onto the - * pipeline then it will over produce unless (A) it is given the same - * {@link BOpStats} each time; and (B) it is not invoked for two chunks - * concurrently. - * <p> - * A safer way to impose the slice constraint is by wrapping the query - * buffer on the query controller. Once the slice is satisfied, it can - * just cancel the query. The only drawback of this approach is that the - * wrapping a buffer is not really the same as applying a {@link BOp} to - * the pipeline so it falls outside of the standard operator evaluation - * logic. + * @todo What is sufficient serialization to make SLICE(ORDER_BY(...)) stable? + * The {@link SortOp} will impose a total ordering and will know how to + * deliver that total ordering to another operator. The {@link SliceOp} + * needs to accept the chunks from the {@link SortOp} in the order in + * which they were sent. This should work as long as we do not reorder the + * chunks for a given operator in the {@link QueryEngine} when they are + * received by the query controller. * * @todo If we allow complex operator trees in which "subqueries" can also use a * slice then either then need to run as their own query with their own @@ -175,8 +175,10 @@ */ private static final long serialVersionUID = 1L; + /** #of solutions visited. */ public final AtomicLong nseen = new AtomicLong(); + /** #of solutions accepted. */ public final AtomicLong naccepted = new AtomicLong(); @Override @@ -235,6 +237,8 @@ /** #of solutions to accept. */ private final long limit; + private final long last; + // /** #of solutions visited. */ // private long nseen; // @@ -261,6 +265,11 @@ this.stats = (SliceStats) context.getStats(); +// this.last = offset + limit; + this.last = BigInteger.valueOf(offset).add( + BigInteger.valueOf(limit)).min( + BigInteger.valueOf(Long.MAX_VALUE)).longValue(); + } public Void call() throws Exception { @@ -277,96 +286,175 @@ try { - // buffer forms chunks which get flushed onto the sink. + /* + * buffer forms chunks which get flushed onto the sink. + * + * @todo if we have visibility into the #of source chunks, then + * do not buffer more than min(#source,#needed). + */ final UnsynchronizedArrayBuffer<IBindingSet> out = new UnsynchronizedArrayBuffer<IBindingSet>( sink, op.getChunkCapacity()); boolean halt = false; - while (source.hasNext()) { + while (source.hasNext() && !halt) { + final IBindingSet[] chunk = source.next(); + /* - * @todo batch each chunk through a lock for better - * concurrency (avoids CAS contention). + * Batch each chunk through a lock for better concurrency + * (avoids CAS contention). + * + * Note: This is safe because the source chunk is already + * materialized and the sink will not block (that is part of + * the bop evaluation contract). */ - final IBindingSet[] chunk = source.next(); - - stats.chunksIn.increment(); - - for (int i = 0; i < chunk.length; i++) { - - stats.unitsIn.increment(); - - if (stats.nseen.incrementAndGet() <= offset) { - // skip solution. - if(log.isTraceEnabled()) - log.trace(toString()); - continue; - } - - final IBindingSet bset = chunk[i]; + synchronized (stats) { - if (out.add2(bset)) { - // chunk was output. -// stats.chunksOut.increment(); - } + if (handleChunk(out, chunk)) { - if(log.isTraceEnabled()) - log.trace(toString() + ":" + bset); + halt = true; -// stats.unitsOut.increment(); - - if (stats.naccepted.incrementAndGet() >= limit) { - if (!out.isEmpty()) { - out.flush(); -// stats.chunksOut.increment(); - } - halt = true; - break; } } - + } - out.flush(); + if (!out.isEmpty()) + out.flush(); + sink.flush(); -// stats.chunksOut.increment(); if (halt) throw new InterruptedException(); -// cancelQuery(); - + // cancelQuery(); + return null; - + } finally { - + sink.close(); - + } } -// /** -// * Cancel the query evaluation. This is invoked when the slice has been -// * satisfied. At that point we want to halt not only the {@link SliceOp} -// * but also the entire query since it does not need to produce any more -// * results. -// */ -// private void cancelQuery() { -// -// context.halt(); -// -// } + /** + * <p> + * Apply the slice semantics to a chunk of binding sets. + * </p> + * <h2>example</h2> + * <p> + * offset=2, limit=3, last=3+2=5. The number line represents the + * observed binding sets. The first binding set is at index ZERO (0). + * The initial conditions are: nseen(S)=0 and naccepted(A)=0. S is + * placed beneath each observation and paired with the value of A for + * that observation. The offset is satisfied when S=2 and observation + * ONE (1) is the first observation accepted. The limit is satisfied + * when A=3, which occurs at observation FOUR (4) which is also + * S=last=5. The observation on which the limit is satisfied is accepted + * and the slice halts as no more observations should be made. {2,3,4} + * are accepted. + * </p> + * + * <pre> + * 0 1 2 3 4 5 6 7 8 9 + * S=1, A=0 + * </pre> + * + * <pre> + * 0 1 2 3 4 5 6 7 8 9 + * S=2, A=0 + * </pre> + * + * <pre> + * 0 1 2 3 4 5 6 7 8 9 + * S=3, A=1 {2} + * </pre> + * + * <pre> + * 0 1 2 3 4 5 6 7 8 9 + * S=4, A=2 {2,3} + * </pre> + * + * <pre> + * 0 1 2 3 4 5 6 7 8 9 + * S=5, A=3 {2,3,4} + * </pre> + * <p> + * Note: The caller MUST be synchronized on the <em>shared</em> + * {@link SliceStats} in order for the decision process to be thread + * safe. + * + * @param chunk + * The chunk of binding sets. + * + * @return <code>true</code> if the slice is satisfied and the query + * should halt. + */ + private boolean handleChunk( + final UnsynchronizedArrayBuffer<IBindingSet> out, + final IBindingSet[] chunk) { + stats.chunksIn.increment(); + + for (int i = 0; i < chunk.length; i++) { + + if (stats.naccepted.get() >= limit) + return true; // nothing more will be accepted. + + stats.unitsIn.increment(); + + final long S = stats.nseen.incrementAndGet(); + + if (S <= offset) + continue; // skip solution. + + final long A = stats.naccepted.get(); + + if (A < limit) { + + final IBindingSet bset = chunk[i]; + + out.add(bset); + + stats.naccepted.incrementAndGet(); + + if (log.isTraceEnabled()) + log.trace(toString() + ":" + bset); + + } + + } // next bindingSet + + return false; + + } + + // /** + // * Cancel the query evaluation. This is invoked when the slice has + // been + // * satisfied. At that point we want to halt not only the {@link + // SliceOp} + // * but also the entire query since it does not need to produce any + // more + // * results. + // */ + // private void cancelQuery() { + // + // context.halt(); + // + // } + public String toString() { return getClass().getName() + "{offset=" + offset + ",limit=" + limit + ",nseen=" + stats.nseen + ",naccepted=" + stats.naccepted + "}"; - + } - + } /** @@ -374,9 +462,9 @@ */ @Override public BOpEvaluationContext getEvaluationContext() { - + return BOpEvaluationContext.CONTROLLER; - + } } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java 2010-09-17 10:19:27 UTC (rev 3576) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java 2010-09-17 10:44:01 UTC (rev 3577) @@ -28,8 +28,16 @@ package com.bigdata.bop.solutions; import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; import java.util.concurrent.FutureTask; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import junit.framework.TestCase2; @@ -37,6 +45,7 @@ import com.bigdata.bop.BOp; import com.bigdata.bop.BOpContext; import com.bigdata.bop.Constant; +import com.bigdata.bop.EmptyBindingSet; import com.bigdata.bop.HashBindingSet; import com.bigdata.bop.IBindingSet; import com.bigdata.bop.IConstant; @@ -44,8 +53,11 @@ import com.bigdata.bop.NV; import com.bigdata.bop.Var; import com.bigdata.bop.engine.BOpStats; +import com.bigdata.bop.engine.IRunningQuery; import com.bigdata.bop.engine.MockRunningQuery; import com.bigdata.bop.engine.TestQueryEngine; +import com.bigdata.bop.solutions.SliceOp.SliceStats; +import com.bigdata.relation.accesspath.BlockingBuffer; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; import com.bigdata.relation.accesspath.ThickAsynchronousIterator; @@ -142,6 +154,113 @@ * @throws ExecutionException * @throws InterruptedException */ + public void test_slice_offset2_limit3() throws InterruptedException, + ExecutionException { + + final Var<?> x = Var.var("x"); + final Var<?> y = Var.var("y"); + + final int bopId = 1; + + final long offset = 2L; + final long limit = 3L; + + final SliceOp query = new SliceOp(new BOp[]{}, + NV.asMap(new NV[]{// + new NV(SliceOp.Annotations.BOP_ID, bopId),// + new NV(SliceOp.Annotations.OFFSET, offset),// + new NV(SliceOp.Annotations.LIMIT, limit),// + })); + + assertEquals("offset", offset, query.getOffset()); + + assertEquals("limit", limit, query.getLimit()); + + // the expected solutions + final IBindingSet[] expected = new IBindingSet[] {// +// new ArrayBindingSet(// +// new IVariable[] { x, y },// +// new IConstant[] { new Constant<String>("John"), +// new Constant<String>("Mary"), }// +// ), +// new ArrayBindingSet(// +// new IVariable[] { x, y },// +// new IConstant[] { new Constant<String>("Mary"), +// new Constant<String>("Paul"), }// +// ), + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { new Constant<String>("Mary"), + new Constant<String>("Jane") }// + ), + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { new Constant<String>("Paul"), + new Constant<String>("Leon") }// + ), + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { new Constant<String>("Paul"), + new Constant<String>("John") }// + ), +// new ArrayBindingSet(// +// new IVariable[] { x, y },// +// new IConstant[] { new Constant<String>("Leon"), +// new Constant<String>("Paul") }// +// ), + }; + + final SliceStats stats = query.newStats(); + + final IAsynchronousIterator<IBindingSet[]> source = new ThickAsynchronousIterator<IBindingSet[]>( + new IBindingSet[][] { data.toArray(new IBindingSet[0]) }); + + final IBlockingBuffer<IBindingSet[]> sink = query.newBuffer(stats); + + final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( + new MockRunningQuery(null/* fed */, null/* indexManager */ + ), -1/* partitionId */, stats, + source, sink, null/* sink2 */); + + // get task. + final FutureTask<Void> ft = query.eval(context); + + ft.run(); + + TestQueryEngine.assertSameSolutions(expected, sink.iterator()); + + assertTrue(ft.isDone()); + assertFalse(ft.isCancelled()); + try { + ft.get(); // verify nothing thrown. + fail("Expecting inner cause : " + InterruptedException.class); + } catch (Throwable t) { + if (InnerCause.isInnerCause(t, InterruptedException.class)) { + if (log.isInfoEnabled()) + log.info("Ignoring expected exception: " + t, t); + } else { + fail("Expecting inner cause : " + InterruptedException.class); + } + } + + // check the slice stats first. + assertEquals(limit, stats.naccepted.get()); + assertEquals(offset+limit, stats.nseen.get()); + + // then the general purpose bop stats (less critical). + assertEquals(1L, stats.chunksIn.get()); + assertEquals(offset+limit, stats.unitsIn.get()); + assertEquals(limit, stats.unitsOut.get()); + assertEquals(1L, stats.chunksOut.get()); + + } + + /** + * Unit test for correct visitation for a variety of offset/limit values. + * + * @throws ExecutionException + * @throws InterruptedException + */ public void test_slice_offset1_limit3() throws InterruptedException, ExecutionException { @@ -149,17 +268,20 @@ final Var<?> y = Var.var("y"); final int bopId = 1; + + final long offset = 1; + final long limit = 3; final SliceOp query = new SliceOp(new BOp[]{}, NV.asMap(new NV[]{// new NV(SliceOp.Annotations.BOP_ID, bopId),// - new NV(SliceOp.Annotations.OFFSET, 1L),// - new NV(SliceOp.Annotations.LIMIT, 3L),// + new NV(SliceOp.Annotations.OFFSET, offset),// + new NV(SliceOp.Annotations.LIMIT, limit),// })); - assertEquals("offset", 1L, query.getOffset()); + assertEquals("offset", offset, query.getOffset()); - assertEquals("limit", 3L, query.getLimit()); + assertEquals("limit", limit, query.getLimit()); // the expected solutions final IBindingSet[] expected = new IBindingSet[] {// @@ -179,7 +301,7 @@ new Constant<String>("Leon") }// ), }; - final BOpStats stats = query.newStats(); + final SliceStats stats = query.newStats(); final IAsynchronousIterator<IBindingSet[]> source = new ThickAsynchronousIterator<IBindingSet[]>( new IBindingSet[][] { data.toArray(new IBindingSet[0]) }); @@ -212,6 +334,9 @@ } } + assertEquals(limit, stats.naccepted.get()); + assertEquals(offset+limit, stats.nseen.get()); + assertEquals(1L, stats.chunksIn.get()); assertEquals(4L, stats.unitsIn.get()); assertEquals(3L, stats.unitsOut.get()); @@ -222,9 +347,6 @@ public void test_slice_offset0_limitAll() throws InterruptedException, ExecutionException { - final Var<?> x = Var.var("x"); - final Var<?> y = Var.var("y"); - final int bopId = 1; final SliceOp query = new SliceOp(new BOp[] {}, NV.asMap(new NV[] {// @@ -240,7 +362,7 @@ // the expected solutions final IBindingSet[] expected = data.toArray(new IBindingSet[0]); - final BOpStats stats = query.newStats(); + final SliceStats stats = query.newStats(); final IAsynchronousIterator<IBindingSet[]> source = new ThickAsynchronousIterator<IBindingSet[]>( new IBindingSet[][] { data.toArray(new IBindingSet[0]) }); @@ -266,6 +388,8 @@ assertEquals(6L, stats.unitsIn.get()); assertEquals(6L, stats.unitsOut.get()); assertEquals(1L, stats.chunksOut.get()); + assertEquals(6L, stats.nseen.get()); + assertEquals(6L, stats.naccepted.get()); } @@ -342,4 +466,146 @@ } + public void test_slice_threadSafe() throws Exception { + + final long timeout = 10000; // ms + + final int ntrials = 10000; + + final int poolSize = 10; + + doStressTest(500L/* offset */, 1500L/* limit */, timeout, ntrials, + poolSize); + + } + + /** + * + * @param timeout + * @param ntrials + * @param poolSize + * + * @return The #of successful trials. + * + * @throws Exception + */ + protected int doStressTest(final long offset, final long limit, + final long timeout, final int ntrials, final int poolSize) + throws Exception { + + final IBindingSet[][] chunks = new IBindingSet[ntrials][]; + { + final Random r = new Random(); + final IBindingSet bset = EmptyBindingSet.INSTANCE; + for (int i = 0; i < chunks.length; i++) { + // random non-zero chunk size + chunks[i] = new IBindingSet[r.nextInt(10) + 1]; + for (int j = 0; j < chunks[i].length; j++) { + chunks[i][j] = bset; + } + } + } + final int bopId = 1; + final SliceOp query = new SliceOp(new BOp[] {}, NV.asMap(new NV[] {// + new NV(SliceOp.Annotations.BOP_ID, bopId),// + new NV(SliceOp.Annotations.OFFSET, offset),// + new NV(SliceOp.Annotations.LIMIT, limit),// + })); + + final SliceStats stats = query.newStats(); + + final IRunningQuery q = new MockRunningQuery(null/* fed */, null/* indexManager */); + + // start time in nanos. + final long begin = System.nanoTime(); + + // timeout in nanos. + final long nanos = TimeUnit.MILLISECONDS.toNanos(timeout); + + final ThreadPoolExecutor service = (ThreadPoolExecutor) Executors + .newFixedThreadPool(poolSize); + + try { + + service.prestartAllCoreThreads(); + + final List<FutureTask<Void>> futures = new LinkedList<FutureTask<Void>>(); + + for (int i = 0; i < ntrials; i++) { + + final IBindingSet[] chunk = chunks[i]; + + final IAsynchronousIterator<IBindingSet[]> source = new ThickAsynchronousIterator<IBindingSet[]>( + new IBindingSet[][] { chunk }); + + final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( + q, -1/* partitionId */, stats, source, + new BlockingBuffer<IBindingSet[]>(chunk.length), null/* sink2 */); + + final FutureTask<Void> ft = query.eval(context); + + futures.add(ft); + + service.execute(ft); + + } + + int nerror = 0; + int ncancel = 0; + int ntimeout = 0; + int nsuccess = 0; + int ninterrupt = 0; + for (FutureTask<Void> ft : futures) { + // remaining nanoseconds. + final long remaining = nanos - (System.nanoTime() - begin); + if (remaining <= 0) + ft.cancel(true/* mayInterruptIfRunning */); + try { + ft.get(remaining, TimeUnit.NANOSECONDS); + nsuccess++; + } catch (CancellationException ex) { + ncancel++; + } catch (TimeoutException ex) { + ntimeout++; + } catch (ExecutionException ex) { + if (InnerCause.isInnerCause(ex, InterruptedException.class)) { + ninterrupt++; + } else { + log.error(ex, ex); + nerror++; + } + } + } + + final long nseen = stats.nseen.get(); + + final long naccepted = stats.naccepted.get(); + + final long nexpected = limit; + + final String msg = "offset=" + offset + ", limit=" + limit + + ", nseen=" + nseen + ",naccepted=" + naccepted + + ", nexpected=" + nexpected + ", nerror=" + nerror + + ", ncancel=" + ncancel + ", ntimeout=" + ntimeout + + ", ninterrupt=" + ninterrupt + ", nsuccess=" + nsuccess; + + System.err.println(getClass().getName() + "." + getName() + " : " + + msg); + + if (nerror > 0) + fail(msg); + + if (nexpected != naccepted) + fail(msg); + + return nsuccess; + + } finally { + + service.shutdownNow(); + + } + + } + } This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <dm...@us...> - 2010-09-17 11:22:36
|
Revision: 3578 http://bigdata.svn.sourceforge.net/bigdata/?rev=3578&view=rev Author: dmacgbr Date: 2010-09-17 11:22:28 +0000 (Fri, 17 Sep 2010) Log Message: ----------- Add some unit tests Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ArrayBindingSet.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/HashBindingSet.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/MemorySortOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestAll.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestMemorySortOp.java Added Paths: ----------- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestArrayBindingSet.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestHashBindingSet.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestIBindingSet.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ArrayBindingSet.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ArrayBindingSet.java 2010-09-17 10:44:01 UTC (rev 3577) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ArrayBindingSet.java 2010-09-17 11:22:28 UTC (rev 3578) @@ -435,10 +435,8 @@ for(int i=0; i<nbound; i++) { -// if (!o.isBound(vars[i])) -// return false; - - if (!vals[i].equals(o.get(vars[i]))) + IConstant<?> o_val = o.get ( vars [ i ] ) ; + if ( null == o_val || !vals[i].equals( o_val )) return false; } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/HashBindingSet.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/HashBindingSet.java 2010-09-17 10:44:01 UTC (rev 3577) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/HashBindingSet.java 2010-09-17 11:22:28 UTC (rev 3578) @@ -283,8 +283,8 @@ // if (!o.isBound(vars[i])) // return false; - - if (!val.equals(o.get(var))) + IConstant<?> o_val = o.get ( var ) ; + if (null == o_val || !val.equals(o_val)) return false; } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/MemorySortOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/MemorySortOp.java 2010-09-17 10:44:01 UTC (rev 3577) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/MemorySortOp.java 2010-09-17 11:22:28 UTC (rev 3578) @@ -88,10 +88,6 @@ // sort. Arrays.sort(all, comparator); - // update counters. - stats.unitsOut.add(all.length); - stats.chunksOut.increment(); - // write output and flush. sink.add(all); sink.flush(); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestAll.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestAll.java 2010-09-17 10:44:01 UTC (rev 3577) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestAll.java 2010-09-17 11:22:28 UTC (rev 3578) @@ -65,7 +65,8 @@ suite.addTestSuite(TestConstant.class); // test binding set impls. - suite.addTestSuite(TestBindingSet.class); + suite.addTestSuite(TestArrayBindingSet.class); + suite.addTestSuite(TestHashBindingSet.class); // unit tests for ctor existence and deep copy semantics suite.addTestSuite(TestDeepCopy.class); Added: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestArrayBindingSet.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestArrayBindingSet.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestArrayBindingSet.java 2010-09-17 11:22:28 UTC (rev 3578) @@ -0,0 +1,110 @@ +/* + +Copyright (C) SYSTAP, LLC 2006-2008. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA + +*/ +/* + * Created on Jun 19, 2008 + */ + +package com.bigdata.bop; + + +/** + * Unit tests for {@link ArrayBindingSet}. + * + * Note: + * a) these tests assume that the values held for a given key are not cloned, + * i.e. comparison is done by '==' and not '.equals' + * b) keys with the same 'name' are a unique object. + * + * @author <a href="mailto:dm...@us...">David MacMillan</a> + * @version $Id$ + */ +public class TestArrayBindingSet extends TestIBindingSet +{ + /** + * + */ + public TestArrayBindingSet () {} + + /** + * @param name + */ + public TestArrayBindingSet ( String name ) { super ( name ) ; } + + /** + * Unit test for {@link ArrayBindingSet#ArrayBindingSet(ArrayBindingSet)} + */ + public void testConstructorArrayBindingSet () + { + try { assertTrue ( null != new ArrayBindingSet ( null ) ) ; fail ( "IllegalArgumentException expected, copy from was null" ) ; } + catch ( IllegalArgumentException e ) {} + + Var<?> var1 = Var.var ( "a" ) ; + Var<?> var2 = Var.var ( "b" ) ; + Constant<Integer> val1 = new Constant<Integer> ( 1 ) ; + Constant<Integer> val2 = new Constant<Integer> ( 2 ) ; + IVariable<?> vars [] = new IVariable [] { var1, var2 } ; + IConstant<?> vals [] = new IConstant [] { val1, val2 } ; + + assertEqual ( new ArrayBindingSet ( new ArrayBindingSet ( vars, vals ) ), vars, vals ) ; + } + + /** + * Unit test for {@link ArrayBindingSet#ArrayBindingSet(IVariable[],IConstant[])} + */ + public void testConstructorVariablesConstants () + { + Var<?> var1 = Var.var ( "a" ) ; + Var<?> var2 = Var.var ( "b" ) ; + Constant<Integer> val1 = new Constant<Integer> ( 1 ) ; + Constant<Integer> val2 = new Constant<Integer> ( 2 ) ; + IVariable<?> vars [] = new IVariable [] { var1, var2 } ; + IConstant<?> vals [] = new IConstant [] { val1, val2 } ; + + try { assertTrue ( null != new ArrayBindingSet ( null, vals ) ) ; fail ( "IllegalArgumentException expected, vars was null" ) ; } + catch ( IllegalArgumentException e ) {} + + try { assertTrue ( null != new ArrayBindingSet ( vars, null ) ) ; fail ( "IllegalArgumentException expected, vals was null" ) ; } + catch ( IllegalArgumentException e ) {} + + try { assertTrue ( null != new ArrayBindingSet ( vars, new IConstant [] { val1 } ) ) ; fail ( "IllegalArgumentException expected, vars and vals were different sizes" ) ; } + catch ( IllegalArgumentException e ) {} + + assertEqual ( new ArrayBindingSet ( vars, vals ), vars, vals ) ; + } + + /** + * Unit test for {@link ArrayBindingSet#ArrayBindingSet(int)} + */ + public void testConstructorInt () + { + try { assertTrue ( null != new ArrayBindingSet ( -1 ) ) ; fail ( "IllegalArgumentException expected, capacity was negative" ) ; } + catch ( IllegalArgumentException e ) {} + + assertEqual ( new ArrayBindingSet ( 2 ), new IVariable [] {}, new IConstant [] {} ) ; + } + + @Override protected IBindingSet newBindingSet ( IVariable<?> vars [], IConstant<?> vals [] ) { return new ArrayBindingSet ( vars, vals ) ; } + @Override protected IBindingSet newBindingSet ( int size ) { return new ArrayBindingSet ( size ) ; } +} \ No newline at end of file Added: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestHashBindingSet.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestHashBindingSet.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestHashBindingSet.java 2010-09-17 11:22:28 UTC (rev 3578) @@ -0,0 +1,128 @@ +/* + +Copyright (C) SYSTAP, LLC 2006-2008. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA + +*/ +/* + * Created on Jun 19, 2008 + */ + +package com.bigdata.bop; + + +/** + * Unit tests for {@link HashBindingSet}. + * + * Note: + * a) these tests assume that the values held for a given key are not cloned, + * i.e. comparison is done by '==' and not '.equals' + * b) keys with the same 'name' are a unique object. + * + * @author <a href="mailto:dm...@us...">David MacMillan</a> + * @version $Id$ + */ +public class TestHashBindingSet extends TestIBindingSet +{ + + /** + * + */ + public TestHashBindingSet () {} + + /** + * @param name + */ + public TestHashBindingSet ( String name ) { super ( name ) ; } + + /** + * Unit test for {@link HashBindingSet#HashBindingSet()} + */ + public void testConstructorHashBindingSet () + { + assertTrue ( null != new HashBindingSet () ) ; + } + + /** + * Unit test for {@link HashBindingSet#HashBindingSet(HashBindingSet)} + */ + public void testConstructorHashBindingSetHashBindingSet () + { + // TODO what is our philosophy on argument validation? +// try { assertTrue ( null != new HashBindingSet ( null ) ) ; fail ( "IllegalArgumentException expected, copy from was null" ) ; } +// catch ( IllegalArgumentException e ) {} + + Var<?> var1 = Var.var ( "a" ) ; + Var<?> var2 = Var.var ( "b" ) ; + Constant<Integer> val1 = new Constant<Integer> ( 1 ) ; + Constant<Integer> val2 = new Constant<Integer> ( 2 ) ; + IVariable<?> vars [] = new IVariable [] { var1, var2 } ; + IConstant<?> vals [] = new IConstant [] { val1, val2 } ; + + assertEqual ( new HashBindingSet ( new HashBindingSet ( vars, vals ) ), vars, vals ) ; + } + + /** + * Unit test for {@link HashBindingSet#HashBindingSet(IBindingSet)} + */ + public void testConstructorHashBindingSetIBindingSet () + { + // TODO what is our philosophy on argument validation? +// try { assertTrue ( null != new HashBindingSet ( null ) ) ; fail ( "IllegalArgumentException expected, copy from was null" ) ; } +// catch ( IllegalArgumentException e ) {} + + Var<?> var1 = Var.var ( "a" ) ; + Var<?> var2 = Var.var ( "b" ) ; + Constant<Integer> val1 = new Constant<Integer> ( 1 ) ; + Constant<Integer> val2 = new Constant<Integer> ( 2 ) ; + IVariable<?> vars [] = new IVariable [] { var1, var2 } ; + IConstant<?> vals [] = new IConstant [] { val1, val2 } ; + + assertEqual ( new HashBindingSet ( new ArrayBindingSet ( vars, vals ) ), vars, vals ) ; + } + + /** + * Unit test for {@link HashBindingSet#HashBindingSet(IVariable[],IConstant[])} + */ + public void testConstructorVariablesConstants () + { + Var<?> var1 = Var.var ( "a" ) ; + Var<?> var2 = Var.var ( "b" ) ; + Constant<Integer> val1 = new Constant<Integer> ( 1 ) ; + Constant<Integer> val2 = new Constant<Integer> ( 2 ) ; + IVariable<?> vars [] = new IVariable [] { var1, var2 } ; + IConstant<?> vals [] = new IConstant [] { val1, val2 } ; + + try { assertTrue ( null != new HashBindingSet ( null, vals ) ) ; fail ( "IllegalArgumentException expected, vars was null" ) ; } + catch ( IllegalArgumentException e ) {} + + try { assertTrue ( null != new HashBindingSet ( vars, null ) ) ; fail ( "IllegalArgumentException expected, vals was null" ) ; } + catch ( IllegalArgumentException e ) {} + + try { assertTrue ( null != new HashBindingSet ( vars, new IConstant [] { val1 } ) ) ; fail ( "IllegalArgumentException expected, vars and vals were different sizes" ) ; } + catch ( IllegalArgumentException e ) {} + + assertEqual ( new HashBindingSet ( vars, vals ), vars, vals ) ; + } + + @Override protected IBindingSet newBindingSet ( IVariable<?> vars [], IConstant<?> vals [] ) { return new HashBindingSet ( vars, vals ) ; } + @Override protected IBindingSet newBindingSet ( int size ) { return new HashBindingSet () ; } +} \ No newline at end of file Added: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestIBindingSet.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestIBindingSet.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestIBindingSet.java 2010-09-17 11:22:28 UTC (rev 3578) @@ -0,0 +1,327 @@ +/* + +Copyright (C) SYSTAP, LLC 2006-2008. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA + +*/ +/* + * Created on Jun 19, 2008 + */ + +package com.bigdata.bop; + +import java.util.Iterator; +import java.util.Map; + +import junit.framework.TestCase2; + +/** + * Unit tests for {@link IBindingSet}. + * + * Note: + * a) these tests assume that the values held for a given key are not cloned, + * i.e. comparison is done by '==' and not '.equals' + * b) keys with the same 'name' are a unique object. + * + * @author <a href="mailto:dm...@us...">David MacMillan</a> + * @version $Id$ + */ +public abstract class TestIBindingSet extends TestCase2 { + + /** + * + */ + public TestIBindingSet () {} + + /** + * @param name + */ + public TestIBindingSet ( String name ) { super ( name ) ; } + + /** + * Unit test for {@link IBindingSet#isBound(IVariable)} + */ + public void testIsBound () + { + Var<?> a = Var.var ( "a" ) ; + Var<?> b = Var.var ( "b" ) ; + Var<?> c = Var.var ( "a" ) ; + + IBindingSet bs = newBindingSet ( new IVariable [] { a }, new IConstant [] { new Constant<Integer> ( 1 ) } ) ; + + assertTrue ( "bound expected, same variable", bs.isBound ( a ) ) ; + assertFalse ( "not bound expected", bs.isBound ( b ) ) ; + assertTrue ( "bound expected, equivalent variable", bs.isBound ( c ) ) ; + } + + /** + * Unit test for {@link IBindingSet#set(IVariable,IConstant)} + */ + public void testSet () + { + Var<?> var = Var.var ( "a" ) ; + Constant<Integer> val1 = new Constant<Integer> ( 1 ) ; + Constant<Integer> val2 = new Constant<Integer> ( 2 ) ; + + IBindingSet bs = newBindingSet ( 2 ) ; + + try { bs.set ( null, val1 ) ; fail ( "IllegalArgumentException expected, var was null" ) ; } + catch ( IllegalArgumentException e ) {} + + try { bs.set ( var, null ) ; fail ( "IllegalArgumentException expected, val was null" ) ; } + catch ( IllegalArgumentException e ) {} + + bs.set ( var, val1 ) ; + assertTrue ( val1 == bs.get ( var ) ) ; + + bs.set ( var, val2 ) ; + assertTrue ( val2 == bs.get ( var ) ) ; + } + + /** + * Unit test for {@link IBindingSet#get(IVariable)} + */ + public void testGet () + { + Var<?> var1 = Var.var ( "a" ) ; + Var<?> var2 = Var.var ( "b" ) ; + Constant<Integer> val1 = new Constant<Integer> ( 1 ) ; + + IBindingSet bs = newBindingSet ( new IVariable [] { var1 }, new IConstant [] { val1 } ) ; + + try { bs.get ( null ) ; fail ( "IllegalArgumentException expected, var was null" ) ; } + catch ( IllegalArgumentException e ) {} + + assertTrue ( val1 == bs.get ( var1 ) ) ; + assertTrue ( null == bs.get ( var2 ) ) ; + } + + /** + * Unit test for {@link IBindingSet#clear(IVariable)} + */ + public void testClear () + { + Var<?> var1 = Var.var ( "a" ) ; + Var<?> var2 = Var.var ( "b" ) ; + Constant<Integer> val1 = new Constant<Integer> ( 1 ) ; + Constant<Integer> val2 = new Constant<Integer> ( 2 ) ; + + IBindingSet bs = newBindingSet ( new IVariable [] { var1, var2 }, new IConstant [] { val1, val2 } ) ; + + try { bs.clear ( null ) ; fail ( "IllegalArgumentException expected, var was null" ) ; } + catch ( IllegalArgumentException e ) {} + + bs.clear ( var1 ) ; + assertTrue ( null == bs.get ( var1 ) ) ; + assertTrue ( val2 == bs.get ( var2 ) ) ; + + bs.clear ( var2 ) ; + assertTrue ( null == bs.get ( var2 ) ) ; + assertTrue ( 0 == bs.size () ) ; + } + + /** + * Unit test for {@link IBindingSet#clearAll()} + */ + public void testClearAll () + { + Var<?> var1 = Var.var ( "a" ) ; + Var<?> var2 = Var.var ( "b" ) ; + Constant<Integer> val1 = new Constant<Integer> ( 1 ) ; + Constant<Integer> val2 = new Constant<Integer> ( 2 ) ; + + IBindingSet bs = newBindingSet ( new IVariable [] { var1, var2 }, new IConstant [] { val1, val2 } ) ; + + bs.clearAll () ; + assertTrue ( null == bs.get ( var1 ) ) ; + assertTrue ( null == bs.get ( var2 ) ) ; + assertTrue ( 0 == bs.size () ) ; + } + + /** + * Unit test for {@link IBindingSet#size()} + */ + public void testSize () + { + Var<?> var1 = Var.var ( "a" ) ; + Var<?> var2 = Var.var ( "b" ) ; + Constant<Integer> val1 = new Constant<Integer> ( 1 ) ; + Constant<Integer> val2 = new Constant<Integer> ( 2 ) ; + + IBindingSet bs = newBindingSet ( 2 ) ; + + assertTrue ( 0 == bs.size () ) ; + + bs.set ( var1, val1 ) ; + bs.set ( var2, val2 ) ; + assertTrue ( 2 == bs.size () ) ; + + bs.clear ( var2 ) ; + assertTrue ( 1 == bs.size () ) ; + } + + /** + * Unit test for {@link IBindingSet#iterator()} + */ + public void testIterator () + { + Var<?> var1 = Var.var ( "a" ) ; + Var<?> var2 = Var.var ( "b" ) ; + Constant<Integer> val1 = new Constant<Integer> ( 1 ) ; + Constant<Integer> val2 = new Constant<Integer> ( 2 ) ; + + IBindingSet bs = newBindingSet ( new IVariable [] { var1, var2 }, new IConstant [] { val1, val2 } ) ; + + int n = 0 ; + for ( Iterator<Map.Entry<IVariable,IConstant>> i = bs.iterator (); i.hasNext (); ) + { + Map.Entry<IVariable,IConstant> e = i.next () ; + IVariable<?> var = e.getKey () ; + + if ( var1 == var ) assertTrue ( "wrong value", val1 == e.getValue () ) ; + else if ( var2 == var ) assertTrue ( "wrong value", val2 == e.getValue () ) ; + else fail ( "unexpected variable: " + var ) ; + + try { i.remove () ; fail ( "UnsupportedOperationException expected, iterator remove" ) ; } + catch ( UnsupportedOperationException ex ) {} + n++ ; + } + assertTrue ( "wrong count", 2 == n ) ; + } + + /** + * Unit test for {@link IBindingSet#vars()} + */ + public void testVars () + { + Var<?> var1 = Var.var ( "a" ) ; + Var<?> var2 = Var.var ( "b" ) ; + Constant<Integer> val1 = new Constant<Integer> ( 1 ) ; + Constant<Integer> val2 = new Constant<Integer> ( 2 ) ; + + IBindingSet bs = newBindingSet ( new IVariable [] { var1, var2 }, new IConstant [] { val1, val2 } ) ; + + int n = 0 ; + for ( Iterator<IVariable> i = bs.vars (); i.hasNext (); ) + { + IVariable<?> var = i.next () ; + + if ( var1 != var && var2 != var ) + fail ( "unexpected variable: " + var ) ; + + try { i.remove () ; fail ( "UnsupportedOperationException expected, iterator remove" ) ; } + catch ( UnsupportedOperationException e ) {} + n++ ; + } + assertTrue ( "wrong count", 2 == n ) ; + } + + /** + * Unit test for {@link IBindingSet#copy(IVariable[])} + */ + public void testCopy () + { + Var<?> var1 = Var.var ( "a" ) ; + Var<?> var2 = Var.var ( "b" ) ; + Var<?> var3 = Var.var ( "c" ) ; + Var<?> var4 = Var.var ( "d" ) ; + Var<?> var5 = Var.var ( "e" ) ; + Constant<Integer> val1 = new Constant<Integer> ( 1 ) ; + Constant<Integer> val2 = new Constant<Integer> ( 2 ) ; + Constant<Integer> val3 = new Constant<Integer> ( 3 ) ; + Constant<Integer> val4 = new Constant<Integer> ( 4 ) ; + Constant<Integer> val5 = new Constant<Integer> ( 5 ) ; + + IBindingSet bs = newBindingSet ( new IVariable [] { var1, var2, var3, var4, var5 } + , new IConstant [] { val1, val2, val3, val4, val5 } + ) ; + + IBindingSet bs2 = bs.copy ( new IVariable [] { var1, var3, var5 } ) ; + + assertTrue ( 3 == bs2.size () ) ; + for ( IVariable<?> v : new IVariable [] { var1, var3, var5 } ) + assertTrue ( bs2.get ( v ).equals ( bs.get ( v ) ) ) ; + } + + /** + * Unit test for {@link IBindingSet#equals(Object)} + */ + public void testEquals () + { + Var<?> var1 = Var.var ( "a" ) ; + Var<?> var2 = Var.var ( "b" ) ; + Var<?> var3 = Var.var ( "c" ) ; + Constant<Integer> val1 = new Constant<Integer> ( 1 ) ; + Constant<Integer> val2 = new Constant<Integer> ( 2 ) ; + Constant<Integer> val3 = new Constant<Integer> ( 3 ) ; + + IBindingSet bs1 = newBindingSet ( new IVariable [] { var1, var2 }, new IConstant [] { val1, val2 } ) ; + IBindingSet bs2 = newBindingSet ( new IVariable [] { var1, var2 }, new IConstant [] { val1, val2 } ) ; + IBindingSet bs3 = newBindingSet ( new IVariable [] { var2, var1 }, new IConstant [] { val2, val1 } ) ; + IBindingSet bs4 = newBindingSet ( new IVariable [] { var1, var2 }, new IConstant [] { val1, val3 } ) ; + IBindingSet bs5 = newBindingSet ( new IVariable [] { var1, var3 }, new IConstant [] { val1, val3 } ) ; + IBindingSet bs6 = newBindingSet ( new IVariable [] { var1, var2, var3 }, new IConstant [] { val1, val2, val3 } ) ; + IBindingSet bs7 = newBindingSet ( new IVariable [] { var1 }, new IConstant [] { val1 } ) ; + + assertTrue ( "expected equal: same bindings, same order", bs1.equals ( bs2 ) ) ; + assertTrue ( "expected equal: same bindings, different order", bs1.equals ( bs3 ) ) ; + assertTrue ( "expected not equal: different value", !bs1.equals ( bs4 ) ) ; + assertTrue ( "expected not equal: different variable", !bs1.equals ( bs5 ) ) ; + assertTrue ( "expected not equal: subsetOf ( this, that )", !bs1.equals ( bs6 ) ) ; + assertTrue ( "expected not equal: subsetOf ( that, this )", !bs1.equals ( bs7 ) ) ; + } + + /** + * Unit test for {@link IBindingSet#hashCode()} + */ + public void testHashCode () + { + Var<?> var1 = Var.var ( "a" ) ; + Var<?> var2 = Var.var ( "b" ) ; + Constant<Integer> val1 = new Constant<Integer> ( 1 ) ; + Constant<Integer> val2 = new Constant<Integer> ( 2 ) ; + + IBindingSet bs1 = newBindingSet ( new IVariable [] { var1, var2 }, new IConstant [] { val1, val2 } ) ; + IBindingSet bs2 = newBindingSet ( new IVariable [] { var1, var2 }, new IConstant [] { val1, val2 } ) ; + IBindingSet bs3 = newBindingSet ( new IVariable [] { var2, var1 }, new IConstant [] { val2, val1 } ) ; + IBindingSet bs4 = newBindingSet ( new IVariable [] { var2 }, new IConstant [] { val2 } ) ; + + assertTrue ( "expected equal: same bindings, same order", bs1.hashCode () == bs2.hashCode () ) ; + assertTrue ( "expected equal: same bindings, different order", bs1.hashCode () == bs3.hashCode () ) ; + + // + // After mutation. Not sure that this really proves anything, although in most cases I guess that + // the original value of bs1.hasCode () will not equal the subsequent value or that of bs4.hashCode () + // + bs1.clear ( var1 ) ; + assertTrue ( "expected equal: same bindings after mutation", bs1.hashCode () == bs4.hashCode () ) ; + } + + protected abstract IBindingSet newBindingSet ( IVariable<?> vars [], IConstant<?> vals [] ) ; + protected abstract IBindingSet newBindingSet ( int size ) ; + + protected void assertEqual ( IBindingSet actual, IVariable<?> vars [], IConstant<?> vals [] ) + { + assertTrue ( "wrong size", actual.size () == vars.length ) ; + for ( int i = 0; i < vars.length; i++ ) + assertTrue ( "wrong value", vals [ i ] == actual.get ( vars [ i ] ) ) ; + } +} \ No newline at end of file Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestMemorySortOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestMemorySortOp.java 2010-09-17 10:44:01 UTC (rev 3577) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestMemorySortOp.java 2010-09-17 11:22:28 UTC (rev 3578) @@ -27,11 +27,24 @@ package com.bigdata.bop.solutions; -import com.bigdata.bop.solutions.ISortOrder; -import com.bigdata.bop.solutions.MemorySortOp; - import junit.framework.TestCase2; +import com.bigdata.bop.ArrayBindingSet; +import com.bigdata.bop.BOp; +import com.bigdata.bop.BOpContext; +import com.bigdata.bop.Constant; +import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.IConstant; +import com.bigdata.bop.IVariable; +import com.bigdata.bop.NV; +import com.bigdata.bop.Var; +import com.bigdata.bop.engine.BOpStats; +import com.bigdata.bop.engine.MockRunningQuery; +import com.bigdata.bop.engine.TestQueryEngine; +import com.bigdata.relation.accesspath.IAsynchronousIterator; +import com.bigdata.relation.accesspath.IBlockingBuffer; +import com.bigdata.relation.accesspath.ThickAsynchronousIterator; + /** * Unit tests for the {@link MemorySortOp}. * @@ -43,26 +56,137 @@ /** * */ - public TestMemorySortOp() { - } + public TestMemorySortOp () {} /** * @param name */ - public TestMemorySortOp(String name) { - super(name); + public TestMemorySortOp ( String name ) + { + super ( name ) ; } - /** - * @todo unit tests for the in-memory sort operator. These tests should not - * focus on SPARQL semantics. Instead, just test the ability to impose - * the appropriate {@link ISortOrder}[] on some in-memory binding - * sets. - */ - public void test_something() { + public void testEval () + { + IVariable<?> x = Var.var ( "x" ) ; + IVariable<?> y = Var.var ( "y" ) ; + IConstant<String> a = new Constant<String> ( "a" ) ; + IConstant<String> b = new Constant<String> ( "b" ) ; + IConstant<String> c = new Constant<String> ( "c" ) ; + IConstant<String> d = new Constant<String> ( "d" ) ; + IConstant<String> e = new Constant<String> ( "e" ) ; - fail("write tests"); + ISortOrder<?> sors [] = new ISortOrder [] { new SortOrder ( x, true ), new SortOrder ( y, false ) } ; + + SortOp query = new MemorySortOp ( new BOp [] {} + , NV.asMap ( new NV [] { new NV ( MemorySortOp.Annotations.BOP_ID, 1 ) + , new NV ( MemorySortOp.Annotations.COMPARATOR, new StringComparatorOp ( sors ) ) + } + ) + ) ; + + // + // the test data + // + IBindingSet data [] = new IBindingSet [] + { + new ArrayBindingSet ( new IVariable<?> [] { x, y }, new IConstant [] { a, a } ) + , new ArrayBindingSet ( new IVariable<?> [] { x, y }, new IConstant [] { a, e } ) + , new ArrayBindingSet ( new IVariable<?> [] { x }, new IConstant [] { c } ) + , new ArrayBindingSet ( new IVariable<?> [] { x, y }, new IConstant [] { d, a } ) + , new ArrayBindingSet ( new IVariable<?> [] { x, y }, new IConstant [] { d, b } ) + , new ArrayBindingSet ( new IVariable<?> [] {}, new IConstant [] {} ) + , new ArrayBindingSet ( new IVariable<?> [] { x, y }, new IConstant [] { a, c } ) + , new ArrayBindingSet ( new IVariable<?> [] { x, y }, new IConstant [] { b, d } ) + , new ArrayBindingSet ( new IVariable<?> [] { y }, new IConstant [] { a } ) + , new ArrayBindingSet ( new IVariable<?> [] { x, y }, new IConstant [] { b, b } ) + } ; + + // + // the expected solutions + // + IBindingSet expected [] = new IBindingSet [] + { + new ArrayBindingSet ( new IVariable<?> [] { y }, new IConstant [] { a } ) + , new ArrayBindingSet ( new IVariable<?> [] {}, new IConstant [] {} ) + , new ArrayBindingSet ( new IVariable<?> [] { x, y }, new IConstant [] { a, e } ) + , new ArrayBindingSet ( new IVariable<?> [] { x, y }, new IConstant [] { a, c } ) + , new ArrayBindingSet ( new IVariable<?> [] { x, y }, new IConstant [] { a, a } ) + , new ArrayBindingSet ( new IVariable<?> [] { x, y }, new IConstant [] { b, d } ) + , new ArrayBindingSet ( new IVariable<?> [] { x, y }, new IConstant [] { b, b } ) + , new ArrayBindingSet ( new IVariable<?> [] { x }, new IConstant [] { c } ) + , new ArrayBindingSet ( new IVariable<?> [] { x, y }, new IConstant [] { d, b } ) + , new ArrayBindingSet ( new IVariable<?> [] { x, y }, new IConstant [] { d, a } ) + } ; + + BOpStats stats = query.newStats () ; + + IAsynchronousIterator<IBindingSet[]> source = new ThickAsynchronousIterator<IBindingSet[]> ( new IBindingSet [][] { data } ) ; + + IBlockingBuffer<IBindingSet[]> sink = query.newBuffer ( stats ) ; + + BOpContext<IBindingSet> context = new BOpContext<IBindingSet> ( new MockRunningQuery ( null/* fed */ + , null/* indexManager */ + ) + , -1/* partitionId */ + , stats + , source + , sink + , null/* sink2 */ + ) ; + + query.eval ( context ).run () ; + + TestQueryEngine.assertSameSolutions ( expected, sink.iterator () ) ; + + assertEquals ( 1, stats.chunksIn.get () ) ; + assertEquals ( 10, stats.unitsIn.get () ) ; + assertEquals ( 10, stats.unitsOut.get () ) ; + assertEquals ( 1, stats.chunksOut.get () ) ; + } + + /////////////////////////////////////////////////////////////////// + /////////////////////////////////////////////////////////////////// + /////////////////////////////////////////////////////////////////// + @SuppressWarnings("serial") + private class StringComparatorOp extends ComparatorOp + { + public StringComparatorOp ( ISortOrder<?> sors [] ) + { + super ( new BOp [] {}, NV.asMap ( new NV [] { new NV ( ComparatorOp.Annotations.ORDER, sors ) } ) ) ; + _sors = sors ; + } + + public int compare ( IBindingSet o1, IBindingSet o2 ) + { + for ( ISortOrder<?> sor : _sors ) + { + int ret = compare ( sor, o1, o2 ) ; + if ( 0 != ret ) + return ret ; + } + return 0 ; + } + + private int compare ( ISortOrder<?> sor, IBindingSet lhs, IBindingSet rhs ) + { + int compare = 0 ; + + IConstant<?> lhsv = lhs.get ( sor.getVariable () ) ; + IConstant<?> rhsv = rhs.get ( sor.getVariable () ) ; + + if ( null == lhsv && null == rhsv ) + return 0 ; + else if ( null == lhsv ) + compare = -1 ; + else if ( null == rhsv ) + compare = 1 ; + else + compare = lhsv.toString ().compareTo ( rhsv.toString () ) ; + + return compare * ( sor.isAscending () ? 1 : -1 ) ; + } + private ISortOrder<?> [] _sors = null ; } - -} +} \ No newline at end of file This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2010-09-17 15:05:04
|
Revision: 3583 http://bigdata.svn.sourceforge.net/bigdata/?rev=3583&view=rev Author: thompsonbry Date: 2010-09-17 15:04:56 +0000 (Fri, 17 Sep 2010) Log Message: ----------- Added BOp.isSharedState() so the query controller will know to invoke SliceOp with the same SliceStats instance each time. This fixes the problems with slice. I have replicated a problem with multiple join invocations first observed in scale-out against a local query engine. I will work to debug that next. Several new unit tests for query. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/CopyBindingSetOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/PipelineUtility.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestAll.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestBOpStats.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java Added Paths: ----------- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine_Slice.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java 2010-09-17 14:54:26 UTC (rev 3582) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java 2010-09-17 15:04:56 UTC (rev 3583) @@ -31,6 +31,7 @@ import java.util.concurrent.TimeUnit; import com.bigdata.bop.engine.BOpStats; +import com.bigdata.bop.engine.QueryEngine; import com.bigdata.btree.IRangeQuery; import com.bigdata.relation.accesspath.AccessPath; import com.bigdata.relation.accesspath.BlockingBuffer; @@ -107,9 +108,12 @@ /** * The default for {@link #CHUNK_TIMEOUT}. * - * @todo this is probably much larger than we want. Try 10ms. + * @todo Experiment with values for this. Low values will push chunks + * through quickly. High values will cause chunks to be combined + * and move larger chunks around. [But if we factor BlockingBuffer + * out of the query engine then this will go away]. */ - int DEFAULT_CHUNK_TIMEOUT = 1000; + int DEFAULT_CHUNK_TIMEOUT = 20; /** * If the estimated rangeCount for an {@link AccessPath#iterator()} is @@ -125,9 +129,10 @@ /** * Default for {@link #FULLY_BUFFERED_READ_THRESHOLD}. * - * @todo try something closer to the branching factor, e.g., 100. + * @todo Experiment with this. It should probably be something close to + * the branching factor, e.g., 100. */ - int DEFAULT_FULLY_BUFFERED_READ_THRESHOLD = 1000; + int DEFAULT_FULLY_BUFFERED_READ_THRESHOLD = 100; /** * Flags for the iterator ({@link IRangeQuery#KEYS}, @@ -203,6 +208,24 @@ */ protected static transient final TimeUnit chunkTimeoutUnit = TimeUnit.MILLISECONDS; + /** + * Return <code>true</code> iff {@link #newStats()} must be shared across + * all invocations of {@link #eval(BOpContext)} for this operator for a + * given query (default <code>false</code>). + * <p> + * Note: {@link BOp#getEvaluationContext()} MUST be overridden to return + * {@link BOpEvaluationContext#CONTROLLER} if this method is overridden to + * return <code>true</code>. + * <p> + * When <code>true</code>, the {@link QueryEngine} will impose the necessary + * constraints when the operator is evaluated. + */ + public boolean isSharedState() { + + return false; + + } + public BOpStats newStats() { return new BOpStats(); @@ -227,13 +250,16 @@ * @param chunkOfChunksCapacity * @param chunkCapacity * @param chunkTimeout - * @param chunktimeoutunit + * @param chunkTimeoutUnit * @param stats */ public BlockingBufferWithStats(int chunkOfChunksCapacity, int chunkCapacity, long chunkTimeout, - TimeUnit chunktimeoutunit, final BOpStats stats) { + TimeUnit chunkTimeoutUnit, final BOpStats stats) { + super(chunkOfChunksCapacity, chunkCapacity, chunkTimeout, + chunkTimeoutUnit); + this.stats = stats; } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/CopyBindingSetOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/CopyBindingSetOp.java 2010-09-17 14:54:26 UTC (rev 3582) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/CopyBindingSetOp.java 2010-09-17 15:04:56 UTC (rev 3583) @@ -108,8 +108,6 @@ stats.chunksIn.increment(); stats.unitsIn.add(chunk.length); sink.add(chunk); -// stats.chunksOut.increment(); -// stats.unitsOut.add(chunk.length); } sink.flush(); return null; Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java 2010-09-17 14:54:26 UTC (rev 3582) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java 2010-09-17 15:04:56 UTC (rev 3583) @@ -91,12 +91,16 @@ } /** - * Combine the statistics (addition). + * Combine the statistics (addition), but do NOT add to self. * * @param o * Another statistics object. */ public void add(final BOpStats o) { + if (this == o) { + // Do not add to self! + return; + } chunksIn.add(o.chunksIn.get()); unitsIn.add(o.unitsIn.get()); unitsOut.add(o.unitsOut.get()); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/PipelineUtility.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/PipelineUtility.java 2010-09-17 14:54:26 UTC (rev 3582) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/PipelineUtility.java 2010-09-17 15:04:56 UTC (rev 3583) @@ -149,6 +149,9 @@ } + if (log.isInfoEnabled()) + log.info("Operator can not be triggered: op=" + op); + return true; } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-17 14:54:26 UTC (rev 3582) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-17 15:04:56 UTC (rev 3583) @@ -55,6 +55,7 @@ import com.bigdata.bop.solutions.SliceOp; import com.bigdata.journal.IIndexManager; import com.bigdata.journal.ITx; +import com.bigdata.relation.accesspath.BlockingBuffer; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; import com.bigdata.service.IBigdataFederation; @@ -135,8 +136,8 @@ /** * The buffer used for the overall output of the query pipeline. * <p> - * Note: In scale out, this only exists on the query controller. In order to - * ensure that the results are transferred to the query controller, the + * Note: This only exists on the query controller. In order to ensure that + * the results are transferred to the query controller in scale-out, the * top-level operator in the query plan must specify * {@link BOpEvaluationContext#CONTROLLER}. For example, {@link SliceOp} * uses this {@link BOpEvaluationContext}. @@ -393,8 +394,8 @@ * <p> * The default implementation supports a standalone database. The generated * chunk is left on the Java heap and handed off synchronously using - * {@link QueryEngine#add(IChunkMessage)}. That method will queue the chunk - * for asynchronous processing. + * {@link QueryEngine#acceptChunk(IChunkMessage)}. That method will queue + * the chunk for asynchronous processing. * * @param sinkId * The identifier of the target operator. @@ -412,9 +413,44 @@ /* * Note: The partitionId will always be -1 in scale-up. */ + final int partitionId = -1; + + /* + * FIXME Raise this into an annotation that we can tweak from the unit + * tests and then debug the problem. + */ + final boolean oneMessagePerChunk = false; + + if (oneMessagePerChunk) { + + final IAsynchronousIterator<IBindingSet[]> itr = sink.iterator(); + + int nchunks = 0; + + while (itr.hasNext()) { + + final IBlockingBuffer<IBindingSet[]> tmp = new BlockingBuffer<IBindingSet[]>( + 1); + + tmp.add(itr.next()); + + tmp.close(); + + final LocalChunkMessage<IBindingSet> chunk = new LocalChunkMessage<IBindingSet>( + clientProxy, queryId, sinkId, partitionId, tmp.iterator()); + + queryEngine.acceptChunk(chunk); + + nchunks++; + + } + + return nchunks; + + } + final LocalChunkMessage<IBindingSet> chunk = new LocalChunkMessage<IBindingSet>( - clientProxy, queryId, sinkId, -1/* partitionId */, sink - .iterator()); + clientProxy, queryId, sinkId, partitionId, sink.iterator()); queryEngine.acceptChunk(chunk); @@ -539,8 +575,10 @@ // update per-operator statistics. final BOpStats tmp = statsMap.putIfAbsent(msg.bopId, msg.taskStats); - if (tmp != null) + if (tmp != null && tmp != msg.taskStats) { + // combine, but do not add to self. tmp.add(msg.taskStats); + } Throwable cause = null; boolean allDone = false; @@ -828,7 +866,25 @@ + bop); } - final BOpStats stats = op.newStats(); + /* + * Setup the BOpStats object. For some operators, e.g., SliceOp, + * this MUST be the same object across all invocations of that + * instance of that operator for this query. This is marked by the + * PipelineOp#isSharedState() method and is handled by a + * putIfAbsent() pattern when that method returns true. + * + * Note: RunState#haltOp() avoids adding a BOpStats object to itself + * since that would cause double counting when the same object is + * used for each invocation of the operator. + */ + final BOpStats stats; + if (((PipelineOp<?>) bop).isSharedState()) { + final BOpStats foo = op.newStats(); + final BOpStats bar = statsMap.putIfAbsent(bopId, foo); + stats = (bar == null ? foo : bar); + } else { + stats = op.newStats(); + } sink = (p == null ? queryBuffer : op.newBuffer(stats)); @@ -853,8 +909,8 @@ int sinkChunksOut = 0; int altSinkChunksOut = 0; try { - clientProxy.startOp(new StartOpMessage(queryId, - bopId, partitionId, serviceId, fanIn)); + clientProxy.startOp(new StartOpMessage(queryId, bopId, + partitionId, serviceId, fanIn)); if (log.isDebugEnabled()) log.debug("Running chunk: " + msg); ft.run(); // run Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java 2010-09-17 14:54:26 UTC (rev 3582) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java 2010-09-17 15:04:56 UTC (rev 3583) @@ -27,7 +27,6 @@ package com.bigdata.bop.solutions; -import java.math.BigInteger; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.FutureTask; @@ -41,7 +40,6 @@ import com.bigdata.bop.BindingSetPipelineOp; import com.bigdata.bop.IBindingSet; import com.bigdata.bop.engine.BOpStats; -import com.bigdata.bop.engine.QueryEngine; import com.bigdata.bop.engine.RunningQuery; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; @@ -67,21 +65,11 @@ * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id$ * - * @todo Unit test with stress test for concurrent {@link SliceOp} invocations - * against a streaming chunk producer. Make sure that the same - * {@link SliceStats} are used for each concurrent invocation of the same - * query. - * - * @todo What is sufficient serialization to make SLICE(ORDER_BY(...)) stable? - * The {@link SortOp} will impose a total ordering and will know how to - * deliver that total ordering to another operator. The {@link SliceOp} - * needs to accept the chunks from the {@link SortOp} in the order in - * which they were sent. This should work as long as we do not reorder the - * chunks for a given operator in the {@link QueryEngine} when they are - * received by the query controller. - * - * @todo If we allow complex operator trees in which "subqueries" can also use a - * slice then either then need to run as their own query with their own + * @todo Slice is not safe for subqueries - the entire query is cancelled when + * the slice is satisfied. + * <p> + * If we allow complex operator trees in which "subqueries" can also use a + * slice then either they need to run as their own query with their own * {@link RunningQuery} state or the API for cancelling a running query as * used here needs to only cancel evaluation of the child operators. * Otherwise we could cancel all operator evaluation for the query, @@ -166,6 +154,20 @@ } /** + * Overridden to return <code>true</code> since the correct decision + * semantics for the slice depend on concurrent invocations for the same + * query having the same {@link SliceStats} object. + * <p> + * {@inheritDoc} + */ + @Override + final public boolean isSharedState() { + + return true; + + } + + /** * Extends {@link BOpStats} to capture the state of the {@link SliceOp}. */ public static class SliceStats extends BOpStats { @@ -184,6 +186,11 @@ @Override public void add(final BOpStats o) { + if (this == o) { + // Do not add to self! + return; + } + super.add(o); if (o instanceof SliceStats) { @@ -201,7 +208,7 @@ @Override protected void toString(final StringBuilder sb) { - sb.append(",nseed=" + nseen); + sb.append(",nseen=" + nseen); sb.append(",naccepted=" + naccepted); @@ -237,14 +244,8 @@ /** #of solutions to accept. */ private final long limit; - private final long last; - -// /** #of solutions visited. */ -// private long nseen; -// -// /** #of solutions accepted. */ -// private long naccepted; -// +// private final long last; + private final SliceStats stats; SliceTask(final SliceOp op, final BOpContext<IBindingSet> context) { @@ -266,9 +267,9 @@ this.stats = (SliceStats) context.getStats(); // this.last = offset + limit; - this.last = BigInteger.valueOf(offset).add( - BigInteger.valueOf(limit)).min( - BigInteger.valueOf(Long.MAX_VALUE)).longValue(); +// this.last = BigInteger.valueOf(offset).add( +// BigInteger.valueOf(limit)).min( +// BigInteger.valueOf(Long.MAX_VALUE)).longValue(); } @@ -282,8 +283,6 @@ final IBlockingBuffer<IBindingSet[]> sink = context.getSink(); -// final BOpStats stats = context.getStats(); - try { /* @@ -326,8 +325,10 @@ sink.flush(); - if (halt) + if (halt) { + log.error("Slice will interrupt query.");// @todo remove. throw new InterruptedException(); + } // cancelQuery(); return null; Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestAll.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestAll.java 2010-09-17 14:54:26 UTC (rev 3582) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestAll.java 2010-09-17 15:04:56 UTC (rev 3583) @@ -69,6 +69,9 @@ // test suite for query evaluation (basic JOINs). suite.addTestSuite(TestQueryEngine.class); + // stress test for SliceOp. + suite.addTestSuite(TestQueryEngine_Slice.class); + // test suite for query evaluation (DISTINCT, ORDER BY, GROUP BY). suite.addTestSuite(TestQueryEngine2.class); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestBOpStats.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestBOpStats.java 2010-09-17 14:54:26 UTC (rev 3582) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestBOpStats.java 2010-09-17 15:04:56 UTC (rev 3583) @@ -99,5 +99,34 @@ assertEquals("chunksOut", 4L, totals.chunksOut.get()); } + + public void test_addToSelf() { + + final BOpStats stats = new BOpStats(); + + assertEquals("chunksIn", 0L, stats.chunksIn.get()); + assertEquals("unitsIn", 0L, stats.unitsIn.get()); + assertEquals("unitsOut", 0L, stats.unitsOut.get()); + assertEquals("chunksOut", 0L, stats.chunksOut.get()); + + stats.chunksIn.increment(); + stats.unitsIn.increment(); + stats.unitsIn.increment(); + + assertEquals("chunksIn", 1L, stats.chunksIn.get()); + assertEquals("unitsIn", 2L, stats.unitsIn.get()); + assertEquals("unitsOut", 0L, stats.unitsOut.get()); + assertEquals("chunksOut", 0L, stats.chunksOut.get()); + + // add to self. + stats.add(stats); + + // verify no change. + assertEquals("chunksIn", 1L, stats.chunksIn.get()); + assertEquals("unitsIn", 2L, stats.unitsIn.get()); + assertEquals("unitsOut", 0L, stats.unitsOut.get()); + assertEquals("chunksOut", 0L, stats.chunksOut.get()); + + } } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-09-17 14:54:26 UTC (rev 3582) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-09-17 15:04:56 UTC (rev 3583) @@ -54,6 +54,7 @@ import com.bigdata.bop.IVariable; import com.bigdata.bop.IVariableOrConstant; import com.bigdata.bop.NV; +import com.bigdata.bop.PipelineOp; import com.bigdata.bop.Var; import com.bigdata.bop.ap.E; import com.bigdata.bop.ap.Predicate; @@ -64,6 +65,7 @@ import com.bigdata.bop.fed.TestFederatedQueryEngine; import com.bigdata.bop.join.PipelineJoin; import com.bigdata.bop.solutions.SliceOp; +import com.bigdata.bop.solutions.SliceOp.SliceStats; import com.bigdata.journal.BufferMode; import com.bigdata.journal.ITx; import com.bigdata.journal.Journal; @@ -90,6 +92,8 @@ * @version $Id$ * * @see TestFederatedQueryEngine + * + * @todo write a unit and stress tests for deadlines. */ public class TestQueryEngine extends TestCase2 { @@ -189,6 +193,35 @@ } /** + * Return an {@link IAsynchronousIterator} that will read a single, chunk + * containing all of the specified {@link IBindingSet}s. + * + * @param bindingSets + * the binding sets. + */ + protected ThickAsynchronousIterator<IBindingSet[]> newBindingSetIterator( + final IBindingSet[] bindingSets) { + + return new ThickAsynchronousIterator<IBindingSet[]>( + new IBindingSet[][] { bindingSets }); + + } + + /** + * Return an {@link IAsynchronousIterator} that will read a single, chunk + * containing all of the specified {@link IBindingSet}s. + * + * @param bindingSetChunks + * the chunks of binding sets. + */ + protected ThickAsynchronousIterator<IBindingSet[]> newBindingSetIterator( + final IBindingSet[][] bindingSetChunks) { + + return new ThickAsynchronousIterator<IBindingSet[]>(bindingSetChunks); + + } + + /** * Starts and stops the {@link QueryEngine}, but does not validate the * semantics of shutdown() versus shutdownNow() since we need to be * evaluating query mixes in order to verify the semantics of those @@ -357,6 +390,187 @@ } /** + * Test the ability run a simple join when multiple binding sets are + * submitted as the initial input. The access path associated with the join + * does not have any constants but the join picks up bindings from the input + * binding sets and uses them to constrain the access path. + */ + public void test_query_join1_multipleChunksIn() throws Exception { + + final Var<?> x = Var.var("x"); + final Var<?> y = Var.var("y"); + + final int startId = 1; + final int joinId = 2; + final int predId = 3; + + /* + * Enforce a constraint on the source such that it hands 3 each source + * chunk to the join operator as a separate chunk + * + * @todo This is not enough to force the query engine to run the join + * operator once per source chunk. Instead, it takes the output of the + * source operator, which is N chunks, and sends them all to a single + * invocation of the join task. To do better than that we have to send + * multiple chunk messages rather than just one. + */ + final int nsources = 3; + final StartOp startOp = new StartOp(new BOp[] {}, NV.asMap(new NV[] {// + new NV(Predicate.Annotations.BOP_ID, startId),// + new NV(PipelineOp.Annotations.CHUNK_CAPACITY, 1),// + new NV(PipelineOp.Annotations.CHUNK_OF_CHUNKS_CAPACITY, nsources),// + })); + + final Predicate<E> predOp = new Predicate<E>(new IVariableOrConstant[] { + x, y }, NV.asMap(new NV[] {// + new NV(Predicate.Annotations.RELATION_NAME, + new String[] { namespace }),// + new NV(Predicate.Annotations.PARTITION_ID, Integer + .valueOf(-1)),// + new NV(Predicate.Annotations.OPTIONAL, Boolean.FALSE),// + new NV(Predicate.Annotations.CONSTRAINT, null),// + new NV(Predicate.Annotations.EXPANDER, null),// + new NV(Predicate.Annotations.BOP_ID, predId),// + new NV(Predicate.Annotations.TIMESTAMP, + ITx.READ_COMMITTED),// + })); + + final PipelineJoin<E> joinOp = new PipelineJoin<E>( + startOp/* left */, predOp/* right */, + // join annotations + NV.asMap(new NV[] { // + new NV(Predicate.Annotations.BOP_ID, joinId),// +// new NV(PipelineOp.Annotations.CHUNK_CAPACITY, 1),// +// new NV(PipelineOp.Annotations.CHUNK_OF_CHUNKS_CAPACITY, 1),// + })// + ); + + final int sliceId = 4; + final SliceOp sliceOp = new SliceOp(new BOp[] { joinOp }, + // slice annotations + NV.asMap(new NV[] {// + new NV(Predicate.Annotations.BOP_ID, sliceId),// + })// + ); + + final BindingSetPipelineOp query = sliceOp; + + /* + * Source binding sets. + * + * Note: We can't bind y in advance for the primary index! + */ + final IBindingSet[] source = new IBindingSet[] {// + new HashBindingSet(new ArrayBindingSet(// + new IVariable[] { x },// + new IConstant[] { new Constant<String>("Paul") }// + )),// + new HashBindingSet(new ArrayBindingSet(// + new IVariable[] { x },// + new IConstant[] { new Constant<String>("Leon") }// + )), + new HashBindingSet(new ArrayBindingSet(// + new IVariable[] { x },// + new IConstant[] { new Constant<String>("Mary") }// + )), + }; + // Put each source binding set into a chunk by itself. + final IBindingSet[][] sources = new IBindingSet[source.length][]; + for (int i = 0; i < sources.length; i++) { + sources[i] = new IBindingSet[] { source[i] }; + } + assertEquals(nsources,source.length); + assertEquals(nsources,sources.length); + +// new E("John", "Mary"),// [0] +// new E("Leon", "Paul"),// [1] +// new E("Mary", "Paul"),// [2] +// new E("Paul", "Leon"),// [3] + + // the expected solution (just one). + final IBindingSet[] expected = new IBindingSet[] {// + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { // + new Constant<String>("Paul"), new Constant<String>("Leon") }// + ),// + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { // + new Constant<String>("Leon"), new Constant<String>("Paul") }// + ), + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { // + new Constant<String>("Mary"), new Constant<String>("Paul") }// + ), + }; + + final UUID queryId = UUID.randomUUID(); + final RunningQuery runningQuery = queryEngine.eval(queryId, query, + new LocalChunkMessage<IBindingSet>(queryEngine, queryId, + startId, -1 /* partitionId */, + newBindingSetIterator(sources))); + + // verify solutions. + assertSameSolutionsAnyOrder(expected, new Dechunkerator<IBindingSet>( + runningQuery.iterator())); + + // Wait until the query is done. + runningQuery.get(); + final Map<Integer, BOpStats> statsMap = runningQuery.getStats(); + { + // validate the stats map. + assertNotNull(statsMap); + assertEquals(3, statsMap.size()); + if (log.isInfoEnabled()) + log.info(statsMap.toString()); + } + + // validate the stats for the start operator. + { + final BOpStats stats = statsMap.get(startId); + assertNotNull(stats); + if (log.isInfoEnabled()) + log.info("start: "+stats.toString()); + + assertEquals(3L, stats.chunksIn.get()); + assertEquals(3L, stats.unitsIn.get()); + assertEquals(3L, stats.unitsOut.get()); + assertEquals(3L, stats.chunksOut.get()); + } + + // validate the stats for the join operator. + { + final BOpStats stats = statsMap.get(joinId); + assertNotNull(stats); + if (log.isInfoEnabled()) + log.info("join : "+stats.toString()); + +// assertEquals(3L, stats.chunksIn.get()); + assertEquals(3L, stats.unitsIn.get()); + assertEquals(3L, stats.unitsOut.get()); +// assertEquals(3L, stats.chunksOut.get()); + } + + // validate the stats for the slice operator. + { + final SliceStats stats = (SliceStats) statsMap.get(sliceId); + assertNotNull(stats); + if (log.isInfoEnabled()) + log.info("slice: "+stats.toString()); + + assertEquals(3L, stats.nseen.get()); + assertEquals(3L, stats.naccepted.get()); +// assertEquals(3L, stats.chunksIn.get()); + assertEquals(3L, stats.unitsIn.get()); + assertEquals(3L, stats.unitsOut.get()); +// assertEquals(3L, stats.chunksOut.get()); + } + + } + + /** * @todo Test the ability close the iterator draining a result set before * the query has finished executing and verify that the query is * correctly terminated [this is difficult to test without having Added: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine_Slice.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine_Slice.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine_Slice.java 2010-09-17 15:04:56 UTC (rev 3583) @@ -0,0 +1,197 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Sep 17, 2010 + */ + +package com.bigdata.bop.engine; + +import java.util.Properties; +import java.util.Random; +import java.util.UUID; + +import junit.framework.TestCase2; + +import com.bigdata.bop.BOp; +import com.bigdata.bop.EmptyBindingSet; +import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.NV; +import com.bigdata.bop.solutions.SliceOp; +import com.bigdata.bop.solutions.SliceOp.SliceStats; +import com.bigdata.journal.BufferMode; +import com.bigdata.journal.Journal; +import com.bigdata.relation.accesspath.IAsynchronousIterator; +import com.bigdata.relation.accesspath.ThickAsynchronousIterator; + +/** + * Stress test for {@link SliceOp} in which a large number of small chunks are + * fed into the query such that the concurrency constraints of the slice are + * stress tested. {@link SliceOp#isSharedState()} returns <code>true</code> so + * each invocation of the same {@link SliceOp} operator instance should use the + * same {@link SliceStats} object. This test will fail if that is not true. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +public class TestQueryEngine_Slice extends TestCase2 { + + /** + * + */ + public TestQueryEngine_Slice() { + } + + /** + * @param name + */ + public TestQueryEngine_Slice(String name) { + super(name); + } + + @Override + public Properties getProperties() { + + final Properties p = new Properties(super.getProperties()); + + p.setProperty(Journal.Options.BUFFER_MODE, BufferMode.Transient + .toString()); + + return p; + + } + + Journal jnl; + QueryEngine queryEngine; + + public void setUp() throws Exception { + + jnl = new Journal(getProperties()); + + queryEngine = new QueryEngine(jnl); + + queryEngine.init(); + + } + + public void tearDown() throws Exception { + + if (queryEngine != null) { + queryEngine.shutdownNow(); + queryEngine = null; + } + + if (jnl != null) { + jnl.destroy(); + jnl = null; + } + + } + + /** + * Return an {@link IAsynchronousIterator} that will read a single, chunk + * containing all of the specified {@link IBindingSet}s. + * + * @param bindingSetChunks + * the chunks of binding sets. + */ + protected ThickAsynchronousIterator<IBindingSet[]> newBindingSetIterator( + final IBindingSet[][] bindingSetChunks) { + + return new ThickAsynchronousIterator<IBindingSet[]>(bindingSetChunks); + + } + + public void test_slice_threadSafe() throws Exception { + + final long timeout = 10000; // ms + + final int ntrials = 10000; + + final int poolSize = 10; + + doSliceTest(500L/* offset */, 1500L/* limit */, timeout, ntrials, + poolSize); + + } + + /** + * + * @param timeout + * @param ntrials + * @param poolSize + * + * @return The #of successful trials. + * + * @throws Exception + */ + protected void doSliceTest(final long offset, final long limit, + final long timeout, final int ntrials, final int poolSize) + throws Exception { + + final IBindingSet[][] chunks = new IBindingSet[ntrials][]; + { + final Random r = new Random(); + final IBindingSet bset = EmptyBindingSet.INSTANCE; + for (int i = 0; i < chunks.length; i++) { + // random non-zero chunk size + chunks[i] = new IBindingSet[r.nextInt(10) + 1]; + for (int j = 0; j < chunks[i].length; j++) { + chunks[i][j] = bset; + } + } + } + final int sliceId = 1; + final SliceOp query = new SliceOp(new BOp[] {}, NV.asMap(new NV[] {// + new NV(SliceOp.Annotations.BOP_ID, sliceId),// + new NV(SliceOp.Annotations.OFFSET, offset),// + new NV(SliceOp.Annotations.LIMIT, limit),// + })); + + final UUID queryId = UUID.randomUUID(); + final RunningQuery q = queryEngine.eval(queryId, query, + new LocalChunkMessage<IBindingSet>(queryEngine, queryId, + sliceId, -1/* partitionId */, + newBindingSetIterator(chunks))); + + // consume solutions. + int nsolutions = 0; + final IAsynchronousIterator<IBindingSet[]> itr = q.iterator(); + while (itr.hasNext()) { + nsolutions += itr.next().length; + } + + // wait for the query to terminate. + q.get(); + + // Verify stats. + final SliceStats stats = (SliceStats) q.getStats().get(sliceId); + System.err.println(getClass().getName() + "." + getName() + " : " + + stats); + assertNotNull(stats); + assertEquals(limit, stats.naccepted.get()); + assertEquals(limit, nsolutions); + + } + +} Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine_Slice.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java 2010-09-17 14:54:26 UTC (rev 3582) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java 2010-09-17 15:04:56 UTC (rev 3583) @@ -65,7 +65,6 @@ import com.bigdata.bop.solutions.SliceOp; import com.bigdata.bop.solutions.SortOp; import com.bigdata.btree.keys.KeyBuilder; -import com.bigdata.counters.CAT; import com.bigdata.journal.BufferMode; import com.bigdata.journal.ITx; import com.bigdata.journal.Journal; @@ -106,6 +105,9 @@ * thompsonbry $ * * @todo reuse the stress test from {@link TestQueryEngine}. + * + * @todo there should be a stress test of {@link SliceOp} with a non-zero offset + * in order to verify that it is properly synchronized. */ public class TestFederatedQueryEngine extends AbstractEmbeddedFederationTestCase { @@ -428,7 +430,7 @@ * * @throws Exception */ - public void test_query_startThenSlice() throws Exception { + public void test_query_startThenSlice_noJoins() throws Exception { final int startId = 1; final int sliceId = 4; @@ -508,15 +510,19 @@ } /** - * Test the ability run a simple join. There are three operators. One feeds - * an empty binding set[] into the join, another is the predicate for the - * access path on which the join will read (it probes the index once for - * "Mary" and binds "Paul" and "John" when it does so), and the third is the - * join itself (there are two solutions, which are "value=Paul" and - * value="John"). + * Test the ability run a simple join which is mapped across two index + * partitions. There are three operators. One feeds an empty binding set[] + * into the join, another is the predicate for the access path on which the + * join will read (no variables are bound so it will read everything), and + * the third is the join itself. + * + * @throws Exception */ - public void test_query_join1() throws Exception { + public void test_query_join_2shards_nothingBoundOnAccessPath() throws Exception { + final Var<?> x = Var.var("x") ; + final Var<?> y = Var.var("y") ; + final int startId = 1; final int joinId = 2; final int predId = 3; @@ -526,13 +532,12 @@ new NV(Predicate.Annotations.BOP_ID, startId),// })); + // access path has has no constants and no constraint. final Predicate<E> predOp = new Predicate<E>(new IVariableOrConstant[] { - new Constant<String>("Mary"), Var.var("value") }, NV + x, y}, NV .asMap(new NV[] {// new NV(Predicate.Annotations.RELATION_NAME, new String[] { namespace }),// - new NV(Predicate.Annotations.KEY_ORDER, - R.primaryKeyOrder),// new NV(Predicate.Annotations.PARTITION_ID, Integer .valueOf(-1)),// new NV(Predicate.Annotations.OPTIONAL, Boolean.FALSE),// @@ -541,6 +546,8 @@ new NV(Predicate.Annotations.BOP_ID, predId),// new NV(Predicate.Annotations.TIMESTAMP, ITx.READ_COMMITTED),// + new NV(Predicate.Annotations.KEY_ORDER, + R.primaryKeyOrder),// })); final PipelineJoin<E> joinOp = new PipelineJoin<E>(startOp/* left */, @@ -552,7 +559,7 @@ ); final BindingSetPipelineOp query = new SliceOp(new BOp[] { joinOp }, - // slice annotations + // slice annotations NV.asMap(new NV[] {// new NV(Predicate.Annotations.BOP_ID, sliceId),// })// @@ -560,15 +567,46 @@ // the expected solutions (order is not reliable due to concurrency). final IBindingSet[] expected = new IBindingSet[] {// - new ArrayBindingSet(// - new IVariable[] { Var.var("value") },// - new IConstant[] { new Constant<String>("Paul") }// + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { // + new Constant<String>("John"), + new Constant<String>("Mary") }// ), // - new ArrayBindingSet(// - new IVariable[] { Var.var("value") },// - new IConstant[] { new Constant<String>("John") }// - ) }; + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] {// + new Constant<String>("Leon"), + new Constant<String>("Paul") }// + ), // + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { // + new Constant<String>("Mary"), + new Constant<String>("John") }// + ), // + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] {// + new Constant<String>("Mary"), + new Constant<String>("Paul") }// + ), // + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { // + new Constant<String>("Paul"), + new Constant<String>("Leon") }// + ), // + }; +// // partition0 +// new E("John", "Mary"),// +// new E("Leon", "Paul"),// +// // partition1 +// new E("Mary", "John"),// +// new E("Mary", "Paul"),// +// new E("Paul", "Leon"),// + // run query with empty binding set, so nothing is bound on the join. final UUID queryId = UUID.randomUUID(); final RunningQuery runningQuery = queryEngine.eval(queryId, query, new LocalChunkMessage<IBindingSet>(queryEngine, queryId, @@ -615,8 +653,8 @@ // verify query solution stats details. assertEquals(1L, stats.chunksIn.get()); assertEquals(1L, stats.unitsIn.get()); - assertEquals(2L, stats.unitsOut.get()); - assertEquals(1L, stats.chunksOut.get()); // @todo this depends on which index partitions we read on. + assertEquals(5L, stats.unitsOut.get()); + assertEquals(2L, stats.chunksOut.get()); // since we read on both shards. } // validate the stats for the slice operator. @@ -627,36 +665,21 @@ log.info("slice: "+stats.toString()); // verify query solution stats details. - assertEquals(1L, stats.chunksIn.get()); - assertEquals(2L, stats.unitsIn.get()); - assertEquals(2L, stats.unitsOut.get()); - assertEquals(1L, stats.chunksOut.get()); + assertEquals(2L, stats.chunksIn.get()); // from both shards. + assertEquals(5L, stats.unitsIn.get()); + assertEquals(5L, stats.unitsOut.get()); + assertEquals(2L, stats.chunksOut.get()); } } /** * Test the ability run a simple join which is mapped across two index - * partitions. - * - * FIXME This is failing because the {@link SliceOp} is not remembering its - * state across distinct invocations and is cancelling the query as soon as - * it exhausts its input. In order to have correct decision boundaries, - * slice needs to be invoked either once, concurrently if using {@link CAT} - * s, or in a series of presentations otherwise (single-threaded operator or - * internal locking in the operator implementation on its {@link SliceOp} to - * achieve chunk-wise serialization of processing). - * <p> - * The easiest way to fix this is to have {@link SliceOp} specialize the - * {@link BOpStats}s and carry its state there. That will also make it safe - * for concurrent evaluation within the same query, and we will have to - * write a unit test for that. - * <p> - * I am not yet convinced that the problem with the test failure is double - * invocation of {@link SliceOp}. It could also be that we are not invoking - * it the 2nd time. + * partitions. The join is constrained to filter for only solutions in which + * [y==Paul]. */ - public void test_query_join_withConstraint_readsOn2shards() throws Exception { + public void test_query_join_2shards_nothingBoundOnAccessPath_withConstraint() + throws Exception { final Var<?> x = Var.var("x"); final Var<?> y = Var.var("y"); @@ -671,7 +694,6 @@ })); /* - * * Note: Since the index on which this reads is formed as (column1 + * column2) the probe key will be [null] if it does not bind the first * column. Therefore, in order to have the 2nd column constraint we have @@ -725,7 +747,7 @@ new IConstant[] { new Constant<String>("Mary"), new Constant<String>("Paul") }// ), // - }; + }; // // partition0 // new E("John", "Mary"),// // new E("Leon", "Paul"),// @@ -786,9 +808,9 @@ log.info("join : "+stats.toString()); // verify query solution stats details. - assertEquals(1L, stats.chunksIn.get()); - assertEquals(1L, stats.unitsIn.get()); - assertEquals(5L, stats.unitsOut.get()); + assertEquals(2L, stats.chunksIn.get()); // since we read on two shards. + assertEquals(1L, stats.unitsIn.get()); // a single empty binding set. + assertEquals(5L, stats.unitsOut.get()); // each of the tuples will be read. assertEquals(2L, stats.chunksOut.get()); // since we read on both shards. } @@ -801,27 +823,23 @@ // verify query solution stats details. assertEquals(2L, stats.chunksIn.get()); // from both shards. - assertEquals(5L, stats.unitsIn.get()); - assertEquals(5L, stats.unitsOut.get()); + assertEquals(2L, stats.unitsIn.get()); + assertEquals(2L, stats.unitsOut.get()); assertEquals(2L, stats.chunksOut.get()); } } /** - * Test the ability run a simple join which is mapped across two index - * partitions. There are three operators. One feeds an empty binding set[] - * into the join, another is the predicate for the access path on which the - * join will read (no variables are bound so it will read everything), and - * the third is the join itself. - * - * @throws Exception + * Test the ability run a simple join reading on a single shard. There are + * three operators. One feeds an empty binding set[] into the join, another + * is the predicate for the access path on which the join will read (it + * probes the index once for "Mary" and binds "Paul" and "John" when it does + * so), and the third is the join itself (there are two solutions, which are + * "value=Paul" and value="John"). */ - public void test_query_join1_2shards_nothingBoundOnAccessPath() throws Exception { + public void test_query_join_1shard() throws Exception { - final Var<?> x = Var.var("x") ; - final Var<?> y = Var.var("y") ; - final int startId = 1; final int joinId = 2; final int predId = 3; @@ -831,11 +849,14 @@ new NV(Predicate.Annotations.BOP_ID, startId),// })); + // Note: tuples with "Mary" in the 1st column are on partition1. final Predicate<E> predOp = new Predicate<E>(new IVariableOrConstant[] { - x, y}, NV + new Constant<String>("Mary"), Var.var("value") }, NV .asMap(new NV[] {// new NV(Predicate.Annotations.RELATION_NAME, new String[] { namespace }),// + new NV(Predicate.Annotations.KEY_ORDER, + R.primaryKeyOrder),// new NV(Predicate.Annotations.PARTITION_ID, Integer .valueOf(-1)),// new NV(Predicate.Annotations.OPTIONAL, Boolean.FALSE),// @@ -844,8 +865,6 @@ new NV(Predicate.Annotations.BOP_ID, predId),// new NV(Predicate.Annotations.TIMESTAMP, ITx.READ_COMMITTED),// - new NV(Predicate.Annotations.KEY_ORDER, - R.primaryKeyOrder),// })); final PipelineJoin<E> joinOp = new PipelineJoin<E>(startOp/* left */, @@ -857,7 +876,7 @@ ); final BindingSetPipelineOp query = new SliceOp(new BOp[] { joinOp }, - // slice annotations + // slice annotations NV.asMap(new NV[] {// new NV(Predicate.Annotations.BOP_ID, sliceId),// })// @@ -865,39 +884,14 @@ // the expected solutions (order is not reliable due to concurrency). final IBindingSet[] expected = new IBindingSet[] {// - new ArrayBindingSet(// - new IVariable[] { x, y },// - new IConstant[] { new Constant<String>("John"), - new Constant<String>("Mary") }// + new ArrayBindingSet(// + new IVariable[] { Var.var("value") },// + new IConstant[] { new Constant<String>("Paul") }// ), // - new ArrayBindingSet(// - new IVariable[] { x, y },// - new IConstant[] { new Constant<String>("John"), - new Constant<String>("Paul") }// - ), // - new ArrayBindingSet(// - new IVariable[] { x, y },// - new IConstant[] { new Constant<String>("Mary"), - new Constant<String>("John") }// - ), // - new ArrayBindingSet(// - new IVariable[] { x, y },// - new IConstant[] { new Constant<String>("Mary"), - new Constant<String>("Paul") }// - ), // - new ArrayBindingSet(// - new IVariable[] { x, y },// - new IConstant[] { new Constant<String>("Paul"), - new Constant<String>("Leon") }// - ), // - }; -// // partition0 -// new E("John", "Mary"),// -// new E("Leon", "Paul"),// -// // partition1 -// new E("Mary", "John"),// -// new E("Mary", "Paul"),// -// new E("Paul", "Leon"),// + new ArrayBindingSet(// + new IVariable[] { Var.var("value") },// + new IConstant[] { new Constant<String>("John") }// + ) }; final UUID queryId = UUID.randomUUID(); final RunningQuery runningQuery = queryEngine.eval(queryId, query, @@ -945,8 +939,8 @@ // verify query solution stats details. assertEquals(1L, stats.chunksIn.get()); assertEquals(1L, stats.unitsIn.get()); - assertEquals(5L, stats.unitsOut.get()); - assertEquals(2L, stats.chunksOut.get()); // since we read on both shards. + assertEquals(2L, stats.unitsOut.get()); + assertEquals(1L, stats.chunksOut.get()); // @todo this depends on which index partitions we read on. } // validate the stats for the slice operator. @@ -957,10 +951,10 @@ log.info("slice: "+stats.toString()); // verify query solution stats details. - assertEquals(2L, stats.chunksIn.get()); // from both shards. - assertEquals(5L, stats.unitsIn.get()); - assertEquals(5L, stats.unitsOut.get()); - assertEquals(2L, stats.chunksOut.get()); + assertEquals(1L, stats.chunksIn.get()); + assertEquals(2L, stats.unitsIn.get()); + assertEquals(2L, stats.unitsOut.get()); + assertEquals(1L, stats.chunksOut.get()); } } @@ -988,26 +982,6 @@ } /** - * @todo Test ability to impose a limit/offset slice on a query. - * <p> - * Note: While the logic for visiting only the solutions selected by - * the slice can be tested against a mock object, the integration by - * which a slice halts a query when it is satisfied has to be tested - * against a {@link QueryEngine}. - * <p> - * This must also be tested in scale-out to make sure that the data - * backing the solutions is not discarded before the caller can use - * those data. [This could be handled by materializing binding set - * objects out of a {@link ByteBuffer} rather than using a live decode - * of the data in that {@link ByteBuffer}.] - */ - public void test_query_slice() { - - fail("write test"); - - } - - /** * @todo Test the ability run a query reading on an access path using a * element filter (other than DISTINCT). */ Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java 2010-09-17 14:54:26 UTC (rev 3582) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java 2010-09-17 15:04:56 UTC (rev 3583) @@ -207,7 +207,7 @@ // new IVariable[] { x, y },// // new IConstant[] { new Constant<String>("Leon"), // new Constant<String>("Paul") }// -// ), +// ),ne }; final SliceStats stats = query.newStats(); This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2010-09-17 17:55:39
|
Revision: 3585 http://bigdata.svn.sourceforge.net/bigdata/?rev=3585&view=rev Author: thompsonbry Date: 2010-09-17 17:55:31 +0000 (Fri, 17 Sep 2010) Log Message: ----------- Removed unused test class (TestBindingSet). Added javadoc on IConstant to specify why it does not implement Comparable and updated both it and Constant to clear out old references to that interface. More systematic deep and shallow copy constructors for bops. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/AbstractChunkedOrderedIteratorOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpList.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/Constant.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/IConstant.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/QuoteOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/Var.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java Removed Paths: ------------- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestBindingSet.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/AbstractChunkedOrderedIteratorOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/AbstractChunkedOrderedIteratorOp.java 2010-09-17 17:50:43 UTC (rev 3584) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/AbstractChunkedOrderedIteratorOp.java 2010-09-17 17:55:31 UTC (rev 3585) @@ -46,12 +46,12 @@ } - /** - * @param args - */ - protected AbstractChunkedOrderedIteratorOp(BOp[] args) { - super(args); - } +// /** +// * @param args +// */ +// protected AbstractChunkedOrderedIteratorOp(BOp[] args) { +// super(args); +// } /** * Required shallow copy constructor. Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java 2010-09-17 17:50:43 UTC (rev 3584) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java 2010-09-17 17:55:31 UTC (rev 3585) @@ -135,6 +135,8 @@ } /** + * Deep copy constructor (required). + * <p> * Each {@link BOp} MUST implement a public copy constructor with the * signature: * @@ -168,18 +170,22 @@ // deep copy the annotations. annotations = deepCopy(op.annotations); } - - /** - * @param args - * The arguments to the operator. - */ - public BOpBase(final BOp[] args) { - - this(args, null/* annotations */); - - } +// /** +// * @param args +// * The arguments to the operator. +// * +// * @deprecated Use the shallow copy constructor. +// */ +// public BOpBase(final BOp[] args) { +// +// this(args, null/* annotations */); +// +// } + /** + * Shallow copy constructor (required). + * * @param args * The arguments to the operator. * @param annotations Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpList.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpList.java 2010-09-17 17:50:43 UTC (rev 3584) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpList.java 2010-09-17 17:55:31 UTC (rev 3585) @@ -58,8 +58,8 @@ } - public BOpList(final BOp[] args) { - super(args); - } +// public BOpList(final BOp[] args) { +// super(args); +// } } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/Constant.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/Constant.java 2010-09-17 17:50:43 UTC (rev 3584) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/Constant.java 2010-09-17 17:55:31 UTC (rev 3585) @@ -29,8 +29,7 @@ * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id$ */ -final public class Constant<E/* extends Comparable<E> */> extends BOpBase - implements IConstant<E> { +final public class Constant<E> extends BOpBase implements IConstant<E> { /** * @@ -73,7 +72,7 @@ public Constant(final E value) { - super(new BOp[]{}); + super(new BOp[] {}, null/* annotations */); if (value == null) throw new IllegalArgumentException(); @@ -130,25 +129,6 @@ return value.hashCode(); } - -// public int compareTo(IVariableOrConstant arg0) { -// -// // order vars before ids -// if(arg0 instanceof Var) return 1; -// -// // @todo ordering only among constants of the same class? -// Constant o = (Constant)arg0; -// -// /* -// * Note: logic avoids possible overflow of [long] by not computing the -// * difference between two longs. -// */ -// -// int ret = id < o.id ? -1 : id > o.id ? 1 : 0; -// -// return ret; -// -// } final public E get() { @@ -167,11 +147,5 @@ throw new UnsupportedOperationException(); } - -// public int compareTo(IConstant<E> o) { -// -// return value.compareTo(o.get()); -// -// } } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/IConstant.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/IConstant.java 2010-09-17 17:50:43 UTC (rev 3584) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/IConstant.java 2010-09-17 17:55:31 UTC (rev 3585) @@ -30,12 +30,24 @@ /** * A constant. + * <p> + * Note: {@link IConstant} does not implement {@link Comparable} for two + * reasons: + * <ol> + * <li>{@link Constant}s wrapping different data types are not comparable. Rigid + * schema data models such as SQL do not have this problem since columns have a + * single data type, but schema flexible object models and RDF both have runtime + * determination of the data type.</li> + * <li>The specifics of the ordering to be imposed are generally determined by a + * high level query language (SPARQL, XQUERY, SQL, etc). Thus even if this + * interface was {@link Comparable}, SORT operators generally must provide their + * own ordering semantics.</li> + * </ol> * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id$ */ -public interface IConstant<E/* extends Comparable<E>*/> extends - IVariableOrConstant<E> { //, Comparable<IConstant<E>> { +public interface IConstant<E> extends IVariableOrConstant<E> { /** * The hash code of the value that would be returned by Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/QuoteOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/QuoteOp.java 2010-09-17 17:50:43 UTC (rev 3584) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/QuoteOp.java 2010-09-17 17:55:31 UTC (rev 3585) @@ -72,7 +72,7 @@ */ public QuoteOp(final BOp op) { - super(new BOp[] { op }); + super(new BOp[] { op }, null/*annotations*/); if (op == null) throw new IllegalArgumentException(); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/Var.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/Var.java 2010-09-17 17:50:43 UTC (rev 3584) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/Var.java 2010-09-17 17:55:31 UTC (rev 3585) @@ -58,7 +58,7 @@ */ private Var(final String name) { - super(new BOp[]{}); + super(new BOp[] {}, null/* annotations */); assert name != null; Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java 2010-09-17 17:50:43 UTC (rev 3584) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java 2010-09-17 17:55:31 UTC (rev 3585) @@ -48,7 +48,6 @@ import com.bigdata.bop.IBindingSet; import com.bigdata.bop.IPredicate; import com.bigdata.bop.bset.Union; -import com.bigdata.bop.fed.FederatedQueryEngine; import com.bigdata.btree.BTree; import com.bigdata.btree.IndexSegment; import com.bigdata.btree.view.FusedView; @@ -757,22 +756,6 @@ * @throws IllegalStateException * if the {@link QueryEngine} has been {@link #shutdown()}. * @throws Exception - * @throws RemoteException - * - * FIXME The test suites need to be modified to create a local - * {@link FederatedQueryEngine} object which fronts for an - * {@link IIndexManager} which is local to the client - not on a - * data service at all. This is necessary in order for the unit - * test (or application code) to directly access the - * RunningQuery reference, which is needed to use get() (to wait - * for the query), iterator() (to drain the query), etc. - * <p> - * This will also give us a place to hang query-local resources - * on the client. - * <p> - * This has to be a {@link FederatedQueryEngine} because it - * needs to talk to a federation. There should be nothing DS - * specific about the {@link FederatedQueryEngine}. */ public RunningQuery eval(final UUID queryId, final BindingSetPipelineOp query, Deleted: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestBindingSet.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestBindingSet.java 2010-09-17 17:50:43 UTC (rev 3584) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestBindingSet.java 2010-09-17 17:55:31 UTC (rev 3585) @@ -1,176 +0,0 @@ -/* - -Copyright (C) SYSTAP, LLC 2006-2008. All rights reserved. - -Contact: - SYSTAP, LLC - 4501 Tower Road - Greensboro, NC 27410 - lic...@bi... - -This program is free software; you can redistribute it and/or modify -it under the terms of the GNU General Public License as published by -the Free Software Foundation; version 2 of the License. - -This program is distributed in the hope that it will be useful, -but WITHOUT ANY WARRANTY; without even the implied warranty of -MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -GNU General Public License for more details. - -You should have received a copy of the GNU General Public License -along with this program; if not, write to the Free Software -Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA - -*/ -/* - * Created on Jun 19, 2008 - */ - -package com.bigdata.bop; - -import junit.framework.TestCase2; - -/** - * Unit tests for {@link IBindingSet}s. - * - * @author <a href="mailto:tho...@us...">Bryan Thompson</a> - * @version $Id$ - */ -public class TestBindingSet extends TestCase2 { - - /** - * - */ - public TestBindingSet() { - } - - /** - * @param name - */ - public TestBindingSet(String name) { - super(name); - } - - /** - * Unit test for {@link ArrayBindingSet#copy(IVariable[])} - */ - public void test_copy_abs() { - - IVariable[] vars = new IVariable[] { - Var.var("a"), - Var.var("b"), - Var.var("c"), - Var.var("d"), - Var.var("e") - }; - - IConstant[] vals = new IConstant[] { - new Constant<Integer>(1), - new Constant<Integer>(2), - new Constant<Integer>(3), - new Constant<Integer>(4), - new Constant<Integer>(5) - }; - - ArrayBindingSet bs = new ArrayBindingSet(vars, vals); - - assertTrue(bs.size() == 5); - for (IVariable v : vars) { - assertTrue(bs.isBound(v)); - } - - IVariable[] varsToKeep = new IVariable[] { - Var.var("a"), - Var.var("c"), - Var.var("e") - }; - - ArrayBindingSet bs2 = bs.copy(varsToKeep); - assertTrue(bs2.size() == 3); - for (IVariable v : varsToKeep) { - assertTrue(bs2.isBound(v)); - assertTrue(bs2.get(v).equals(bs.get(v))); - } - assertFalse(bs2.isBound(Var.var("b"))); - assertFalse(bs2.isBound(Var.var("d"))); - - } - - /** - * Unit test for {@link HashBindingSet#copy(IVariable[])} - */ - public void test_copy_hbs() { - - IVariable[] vars = new IVariable[] { - Var.var("a"), - Var.var("b"), - Var.var("c"), - Var.var("d"), - Var.var("e") - }; - - IConstant[] vals = new IConstant[] { - new Constant<Integer>(1), - new Constant<Integer>(2), - new Constant<Integer>(3), - new Constant<Integer>(4), - new Constant<Integer>(5) - }; - - HashBindingSet bs = new HashBindingSet(); - for (int i = 0; i < vars.length; i++) { - bs.set(vars[i], vals[i]); - } - - assertTrue(bs.size() == 5); - for (IVariable v : vars) { - assertTrue(bs.isBound(v)); - } - - IVariable[] varsToKeep = new IVariable[] { - Var.var("a"), - Var.var("c"), - Var.var("e") - }; - - HashBindingSet bs2 = bs.copy(varsToKeep); - assertTrue(bs2.size() == 3); - for (IVariable v : varsToKeep) { - assertTrue(bs2.isBound(v)); - assertTrue(bs2.get(v).equals(bs.get(v))); - } - assertFalse(bs2.isBound(Var.var("b"))); - assertFalse(bs2.isBound(Var.var("d"))); - - } - - /** - * @todo Write unit tests for equals which verify that binding sets are - * equals iff they have the same variables and those variables have - * the same bindings. - */ - public void test_equals() { - fail("write tests"); - } - - /** - * @todo Write unit tests for the hash code which demonstrate that the hash - * code of binding sets with the same bindings on the same variables - * is the same regardless of the order in which those variables - * appear. - */ - public void test_hashCode() { - fail("write tests"); - } - - /** - * @todo Write unit tests for the hash code which demonstrate that the hash - * code of the binding sets is invalidated and recomputed when there - * is a mutation to the binding set. Do this for all - * {@link IBindingSet} implementations. - */ - public void test_hashCode_mutations() { - fail("write tests"); - } - -} This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2010-09-23 20:11:07
|
Revision: 3617 http://bigdata.svn.sourceforge.net/bigdata/?rev=3617&view=rev Author: thompsonbry Date: 2010-09-23 20:10:58 +0000 (Thu, 23 Sep 2010) Log Message: ----------- Finally chased down one bug which I had introduced in QueryResultIterator. I've added a bunch of unit tests and the PipelineType annotation, which will eventually support both vectored and operator at a time evaluation. I am still chasing the bug with multiple chunk messages flowing through the query controller. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContextBase.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpUtility.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ap/Predicate.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/CopyBindingSetOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/Union.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/HaltOpMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/PipelineUtility.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunState.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/StartOpMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/notes.txt branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/join/PipelineJoin.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/rdf/join/DataSetJoin.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/util/concurrent/Haltable.java branches/QUADS_QUERY_BRANCH/bigdata/src/resources/logging/log4j.properties branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestBOpUtility.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/bset/TestCopyBindingSets.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/PipelineDelayOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestAll.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestPipelineUtility.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestNIOChunkMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestThickChunkMessage.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestMemorySortOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java Added Paths: ----------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineType.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngineTestAnnotations.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryResultIterator.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/CancelQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestRunState.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java 2010-09-23 16:09:13 UTC (rev 3616) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -167,7 +167,7 @@ * override this method. */ BOpEvaluationContext getEvaluationContext(); - + /** * Return <code>true</code> iff this operator is an access path which writes * on the database. Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java 2010-09-23 16:09:13 UTC (rev 3616) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpBase.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -344,7 +344,8 @@ public String toString() { final StringBuilder sb = new StringBuilder(); - sb.append(getClass().getName()); +// sb.append(getClass().getName()); + sb.append(super.toString()); sb.append("("); for (int i = 0; i < args.length; i++) { final BOp t = args[i]; Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java 2010-09-23 16:09:13 UTC (rev 3616) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContext.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -27,31 +27,18 @@ */ package com.bigdata.bop; -import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; - import org.apache.log4j.Logger; import com.bigdata.bop.engine.BOpStats; import com.bigdata.bop.engine.IChunkMessage; import com.bigdata.bop.engine.IRunningQuery; import com.bigdata.bop.engine.RunningQuery; -import com.bigdata.btree.IIndex; import com.bigdata.btree.ILocalBTreeView; -import com.bigdata.btree.IRangeQuery; import com.bigdata.journal.IIndexManager; -import com.bigdata.journal.TimestampUtility; -import com.bigdata.relation.IRelation; -import com.bigdata.relation.accesspath.AccessPath; import com.bigdata.relation.accesspath.IAccessPath; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; -import com.bigdata.relation.locator.IResourceLocator; -import com.bigdata.relation.rule.IRule; -import com.bigdata.relation.rule.eval.IJoinNexus; -import com.bigdata.service.DataService; import com.bigdata.service.IBigdataFederation; -import com.bigdata.striterator.IKeyOrder; import com.ibm.icu.impl.ByteBuffer; /** @@ -62,18 +49,10 @@ */ public class BOpContext<E> extends BOpContextBase { - static private final Logger log = Logger.getLogger(BOpContext.class); + static private final transient Logger log = Logger.getLogger(BOpContext.class); private final IRunningQuery runningQuery; -// private final IBigdataFederation<?> fed; -// -// private final IIndexManager indexManager; -// -// private final long readTimestamp; -// -// private final long writeTimestamp; - private final int partitionId; private final BOpStats stats; @@ -95,60 +74,8 @@ public IRunningQuery getRunningQuery() { return runningQuery; } - + /** - * The {@link IBigdataFederation} IFF the operator is being evaluated on an - * {@link IBigdataFederation}. When evaluating operations against an - * {@link IBigdataFederation}, this reference provides access to the - * scale-out view of the indices and to other bigdata services. - */ - @Override - public IBigdataFederation<?> getFederation() { - return runningQuery.getFederation(); - } - - /** - * The <strong>local</strong> {@link IIndexManager}. Query evaluation occurs - * against the local indices. In scale-out, query evaluation proceeds shard - * wise and this {@link IIndexManager} MUST be able to read on the - * {@link ILocalBTreeView}. - */ - @Override - public IIndexManager getIndexManager() { - return runningQuery.getIndexManager(); - } - - /** - * Return the {@link Executor} on to which the operator may submit tasks. - * <p> - * Note: The is the {@link ExecutorService} associated with the - * <em>local</em> {@link #getIndexManager() index manager}. - */ - public final Executor getExecutorService() { - return runningQuery.getIndexManager().getExecutorService(); - } - -// /** -// * The timestamp or transaction identifier against which the query is -// * reading. -// * -// * @deprecated by {@link BOp.Annotations#TIMESTAMP} -// */ -// public final long getReadTimestamp() { -// return runningQuery.getReadTimestamp(); -// } -// -// /** -// * The timestamp or transaction identifier against which the query is -// * writing. -// * -// * @deprecated by {@link BOp.Annotations#TIMESTAMP} -// */ -// public final long getWriteTimestamp() { -// return runningQuery.getWriteTimestamp(); -// } - - /** * The index partition identifier -or- <code>-1</code> if the index is not * sharded. */ Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContextBase.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContextBase.java 2010-09-23 16:09:13 UTC (rev 3616) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpContextBase.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -27,7 +27,11 @@ */ package com.bigdata.bop; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; + import org.apache.log4j.Logger; + import com.bigdata.bop.engine.QueryEngine; import com.bigdata.btree.IIndex; import com.bigdata.btree.ILocalBTreeView; @@ -53,7 +57,7 @@ */ public class BOpContextBase { - static private final Logger log = Logger.getLogger(BOpContextBase.class); + static private final transient Logger log = Logger.getLogger(BOpContextBase.class); private final QueryEngine queryEngine; @@ -63,7 +67,7 @@ * wise and this {@link IIndexManager} MUST be able to read on the * {@link ILocalBTreeView}. */ - public IIndexManager getIndexManager() { + final public IIndexManager getIndexManager() { return queryEngine.getIndexManager(); } @@ -73,11 +77,21 @@ * {@link IBigdataFederation}, this reference provides access to the * scale-out view of the indices and to other bigdata services. */ - public IBigdataFederation<?> getFederation() { + final public IBigdataFederation<?> getFederation() { return queryEngine.getFederation(); } /** + * Return the {@link Executor} on to which the operator may submit tasks. + * <p> + * Note: The is the {@link ExecutorService} associated with the + * <em>local</em> {@link #getIndexManager() index manager}. + */ + public final Executor getExecutorService() { + return getIndexManager().getExecutorService(); + } + + /** * * @param indexManager * The <strong>local</strong> {@link IIndexManager}. Query Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpUtility.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpUtility.java 2010-09-23 16:09:13 UTC (rev 3616) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOpUtility.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -27,6 +27,7 @@ package com.bigdata.bop; +import java.util.Collections; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.LinkedHashSet; @@ -374,7 +375,7 @@ * @param op * A {@link BOp}. * - * @return The index. + * @return The index, which is immutable and thread-safe. * * @throws DuplicateBOpIdException * if there are two or more {@link BOp}s having the same @@ -412,7 +413,8 @@ throw new DuplicateBOpException(t.toString()); } } - return map; + // wrap to ensure immutable and thread-safe. + return Collections.unmodifiableMap(map); } /** Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java 2010-09-23 16:09:13 UTC (rev 3616) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -209,6 +209,16 @@ protected static transient final TimeUnit chunkTimeoutUnit = TimeUnit.MILLISECONDS; /** + * Return the {@link PipelineType} of the operator (default + * {@link PipelineType#Vectored}). + */ + public PipelineType getPipelineType() { + + return PipelineType.Vectored; + + } + + /** * Return <code>true</code> iff {@link #newStats()} must be shared across * all invocations of {@link #eval(BOpContext)} for this operator for a * given query (default <code>false</code>). Added: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineType.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineType.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineType.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -0,0 +1,68 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Sep 21, 2010 + */ + +package com.bigdata.bop; + +/** + * Return the type of pipelining supported by an operator. + * <p> + * Note: bigdata does not support tuple-at-a-time processing. Only vectored and + * operator-at-a-time processing. Tuple at a time processing is generally very + * inefficient. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +public enum PipelineType { + + /** + * Vectored operators stream chunks of intermediate results from one + * operator to the next using producer / consumer pattern. Each time a set + * of intermediate results is available for a vectored operator, it is + * evaluated against those inputs producing another set of intermediate + * results for its target operator(s). Vectored operators may be evaluated + * many times during a given query and often have excellent parallelism due + * to the concurrent evaluation of the different operators on different sets + * of intermediate results. + */ + Vectored, + + /** + * The operator will run exactly once and must wait for all of its inputs to + * be assembled before it runs. + * <p> + * There are some operations for which this is always true, such as SORT. + * Other operations MAY use operator-at-once evaluation in order to benefit + * from a combination of more efficient IO patterns and simpler design. + * However, pipelined operators using large memory blocks have many of the + * benefits of operator-at-once evaluation. By deferring their evaluation + * until some minimum number of source data blocks are available, they may + * be evaluated once or more than once, depending on the data scale. + */ + OneShot; + +} Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineType.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ap/Predicate.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ap/Predicate.java 2010-09-23 16:09:13 UTC (rev 3616) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/ap/Predicate.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -30,6 +30,8 @@ import java.util.Map; +import cern.colt.Arrays; + import com.bigdata.bop.AbstractChunkedOrderedIteratorOp; import com.bigdata.bop.BOp; import com.bigdata.bop.Constant; @@ -414,7 +416,15 @@ for (Map.Entry<String, Object> e : annotations.entrySet()) { if (!first) sb.append(", "); - sb.append(e.getKey() + "=" + e.getValue()); + // @todo remove relation name hack when making relation name a scalar. + if (Annotations.RELATION_NAME.equals(e.getKey()) + && e.getValue() != null + && e.getValue().getClass().isArray()) { + sb.append(e.getKey() + "=" + + Arrays.toString((String[]) e.getValue())); + } else { + sb.append(e.getKey() + "=" + e.getValue()); + } first = false; } sb.append("]"); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/CopyBindingSetOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/CopyBindingSetOp.java 2010-09-23 16:09:13 UTC (rev 3616) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/CopyBindingSetOp.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -35,6 +35,7 @@ import com.bigdata.bop.BOpContext; import com.bigdata.bop.BindingSetPipelineOp; import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.IConstraint; import com.bigdata.bop.engine.BOpStats; import com.bigdata.bop.engine.IChunkAccessor; import com.bigdata.relation.accesspath.IAsynchronousIterator; @@ -56,6 +57,16 @@ */ private static final long serialVersionUID = 1L; + public interface Annotations extends BindingSetPipelineOp.Annotations { + + /** + * An optional {@link IConstraint}[] which places restrictions on the + * legal patterns in the variable bindings. + */ + String CONSTRAINTS = CopyBindingSetOp.class.getName() + ".constraints"; + + } + /** * Deep copy constructor. * @@ -75,10 +86,19 @@ super(args, annotations); } + /** + * @see Annotations#CONSTRAINTS + */ + public IConstraint[] constraints() { + + return getProperty(Annotations.CONSTRAINTS, null/* defaultValue */); + + } + public FutureTask<Void> eval(final BOpContext<IBindingSet> context) { - return new FutureTask<Void>(new CopyTask(context)); - + return new FutureTask<Void>(new CopyTask(this, context)); + } /** @@ -90,11 +110,19 @@ static private class CopyTask implements Callable<Void> { private final BOpContext<IBindingSet> context; - - CopyTask(final BOpContext<IBindingSet> context) { - + + /** + * The constraint (if any) specified for the join operator. + */ + final private IConstraint[] constraints; + + CopyTask(final CopyBindingSetOp op, + final BOpContext<IBindingSet> context) { + this.context = context; - + + this.constraints = op.constraints(); + } public Void call() throws Exception { @@ -108,9 +136,10 @@ final IBindingSet[] chunk = source.next(); stats.chunksIn.increment(); stats.unitsIn.add(chunk.length); - sink.add(chunk); + final IBindingSet[] tmp = applyConstraints(chunk); + sink.add(tmp); if (sink2 != null) - sink2.add(chunk); + sink2.add(tmp); } sink.flush(); if (sink2 != null) @@ -124,6 +153,56 @@ } } - } + private IBindingSet[] applyConstraints(final IBindingSet[] chunk) { + + if (constraints == null) { + /* + * No constraints, copy all binding sets. + */ + + return chunk; + + } + + /* + * Copy binding sets which satisfy the constraint(s). + */ + + IBindingSet[] t = new IBindingSet[chunk.length]; + + int j = 0; + + for (int i = 0; i < chunk.length; i++) { + + final IBindingSet bindingSet = chunk[i]; + + if (context.isConsistent(constraints, bindingSet)) { + + t[j++] = bindingSet; + + } + + } + + if (j != chunk.length) { + + // allocate exact size array. + final IBindingSet[] tmp = (IBindingSet[]) java.lang.reflect.Array + .newInstance(chunk[0].getClass(), j); + + // make a dense copy. + System.arraycopy(t/* src */, 0/* srcPos */, tmp/* dst */, + 0/* dstPos */, j/* len */); + + t = tmp; + + } + + return t; + + } + + } // class CopyTask + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/Union.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/Union.java 2010-09-23 16:09:13 UTC (rev 3616) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/bset/Union.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -54,7 +54,7 @@ * be mapped across shards or nodes as appropriate for the parent. UNION runs on * the query controller. In order to avoid routing intermediate results through * the controller, the {@link BindingSetPipelineOp.Annotations#SINK_REF} of each - * child operand should be overriden to specify the parent of the UNION + * child operand should be overridden to specify the parent of the UNION * operator. * <p> * UNION can not be used when the intermediate results must be routed into the Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java 2010-09-23 16:09:13 UTC (rev 3616) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -67,21 +67,26 @@ * #of chunks in. */ final public CAT chunksIn = new CAT(); +// final public AtomicLong chunksIn = new AtomicLong(); /** * #of units sets in (tuples, elements, binding sets, etc). */ final public CAT unitsIn = new CAT(); +// final public AtomicLong unitsIn = new AtomicLong(); /** * #of chunks out. */ final public CAT chunksOut = new CAT(); +// final public AtomicLong chunksOut = new AtomicLong(); + /** * #of units sets in (tuples, elements, binding sets, etc). */ final public CAT unitsOut = new CAT(); +// final public AtomicLong unitsOut = new AtomicLong(); /** * Constructor. @@ -105,15 +110,20 @@ unitsIn.add(o.unitsIn.get()); unitsOut.add(o.unitsOut.get()); chunksOut.add(o.chunksOut.get()); +// chunksIn.addAndGet(o.chunksIn.get()); +// unitsIn.addAndGet(o.unitsIn.get()); +// unitsOut.addAndGet(o.unitsOut.get()); +// chunksOut.addAndGet(o.chunksOut.get()); } + public String toString() { final StringBuilder sb = new StringBuilder(); sb.append(getClass().getName()); - sb.append("{chunksIn=" + chunksIn.estimate_get()); - sb.append(",unitsIn=" + unitsIn.estimate_get()); - sb.append(",chunksOut=" + chunksOut.estimate_get()); - sb.append(",unitsOut=" + unitsOut.estimate_get()); + sb.append("{chunksIn=" + chunksIn.get()); + sb.append(",unitsIn=" + unitsIn.get()); + sb.append(",chunksOut=" + chunksOut.get()); + sb.append(",unitsOut=" + unitsOut.get()); toString(sb); // extension hook sb.append("}"); return sb.toString(); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/HaltOpMessage.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/HaltOpMessage.java 2010-09-23 16:09:13 UTC (rev 3616) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/HaltOpMessage.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -3,6 +3,8 @@ import java.io.Serializable; import java.util.UUID; +import com.bigdata.bop.BOp; + /** * A message sent to the {@link IQueryClient} when an operator is done executing * for some chunk of inputs. @@ -53,7 +55,7 @@ * scale-out, this is one per index partition over which the intermediate * results were mapped. */ - final public int sinkChunksOut; + final public int sinkMessagesOut; /** * The operator identifier for the alternative sink -or- <code>null</code> @@ -71,7 +73,7 @@ * results were mapped. It is zero if there was no alternative sink for the * operator. */ - final public int altSinkChunksOut; + final public int altSinkMessagesOut; /** * The statistics for the execution of the bop against the partition on the @@ -91,10 +93,19 @@ * The node which executed the operator. * @param cause * <code>null</code> unless execution halted abnormally. - * @param chunksOut - * A map reporting the #of binding set chunks which were output - * for each downstream operator for which at least one chunk of - * output was produced. + * @param sinkId + * The {@link BOp.Annotations#BOP_ID} of the default sink and + * <code>null</code> if there is no sink (for example, if this is + * the last operator in the pipeline). + * @param sinkMessagesOut + * The number of {@link IChunkMessage} which were sent to the + * operator for the default sink. + * @param altSinkId + * The {@link BOp.Annotations#BOP_ID} of the alternative sink and + * <code>null</code> if there is no alternative sink. + * @param altSinkMessagesOut + * The number of {@link IChunkMessage} which were sent to the + * operator for the alternative sink. * @param taskStats * The statistics for the execution of that bop on that shard and * service. @@ -103,8 +114,8 @@ // final UUID queryId, final int bopId, final int partitionId, final UUID serviceId, Throwable cause, // - final Integer sinkId, final int sinkChunksOut,// - final Integer altSinkId, final int altSinkChunksOut,// + final Integer sinkId, final int sinkMessagesOut,// + final Integer altSinkId, final int altSinkMessagesOut,// final BOpStats taskStats) { this.queryId = queryId; @@ -113,9 +124,9 @@ this.serviceId = serviceId; this.cause = cause; this.sinkId = sinkId; - this.sinkChunksOut = sinkChunksOut; + this.sinkMessagesOut = sinkMessagesOut; this.altSinkId = altSinkId; - this.altSinkChunksOut = altSinkChunksOut; + this.altSinkMessagesOut = altSinkMessagesOut; this.taskStats = taskStats; } @@ -128,9 +139,9 @@ if (cause != null) sb.append(",cause=" + cause); sb.append(",sinkId=" + sinkId); - sb.append(",sinkChunksOut=" + sinkChunksOut); + sb.append(",sinkChunksOut=" + sinkMessagesOut); sb.append(",altSinkId=" + altSinkId); - sb.append(",altSinkChunksOut=" + altSinkChunksOut); + sb.append(",altSinkChunksOut=" + altSinkMessagesOut); sb.append(",stats=" + taskStats); sb.append("}"); return sb.toString(); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java 2010-09-23 16:09:13 UTC (rev 3616) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IQueryPeer.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -49,4 +49,18 @@ */ void bufferReady(IChunkMessage<IBindingSet> msg) throws RemoteException; + /** + * Notify a service that the query has been terminated. The peer MUST NOT + * cancel the query synchronously as that can lead to a deadlock with the + * query controller. Instead, the peer should queue a task to cancel the + * query and then return. + * + * @param queryId + * The query identifier. + * @param cause + * The cause. When <code>null</code>, this is presumed to be + * normal query termination. + */ + void cancelQuery(UUID queryId, Throwable cause) throws RemoteException; + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java 2010-09-23 16:09:13 UTC (rev 3616) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/IRunningQuery.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -56,7 +56,14 @@ IIndexManager getIndexManager(); /** - * Terminate query evaluation + * Cancel the running query (normal termination). + * <p> + * Note: This method provides a means for an operator to indicate that the + * query should halt immediately for reasons other than abnormal + * termination. + * <p> + * Note: For abnormal termination of a query, just throw an exception out of + * the query operator implementation. */ void halt(); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/PipelineUtility.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/PipelineUtility.java 2010-09-23 16:09:13 UTC (rev 3616) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/PipelineUtility.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -48,18 +48,16 @@ private static final Logger log = Logger.getLogger(PipelineUtility.class); /** - * Return <code>true</code> iff the <i>runningCountMap</i> AND - * <i>availableChunkMap</i> map are ZERO (0) for both the given operator and - * for all operators which proceed the given operator in the tree structure - * of its operands. + * Return <code>true</code> iff <i>availableChunkMap</i> map is ZERO (0) for + * the given operator and its descendants AND the <i>runningCountMap</i> is + * ZERO (0) for the operator and all descendants of the operator. For the + * purposes of this method, only {@link BOp#args() operands} are considered + * as descendants. * <p> - * Note: The movement of the intermediate binding set chunks forms an - * acyclic directed graph. We can decide whether or not a {@link BOp} in the - * query plan can be triggered by the current activity pattern by inspecting - * the {@link BOp} and its operands recursively. If neither the {@link BOp} - * nor any of its operands (recursively) has non-zero activity then the - * {@link BOp} can not be triggered and this method will return - * <code>true</code>. + * Note: The movement of the intermediate binding set chunks during query + * processing forms an acyclic directed graph. We can decide whether or not + * a {@link BOp} in the query plan can be triggered by the current activity + * pattern by inspecting the {@link BOp} and its operands recursively. * * @param bopId * The identifier for an operator which appears in the query @@ -92,8 +90,10 @@ if (queryPlan == null) throw new IllegalArgumentException(); + if (queryIndex == null) throw new IllegalArgumentException(); + if (availableChunkCountMap == null) throw new IllegalArgumentException(); @@ -103,7 +103,7 @@ throw new NoSuchBOpException(bopId); final Iterator<BOp> itr = BOpUtility.preOrderIterator(op); - + while (itr.hasNext()) { final BOp t = itr.next(); @@ -112,8 +112,17 @@ if (id == null) continue; + { + /* + * If the operator is running then it is, defacto, "not done." + * + * If any descendants of the operator are running, then they + * could cause the operator to be re-triggered and it is "not + * done." + */ + final AtomicLong runningCount = runningCountMap.get(id); if (runningCount != null && runningCount.get() != 0) { @@ -125,11 +134,16 @@ return false; } - + } { - + + /* + * Any chunks available for the operator in question or any of + * its descendants could cause that operator to be triggered. + */ + final AtomicLong availableChunkCount = availableChunkCountMap .get(id); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java 2010-09-23 16:09:13 UTC (rev 3616) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -41,31 +41,16 @@ import org.apache.log4j.Logger; -import alice.tuprolog.Prolog; - import com.bigdata.bop.BOp; import com.bigdata.bop.BindingSetPipelineOp; import com.bigdata.bop.IBindingSet; -import com.bigdata.bop.IPredicate; -import com.bigdata.bop.bset.Union; import com.bigdata.btree.BTree; import com.bigdata.btree.IndexSegment; import com.bigdata.btree.view.FusedView; import com.bigdata.journal.IIndexManager; -import com.bigdata.rdf.internal.IV; -import com.bigdata.rdf.spo.SPORelation; -import com.bigdata.relation.IMutableRelation; -import com.bigdata.relation.IRelation; -import com.bigdata.relation.accesspath.IElementFilter; -import com.bigdata.relation.rule.IRule; -import com.bigdata.relation.rule.Program; -import com.bigdata.relation.rule.eval.pipeline.DistributedJoinTask; import com.bigdata.resources.IndexManager; import com.bigdata.service.IBigdataFederation; import com.bigdata.service.IDataService; -import com.bigdata.service.ndx.IAsynchronousWriteBufferFactory; -import com.bigdata.striterator.ChunkedArrayIterator; -import com.bigdata.striterator.IChunkedOrderedIterator; /** * A class managing execution of concurrent queries against a local @@ -185,132 +170,6 @@ * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id$ * - * - * FIXME Unit tests for non-distinct {@link IElementFilter}s on an - * {@link IPredicate}, unit tests for distinct element filter on an - * {@link IPredicate} which is capable of distributed operations. Do not use - * distinct where not required (SPOC, only one graph, etc). - * <p> - * It seems like the right way to approach this is by unifying the stackable CTC - * striterator pattern with the chunked iterator pattern and passing the query - * engine (or the bop context) into the iterator construction process (or simply - * requesting that the query engine construct the iterator stack). - * <p> - * In terms of harmonization, it is difficult to say which way would work - * better. In the short term we could simply allow both and mask the differences - * in how we construct the filters, but the conversion to/from striterators and - * chunked iterators seems to waste a bit of effort. - * <p> - * The trickiest part of all of this is to allow a distributed filter pattern - * where the filter gets created on a set of nodes identified by the operator - * and the elements move among those nodes using the query engine's buffers. - * <p> - * To actually implement the distributed distinct filter we need to stack the - * following: - * - * <pre> - * - ITupleIterator - * - Resolve ITuple to Element (e.g., SPOC). - * - Layer on optional IElementFilter associated with the IPredicate. - * - Layer on SameVariableConstraint iff required (done by AccessPath) - * - Resolve SPO to SPO, stripping off the context position. - * - Chunk SPOs (SPO[], IKeyOrder), where the key order is from the access path. - * - Filter SPO[] using DHT constructed on specified nodes of the cluster. - * The SPO[] chunks should be packaged into NIO buffers and shipped to those - * nodes. The results should be shipped back as a bit vectors packaged into - * a NIO buffers. - * - Dechunk SPO[] to SPO since that is the current expectation for the filter - * stack. - * - The result then gets wrapped as a {@link IChunkedOrderedIterator} by - * the AccessPath using a {@link ChunkedArrayIterator}. - * </pre> - * - * This stack is a bit complex(!). But it is certainly easy enough to generate - * the necessary bits programmatically. - * - * FIXME Handling the {@link Union} of binding sets. Consider whether the chunk - * combiner logic from the {@link DistributedJoinTask} could be reused. - * - * FIXME INSERT and DELETE which will construct elements using - * {@link IRelation#newElement(java.util.List, IBindingSet)} from a binding set - * and then use {@link IMutableRelation#insert(IChunkedOrderedIterator)} and - * {@link IMutableRelation#delete(IChunkedOrderedIterator)}. For s/o, we first - * need to move the bits into the right places so it makes sense to unpack the - * processing of the loop over the elements and move the data around, writing on - * each index as necessary. There could be eventually consistent approaches to - * this as well. For justifications we need to update some additional indices, - * in which case we are stuck going through {@link IRelation} rather than - * routing data directly or using the {@link IAsynchronousWriteBufferFactory}. - * For example, we could handle routing and writing in s/o as follows: - * - * <pre> - * INSERT(relation,bindingSets) - * - * expands to - * - * SEQUENCE( - * SELECT(s,p,o), // drop bindings that we do not need - * PARALLEL( - * INSERT_INDEX(spo), // construct (s,p,o) elements and insert - * INSERT_INDEX(pos), // construct (p,o,s) elements and insert - * INSERT_INDEX(osp), // construct (o,s,p) elements and insert - * )) - * - * </pre> - * - * The output of the SELECT operator would be automatically mapped against the - * shards on which the next operators need to write. Since there is a nested - * PARALLEL operator, the mapping will be against the shards of each of the - * given indices. (A simpler operator would invoke - * {@link SPORelation#insert(IChunkedOrderedIterator)}. Handling justifications - * requires that we also formulate the justification chain from the pattern of - * variable bindings in the rule). - * - * FIXME Handle {@link Program}s. There are three flavors, which should probably - * be broken into three operators: sequence(ops), set(ops), and closure(op). The - * 'set' version would be parallelized, or at least have an annotation for - * parallel evaluation. These things belong in the same broad category as the - * join graph since they are operators which control the evaluation of other - * operators (the current pipeline join also has that characteristic which it - * uses to do the nested index subqueries). - * - * FIXME SPARQL to BOP translation - * <p> - * The initial pass should translate from {@link IRule} to {@link BOp}s so we - * can immediately begin running SPARQL queries against the {@link QueryEngine}. - * A second pass should explore a rules base translation from the openrdf SPARQL - * operator tree into {@link BOp}s, perhaps using an embedded {@link Prolog} - * engine. What follows is a partial list of special considerations for that - * translation: - * <ul> - * <li>Distinct can be trivially enforced for default graph queries against the - * SPOC index.</li> - * <li>Local distinct should wait until there is more than one tuple from the - * index since a single tuple does not need to be made distinct using a hash - * map.</li> - * <li>Low volume distributed queries should use solution modifiers which - * evaluate on the query controller node rather than using distributed sort, - * distinct, slice, or aggregation operators.</li> - * <li></li> - * <li></li> - * <li></li> - * <li>High volume queries should use special operators (different - * implementations of joins, use an external merge sort, etc).</li> - * </ul> - * - * FIXME SPARQL Coverage: Add native support for all SPARQL operators. A lot of - * this can be picked up from Sesame. Some things, such as isIRI() can be done - * natively against the {@link IV}. Likewise, there is already a set of - * comparison methods for {@link IV}s which are inlined values. Add support for - * <ul> - * <li></li> - * <li></li> - * <li></li> - * <li></li> - * <li></li> - * <li></li> - * </ul> - * * @todo Expander patterns will continue to exist until we handle the standalone * backchainers in a different manner for scale-out so add support for * those for now. @@ -536,6 +395,8 @@ if (q.isCancelled()) continue; final IChunkMessage<IBindingSet> chunk = q.chunksIn.poll(); + if (chunk == null) + continue; if (log.isTraceEnabled()) log.trace("Accepted chunk: " + chunk); try { @@ -820,6 +681,9 @@ */ protected RunningQuery getRunningQuery(final UUID queryId) { + if(queryId == null) + throw new IllegalArgumentException(); + return runningQueries.get(queryId); } @@ -868,4 +732,13 @@ } + /** + * {@inheritDoc} + * <p> + * The default implementation is a NOP. + */ + public void cancelQuery(UUID queryId, Throwable cause) { + // NOP + } + } Added: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngineTestAnnotations.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngineTestAnnotations.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngineTestAnnotations.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -0,0 +1,69 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Sep 21, 2010 + */ + +package com.bigdata.bop.engine; + +import com.bigdata.bop.PipelineOp; + +/** + * Annotations understood by the {@link QueryEngine} which are used for some + * unit tests but which should not be used for real queries. + * <p> + * Note: This class is in the main source tree because {@link QueryEngine} + * references it, but the annotations defined here should only be specified from + * within a unit test. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +public interface QueryEngineTestAnnotations { + + /** + * When <code>true</code>, each chunk will be sent out using its own + * {@link IChunkMessage}. Otherwise the {@link QueryEngine} MAY (and + * generally does) combine the chunks in the output of a given operator + * evaluation pass into a single {@link IChunkMessage} per target query + * peer. + * <p> + * Note: This annotation was introduced to make it easier to control the #of + * {@link IChunkMessage}s output from a given operator and thereby diagnose + * {@link RunState} termination conditions linked to having multiple + * {@link IChunkMessage}s. + * <p> + * Note: Just controlling the {@link PipelineOp.Annotations#CHUNK_CAPACITY} + * and {@link PipelineOp.Annotations#CHUNK_OF_CHUNKS_CAPACITY} is not enough + * to force the {@link QueryEngine} to run the an operator once per source + * chunk. The {@link QueryEngine} normally combines chunks together. You + * MUST also specify this annotation in order for the query engine to send + * multiple {@link IChunkMessage} rather than just one. + */ + String ONE_MESSAGE_PER_CHUNK = QueryEngineTestAnnotations.class.getName() + + ".oneMessagePerChunk"; + + boolean DEFAULT_ONE_MESSAGE_PER_CHUNK = false; + +} Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngineTestAnnotations.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Added: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryResultIterator.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryResultIterator.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryResultIterator.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -0,0 +1,98 @@ +package com.bigdata.bop.engine; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import com.bigdata.relation.accesspath.IAsynchronousIterator; + +/** + * Delegate pattern cancels the {@link RunningQuery} when the iterator is + * {@link #close() closed} and signals normal completion of the query once the + * iterator is exhausted. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * @version $Id$ + */ +class QueryResultIterator<E> implements IAsynchronousIterator<E> { + + private final RunningQuery runningQuery; + + private final IAsynchronousIterator<E> src; + + private final AtomicBoolean open = new AtomicBoolean(true); + + public QueryResultIterator(final RunningQuery runningQuery, + final IAsynchronousIterator<E> src) { + + if (runningQuery == null) + throw new IllegalArgumentException(); + + if (src == null) + throw new IllegalArgumentException(); + + this.runningQuery = runningQuery; + + this.src = src; + + } + + public void close() { + if (open.compareAndSet(true/* expect */, false/* update */)) { + try { + runningQuery.cancel(true/* mayInterruptIfRunning */); + } finally { + src.close(); + } + } + } + + private void normalCompletion() { + if (open.compareAndSet(true/* expect */, false/* update */)) { + /* + * Note: DO NOT halt the query here!!!! That will cause it to not + * accept any more messages. Just close the source iterator. + */ +// try { +// runningQuery.halt(); +// } finally { + src.close(); +// } + } + } + + public boolean isExhausted() { +// return src.isExhausted(); + if (src.isExhausted()) { + normalCompletion(); + return true; + } + return false; + } + + public boolean hasNext() { +// return src.hasNext(); + if (!src.hasNext()) { + normalCompletion(); + return false; + } + return true; + } + + public boolean hasNext(long timeout, TimeUnit unit) + throws InterruptedException { + return src.hasNext(timeout, unit); + } + + public E next(long timeout, TimeUnit unit) throws InterruptedException { + return src.next(timeout, unit); + } + + public E next() { + return src.next(); + } + + public void remove() { + src.remove(); + } + +} Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryResultIterator.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunState.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunState.java 2010-09-23 16:09:13 UTC (rev 3616) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunState.java 2010-09-23 20:10:58 UTC (rev 3617) @@ -29,11 +29,12 @@ import java.rmi.RemoteException; import java.util.Arrays; -import java.util.LinkedHashMap; +import java.util.Collections; import java.util.LinkedHashSet; import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; @@ -42,12 +43,10 @@ import org.apache.log4j.Logger; import com.bigdata.bop.BOp; -import com.bigdata.util.InnerCause; +import com.bigdata.relation.accesspath.IBlockingBuffer; /** - * The run state for a {@link RunningQuery}. This class is NOT thread-safe. - * {@link RunningQuery} uses an internal lock to serialize requests against the - * public methods of this class. + * The run state for a {@link RunningQuery}. This class is thread-safe. * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id$ @@ -66,19 +65,63 @@ } /** - * Note: Due to concurrency, it is possible for an {@link IChunkMessage} to - * be accepted and the corresponding chunk task started, before a - * {@link RunState#startOp(StartOpMessage)} transition has been fully - * processed. This means that the {@link RunState#totalAvailableChunkCount} - * can become transiently negative. This flag disables asserts which would - * otherwise fail on legal transient negatives. + * Message if the query has already started evaluation. */ - static private boolean availableChunkCountMayBeNegative = true; + static private final transient String ERR_QUERY_STARTED = "Query already running."; + + /** + * Message if query evaluation has already halted. + */ + static private final transient String ERR_QUERY_HALTED = "Query already halted."; + + /** + * Message if an operator addressed by a {@link HaltOpMessage} was never started. + */ + static private final transient String ERR_OP_NOT_STARTED = "Operator never ran."; + + /** + * Message if an operator addressed by a message has been halted. + */ + static private final transient String ERR_OP_HALTED = "Operator is not running."; + + /** + * Message if a query deadline has been exceeded. + */ + static private final transient String ERR_DEADLINE = "Query deadline is expired."; + + /** + * {@link RunningQuery#handleOutputChunk(BOp, int, IBlockingBuffer)} drops + * {@link IChunkMessage}s onto {@link RunningQuery#chunksIn} and drops the + * {@link RunningQuery} on {@link QueryEngine#runningQueries} as soon as + * output {@link IChunkMessage}s are generated. A {@link IChunkMessage} MAY + * be taken for evaluation as soon as it is published. This means that the + * operator which will consume that {@link IChunkMessage} can begin to + * execute <em>before</em> {@link RunningQuery#haltOp(HaltOpMessage)} is + * invoked to indicate the end of the operator which produced that + * {@link IChunkMessage}. + * <p> + * This is all fine. However, due to the potential overlap in these + * schedules {@link RunState#totalAvailableCount} can become transiently + * negative. This flag disables asserts which would otherwise fail on legal + * transient negatives. + */ + static private final boolean availableMessageCountMayBeNegative = true; /** + * Flag may be used to turn on stderr output. + */ + static private final boolean debug = true; + + /** * The query. */ - private final RunningQuery query; + private final BOp query; + + /** + * An index from {@link BOp.Annotations#BOP_ID} to {@link BOp} for the + * {@link #query}. + */ + private final Map<Integer,BOp> bopIndex; /** * The query identifier. @@ -94,36 +137,42 @@ private final long deadline; /** + * Set to <code>true</code> iff the query evaluation has begun. + * + * @see #startQuery(IChunkMessage) + */ + private final AtomicBoolean started = new AtomicBoolean(false); + + /** * Set to <code>true</code> iff the query evaluation is complete due to * normal termination. - * <p> - * Note: This is package private to expose it to {@link RunningQuery}. * * @see #haltOp(HaltOpMessage) */ - /*private*/ final AtomicBoolean allDone = new AtomicBoolean(false); + private final AtomicBoolean allDone = new AtomicBoolean(false); /** * The #of run state transitions which have occurred for this query. */ - private long nsteps = 0; + private final AtomicLong nsteps = new AtomicLong(); /** * The #of tasks for this query which have started but not yet halted. */ - private long totalRunningTaskCount = 0; + private final AtomicLong totalRunningCount = new AtomicLong(); /** - * The #of chunks for this query of which a running task has made available - * but which have not yet been accepted for processing by another task. + * The #of {@link IChunkMessage} for the query which a running task has made + * available but which have not yet been accepted for processing by another + * task. */ - private long totalAvailableChunkCount = 0; + private final AtomicLong totalAvailableCount = new AtomicLong(); /** - * A map reporting the #of chunks available for each operator in the - * pipeline (we only report chunks for pipeline operators). The total #of - * chunks available across all operators in the pipeline is reported by - * {@link #totalAvailableChunkCount}. + * A map reporting the #of {@link IChunkMessage} available for each operator + * in the pipeline. The total #of {@link IChunkMessage}s available across + * all operators in the pipeline is reported by {@link #totalAvailableCount} + * . * <p> * The movement of the intermediate binding set chunks forms an acyclic * directed graph. This map is used to track the #of chunks available for @@ -132,62 +181,166 @@ * {@link BOp} had executed informing the {@link QueryEngine} on that node * that it should immediately release all resources associated with that * {@link BOp}. + * <p> + * Note: This collection is package private in order to expose its state to + * the unit tests. Since the map contains {@link AtomicLong}s it can not be + * readily exposed as {@link Map} object. If we were to expose the map, it + * would have to be via a get(key) style interface. */ - private final Map<Integer/* bopId */, AtomicLong/* availableChunkCount */> availableChunkCountMap = new LinkedHashMap<Integer, AtomicLong>(); + /* private */final Map<Integer/* bopId */, AtomicLong/* availableChunkCount */> availableMap = new ConcurrentHashMap<Integer, AtomicLong>(); /** * A collection reporting on the #of instances of a given {@link BOp} which * are concurrently executing. + * <p> + * Note: This collection is package private in order to expose its state to + * the unit tests. Since the map contains {@link AtomicLong}s it can not be + * readily exposed as {@link Map} object. If we were to expose the map, it + * would have to be via a get(key) style interface. */ - private final Map<Integer/* bopId */, AtomicLong/* runningCount */> runningTaskCountMap = new LinkedHashMap<Integer, AtomicLong>(); + /* private */final Map<Integer/* bopId */, AtomicLong/* runningCount */> runningMap = new ConcurrentHashMap<Integer, AtomicLong>(); /** * A collection of the operators which have executed at least once. */ private final Set<Integer/* bopId */> startedSet = new LinkedHashSet<Integer>(); - public RunState(final RunningQuery query) { + /** + * Return the query identifier specified to the constructor. + */ + final public UUID getQueryId() { + return queryId; + } - this.query = query; + /** + * Return the deadline specified to the constructor. + */ + final public long getDeadline() { + return deadline; + } - this.queryId = query.getQueryId(); + /** + * Return <code>true</code> if evaluation of the query has been initiated + * using {@link #startQuery(IChunkMessage)}. + */ + final public boolean isStarted() { + return started.get(); + } - this.deadline = query.getDeadline(); - - // this.nops = query.bopIndex.size(); + /** + * Return <code>true</code> if the query is known to be completed based on + * the {@link #haltOp(HaltOpMessage)}. + */ + final public boolean isAllDone() { + return allDone.get(); + } + /** + * The #of run state transitions which have occurred for this query. + */ + final public long getStepCount() { + return nsteps.get(); } - public void startQuery(final IChunkMessage<?> msg) { + /** + * The #... [truncated message content] |
From: <tho...@us...> - 2010-09-24 13:38:39
|
Revision: 3620 http://bigdata.svn.sourceforge.net/bigdata/?rev=3620&view=rev Author: thompsonbry Date: 2010-09-24 13:38:31 +0000 (Fri, 24 Sep 2010) Log Message: ----------- Fixed a nagging bug when handling multiple small chunks flowing through the pipeline. In the end, it turns out that the problem was the SliceOp. The SliceTask was closing the sink after the first invocation, which was causing the query to be interrupted (cancelled). Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryResultIterator.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunState.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/accesspath/BlockingBuffer.java branches/QUADS_QUERY_BRANCH/bigdata/src/resources/logging/log4j.properties branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/MockRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java 2010-09-23 20:22:50 UTC (rev 3619) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/BOp.java 2010-09-24 13:38:31 UTC (rev 3620) @@ -195,7 +195,7 @@ * identifier for the {@link BOp} within the context of its owning * query. */ - String BOP_ID = BOp.class.getName()+".bopId"; + String BOP_ID = BOp.class.getName() + ".bopId"; /** * The timeout for the operator evaluation (milliseconds). @@ -210,8 +210,8 @@ * be interpreted with respect to the time when the query began to * execute. */ - String TIMEOUT = BOp.class.getName()+".timeout"; - + String TIMEOUT = BOp.class.getName() + ".timeout"; + /** * The default timeout for operator evaluation. */ @@ -233,9 +233,9 @@ * @see #TIMESTAMP */ String MUTATION = BOp.class.getName() + ".mutation"; - + boolean DEFAULT_MUTATION = false; - + /** * The timestamp (or transaction identifier) used by this operator if it * reads or writes on the database. Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java 2010-09-23 20:22:50 UTC (rev 3619) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/PipelineOp.java 2010-09-24 13:38:31 UTC (rev 3620) @@ -30,6 +30,9 @@ import java.util.Map; import java.util.concurrent.TimeUnit; +import org.apache.log4j.Level; +import org.apache.log4j.Priority; + import com.bigdata.bop.engine.BOpStats; import com.bigdata.bop.engine.QueryEngine; import com.bigdata.btree.IRangeQuery; @@ -307,6 +310,22 @@ } + /** + * You can uncomment a line in this method to see who is closing the + * buffer. + * <p> + * {@inheritDoc} + */ + @Override + public void close() { + +// if (isOpen()) +// log.error(toString(), new RuntimeException("STACK TRACE")); + + super.close(); + + } + } } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java 2010-09-23 20:22:50 UTC (rev 3619) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/BOpStats.java 2010-09-24 13:38:31 UTC (rev 3620) @@ -119,7 +119,7 @@ public String toString() { final StringBuilder sb = new StringBuilder(); - sb.append(getClass().getName()); + sb.append(super.toString()); sb.append("{chunksIn=" + chunksIn.get()); sb.append(",unitsIn=" + unitsIn.get()); sb.append(",chunksOut=" + chunksOut.get()); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java 2010-09-23 20:22:50 UTC (rev 3619) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryEngine.java 2010-09-24 13:38:31 UTC (rev 3620) @@ -413,7 +413,17 @@ continue; } } catch (InterruptedException e) { - log.warn("Interrupted."); + /* + * Note: Uncomment the stack trace here if you want to find + * where the query was interrupted. + * + * Note: If you want to find out who interrupted the query, + * then you can instrument BlockingBuffer#close() in + * PipelineOp#newBuffer(stats). + */ + log.warn("Interrupted." +// ,e + ); return; } catch (Throwable ex) { // log and continue Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryResultIterator.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryResultIterator.java 2010-09-23 20:22:50 UTC (rev 3619) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/QueryResultIterator.java 2010-09-24 13:38:31 UTC (rev 3620) @@ -52,10 +52,11 @@ * Note: DO NOT halt the query here!!!! That will cause it to not * accept any more messages. Just close the source iterator. */ + src.close(); // try { // runningQuery.halt(); // } finally { - src.close(); +// src.close(); // } } } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunState.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunState.java 2010-09-23 20:22:50 UTC (rev 3619) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunState.java 2010-09-24 13:38:31 UTC (rev 3620) @@ -338,9 +338,6 @@ messagesProduced(msg.getBOpId(), 1/* nmessages */); - if (log.isInfoEnabled()) - log.info(msg.toString()); - if (TableLog.tableLog.isInfoEnabled()) { /* * Note: RunState is only used by the query controller so this will @@ -359,9 +356,12 @@ null/* cause */, null/* stats */)); } - if (debug) - System.err.println("startQ : " + toString()); + if(log.isInfoEnabled()) + log.info("startQ : " + toString()); + if (log.isTraceEnabled()) + log.trace(msg.toString()); + } /** @@ -397,19 +397,18 @@ messagesConsumed(msg.bopId, msg.nmessages); - if (log.isTraceEnabled()) - log.trace(msg.toString()); - if (TableLog.tableLog.isInfoEnabled()) { TableLog.tableLog.info(getTableRow("startOp", msg.serviceId, msg.bopId, msg.partitionId, msg.nmessages/* fanIn */, null/* cause */, null/* stats */)); } - if (debug) - System.err - .println("startOp: " + toString() + " : bop=" + msg.bopId); + if (log.isInfoEnabled()) + log.info("startOp: " + toString() + " : bop=" + msg.bopId); + if (log.isTraceEnabled()) + log.trace(msg.toString()); + return firstTime; } @@ -470,9 +469,6 @@ if (isAllDone) this.allDone.set(true); - if (log.isTraceEnabled()) - log.trace(msg.toString()); - if (TableLog.tableLog.isInfoEnabled()) { final int fanOut = msg.sinkMessagesOut + msg.altSinkMessagesOut; TableLog.tableLog.info(getTableRow("haltOp", msg.serviceId, @@ -480,10 +476,13 @@ msg.taskStats)); } - if (debug) - System.err.println("haltOp : " + toString() + " : bop=" + msg.bopId + if (log.isInfoEnabled()) + log.info("haltOp : " + toString() + " : bop=" + msg.bopId + ",isOpDone=" + isOpDone); + if (log.isTraceEnabled()) + log.trace(msg.toString()); + if (msg.cause != null) { /* Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java 2010-09-23 20:22:50 UTC (rev 3619) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/solutions/SliceOp.java 2010-09-24 13:38:31 UTC (rev 3620) @@ -275,70 +275,69 @@ public Void call() throws Exception { - if(log.isTraceEnabled()) - log.trace(toString()); - final IAsynchronousIterator<IBindingSet[]> source = context .getSource(); final IBlockingBuffer<IBindingSet[]> sink = context.getSink(); - try { + /* + * buffer forms chunks which get flushed onto the sink. + * + * @todo if we have visibility into the #of source chunks, then do + * not buffer more than min(#source,#needed). + */ + final UnsynchronizedArrayBuffer<IBindingSet> out = new UnsynchronizedArrayBuffer<IBindingSet>( + sink, op.getChunkCapacity()); + while (source.hasNext()) { + + final IBindingSet[] chunk = source.next(); + /* - * buffer forms chunks which get flushed onto the sink. + * Batch each chunk through a lock for better concurrency + * (avoids CAS contention). * - * @todo if we have visibility into the #of source chunks, then - * do not buffer more than min(#source,#needed). + * Note: This is safe because the source chunk is already + * materialized and the sink will not block (that is part of the + * bop evaluation contract). + * + * Note: We need to be careful here with concurrent close of the + * sink (which is the shared queryBuffer) by concurrent + * SliceOps. The problem is that the slice can count off the + * solutions without having them flushed all the way through to + * the queryBuffer, but we can not close the query buffer until + * we actually see the last solution added to the query buffer. + * This is why the slice flushes the buffer while it is + * synchronized. */ - final UnsynchronizedArrayBuffer<IBindingSet> out = new UnsynchronizedArrayBuffer<IBindingSet>( - sink, op.getChunkCapacity()); + synchronized (stats) { - boolean halt = false; - - while (source.hasNext() && !halt) { + if (log.isTraceEnabled()) + log.trace(toString() + ": stats=" + stats + ", sink=" + + sink); - final IBindingSet[] chunk = source.next(); + final boolean halt = handleChunk(out, chunk); - /* - * Batch each chunk through a lock for better concurrency - * (avoids CAS contention). - * - * Note: This is safe because the source chunk is already - * materialized and the sink will not block (that is part of - * the bop evaluation contract). - */ - synchronized (stats) { - - if (handleChunk(out, chunk)) { + if (!out.isEmpty()) + out.flush(); - halt = true; + sink.flush(); - } + if (halt) { - } + if (log.isInfoEnabled()) + log.info("Slice will interrupt query."); - } + context.getRunningQuery().halt(); - if (!out.isEmpty()) - out.flush(); + } - sink.flush(); - - if (halt) { -// log.error("Slice will interrupt query.");// FIXME comment out this line. - context.getRunningQuery().halt();//throw new InterruptedException(); } - // cancelQuery(); - return null; - - } finally { - - sink.close(); - } + return null; + } /** @@ -400,6 +399,8 @@ stats.chunksIn.increment(); +// int nadded = 0; + for (int i = 0; i < chunk.length; i++) { if (stats.naccepted.get() >= limit) @@ -420,6 +421,8 @@ out.add(bset); +// nadded++; + stats.naccepted.incrementAndGet(); if (log.isTraceEnabled()) @@ -428,29 +431,14 @@ } } // next bindingSet - + return false; } - // /** - // * Cancel the query evaluation. This is invoked when the slice has - // been - // * satisfied. At that point we want to halt not only the {@link - // SliceOp} - // * but also the entire query since it does not need to produce any - // more - // * results. - // */ - // private void cancelQuery() { - // - // context.halt(); - // - // } - public String toString() { - return getClass().getName() + "{offset=" + offset + ",limit=" + return super.toString() + "{offset=" + offset + ",limit=" + limit + ",nseen=" + stats.nseen + ",naccepted=" + stats.naccepted + "}"; Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/accesspath/BlockingBuffer.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/accesspath/BlockingBuffer.java 2010-09-23 20:22:50 UTC (rev 3619) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/accesspath/BlockingBuffer.java 2010-09-24 13:38:31 UTC (rev 3620) @@ -475,7 +475,8 @@ final StringBuilder sb = new StringBuilder(); - sb.append("BlockingBuffer"); + sb.append(super.toString()); +// sb.append("BlockingBuffer"); sb.append("{ open=" + open); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/resources/logging/log4j.properties =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/resources/logging/log4j.properties 2010-09-23 20:22:50 UTC (rev 3619) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/resources/logging/log4j.properties 2010-09-24 13:38:31 UTC (rev 3620) @@ -91,13 +91,13 @@ log4j.logger.com.bigdata.util.concurrent.Haltable=ALL -log4j.logger.com.bigdata.bop=ALL +#log4j.logger.com.bigdata.bop=ALL #log4j.logger.com.bigdata.bop.join.PipelineJoin=ALL -#log4j.logger.com.bigdata.bop.solutions.SliceOp=ALL +#log4j.logger.com.bigdata.bop.solutions.SliceOp=ALL,destPlain #log4j.logger.com.bigdata.bop.engine=ALL #log4j.logger.com.bigdata.bop.engine.QueryEngine=ALL #log4j.logger.com.bigdata.bop.engine.RunningQuery=ALL -#log4j.logger.com.bigdata.bop.engine.RunState=ALL +log4j.logger.com.bigdata.bop.engine.RunState=INFO #log4j.logger.com.bigdata.bop.engine.RunningQuery$ChunkTask=ALL #log4j.logger.com.bigdata.bop.fed.FederatedQueryEngine=ALL #log4j.logger.com.bigdata.bop.fed.FederatedRunningQuery=ALL @@ -215,6 +215,11 @@ log4j.appender.dest2.layout=org.apache.log4j.PatternLayout log4j.appender.dest2.layout.ConversionPattern=%-5p: %r %X{hostname} %X{serviceUUID} %X{taskname} %X{timestamp} %X{resources} %t %l: %m%n +## destPlain +#log4j.appender.destPlain=org.apache.log4j.ConsoleAppender +#log4j.appender.destPlain.layout=org.apache.log4j.PatternLayout +#log4j.appender.destPlain.layout.ConversionPattern= + ## # BOp run state trace (tab delimited file). Uncomment the next line to enable. log4j.logger.com.bigdata.bop.engine.RunState$TableLog=INFO,queryRunStateLog Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/MockRunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/MockRunningQuery.java 2010-09-23 20:22:50 UTC (rev 3619) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/MockRunningQuery.java 2010-09-24 13:38:31 UTC (rev 3620) @@ -46,10 +46,6 @@ private final IIndexManager indexManager; -// private final long readTimestamp; -// -// private final long writeTimestamp; - /** * Note: This constructor DOES NOT check its arguments so unit tests may be * written with the minimum dependencies @@ -60,13 +56,10 @@ * @param writeTimestamp */ public MockRunningQuery(final IBigdataFederation<?> fed, - final IIndexManager indexManager/*, final long readTimestamp, - final long writeTimestamp*/) { + final IIndexManager indexManager) { this.fed = fed; this.indexManager = indexManager; -// this.readTimestamp = readTimestamp; -// this.writeTimestamp = writeTimestamp; } @@ -78,14 +71,6 @@ return indexManager; } -// public long getReadTimestamp() { -// return readTimestamp; -// } -// -// public long getWriteTimestamp() { -// return writeTimestamp; -// } - /** * NOP (you have to test things like slices with a full integration). */ Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-09-23 20:22:50 UTC (rev 3619) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/engine/TestQueryEngine.java 2010-09-24 13:38:31 UTC (rev 3620) @@ -405,6 +405,7 @@ final int startId = 1; final int joinId = 2; final int predId = 3; + final int sliceId = 4; /* * Enforce a constraint on the source such that it hands 3 each source @@ -442,7 +443,6 @@ })// ); - final int sliceId = 4; final SliceOp sliceOp = new SliceOp(new BOp[] { joinOp }, // slice annotations NV.asMap(new NV[] {// @@ -509,6 +509,8 @@ startId, -1 /* partitionId */, newBindingSetIterator(sources))); +// runningQuery.get(); + // verify solutions. assertSameSolutionsAnyOrder(expected, new Dechunkerator<IBindingSet>( runningQuery.iterator())); @@ -686,6 +688,131 @@ fail("write test"); } + + /** + * Unit test runs chunks into a slice without a limit. This verifies that + * the query terminates properly even though the slice is willing to accept + * more data. + * + * @throws Exception + */ + public void test_query_slice_noLimit() throws Exception { + + final Var<?> x = Var.var("x"); + final Var<?> y = Var.var("y"); + + final int startId = 1; + final int sliceId = 2; + + /* + * Enforce a constraint on the source such that it hands 3 each source + * chunk to the join operator as a separate chunk + */ + final int nsources = 4; + final StartOp startOp = new StartOp(new BOp[] {}, NV.asMap(new NV[] {// + new NV(Predicate.Annotations.BOP_ID, startId),// + new NV(PipelineOp.Annotations.CHUNK_CAPACITY, 1),// + new NV(PipelineOp.Annotations.CHUNK_OF_CHUNKS_CAPACITY, nsources),// + new NV(QueryEngineTestAnnotations.ONE_MESSAGE_PER_CHUNK, true),// + })); + + final SliceOp sliceOp = new SliceOp(new BOp[] { startOp }, + // slice annotations + NV.asMap(new NV[] { // + new NV(BOp.Annotations.BOP_ID, sliceId),// + new NV(SliceOp.Annotations.OFFSET, 0L),// + new NV(SliceOp.Annotations.LIMIT, Long.MAX_VALUE),// + })// + ); + + // the source data. + final IBindingSet[] source = new IBindingSet[] {// + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { new Constant<String>("John"), + new Constant<String>("Mary") }// + ),// + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { new Constant<String>("Leon"), + new Constant<String>("Paul") }// + ),// + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { new Constant<String>("Paul"), + new Constant<String>("Mary") }// + ),// + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { new Constant<String>("Paul"), + new Constant<String>("Mark") }// + )}; + // Put each source binding set into a chunk by itself. + final IBindingSet[][] sources = new IBindingSet[source.length][]; + for (int i = 0; i < sources.length; i++) { + sources[i] = new IBindingSet[] { source[i] }; + } + assertEquals(nsources, source.length); + assertEquals(nsources, sources.length); + + final BindingSetPipelineOp query = sliceOp; + final UUID queryId = UUID.randomUUID(); + final RunningQuery runningQuery = queryEngine.eval(queryId, query, + new LocalChunkMessage<IBindingSet>(queryEngine, queryId, + startId, -1 /* partitionId */, + newBindingSetIterator(sources))); + + // + // + // + + // the expected solutions. + final IBindingSet[] expected = source; + + // verify solutions. + assertSameSolutionsAnyOrder(expected, new Dechunkerator<IBindingSet>( + runningQuery.iterator())); + + // Wait until the query is done. + runningQuery.get(); + final Map<Integer, BOpStats> statsMap = runningQuery.getStats(); + { + // validate the stats map. + assertNotNull(statsMap); + assertEquals(2, statsMap.size()); + if (log.isInfoEnabled()) + log.info(statsMap.toString()); + } + + // validate the stats for the start operator. + { + final BOpStats stats = statsMap.get(startId); + assertNotNull(stats); + if (log.isInfoEnabled()) + log.info("start: " + stats.toString()); + + // verify query solution stats details. + assertEquals((long)nsources, stats.chunksIn.get()); + assertEquals((long)nsources, stats.unitsIn.get()); + assertEquals((long)nsources, stats.unitsOut.get()); + assertEquals((long)nsources, stats.chunksOut.get()); + } + + // validate the stats for the slice operator. + { + final BOpStats stats = statsMap.get(sliceId); + assertNotNull(stats); + if (log.isInfoEnabled()) + log.info("slice: " + stats.toString()); + + // verify query solution stats details. + assertEquals((long)nsources, stats.chunksIn.get()); + assertEquals((long)nsources, stats.unitsIn.get()); + assertEquals((long)nsources, stats.unitsOut.get()); + assertEquals((long)nsources, stats.chunksOut.get()); + } + + } /** * Run a join with a slice. The slice is always evaluated on the query Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java 2010-09-23 20:22:50 UTC (rev 3619) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/solutions/TestSliceOp.java 2010-09-24 13:38:31 UTC (rev 3620) @@ -54,13 +54,14 @@ import com.bigdata.bop.Var; import com.bigdata.bop.engine.BOpStats; import com.bigdata.bop.engine.IRunningQuery; -import com.bigdata.bop.engine.MockRunningQuery; import com.bigdata.bop.engine.TestQueryEngine; import com.bigdata.bop.solutions.SliceOp.SliceStats; +import com.bigdata.journal.IIndexManager; import com.bigdata.relation.accesspath.BlockingBuffer; import com.bigdata.relation.accesspath.IAsynchronousIterator; import com.bigdata.relation.accesspath.IBlockingBuffer; import com.bigdata.relation.accesspath.ThickAsynchronousIterator; +import com.bigdata.service.IBigdataFederation; import com.bigdata.util.InnerCause; /** @@ -219,7 +220,7 @@ final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( new MockRunningQuery(null/* fed */, null/* indexManager */ - ), -1/* partitionId */, stats, + , sink), -1/* partitionId */, stats, source, sink, null/* sink2 */); // get task. @@ -311,7 +312,7 @@ final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( new MockRunningQuery(null/* fed */, null/* indexManager */ - ), -1/* partitionId */, stats, + , sink), -1/* partitionId */, stats, source, sink, null/* sink2 */); // get task. @@ -323,18 +324,7 @@ assertTrue(ft.isDone()); assertFalse(ft.isCancelled()); -// try { - ft.get(); // verify nothing thrown. -// fail("Expecting inner cause : " + InterruptedException.class); -// } catch (Throwable t) { -// if (InnerCause.isInnerCause(t, InterruptedException.class)) { -// if (log.isInfoEnabled()) -// log.info("Ignoring expected exception: " + t, t); -// } else { -// fail("Expecting inner cause " + InterruptedException.class -// + ", not " + t, t); -// } -// } + ft.get(); // verify nothing thrown. assertEquals(limit, stats.naccepted.get()); assertEquals(offset+limit, stats.nseen.get()); @@ -346,6 +336,171 @@ } + /** + * Unit test where the offset is never satisfied. For this test, all binding + * sets will be consumed but none will be emitted. + * + * @throws InterruptedException + * @throws ExecutionException + */ + public void test_slice_offsetNeverSatisfied() throws InterruptedException, + ExecutionException { + + final int bopId = 1; + + final long offset = 100L; + final long limit = 3L; + + final SliceOp query = new SliceOp(new BOp[] {}, NV.asMap(new NV[] {// + new NV(SliceOp.Annotations.BOP_ID, bopId),// + new NV(SliceOp.Annotations.OFFSET, offset),// + new NV(SliceOp.Annotations.LIMIT, limit),// + })); + + assertEquals("offset", offset, query.getOffset()); + + assertEquals("limit", limit, query.getLimit()); + + // the expected solutions (none) + final IBindingSet[] expected = new IBindingSet[0]; + + final SliceStats stats = query.newStats(); + + final IAsynchronousIterator<IBindingSet[]> source = new ThickAsynchronousIterator<IBindingSet[]>( + new IBindingSet[][] { data.toArray(new IBindingSet[0]) }); + + final IBlockingBuffer<IBindingSet[]> sink = query.newBuffer(stats); + + final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( + new MockRunningQuery(null/* fed */, null/* indexManager */ + , sink), -1/* partitionId */, stats, source, sink, null/* sink2 */); + + // get task. + final FutureTask<Void> ft = query.eval(context); + + ft.run(); + + /* + * Note: When the slice does not have a limit (or if we write a test + * where the #of source binding sets can not satisfy the offset and/or + * limit) then the sink WILL NOT be closed by the slice. Therefore, in + * order for the iterator to terminate we first check the Future of the + * SliceTask and then _close_ the sink before consuming the iterator. + */ + assertTrue(ft.isDone()); + assertFalse(ft.isCancelled()); + ft.get(); // verify nothing thrown. + sink.close(); // close the sink so the iterator will terminate! + + TestQueryEngine.assertSameSolutions(expected, sink.iterator()); + + assertEquals(1L, stats.chunksIn.get()); + assertEquals(6L, stats.unitsIn.get()); + assertEquals(0L, stats.unitsOut.get()); + assertEquals(0L, stats.chunksOut.get()); + assertEquals(6L, stats.nseen.get()); + assertEquals(0L, stats.naccepted.get()); + + } + + /** + * Unit test where the offset plus the limit is never satisfied. For this + * test, all binding sets will be consumed and some will be emitted, but the + * slice is never satisfied. + * + * @throws InterruptedException + * @throws ExecutionException + */ + public void test_slice_offsetPlusLimitNeverSatisfied() throws InterruptedException, + ExecutionException { + + final Var<?> x = Var.var("x"); + final Var<?> y = Var.var("y"); + + final int bopId = 1; + + final long offset = 2L; + final long limit = 10L; + + final SliceOp query = new SliceOp(new BOp[] {}, NV.asMap(new NV[] {// + new NV(SliceOp.Annotations.BOP_ID, bopId),// + new NV(SliceOp.Annotations.OFFSET, offset),// + new NV(SliceOp.Annotations.LIMIT, limit),// + })); + + assertEquals("offset", offset, query.getOffset()); + + assertEquals("limit", limit, query.getLimit()); + + // the expected solutions + final IBindingSet[] expected = new IBindingSet[] {// + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { new Constant<String>("Mary"), + new Constant<String>("Jane") }// + ),// + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { new Constant<String>("Paul"), + new Constant<String>("Leon") }// + ),// + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { new Constant<String>("Paul"), + new Constant<String>("John") }// + ),// + new ArrayBindingSet(// + new IVariable[] { x, y },// + new IConstant[] { new Constant<String>("Leon"), + new Constant<String>("Paul") }// + ),// + }; + + final SliceStats stats = query.newStats(); + + final IAsynchronousIterator<IBindingSet[]> source = new ThickAsynchronousIterator<IBindingSet[]>( + new IBindingSet[][] { data.toArray(new IBindingSet[0]) }); + + final IBlockingBuffer<IBindingSet[]> sink = query.newBuffer(stats); + + final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( + new MockRunningQuery(null/* fed */, null/* indexManager */ + , sink), -1/* partitionId */, stats, source, sink, null/* sink2 */); + + // get task. + final FutureTask<Void> ft = query.eval(context); + + ft.run(); + + /* + * Note: When the slice does not have a limit (or if we write a test + * where the #of source binding sets can not satisfy the offset and/or + * limit) then the sink WILL NOT be closed by the slice. Therefore, in + * order for the iterator to terminate we first check the Future of the + * SliceTask and then _close_ the sink before consuming the iterator. + */ + assertTrue(ft.isDone()); + assertFalse(ft.isCancelled()); + ft.get(); // verify nothing thrown. + sink.close(); // close the sink so the iterator will terminate! + + TestQueryEngine.assertSameSolutions(expected, sink.iterator()); + + assertEquals(1L, stats.chunksIn.get()); + assertEquals(6L, stats.unitsIn.get()); + assertEquals(4L, stats.unitsOut.get()); + assertEquals(1L, stats.chunksOut.get()); + assertEquals(6L, stats.nseen.get()); + assertEquals(4L, stats.naccepted.get()); + + } + + /** + * Unit test where the slice accepts everything. + * + * @throws InterruptedException + * @throws ExecutionException + */ public void test_slice_offset0_limitAll() throws InterruptedException, ExecutionException { @@ -353,8 +508,8 @@ final SliceOp query = new SliceOp(new BOp[] {}, NV.asMap(new NV[] {// new NV(SliceOp.Annotations.BOP_ID, bopId),// -// new NV(SliceOp.Annotations.OFFSET, 1L),// -// new NV(SliceOp.Annotations.LIMIT, 3L),// + // new NV(SliceOp.Annotations.OFFSET, 1L),// + // new NV(SliceOp.Annotations.LIMIT, 3L),// })); assertEquals("offset", 0L, query.getOffset()); @@ -362,8 +517,8 @@ assertEquals("limit", Long.MAX_VALUE, query.getLimit()); // the expected solutions - final IBindingSet[] expected = data.toArray(new IBindingSet[0]); - + final IBindingSet[] expected = data.toArray(new IBindingSet[0]); + final SliceStats stats = query.newStats(); final IAsynchronousIterator<IBindingSet[]> source = new ThickAsynchronousIterator<IBindingSet[]>( @@ -373,19 +528,27 @@ final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( new MockRunningQuery(null/* fed */, null/* indexManager */ - ), -1/* partitionId */, stats, source, sink, null/* sink2 */); + , sink), -1/* partitionId */, stats, source, sink, null/* sink2 */); // get task. final FutureTask<Void> ft = query.eval(context); ft.run(); - TestQueryEngine.assertSameSolutions(expected, sink.iterator()); - + /* + * Note: When the slice does not have a limit (or if we write a test + * where the #of source binding sets can not satisfy the offset and/or + * limit) then the sink WILL NOT be closed by the slice. Therefore, in + * order for the iterator to terminate we first check the Future of the + * SliceTask and then _close_ the sink before consuming the iterator. + */ assertTrue(ft.isDone()); assertFalse(ft.isCancelled()); ft.get(); // verify nothing thrown. + sink.close(); // close the sink so the iterator will terminate! + TestQueryEngine.assertSameSolutions(expected, sink.iterator()); + assertEquals(1L, stats.chunksIn.get()); assertEquals(6L, stats.unitsIn.get()); assertEquals(6L, stats.unitsOut.get()); @@ -395,7 +558,8 @@ } - public void test_slice_correctRejection_badOffset() throws InterruptedException { + public void test_slice_correctRejection_badOffset() + throws InterruptedException { final int bopId = 1; @@ -418,7 +582,7 @@ final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( new MockRunningQuery(null/* fed */, null/* indexManager */ - ), -1/* partitionId */, stats, source, sink, null/* sink2 */); + , sink), -1/* partitionId */, stats, source, sink, null/* sink2 */); // get task. try { @@ -455,7 +619,7 @@ final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( new MockRunningQuery(null/* fed */, null/* indexManager */ - ), -1/* partitionId */, stats, source, sink, null/* sink2 */); + , sink), -1/* partitionId */, stats, source, sink, null/* sink2 */); // get task. try { @@ -516,8 +680,6 @@ final SliceStats stats = query.newStats(); - final IRunningQuery q = new MockRunningQuery(null/* fed */, null/* indexManager */); - // start time in nanos. final long begin = System.nanoTime(); @@ -540,9 +702,14 @@ final IAsynchronousIterator<IBindingSet[]> source = new ThickAsynchronousIterator<IBindingSet[]>( new IBindingSet[][] { chunk }); + final IBlockingBuffer<IBindingSet[]> sink = new BlockingBuffer<IBindingSet[]>( + chunk.length); + + final IRunningQuery q = new MockRunningQuery(null/* fed */, + null/* indexManager */, sink); + final BOpContext<IBindingSet> context = new BOpContext<IBindingSet>( - q, -1/* partitionId */, stats, source, - new BlockingBuffer<IBindingSet[]>(chunk.length), null/* sink2 */); + q, -1/* partitionId */, stats, source, sink, null/* sink2 */); final FutureTask<Void> ft = query.eval(context); @@ -610,4 +777,29 @@ } + private static class MockRunningQuery extends + com.bigdata.bop.engine.MockRunningQuery { + + private final IBlockingBuffer<IBindingSet[]> sink; + + public MockRunningQuery(final IBigdataFederation<?> fed, + final IIndexManager indexManager, + final IBlockingBuffer<IBindingSet[]> sink) { + + super(fed, indexManager); + + this.sink = sink; + + } + + /** + * Overridden to close the sink so the slice will terminate. + */ + @Override + public void halt() { + sink.close(); + } + + } + } This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2010-09-24 14:30:46
|
Revision: 3622 http://bigdata.svn.sourceforge.net/bigdata/?rev=3622&view=rev Author: thompsonbry Date: 2010-09-24 14:30:39 +0000 (Fri, 24 Sep 2010) Log Message: ----------- All implemented unit tests for distributed query now run correctly. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/shards/Bundle.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestAll.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/shards/TestMapBindingSetsOverShards.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-24 13:51:34 UTC (rev 3621) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/RunningQuery.java 2010-09-24 14:30:39 UTC (rev 3622) @@ -78,6 +78,12 @@ .getLogger(ChunkTask.class); /** + * Error message used when an operation which must be performed on the query + * controller is attempted on some other {@link IQueryPeer}. + */ + static protected final String ERR_NOT_CONTROLLER = "Operator only permitted on the query controller"; + + /** * The class executing the query on this node. */ final private QueryEngine queryEngine; @@ -1160,8 +1166,10 @@ */ // cancel any running operators for this query on this node. cancelled |= cancelRunningOperators(mayInterruptIfRunning); - // cancel any running operators for this query on other nodes. - cancelled |= cancelQueryOnPeers(future.getCause()); + if (controller) { + // cancel query on other peers. + cancelled |= cancelQueryOnPeers(future.getCause()); + } if (queryBuffer != null) { /* * Close the query buffer so the iterator draining the query @@ -1216,13 +1224,12 @@ return cancelled; } - + /** * Cancel the query on each node where it is known to be running. * <p> * Note: The default implementation verifies that the caller is holding the - * {@link #lock} but is otherwise a NOP. This is overridden for - * scale-out. + * {@link #lock} but is otherwise a NOP. This is overridden for scale-out. * * @param cause * When non-<code>null</code>, the cause. @@ -1230,11 +1237,15 @@ * @return <code>true</code> iff something was cancelled. * * @throws IllegalMonitorStateException - * unless the {@link #lock} is held by the current - * thread. + * unless the {@link #lock} is held by the current thread. + * @throws UnsupportedOperationException + * unless this is the query controller. */ protected boolean cancelQueryOnPeers(final Throwable cause) { + if (!controller) + throw new UnsupportedOperationException(ERR_NOT_CONTROLLER); + if (!lock.isHeldByCurrentThread()) throw new IllegalMonitorStateException(); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java 2010-09-24 13:51:34 UTC (rev 3621) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/FederatedRunningQuery.java 2010-09-24 14:30:39 UTC (rev 3622) @@ -415,7 +415,7 @@ final BOp targetOp = bopIndex.get(sinkId); - if (bop == null) + if (targetOp == null) throw new IllegalStateException("Not found: " + sinkId); if(log.isTraceEnabled()) @@ -724,12 +724,10 @@ @Override protected boolean cancelQueryOnPeers(final Throwable cause) { - super.cancelQueryOnPeers(cause); + boolean cancelled = super.cancelQueryOnPeers(cause); final UUID queryId = getQueryId(); - boolean cancelled = false; - for (IQueryPeer peer : peers.values()) { try { Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/shards/Bundle.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/shards/Bundle.java 2010-09-24 13:51:34 UTC (rev 3621) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/fed/shards/Bundle.java 2010-09-24 14:30:39 UTC (rev 3622) @@ -37,7 +37,7 @@ this.fromKey = keyOrder.getFromKey(keyBuilder, asBound); - this.toKey = keyOrder.getFromKey(keyBuilder, asBound); + this.toKey = keyOrder.getToKey(keyBuilder, asBound); } @@ -94,4 +94,14 @@ private int hash = 0; + public String toString() { + StringBuilder sb = new StringBuilder(super.toString()); + sb.append("{bindingSet="+bindingSet); + sb.append(",asBound="+asBound); + sb.append(",fromKey="+BytesUtil.toString(fromKey)); + sb.append(",toKey="+BytesUtil.toString(toKey)); + sb.append("}"); + return sb.toString(); + } + } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestAll.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestAll.java 2010-09-24 13:51:34 UTC (rev 3621) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/TestAll.java 2010-09-24 14:30:39 UTC (rev 3622) @@ -108,7 +108,7 @@ * Note: This is tested later once we have gone through the core unit * tests for the services. */ - //suite.addTest( com.bigdata.bop.fed.TestAll.suite() ); + suite.addTest( com.bigdata.bop.fed.TestAll.suite() ); return suite; Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java 2010-09-24 13:51:34 UTC (rev 3621) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestFederatedQueryEngine.java 2010-09-24 14:30:39 UTC (rev 3622) @@ -107,7 +107,10 @@ * @todo reuse the stress tests from {@link TestQueryEngine}. * * @todo verify that the peers notify the query controller when they first - * register + * register + * + * FIXME Write test of an RMI based join (this is used for some default + * graph query patterns). */ public class TestFederatedQueryEngine extends AbstractEmbeddedFederationTestCase { @@ -651,9 +654,9 @@ log.info("join : "+stats.toString()); // verify query solution stats details. - assertEquals(1L, stats.chunksIn.get()); - assertEquals(1L, stats.unitsIn.get()); - assertEquals(5L, stats.unitsOut.get()); + assertEquals(2L, stats.chunksIn.get()); // two shards. + assertEquals(2L, stats.unitsIn.get()); // two shards, one empty bset each. + assertEquals(5L, stats.unitsOut.get()); // total of 5 tuples read across both shards. assertEquals(2L, stats.chunksOut.get()); // since we read on both shards. } @@ -809,9 +812,9 @@ // verify query solution stats details. assertEquals(2L, stats.chunksIn.get()); // since we read on two shards. - assertEquals(1L, stats.unitsIn.get()); // a single empty binding set. - assertEquals(5L, stats.unitsOut.get()); // each of the tuples will be read. - assertEquals(2L, stats.chunksOut.get()); // since we read on both shards. + assertEquals(2L, stats.unitsIn.get()); // a single empty binding set for each. + assertEquals(2L, stats.unitsOut.get()); // one tuple on each shard will satisfy the constraint. + assertEquals(2L, stats.chunksOut.get()); // since we read on both shards and both shards have one tuple which joins. } // validate the stats for the slice operator. @@ -831,12 +834,12 @@ } /** - * Test the ability run a simple join reading on a single shard. There are - * three operators. One feeds an empty binding set[] into the join, another - * is the predicate for the access path on which the join will read (it - * probes the index once for "Mary" and binds "Paul" and "John" when it does - * so), and the third is the join itself (there are two solutions, which are - * "value=Paul" and value="John"). + * Test the ability to run a simple join reading on a single shard. There + * are three operators. One feeds an empty binding set[] into the join, + * another is the predicate for the access path on which the join will read + * (it probes the index once for "Mary" and binds "Paul" and "John" when it + * does so), and the third is the join itself (there are two solutions, + * which are value="Paul" and value="John"). */ public void test_query_join_1shard() throws Exception { @@ -1096,15 +1099,15 @@ // verify solutions. { - // the expected solution (just one). + // the expected solutions. final IBindingSet[] expected = new IBindingSet[] {// - new ArrayBindingSet(// + new ArrayBindingSet(// partition1 new IVariable[] { Var.var("x"), Var.var("y"), Var.var("z") },// new IConstant[] { new Constant<String>("Mary"), new Constant<String>("Paul"), new Constant<String>("Leon") }// ),// - new ArrayBindingSet(// + new ArrayBindingSet(// partition0 new IVariable[] { Var.var("x"), Var.var("y"), Var.var("z") },// new IConstant[] { new Constant<String>("Mary"), new Constant<String>("John"), @@ -1114,6 +1117,13 @@ TestQueryEngine.assertSameSolutionsAnyOrder(expected, new Dechunkerator<IBindingSet>(runningQuery.iterator())); +// // partition0 +// new E("John", "Mary"),// +// new E("Leon", "Paul"),// +// // partition1 +// new E("Mary", "John"),// +// new E("Mary", "Paul"),// +// new E("Paul", "Leon"),// } // Wait until the query is done. @@ -1122,7 +1132,7 @@ { // validate the stats map. assertNotNull(statsMap); - assertEquals(3, statsMap.size()); + assertEquals(4, statsMap.size()); if (log.isInfoEnabled()) log.info(statsMap.toString()); } @@ -1149,10 +1159,10 @@ log.info("join1: " + stats.toString()); // verify query solution stats details. - assertEquals(1L, stats.chunksIn.get()); - assertEquals(1L, stats.unitsIn.get()); + assertEquals(1L, stats.chunksIn.get()); // reads only on one shard. + assertEquals(1L, stats.unitsIn.get()); // the initial binding set. assertEquals(2L, stats.unitsOut.get()); - assertEquals(1L, stats.chunksOut.get()); // @todo depends on where the shards are. + assertEquals(1L, stats.chunksOut.get()); // one chunk out, but will be mapped over two shards. } // validate the stats for the 2nd join operator. @@ -1163,10 +1173,10 @@ log.info("join2: " + stats.toString()); // verify query solution stats details. - assertEquals(1L, stats.chunksIn.get()); // @todo depends on where the shards are. - assertEquals(2L, stats.unitsIn.get()); - assertEquals(2L, stats.unitsOut.get()); - assertEquals(1L, stats.chunksOut.get()); // @todo depends on where the shards are. + assertEquals(2L, stats.chunksIn.get()); // one chunk per shard on which we will read. + assertEquals(2L, stats.unitsIn.get()); // one binding set in per shard. + assertEquals(2L, stats.unitsOut.get()); // one solution per shard. + assertEquals(2L, stats.chunksOut.get()); // since join ran on two shards and each had one solution. } // validate stats for the sliceOp (on the query controller) @@ -1177,10 +1187,10 @@ log.info("slice: " + stats.toString()); // verify query solution stats details. - assertEquals(1L, stats.chunksIn.get()); // @todo? - assertEquals(2L, stats.unitsIn.get()); - assertEquals(2L, stats.unitsOut.get()); - assertEquals(1L, stats.chunksOut.get()); // @todo? + assertEquals(2L, stats.chunksIn.get()); // one chunk from each shard of join2 with a solution. + assertEquals(2L, stats.unitsIn.get()); // one solution per shard for join2. + assertEquals(2L, stats.unitsOut.get()); // slice passes all units. + assertEquals(2L, stats.chunksOut.get()); // slice runs twice. } } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/shards/TestMapBindingSetsOverShards.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/shards/TestMapBindingSetsOverShards.java 2010-09-24 13:51:34 UTC (rev 3621) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/shards/TestMapBindingSetsOverShards.java 2010-09-24 14:30:39 UTC (rev 3622) @@ -64,11 +64,15 @@ import com.bigdata.striterator.IKeyOrder; /** - * Unit tests for {@link MapBindingSetsOverShardsBuffer}. + * Unit tests for {@link MapBindingSetsOverShardsBuffer}. * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id: TestMapBindingSetsOverShards.java 3448 2010-08-18 20:55:58Z * thompsonbry $ + * + * FIXME More unit tests. It appears that none of these tests cover the + * case where there is a shared prefix, e.g., because at least one + * component of the selected key order is bound. */ public class TestMapBindingSetsOverShards extends AbstractEmbeddedFederationTestCase { This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2010-09-25 17:42:00
|
Revision: 3631 http://bigdata.svn.sourceforge.net/bigdata/?rev=3631&view=rev Author: thompsonbry Date: 2010-09-25 17:41:54 +0000 (Sat, 25 Sep 2010) Log Message: ----------- Added worksheet for query cost models. Updates notes on the quads query branch. Modified the remote access path test to run using READ_COMMITTED views. Removed a worksheet with old performance data. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/notes.txt branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestRemoteAccessPath.java Added Paths: ----------- branches/QUADS_QUERY_BRANCH/bigdata/src/architecture/query-cost-model.xls Removed Paths: ------------- branches/QUADS_QUERY_BRANCH/bigdata/src/architecture/performance.xls Deleted: branches/QUADS_QUERY_BRANCH/bigdata/src/architecture/performance.xls =================================================================== (Binary files differ) Added: branches/QUADS_QUERY_BRANCH/bigdata/src/architecture/query-cost-model.xls =================================================================== (Binary files differ) Property changes on: branches/QUADS_QUERY_BRANCH/bigdata/src/architecture/query-cost-model.xls ___________________________________________________________________ Added: svn:mime-type + application/octet-stream Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/notes.txt =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/notes.txt 2010-09-25 00:31:32 UTC (rev 3630) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/engine/notes.txt 2010-09-25 17:41:54 UTC (rev 3631) @@ -123,7 +123,7 @@ For standalone the decision should be based on whether more leaves would be read (the tuple range count may be used as a proxy for this) by issuing individual subqueries for the - specific as bound predicates or by leaving with C unbound. + specific as bound predicates or by reading with C unbound. For scale-out, the decision is different since we can use multi-block iterators on the index segments and do fewer @@ -158,105 +158,79 @@ SPARQL default graph query patterns. Note: Default graph queries require us to apply a distinct {s,p,o} -filter to each default graph access path. +filter to each default graph access path. The default graph queries +uses an expander pattern. See DefaultGraphSolutionExpander and its +inner classes. -Standalone: +The following special cases exist: - The default graph queries uses an expander pattern. See - DefaultGraphSolutionExpander and its inner classes. - - The following special cases exist: - - The default graph data set is empty (no graphs were identified which are known to the database), in which case an empty access path is used. - The default graph data set includes a single graph which is known to the database. C is bound and we impose a filter which strips - off the context position. See StripContextAccessPath. + off the context position. Because C takes on only one value, a + distinct filter is not required. This means that scale-out can + use normal pipeline joins. - - The default graph data set includes all graphs. C is left unbound - and we impose a distinct SPO filter which strips off the context - position. See MergeAllGraphsAccessPath. + See StripContextAccessPath. - - The default graph data set includes more than a threshold number - of graphs. C is left unbound and we impose a distinct SPO filter - which strips off the context position. See - MergeAllGraphsAccessPath. + - ___C index: We know that C is strictly ascending in index order + within each triple. Use an advancer pattern or ignore quads until + the data changes to a new triple. Apply a filter to strip off the + context position. - FIXME The threshold for this case is 200, which is WAY too - low. - - For standalone the decision should be based on whether more - leaves would be read (the tuple range count may be used as a - proxy for this) by issuing individual subqueries for the - specific as bound predicates or by leaving with C unbound. + The same optimization works in scale-out using shard-wise pipeline + joins if the ___C index was created with the constraint that the + all quads for a given triple are on the same shard. - For scale-out, the decision is different since we can use - multi-block iterators on the index segments and do fewer - disk seeks. + - SCALEOUT and VERY HIGH VOLUME: Use a distributed external merge + sort to impose distinct and do operator at a time processing. - - The default graph data set includes more than one graph but less - than some threshold #of graphs. Parallel subtasks are evaluated - for each graph in the data set and write on a shared - BlockingBuffer. The BlockingBuffer is wrapped with an - SPORelation.distinctSPOIterator(). See - DefaultGraphParallelEvaluationAccessPath. + - SCAN and FILTER: The default graph data set includes all graphs OR + the cost of scanning with C unbound is less than the cost of + subqueries with C bound (for scale-out, subquery cost must be + estimated for a remote access path). C is left unbound and we + impose a distinct SPO filter which strips off the context + position. Unless all graphs are being merged, we also apply an IN + filter. - Note: This works out better for standalone because the queries are - much more selective, even when there are a large number of graphs - in the default graph data set. + SCALEOUT: The join evaluation context is ANY, uses a remote access + path, and the access path should be configured to move a + lot of data efficiently over the remote range iterator. -Scale-out: + - It is possible to partition the IN filter based on the + shard on which it will be applied (split the ordered + list of contexts based on the contexts found in a + given shard). - Special cases exist for: - - - The SPOC index when s, p, and o are bound and we have a guarantee - that an SPO prefix is never split (standalone or when the - appropriate ISimpleSplitHandler is defined for the SPOC index). - In this case we can use the normal pipelined join and apply an - optimized distinct filter because we know that C is strictly - ascending. + See MergeAllGraphsAccessPath. - - High volume access paths: Based on a cost analysis, leave C - unbound and then applies an IN filter. This works better because - we can handle the reads on the SPOC index with C unbound very - efficiently in scale-out by using a multi-block iterator on the - index segments. [However, we must still impose DISTINCT on the - access path.] + - SUBQUERY: Parallel subtasks are evaluated for each graph in the + data set and write on a shared BlockingBuffer. The BlockingBuffer + is wrapped with an SPORelation.distinctSPOIterator(). - - For very high volume operations we could do distributed merge - sorts to impose distinct and do operator at a time processing. + SCALEOUT: Mark the join evaluation context as ANY and mark the + access path as remote. -- @todo Add annotation to Predicate to indicate the use of an RMI - access path in scale-out. Modify PipelineJoin such that it can be - used as an ANY operator -or- a SHARDED operator. For default graph - queries, we need to make the join ANY and the predicate use RMI. + Tune the capacity for the remote access path iterator. + When the remote access path will be selective, the + capacity should be small and we will pay a high price if + there are a lot of nested subqueries. When the remote + access path is less selective the capacity should be + larger to reduce the #of RMI requests made per access + path. -- @todo Create a cost plan for leaving C unbound and then filtering - for standalone and scale-out. These will have different cost - profiles. Normalize the range count to the expected number of - leaves. For standalone, the expected number of seeks is predicted by - the standard log linear model and the expected #of transfers is one - per seek. For scale-out, the expected #of seeks is two (per shard) - and the expected #of transfers is one per 1M of leaves. For - scale-out, we must also factor in the RMI cost for doing RMI based - subqueries. This has been pretty substantial in the past, but that - might also have to do with unisolated vs read committed vs read-only - tx views. + Note: The way the code is written, the access path will + do RMI for the range count before issuing the RMI + iterator request. Look at ways to optimize this. -- @todo The current named and default graph solution expanders use a - threshold for filtering which is WAY too low. Raise this into the - query rewrite based on the cost plan, per above. + See DefaultGraphParallelEvaluationAccessPath. - @todo Lazily create the hash map for the distinctSPOIterator when we observe the 2nd distinct SPO value. -- @todo An optimized distinct filter for SPOC. We can note the last - observed {s,p,o} and skip to the next possible o in the index - (o:=o+1) using an advancer pattern (this could also just scan until - o changes). - --- UNION(ops)[maxParallel(default all)] Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestRemoteAccessPath.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestRemoteAccessPath.java 2010-09-25 00:31:32 UTC (rev 3630) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/fed/TestRemoteAccessPath.java 2010-09-25 17:41:54 UTC (rev 3631) @@ -67,7 +67,7 @@ * @version $Id$ * * @todo test read-committed access paths. - * @todo test read historical access paths. + * @todo test read historical access paths (read-only tx). * @todo test unisolated (writable) access paths. * @todo test fully isolated access paths. */ @@ -97,12 +97,11 @@ /** The query controller. */ private FederatedQueryEngine queryEngine; - + /** - * The read only transaction identifier used for the test (initially set to - * an invalid value). + * The timestamp or transaction identifier used for the test. */ - private long tx = Long.MAX_VALUE; + private long tx = ITx.READ_COMMITTED; public Properties getProperties() { @@ -179,8 +178,11 @@ loadData(); - // read-only transaction from the most recent commit point on the db. - tx = fed.getTransactionService().newTx(ITx.READ_COMMITTED); + /* + * Optionally obtain a read-only transaction from the some commit point + * on the db. + */ +// tx = fed.getTransactionService().newTx(ITx.READ_COMMITTED); } @@ -198,8 +200,10 @@ queryEngine = null; } - if (tx != Long.MAX_VALUE) + if (tx != ITx.READ_COMMITTED && tx != ITx.UNISOLATED) { + // Some kind of transaction. fed.getTransactionService().abort(tx); + } super.tearDown(); This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <dm...@us...> - 2010-09-27 16:12:20
|
Revision: 3636 http://bigdata.svn.sourceforge.net/bigdata/?rev=3636&view=rev Author: dmacgbr Date: 2010-09-27 16:12:12 +0000 (Mon, 27 Sep 2010) Log Message: ----------- Added some tests. Updated operators w.r.t the 'not bound yet' case as discussed. e.g. when, say EQ, is evaluated and EQ's variable is not in the IBindingSet, it now returns false rather than true. It is expected that all variables that are going to be bound will have been bound prior to evaluation. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/EQ.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/EQConstant.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/INBinarySearch.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/INHashMap.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/NE.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/NEConstant.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestEQ.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestEQConstant.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestInBinarySearch.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestInHashMap.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestNE.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestNEConstant.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestOR.java Added Paths: ----------- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestINConstraint.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/EQ.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/EQ.java 2010-09-27 14:29:21 UTC (rev 3635) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/EQ.java 2010-09-27 16:12:12 UTC (rev 3636) @@ -73,13 +73,13 @@ final IConstant<?> x = s.get((IVariable<?>) get(0)/* x */); if (x == null) - return true; // not yet bound. + return false; // not bound. // get binding for "y". final IConstant<?> y = s.get((IVariable<?>) get(1)/* y */); if (y == null) - return true; // not yet bound. + return false; // not bound. return x.equals(y); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/EQConstant.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/EQConstant.java 2010-09-27 14:29:21 UTC (rev 3635) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/EQConstant.java 2010-09-27 16:12:12 UTC (rev 3636) @@ -78,7 +78,7 @@ final IConstant<?> asBound = bset.get(var); if (asBound == null) - return true; // not yet bound. + return false; // not bound. final IConstant<?> cnst = (IConstant<?>) get(1); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/INBinarySearch.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/INBinarySearch.java 2010-09-27 14:29:21 UTC (rev 3635) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/INBinarySearch.java 2010-09-27 16:12:12 UTC (rev 3636) @@ -181,8 +181,8 @@ if (x == null) { - // not yet bound : @todo should this reject an unbound variable? - return true; + // not bound + return false; } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/INHashMap.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/INHashMap.java 2010-09-27 14:29:21 UTC (rev 3635) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/INHashMap.java 2010-09-27 16:12:12 UTC (rev 3636) @@ -159,8 +159,8 @@ if (x == null) { - // not yet bound. - return true; + // not bound. + return false; } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/NE.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/NE.java 2010-09-27 14:29:21 UTC (rev 3635) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/NE.java 2010-09-27 16:12:12 UTC (rev 3636) @@ -26,11 +26,9 @@ import java.util.Map; -import com.bigdata.bop.BOpBase; import com.bigdata.bop.BOp; import com.bigdata.bop.IBindingSet; import com.bigdata.bop.IConstant; -import com.bigdata.bop.IConstraint; import com.bigdata.bop.IVariable; /** Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/NEConstant.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/NEConstant.java 2010-09-27 14:29:21 UTC (rev 3635) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/constraint/NEConstant.java 2010-09-27 16:12:12 UTC (rev 3636) @@ -26,11 +26,9 @@ import java.util.Map; -import com.bigdata.bop.BOpBase; import com.bigdata.bop.BOp; import com.bigdata.bop.IBindingSet; import com.bigdata.bop.IConstant; -import com.bigdata.bop.IConstraint; import com.bigdata.bop.IVariable; /** Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestEQ.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestEQ.java 2010-09-27 14:29:21 UTC (rev 3635) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestEQ.java 2010-09-27 16:12:12 UTC (rev 3636) @@ -114,4 +114,15 @@ } + public void test_eval_correct_unblound() { + + final EQ op = new EQ(Var.var("x"), Var.var("y")); + + final IBindingSet bs1 = new ArrayBindingSet(// + new IVariable[] { Var.var("x") }, // + new IConstant[] { new Constant<String>("1") }); + + assertFalse(op.accept(bs1)); + + } } Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestEQConstant.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestEQConstant.java 2010-09-27 14:29:21 UTC (rev 3635) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestEQConstant.java 2010-09-27 16:12:12 UTC (rev 3636) @@ -29,6 +29,13 @@ import junit.framework.TestCase2; +import com.bigdata.bop.ArrayBindingSet; +import com.bigdata.bop.Constant; +import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.IConstant; +import com.bigdata.bop.IVariable; +import com.bigdata.bop.Var; + /** * Unit tests for {@link EQConstant}. * @@ -50,8 +57,40 @@ super(name); } - public void test_something() { - fail("write tests"); + /** + * Unit test for {@link EQConstant#EQConstant(IVariable,IConstant)} + */ + public void testConstructor () + { + try { assertTrue ( null != new EQConstant ( null, new Constant<String> ( "1" ) ) ) ; fail ( "IllegalArgumentException expected, lhs was null" ) ; } + catch ( IllegalArgumentException e ) {} + + try { assertTrue ( null != new EQConstant ( Var.var ( "x" ), null ) ) ; fail ( "IllegalArgumentException expected, rhs was null" ) ; } + catch ( IllegalArgumentException e ) {} + + assertTrue ( null != new EQConstant ( Var.var ( "x" ), new Constant<String> ( "1" ) ) ) ; } - -} + + /** + * Unit test for {@link EQConstant#accept(IBindingSet)} + */ + public void testAccept () + { + Var<?> var = Var.var ( "x" ) ; + Constant<String> val1 = new Constant<String> ( "1" ) ; + Constant<String> val2 = new Constant<String> ( "2" ) ; + Constant<Integer> val3 = new Constant<Integer> ( 1 ) ; + + EQConstant op = new EQConstant ( var, val1 ) ; + + IBindingSet eq = new ArrayBindingSet ( new IVariable<?> [] { var }, new IConstant [] { val1 } ) ; + IBindingSet ne1 = new ArrayBindingSet ( new IVariable<?> [] { var }, new IConstant [] { val2 } ) ; + IBindingSet ne2 = new ArrayBindingSet ( new IVariable<?> [] { var }, new IConstant [] { val3 } ) ; + IBindingSet nb = new ArrayBindingSet ( new IVariable<?> [] {}, new IConstant [] {} ) ; + + assertTrue ( op.accept ( eq ) ) ; + assertFalse ( op.accept ( ne1 ) ) ; + assertFalse ( op.accept ( ne2 ) ) ; + assertFalse ( op.accept ( nb ) ) ; + } +} \ No newline at end of file Added: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestINConstraint.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestINConstraint.java (rev 0) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestINConstraint.java 2010-09-27 16:12:12 UTC (rev 3636) @@ -0,0 +1,132 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Sep 2, 2010 + */ + +package com.bigdata.bop.constraint; + +import java.util.Arrays; + +import junit.framework.TestCase2; + +import com.bigdata.bop.ArrayBindingSet; +import com.bigdata.bop.Constant; +import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.IConstant; +import com.bigdata.bop.IVariable; +import com.bigdata.bop.Var; + +/** + * Unit tests for {@link INHashMap}. + * + * @author <a href="mailto:dm...@us...">David MacMillan</a> + * @version $Id:$ + */ +public abstract class TestINConstraint extends TestCase2 +{ + /** + * + */ + public TestINConstraint () + { + } + + /** + * @param name + */ + public TestINConstraint ( String name ) + { + super ( name ) ; + } + + /** + * Unit test for {@link INHashMap#INHashMap(IVariable<T>,IConstant<T>[])} + */ + public void testConstructor () + { + IVariable<Integer> var = Var.var ( "x" ) ; + IConstant<Integer> vals [] = new IConstant [] { new Constant<Integer> ( 1 ) } ; + + try { assertTrue ( null != newINConstraint ( null, vals ) ) ; fail ( "IllegalArgumentException expected, lhs was null" ) ; } + catch ( IllegalArgumentException e ) {} + + try { assertTrue ( null != newINConstraint ( var, null ) ) ; fail ( "IllegalArgumentException expected, rhs was null" ) ; } + catch ( IllegalArgumentException e ) {} + + try { assertTrue ( null != newINConstraint ( var, new IConstant [] {} ) ) ; fail ( "IllegalArgumentException expected, set was empty" ) ; } + catch ( IllegalArgumentException e ) {} + + assertTrue ( null != newINConstraint ( var, vals ) ) ; + } + + /** + * Unit test for {@link INConstraint#getVariable()} + */ + public void testGetVariable () + { + Var<?> x = Var.var ( "x" ) ; + IConstant vals [] = new Constant [] { new Constant<Integer> ( 1 ) } ; + + INConstraint op = newINConstraint ( x, vals ) ; + + assertTrue ( x.equals ( op.getVariable () ) ) ; + } + + /** + * Unit test for {@link INConstraint#getSet()} + */ + public void testGetSet () + { + Var<?> x = Var.var ( "x" ) ; + IConstant vals [] = new Constant [] { new Constant<Integer> ( 1 ) } ; + + INConstraint op = newINConstraint ( x, vals ) ; + + assertTrue ( Arrays.equals ( vals, op.getSet () ) ) ; + } + + /** + * Unit test for {@link INConstraint#accept(IBindingSet)} + */ + public void testAccept () + { + Var<?> x = Var.var ( "x" ) ; + Constant<Integer> val1 = new Constant<Integer> ( 1 ) ; + Constant<Integer> val2 = new Constant<Integer> ( 2 ) ; + Constant<Integer> val3 = new Constant<Integer> ( 3 ) ; + + INConstraint op = newINConstraint ( x, new IConstant [] { val1, val2 } ) ; + + IBindingSet in = new ArrayBindingSet ( new IVariable<?> [] { x }, new IConstant [] { val1 } ) ; + IBindingSet notin = new ArrayBindingSet ( new IVariable<?> [] { x }, new IConstant [] { val3 } ) ; + IBindingSet nb = new ArrayBindingSet ( new IVariable<?> [] {}, new IConstant [] {} ) ; + + assertTrue ( op.accept ( in ) ) ; + assertFalse ( op.accept ( notin ) ) ; + assertFalse ( op.accept ( nb ) ) ; + } + + protected abstract INConstraint newINConstraint ( IVariable<?> var, IConstant<?> vals [] ) ; +} \ No newline at end of file Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestInBinarySearch.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestInBinarySearch.java 2010-09-27 14:29:21 UTC (rev 3635) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestInBinarySearch.java 2010-09-27 16:12:12 UTC (rev 3636) @@ -27,15 +27,24 @@ package com.bigdata.bop.constraint; -import junit.framework.TestCase2; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import com.bigdata.bop.ArrayBindingSet; +import com.bigdata.bop.Constant; +import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.IConstant; +import com.bigdata.bop.IVariable; +import com.bigdata.bop.Var; + /** * Unit tests for {@link INBinarySearch}. * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id$ */ -public class TestInBinarySearch extends TestCase2 { +public class TestInBinarySearch extends TestINConstraint { /** * @@ -50,8 +59,34 @@ super(name); } - public void test_something() { - fail("write tests"); + /** + * Unit test for {@link INConstraint#accept(IBindingSet)} + */ + public void testAccept () + { + super.testAccept () ; + + // specific test because the implementation does a sort, etc... + + Var<?> x = Var.var ( "x" ) ; + + IConstant [] vals = new IConstant [ 100 ] ; + for ( int i = 0; i < vals.length; i++ ) + vals [ i ] = new Constant<Integer> ( i ) ; + List<IConstant> list = Arrays.asList ( vals ) ; + Collections.shuffle ( list ) ; + vals = list.toArray ( vals ) ; + + INConstraint op = new INBinarySearch ( x, vals ) ; + + assertTrue ( op.accept ( new ArrayBindingSet ( new IVariable<?> [] { x }, new IConstant [] { new Constant<Integer> ( 21 ) } ) ) ) ; + assertTrue ( op.accept ( new ArrayBindingSet ( new IVariable<?> [] { x }, new IConstant [] { new Constant<Integer> ( 37 ) } ) ) ) ; + assertTrue ( op.accept ( new ArrayBindingSet ( new IVariable<?> [] { x }, new IConstant [] { new Constant<Integer> ( 75 ) } ) ) ) ; + assertFalse ( op.accept ( new ArrayBindingSet ( new IVariable<?> [] { x }, new IConstant [] { new Constant<Integer> ( 101 ) } ) ) ) ; } - -} + + @Override protected INConstraint newINConstraint ( IVariable<?> var, IConstant<?> vals [] ) + { + return new INBinarySearch ( var, vals ) ; + } +} \ No newline at end of file Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestInHashMap.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestInHashMap.java 2010-09-27 14:29:21 UTC (rev 3635) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestInHashMap.java 2010-09-27 16:12:12 UTC (rev 3636) @@ -27,7 +27,8 @@ package com.bigdata.bop.constraint; -import junit.framework.TestCase2; +import com.bigdata.bop.IConstant; +import com.bigdata.bop.IVariable; /** * Unit tests for {@link INHashMap}. @@ -35,8 +36,8 @@ * @author <a href="mailto:tho...@us...">Bryan Thompson</a> * @version $Id$ */ -public class TestInHashMap extends TestCase2 { - +public class TestInHashMap extends TestINConstraint +{ /** * */ @@ -50,8 +51,8 @@ super(name); } - public void test_something() { - fail("write tests"); + @Override protected INConstraint newINConstraint ( IVariable<?> var, IConstant<?> vals [] ) + { + return new INHashMap ( var, vals ) ; } - -} +} \ No newline at end of file Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestNE.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestNE.java 2010-09-27 14:29:21 UTC (rev 3635) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestNE.java 2010-09-27 16:12:12 UTC (rev 3636) @@ -29,6 +29,13 @@ import junit.framework.TestCase2; +import com.bigdata.bop.ArrayBindingSet; +import com.bigdata.bop.Constant; +import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.IConstant; +import com.bigdata.bop.IVariable; +import com.bigdata.bop.Var; + /** * Unit tests for {@link NE}. * @@ -50,8 +57,40 @@ super(name); } - public void test_something() { - fail("write tests"); + /** + * Unit test for {@link NE#NE(IVariable,IVariable)} + */ + public void testConstructor () + { + try { assertTrue ( null != new NE ( null, Var.var ( "y" ) ) ) ; fail ( "IllegalArgumentException expected, lhs was null" ) ; } + catch ( IllegalArgumentException e ) {} + + try { assertTrue ( null != new NE ( Var.var ( "x" ), null ) ) ; fail ( "IllegalArgumentException expected, rhs was null" ) ; } + catch ( IllegalArgumentException e ) {} + + try { assertTrue ( null != new NE ( Var.var ( "x" ), Var.var ( "x" ) ) ) ; fail ( "IllegalArgumentException expected, lhs identical to rhs" ) ; } + catch ( IllegalArgumentException e ) {} + + assertTrue ( null != new NE ( Var.var ( "x" ), Var.var ( "y" ) ) ) ; } - -} + + /** + * Unit test for {@link NE#accept(IBindingSet)} + */ + public void testAccept () + { + Var<?> x = Var.var ( "x" ) ; + Var<?> y = Var.var ( "y" ) ; + Var<?> vars [] = new Var<?> [] { x, y } ; + + NE op = new NE ( x, y ) ; + + IBindingSet eq = new ArrayBindingSet ( vars, new IConstant [] { new Constant<String> ( "1" ), new Constant<String> ( "1" ) } ) ; + IBindingSet ne = new ArrayBindingSet ( vars, new IConstant [] { new Constant<String> ( "1" ), new Constant<String> ( "2" ) } ) ; + IBindingSet nb = new ArrayBindingSet ( new IVariable<?> [] { x }, new IConstant [] { new Constant<String> ( "1" ) } ) ; + + assertTrue ( op.accept ( ne ) ) ; + assertFalse ( op.accept ( eq ) ) ; + assertTrue ( op.accept ( nb ) ) ; + } +} \ No newline at end of file Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestNEConstant.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestNEConstant.java 2010-09-27 14:29:21 UTC (rev 3635) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestNEConstant.java 2010-09-27 16:12:12 UTC (rev 3636) @@ -29,6 +29,13 @@ import junit.framework.TestCase2; +import com.bigdata.bop.ArrayBindingSet; +import com.bigdata.bop.Constant; +import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.IConstant; +import com.bigdata.bop.IVariable; +import com.bigdata.bop.Var; + /** * Unit tests for {@link NEConstant}. * @@ -50,8 +57,40 @@ super(name); } - public void test_something() { - fail("write tests"); + /** + * Unit test for {@link NEConstant#NEConstant(IVariable,IConstant)} + */ + public void testConstructor () + { + try { assertTrue ( null != new NEConstant ( null, new Constant<String> ( "1" ) ) ) ; fail ( "IllegalArgumentException expected, lhs was null" ) ; } + catch ( IllegalArgumentException e ) {} + + try { assertTrue ( null != new NEConstant ( Var.var ( "x" ), null ) ) ; fail ( "IllegalArgumentException expected, rhs was null" ) ; } + catch ( IllegalArgumentException e ) {} + + assertTrue ( null != new NEConstant ( Var.var ( "x" ), new Constant<String> ( "1" ) ) ) ; } - -} + + /** + * Unit test for {@link NEConstant#accept(IBindingSet)} + */ + public void testAccept () + { + Var<?> var = Var.var ( "x" ) ; + Constant<String> val1 = new Constant<String> ( "1" ) ; + Constant<String> val2 = new Constant<String> ( "2" ) ; + Constant<Integer> val3 = new Constant<Integer> ( 1 ) ; + + NEConstant op = new NEConstant ( var, val1 ) ; + + IBindingSet eq = new ArrayBindingSet ( new IVariable<?> [] { var }, new IConstant [] { val1 } ) ; + IBindingSet ne1 = new ArrayBindingSet ( new IVariable<?> [] { var }, new IConstant [] { val2 } ) ; + IBindingSet ne2 = new ArrayBindingSet ( new IVariable<?> [] { var }, new IConstant [] { val3 } ) ; + IBindingSet nb = new ArrayBindingSet ( new IVariable<?> [] {}, new IConstant [] {} ) ; + + assertFalse ( op.accept ( eq ) ) ; + assertTrue ( op.accept ( ne1 ) ) ; + assertTrue ( op.accept ( ne2 ) ) ; + assertTrue ( op.accept ( nb ) ) ; + } +} \ No newline at end of file Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestOR.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestOR.java 2010-09-27 14:29:21 UTC (rev 3635) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/constraint/TestOR.java 2010-09-27 16:12:12 UTC (rev 3636) @@ -29,6 +29,14 @@ import junit.framework.TestCase2; +import com.bigdata.bop.ArrayBindingSet; +import com.bigdata.bop.Constant; +import com.bigdata.bop.IBindingSet; +import com.bigdata.bop.IConstant; +import com.bigdata.bop.IConstraint; +import com.bigdata.bop.IVariable; +import com.bigdata.bop.Var; + /** * Unit tests for {@link OR}. * @@ -50,8 +58,44 @@ super(name); } - public void test_something() { - fail("write tests"); + /** + * Unit test for {@link OR#OR(IConstraint,IConstraint)} + */ + public void testConstructor () + { + IConstraint eq = new EQ ( Var.var ( "x" ), Var.var ( "y" ) ) ; + IConstraint ne = new EQ ( Var.var ( "x" ), Var.var ( "y" ) ) ; + + try { assertTrue ( null != new OR ( null, eq ) ) ; fail ( "IllegalArgumentException expected, lhs was null" ) ; } + catch ( IllegalArgumentException e ) {} + + try { assertTrue ( null != new OR ( eq, null ) ) ; fail ( "IllegalArgumentException expected, rhs was null" ) ; } + catch ( IllegalArgumentException e ) {} + + assertTrue ( null != new OR ( eq, ne ) ) ; } - -} + + /** + * Unit test for {@link OR#accept(IBindingSet)} + */ + public void testAccept () + { + Var<?> x = Var.var ( "x" ) ; + Var<?> y = Var.var ( "y" ) ; + Constant<Integer> val1 = new Constant<Integer> ( 1 ) ; + Constant<Integer> val2 = new Constant<Integer> ( 2 ) ; + + IConstraint eq = new EQ ( x, y ) ; + IConstraint eqc = new EQConstant ( y, val2 ) ; + + OR op = new OR ( eq, eqc ) ; + + IBindingSet eqlhs = new ArrayBindingSet ( new IVariable<?> [] { x, y }, new IConstant [] { val1, val1 } ) ; + IBindingSet eqrhs = new ArrayBindingSet ( new IVariable<?> [] { x, y }, new IConstant [] { val1, val2 } ) ; + IBindingSet ne = new ArrayBindingSet ( new IVariable<?> [] { x, y }, new IConstant [] { val2, val1 } ) ; + + assertTrue ( op.accept ( eqlhs ) ) ; + assertTrue ( op.accept ( eqrhs ) ) ; + assertFalse ( op.accept ( ne ) ) ; + } +} \ No newline at end of file This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |