From: <tho...@us...> - 2014-01-10 21:10:54
|
Revision: 7761 http://bigdata.svn.sourceforge.net/bigdata/?rev=7761&view=rev Author: thompsonbry Date: 2014-01-10 21:10:45 +0000 (Fri, 10 Jan 2014) Log Message: ----------- Checkpoint on the RTO development. See #64. Modified Paths: -------------- branches/BIGDATA_RELEASE_1_3_0/bigdata/src/java/com/bigdata/bop/engine/ChunkedRunningQuery.java branches/BIGDATA_RELEASE_1_3_0/bigdata/src/java/com/bigdata/bop/joinGraph/rto/JGraph.java branches/BIGDATA_RELEASE_1_3_0/bigdata/src/java/com/bigdata/bop/joinGraph/rto/JoinGraph.java branches/BIGDATA_RELEASE_1_3_0/bigdata/src/java/com/bigdata/bop/joinGraph/rto/Path.java branches/BIGDATA_RELEASE_1_3_0/bigdata/src/test/com/bigdata/bop/joinGraph/rto/TestJoinGraph.java branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/eval/AST2BOpRTO.java branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/eval/rto/AbstractRTOTestCase.java branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/eval/rto/BSBM-Q4.rq branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/eval/rto/TestAll.java branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/eval/rto/TestRTO_BSBM.java Modified: branches/BIGDATA_RELEASE_1_3_0/bigdata/src/java/com/bigdata/bop/engine/ChunkedRunningQuery.java =================================================================== --- branches/BIGDATA_RELEASE_1_3_0/bigdata/src/java/com/bigdata/bop/engine/ChunkedRunningQuery.java 2014-01-10 17:25:56 UTC (rev 7760) +++ branches/BIGDATA_RELEASE_1_3_0/bigdata/src/java/com/bigdata/bop/engine/ChunkedRunningQuery.java 2014-01-10 21:10:45 UTC (rev 7761) @@ -42,6 +42,8 @@ import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.atomic.AtomicInteger; +import junit.framework.AssertionFailedError; + import org.apache.log4j.Logger; import com.bigdata.bop.BOp; @@ -1169,7 +1171,9 @@ stats = op.newStats(); // log.warn("bopId=" + bopId + ", stats=" + stats); } - assert stats != null : "No stats: op=" + op; + if (stats == null) { + throw new AssertionError("No stats: op=" + op); + } // // The groupId (if any) for this operator. // final Integer fromGroupId = (Integer) op Modified: branches/BIGDATA_RELEASE_1_3_0/bigdata/src/java/com/bigdata/bop/joinGraph/rto/JGraph.java =================================================================== --- branches/BIGDATA_RELEASE_1_3_0/bigdata/src/java/com/bigdata/bop/joinGraph/rto/JGraph.java 2014-01-10 17:25:56 UTC (rev 7760) +++ branches/BIGDATA_RELEASE_1_3_0/bigdata/src/java/com/bigdata/bop/joinGraph/rto/JGraph.java 2014-01-10 21:10:45 UTC (rev 7761) @@ -1387,7 +1387,7 @@ * those get applied when we evaluate the cutoff joins from one * vertex to another. */ - public void sampleAllVertices(final QueryEngine queryEngine, final int limit) { + private void sampleAllVertices(final QueryEngine queryEngine, final int limit) { final Map<Vertex, AtomicInteger> vertexLimit = new LinkedHashMap<Vertex, AtomicInteger>(); Modified: branches/BIGDATA_RELEASE_1_3_0/bigdata/src/java/com/bigdata/bop/joinGraph/rto/JoinGraph.java =================================================================== --- branches/BIGDATA_RELEASE_1_3_0/bigdata/src/java/com/bigdata/bop/joinGraph/rto/JoinGraph.java 2014-01-10 17:25:56 UTC (rev 7760) +++ branches/BIGDATA_RELEASE_1_3_0/bigdata/src/java/com/bigdata/bop/joinGraph/rto/JoinGraph.java 2014-01-10 21:10:45 UTC (rev 7761) @@ -32,10 +32,7 @@ import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.FutureTask; -import java.util.concurrent.TimeUnit; -import org.apache.log4j.Logger; - import com.bigdata.bop.BOp; import com.bigdata.bop.BOpContext; import com.bigdata.bop.BOpUtility; @@ -50,14 +47,12 @@ import com.bigdata.bop.ap.SampleIndex.SampleType; import com.bigdata.bop.controller.AbstractSubqueryOp; import com.bigdata.bop.engine.AbstractRunningQuery; -import com.bigdata.bop.engine.BOpStats; import com.bigdata.bop.engine.IRunningQuery; import com.bigdata.bop.engine.QueryEngine; -import com.bigdata.rdf.sparql.ast.IJoinNode; import com.bigdata.rdf.sparql.ast.JoinGroupNode; import com.bigdata.rdf.sparql.ast.eval.AST2BOpContext; import com.bigdata.rdf.sparql.ast.eval.AST2BOpRTO; -import com.bigdata.rdf.sparql.ast.optimizers.IASTOptimizer; +import com.bigdata.rdf.sparql.ast.eval.AST2BOpUtility; import com.bigdata.util.NT; import com.bigdata.util.concurrent.Haltable; @@ -65,13 +60,33 @@ /** * A join graph with annotations for estimated cardinality and other details in - * support of runtime query optimization. A join graph is a collection of - * relations and joins which connect those relations. This boils down to a - * collection of {@link IPredicate}s (selects on relations), shared variables - * (which identify joins), and {@link IConstraint}s (which limit solutions). - * Operators other than standard joins (including optional joins, sort, order - * by, etc.) must be handled downstream from the join graph in a "tail plan". + * support of runtime query optimization. A join graph is a collection of access + * paths reading on relations (the vertices of the join graph) and joins which + * connect those relations (the edges of the join graph). This boils down to a + * collection of {@link IPredicate}s (access paths reading on on relations), + * shared variables (which identify joins), and {@link IConstraint}s (which may + * reject some solutions for those joins). Operators other than standard joins + * (including optional joins, sort, order by, etc.) must be handled downstream + * from the join graph in a "tail plan". + * <p> + * The {@link JoinGraph} operator works in two phases. On its first invocation, + * it constructs a {@link JGraph join graph} and identifies a join path having a + * low cost join ordering. This join path is converted into a query plan and set + * as the {@link Attributes#QUERY_PLAN} attribute on the {@link IRunningQuery}. + * The upstream solutions are then flooded into sub-query that executes the + * chosen query plan. The solutions from the sub-query are simply copied to the + * output sink of the {@link JoinGraph} operator. Once the query plan has been + * identified by the first invocation, subsequent invocations of this operator + * simply push more data into the sub-query using the pre-identified query plan. * + * TODO This approach amounts to bottom-up evaluation of the {@link JGraph}. + * Thus, the RTO is not using information from the upstream query when it + * decides on a query plan. Therefore, we could lift-out the RTO sections of the + * query into named subqueries, run them first in parallel, and then INCLUDE + * their results into the main query. This would require an AST optimizer to + * modify the AST. (Currently the RTO is integrated when the query plan is + * generated in {@link AST2BOpUtility} rather than as an AST optimizer.) + * * @see http://arxiv.org/PS_cache/arxiv/pdf/0810/0810.4809v1.pdf, XQuery Join * Graph Isolation. * @@ -87,19 +102,19 @@ private static final long serialVersionUID = 1L; - private static final transient Logger log = Logger - .getLogger(JoinGraph.class); +// private static final transient Logger log = Logger +// .getLogger(JoinGraph.class); /** * Known annotations. */ public interface Annotations extends PipelineOp.Annotations { - /** - * The variables to be projected out of the join graph (optional). When - * <code>null</code>, all variables will be projected out. - */ - String SELECTED = JoinGraph.class.getName() + ".selected"; +// /** +// * The variables to be projected out of the join graph (optional). When +// * <code>null</code>, all variables will be projected out. +// */ +// String SELECTED = JoinGraph.class.getName() + ".selected"; /** * The vertices of the join graph, expressed an an {@link IPredicate}[] @@ -147,44 +162,12 @@ /** * The set of variables that are known to have already been materialized * in the context in which the RTO was invoked. - * - * FIXME In order to support left-to-right evaluation fully, the - * {@link JGraph} needs to accept this, track it as it binds variables, - * and pass it through when doing cutoff joins to avoid pipeline - * materialization steps for variables that are already known to be - * materialized. Otherwise the RTO will assume that it needs to - * materialize everything that needs to be materialized for a FILTER and - * thus do too much work (which is basically the assumption of bottom-up - * evaluation, or if you prefer that it is executing in its own little - * world). */ String DONE_SET = JoinGraph.class.getName() + ".doneSet"; -// /** -// * The query hints from the dominating AST node (if any). These query -// * hints will be passed through and made available when we compile the -// * query plan once the RTO has decided on the join ordering. While the -// * RTO is running, it needs to override many of the query hints for the -// * {@link IPredicate}s, {@link PipelineJoin}s, etc. in order to ensure -// * that the cutoff evaluation semantics are correctly applied while it -// * is exploring the plan state space for the join graph. -// */ -// String AST_QUERY_HINTS = JoinGraph.class.getName() + ".astQueryHints"; - /** * The AST {@link JoinGroupNode} for the joins and filters that we are * running through the RTO (required). - * - * FIXME This should be set by an ASTRTOOptimizer. That class should - * rewrite the original join group, replacing some set of joins with a - * JoinGraphNode which implements {@link IJoinNode} and gets hooked into - * AST2BOpUtility#convertJoinGroup() normally rather than through - * expectional processing. This will simplify the code and adhere to the - * general {@link IASTOptimizer} pattern and avoid problems with cloning - * children out of the {@link JoinGroupNode} when we set it up to run - * the RTO. [Eventually, we will need to pass this in rather than the - * {@link IPredicate}[] in order to handle JOINs that are not SPs, e.g., - * sub-selects, etc.] */ String JOIN_GROUP = JoinGraph.class.getName() + ".joinGroup"; @@ -207,9 +190,6 @@ * * @author <a href="mailto:tho...@us...">Bryan * Thompson</a> - * - * TODO This could also be put on a {@link BOpStats} interface, - * which is the other way for accessing shared state. */ public interface Attributes { @@ -236,15 +216,15 @@ * JoinGraph operator annotations. */ - /** - * @see Annotations#SELECTED - */ - public IVariable<?>[] getSelected() { +// /** +// * @see Annotations#SELECTED +// */ +// public IVariable<?>[] getSelected() { +// +// return (IVariable[]) getRequiredProperty(Annotations.SELECTED); +// +// } - return (IVariable[]) getRequiredProperty(Annotations.SELECTED); - - } - /** * @see Annotations#VERTICES */ @@ -462,72 +442,99 @@ } - /** - * {@inheritDoc} - * - * FIXME When run as sub-query, we need to fix point the upstream - * solutions and then flood them into the join graph. Samples of the - * known bound variables can be pulled from those initial solutions. - */ @Override public Void call() throws Exception { - final long begin = System.nanoTime(); - - // Create the join graph. - final JGraph g = new JGraph(JoinGraph.this); + if (getQueryPlan(context.getRunningQuery()) == null) { + + /* + * Use the RTO to generate a query plan. + * + * TODO Make sure that the JoinGraph can not be triggered + * concurrently, e.g., that the CONTROLLER attribute prevents + * concurrent evaluation, just like MAX_PARALLEL. + */ + + // final long begin = System.nanoTime(); - /* - * This map is used to associate join path segments (expressed as an - * ordered array of bopIds) with edge sample to avoid redundant effort. - * - * FIXME RTO: HEAP MANAGMENT : This map holds references to the cutoff - * join samples. To ensure that the map has the minimum heap footprint, - * it must be scanned each time we prune the set of active paths and any - * entry which is not a prefix of an active path should be removed. - * - * TODO RTO: MEMORY MANAGER : When an entry is cleared from this map, - * the corresponding allocation in the memory manager (if any) must be - * released. The life cycle of the map needs to be bracketed by a - * try/finally in order to ensure that all allocations associated with - * the map are released no later than when we leave the lexicon scope of - * that clause. - */ - final Map<PathIds, EdgeSample> edgeSamples = new LinkedHashMap<PathIds, EdgeSample>(); + // Create the join graph. + final JGraph g = new JGraph(JoinGraph.this); - // Find the best join path. - final Path path = g.runtimeOptimizer(context.getRunningQuery() - .getQueryEngine(), getLimit(), getNEdges(), edgeSamples); + /* + * This map is used to associate join path segments (expressed + * as an ordered array of bopIds) with edge sample to avoid + * redundant effort. + */ + final Map<PathIds, EdgeSample> edgeSamples = new LinkedHashMap<PathIds, EdgeSample>(); - // Set attribute for the join path result. - setPath(context.getRunningQuery(), path); + // Find the best join path. + final Path path = g + .runtimeOptimizer(context.getRunningQuery() + .getQueryEngine(), getLimit(), getNEdges(), + edgeSamples); - // Set attribute for the join path samples. - setSamples(context.getRunningQuery(), edgeSamples); + /* + * Release samples. + * + * TODO If we have fully sampled some vertices or edges, then we + * could replace the JOIN with the sample. For this to work, we + * would need to access path that could read the sample and we + * would have to NOT release the samples until the RTO was done + * executing sub-queries against the generated query plan. Since + * we can flow multiple chunks into the sub-query, this amounts + * to having a LAST_PASS annotation. + */ + + for (EdgeSample s : edgeSamples.values()) { - final long mark = System.nanoTime(); - - final long elapsed_queryOptimizer = mark - begin; - - /* - * Generate the query from the selected join path. - */ - final PipelineOp queryOp = AST2BOpRTO.compileJoinGraph(context - .getRunningQuery().getQueryEngine(), JoinGraph.this, path); + s.releaseSample(); + + } + + for (Vertex v : g.getVertices()) { - // Set attribute for the join path samples. - setQueryPlan(context.getRunningQuery(), queryOp); + if (v.sample != null) { + v.sample.releaseSample(); + + } + + } + + // Set attribute for the join path result. + setPath(context.getRunningQuery(), path); + // Set attribute for the join path samples. + setSamples(context.getRunningQuery(), edgeSamples); + + // final long mark = System.nanoTime(); + // + // final long elapsed_queryOptimizer = mark - begin; + + /* + * Generate the query from the selected join path. + */ + final PipelineOp queryOp = AST2BOpRTO.compileJoinGraph(context + .getRunningQuery().getQueryEngine(), JoinGraph.this, + path); + + // Set attribute for the join path samples. + setQueryPlan(context.getRunningQuery(), queryOp); + + } + + // The query plan. + final PipelineOp queryOp = getQueryPlan(context.getRunningQuery()); + // Run the query, blocking until it is done. JoinGraph.runSubquery(context, queryOp); - final long elapsed_queryExecution = System.nanoTime() - mark; - - if (log.isInfoEnabled()) - log.info("RTO: queryOptimizer=" - + TimeUnit.NANOSECONDS.toMillis(elapsed_queryOptimizer) - + ", queryExecution=" - + TimeUnit.NANOSECONDS.toMillis(elapsed_queryExecution)); +// final long elapsed_queryExecution = System.nanoTime() - mark; +// +// if (log.isInfoEnabled()) +// log.info("RTO: queryOptimizer=" +// + TimeUnit.NANOSECONDS.toMillis(elapsed_queryOptimizer) +// + ", queryExecution=" +// + TimeUnit.NANOSECONDS.toMillis(elapsed_queryExecution)); return null; @@ -542,42 +549,48 @@ * subquery. Therefore we have to take appropriate care to ensure that the * results are copied out of the subquery and into the parent query. See * {@link AbstractSubqueryOp} for how this is done. - * - * @throws Exception - * - * @todo When we execute the query, we should clear the references to the - * samples (unless they are exact, in which case they can be used as - * is) in order to release memory associated with those samples if the - * query is long running. Samples must be held until we have - * identified the final join path since each vertex will be used by - * each maximum length join path and we use the samples from the - * vertices to re-sample the surviving join paths in each round. [In - * fact, the samples are not being provided to this evaluation context - * right now.] - * - * @todo If there are source binding sets then they need to be applied above - * (when we are sampling) and below (when we evaluate the selected - * join path). */ static private void runSubquery( final BOpContext<IBindingSet> parentContext, final PipelineOp queryOp) throws Exception { + if(parentContext==null) + throw new IllegalArgumentException(); + + if(queryOp==null) + throw new IllegalArgumentException(); + final QueryEngine queryEngine = parentContext.getRunningQuery() .getQueryEngine(); /* - * Run the query. - * - * TODO Pass in the source binding sets here and also when sampling the - * vertices? Otherwise it is as if we are doing bottom-up evaluation (in - * which case the doneSet should be empty on entry). + * Run the sub-query. */ ICloseableIterator<IBindingSet[]> subquerySolutionItr = null; - final IRunningQuery runningSubquery = queryEngine.eval(queryOp); + // Fully materialize the upstream solutions. + final IBindingSet[] bindingSets = BOpUtility.toArray( + parentContext.getSource(), parentContext.getStats()); + /* + * Run on all available upstream solutions. + * + * Note: The subquery will run for each chunk of upstream solutions, so + * it could make sense to increase the vector size or to collect all + * upstream solutions into a SolutionSet and then flood them into the + * sub-query. + * + * Note: We do not need to do a hash join with the output of the + * sub-query. This amounts to pipelined evaluation. Solutions flow into + * a subquery and then back out. The only reason for a hash join would + * be if we project in only a subset of the variables that were in scope + * in the parent context and then needed to pick up the correlated + * variables after running the query plan generated by the RTO. + */ + final IRunningQuery runningSubquery = queryEngine.eval(queryOp, + bindingSets); + try { // Declare the child query to the parent. Modified: branches/BIGDATA_RELEASE_1_3_0/bigdata/src/java/com/bigdata/bop/joinGraph/rto/Path.java =================================================================== --- branches/BIGDATA_RELEASE_1_3_0/bigdata/src/java/com/bigdata/bop/joinGraph/rto/Path.java 2014-01-10 17:25:56 UTC (rev 7760) +++ branches/BIGDATA_RELEASE_1_3_0/bigdata/src/java/com/bigdata/bop/joinGraph/rto/Path.java 2014-01-10 21:10:45 UTC (rev 7761) @@ -27,8 +27,6 @@ import java.util.Collections; import java.util.List; -import org.apache.log4j.Logger; - import com.bigdata.bop.BOp; import com.bigdata.bop.BOpUtility; import com.bigdata.bop.IConstraint; @@ -87,7 +85,7 @@ */ public class Path { - private static final transient Logger log = Logger.getLogger(Path.class); +// private static final transient Logger log = Logger.getLogger(Path.class); /** * An ordered list of the vertices in the {@link Path}. @@ -180,6 +178,13 @@ * the JGraph trace appropriately. [Refactor into an IPathCost * interface. It should have visibility into the full path and also * allow visibility into the vertex cost for generality.] + * + * TODO Add a cost function API, e.g., IPathCost. This gets passed + * into Path to compute a score. We also compute a score for a + * vertex. Add query hints for both so we can control the behavior. + * The default should be estCard, but estRead or a weighted + * combination of estCard and estRead are also possible cost + * functions. */ private static long getCost(final long sumEstRead, final long sumEstCard) { Modified: branches/BIGDATA_RELEASE_1_3_0/bigdata/src/test/com/bigdata/bop/joinGraph/rto/TestJoinGraph.java =================================================================== --- branches/BIGDATA_RELEASE_1_3_0/bigdata/src/test/com/bigdata/bop/joinGraph/rto/TestJoinGraph.java 2014-01-10 17:25:56 UTC (rev 7760) +++ branches/BIGDATA_RELEASE_1_3_0/bigdata/src/test/com/bigdata/bop/joinGraph/rto/TestJoinGraph.java 2014-01-10 21:10:45 UTC (rev 7761) @@ -76,7 +76,7 @@ }; final IConstraint[] constraints = null; final JoinGraph joinGraph = new JoinGraph(new BOp[0],// - new NV(JoinGraph.Annotations.SELECTED, new IVariable[]{}),// +// new NV(JoinGraph.Annotations.SELECTED, new IVariable[]{}),// new NV(JoinGraph.Annotations.VERTICES, vertices),// new NV(JoinGraph.Annotations.CONTROLLER, true), // new NV(JoinGraph.Annotations.EVALUATION_CONTEXT, Modified: branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/eval/AST2BOpRTO.java =================================================================== --- branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/eval/AST2BOpRTO.java 2014-01-10 17:25:56 UTC (rev 7760) +++ branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/eval/AST2BOpRTO.java 2014-01-10 21:10:45 UTC (rev 7761) @@ -75,13 +75,12 @@ import com.bigdata.bop.joinGraph.rto.SampleBase; import com.bigdata.bop.joinGraph.rto.VertexSample; import com.bigdata.bop.rdf.join.ChunkedMaterializationOp; +import com.bigdata.bop.rdf.join.DataSetJoin; import com.bigdata.bop.solutions.MemorySortOp; -import com.bigdata.bop.solutions.ProjectionOp; import com.bigdata.bop.solutions.SliceOp; import com.bigdata.journal.IIndexManager; import com.bigdata.rdf.internal.IV; import com.bigdata.rdf.sparql.ast.ASTContainer; -import com.bigdata.rdf.sparql.ast.IBindingProducerNode; import com.bigdata.rdf.sparql.ast.IGroupMemberNode; import com.bigdata.rdf.sparql.ast.JoinGroupNode; import com.bigdata.rdf.sparql.ast.QueryHints; @@ -92,6 +91,11 @@ /** * Integration with the Runtime Optimizer (RTO). + * <p> + * Note: The RTO currently uses bottom-up evaluation to solve the join graph and + * generate a sub-query plan with an optimized join ordering. It uses + * left-to-right evaluation to pass pipeline solutions through the optimized + * subquery. * * @see <a href="https://sourceforge.net/apps/trac/bigdata/ticket/64">Runtime * Query Optimization</a> @@ -105,11 +109,6 @@ * @see JGraph * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> - * - * TODO Can the RTO give us information about the locality of joins that - * we could use to decide on vectoring (chunkSize) for those joins or to - * decide when to use a hash join against an access path as opposed to a - * nested index join? */ public class AST2BOpRTO extends AST2BOpJoins { @@ -180,33 +179,6 @@ static private final boolean onlySPs = true; /** - * When <code>true</code>, the RTO will be applied as in we were doing - * bottom-up query optimization. In this case, it WILL NOT receive any - * solutions from the upstream operators in the pipeline when it performs - * its runtime sampling and it will ignore the <code>doneSet</code> for the - * context in which it is invoked. When run in this manner, the RTO *could* - * be run before the main query is executed. The only way to facilitate this - * at this time would be to lift out the joins on which the RTO would be run - * into a named subquery and then optimize that named subquery before the - * rest of the query. - * <p> - * When <code>false</code>, the RTO solutions from upstream operators will - * flow into the RTO. - * - * TODO We could still pass in exogenous solutions for bottom up evaluation. - * This would help constraint the RTOs exploration. - * - * TODO The RTO is not operating 100% in either an left-to-right or a - * bottom-up fashion, primarily because we are not passing in either - * exogenous bindings or meaningfully using the bindings from the upstream - * operator when exploring the join graph. In fact, the RTO could accept a - * new sample from the upstream operator in each iteration drawing from - * amoung those solutions which had already been materialized by the - * upstream operator. - */ - static private final boolean bottomUp = true; - - /** * When <code>true</code>, even simple JOINs will run through the code path * for the evaluation of complex joins. * <p> @@ -225,7 +197,7 @@ * @see #runSimpleJoin(QueryEngine, SampleBase, int, PipelineJoin) * @see #runComplexJoin(QueryEngine, SampleBase, int, PipelineOp) * - * FIXME RTO: Measure performance when using the complex join code path + * TODO RTO: Measure performance when using the complex join code path * as opposed to the simple join code path. If there is a performance * win for the simple join code path, then set this to false in * committed code. If not, then we might as well run everything in the @@ -235,7 +207,7 @@ * somewhat in the end result that they produce. This can be observed * in BSBM Q1 on the pc100 data set and the BAR query. */ - static final boolean runAllJoinsAsComplexJoins = false; + static final boolean runAllJoinsAsComplexJoins = false; /** * Inspect the remainder of the join group. If we can isolate a join graph @@ -400,37 +372,6 @@ } - /* - * Figure out which variables are projected out of the RTO. - * - * TODO This should only include things that are not reused later in the - * query. - */ - final Set<IVariable<?>> selectVars = new LinkedHashSet<IVariable<?>>(); - { - - for (IGroupMemberNode child : rtoJoinGroup.getChildren()) { - - if (!(child instanceof IBindingProducerNode)) - continue; - - // Note: recursive only matters for complex nodes, not SPs. - ctx.sa.getDefinitelyProducedBindings( - (IBindingProducerNode) child, selectVars, true/* recursive */); - - } - - } - - /* - * FIXME RTO: Sub-Groups: When running the RTO as anything other than - * the top-level join group in the query plan and for the *FIRST* joins - * in the query plan, we need to flow in any solutions that are already - * in the pipeline (unless we are going to run the RTO "bottom up") and - * build a hash index. When the hash index is ready, we can execute the - * join group. - */ - final SampleType sampleType = joinGroup.getProperty( QueryHints.RTO_SAMPLE_TYPE, QueryHints.DEFAULT_RTO_SAMPLE_TYPE); @@ -447,8 +388,8 @@ new NV(BOp.Annotations.CONTROLLER, true),// Drop "CONTROLLER" annotation? // new NV(PipelineOp.Annotations.MAX_PARALLEL, 1),// // new NV(PipelineOp.Annotations.LAST_PASS, true),// required - new NV(JoinGraph.Annotations.SELECTED, selectVars - .toArray(new IVariable[selectVars.size()])),// +// new NV(JoinGraph.Annotations.SELECTED, selectVars +// .toArray(new IVariable[selectVars.size()])),// new NV(JoinGraph.Annotations.VERTICES, preds.toArray(new Predicate[preds.size()])),// new NV(JoinGraph.Annotations.CONSTRAINTS, constraints @@ -495,32 +436,12 @@ if (path == null) throw new IllegalArgumentException(); - final IVariable<?>[] selected = joinGraph.getSelected(); +// final IVariable<?>[] selected = joinGraph.getSelected(); final IPredicate<?>[] predicates = path.getPredicates(); final IConstraint[] constraints = joinGraph.getConstraints(); - -// if (onlySimpleJoins) { -// -// /* -// * This is the old code. It does not handle variable materialization -// * for filters. -// */ -// -// // Factory avoids reuse of bopIds assigned to the predicates. -// final BOpIdFactory idFactory = new BOpIdFactory(); -// -// return PartitionedJoinGroup.getQuery(idFactory, -// false/* distinct */, selected, predicates, constraints); -// -// } - /* - * FIXME RTO: doneSet: The RTO is ignoring the doneSet so it always runs - * all materialization steps even if some variable is known to be - * materialized on entry. - */ final Set<IVariable<?>> doneSet = joinGraph.getDoneSet(); /* @@ -584,16 +505,16 @@ } - if (selected != null && selected.length != 0) { - - // Drop variables that are not projected out. - left = applyQueryHints(new ProjectionOp(// - leftOrEmpty(left), // - new NV(ProjectionOp.Annotations.BOP_ID, idFactory.nextId()),// - new NV(ProjectionOp.Annotations.SELECT, selected)// - ), rtoJoinGroup, ctx); - - } +// if (selected != null && selected.length != 0) { +// +// // Drop variables that are not projected out. +// left = applyQueryHints(new ProjectionOp(// +// leftOrEmpty(left), // +// new NV(ProjectionOp.Annotations.BOP_ID, idFactory.nextId()),// +// new NV(ProjectionOp.Annotations.SELECT, selected)// +// ), rtoJoinGroup, ctx); +// +// } return left; @@ -729,27 +650,6 @@ * the {@link EdgeSample} for that {@link Path}. * * @return The result of sampling that edge. - * - * TODO TESTS: Provide test coverage for running queries with - * complex FILTERs (triples mode is Ok). - * - * TODO TESTS: Test with FILTERs that can not run until after all - * joins. Such filters are only attached when the [pathIsComplete] - * flag is set. This might only occur when we have FILTERs that - * depend on variables that are only "maybe" bound by an OPTIONAL - * join. - * - * TODO TESTS: Quads mode tests. We need to look in depth at how the - * quads mode access paths are evaluated. There are several - * different conditions. We need to look at each condition and at - * whether and how it can be made compatible with cutoff evaluation. - * (This is somewhat similar to the old scan+filter versus nested - * query debate on quads mode joins.) - * - * TODO TESTS: Scale-out tests. For scale-out, we need to either - * mark the join's evaluation context based on whether or not the - * access path is local or remote (and whether the index is - * key-range distributed or hash partitioned). */ static public EdgeSample cutoffJoin(// final QueryEngine queryEngine,// @@ -900,7 +800,11 @@ * that is known to have been materialized based on an analysis of the * join path (as executed) up to this point in the path. This will let * us potentially do less work. This will require tracking the doneSet - * in the Path and passing the Path into cutoffJoin(). + * in the Path and passing the Path into cutoffJoin(). The simplest way + * to manage this is to just annotate the Path as we go, which means + * making the Path more AST aware - or at least doneSet aware. Or we can + * just apply the analysis to each step in the path to figure out what + * is done each time we setup cutoff evaluation of an operator. */ final Set<IVariable<?>> doneSet = new LinkedHashSet<IVariable<?>>( joinGraph.getDoneSet()); @@ -1101,6 +1005,19 @@ * the manner in which the query plan is constructed and the * parallelism in the query plan. Any parallelism or reordering * will trip this error. + * + * TODO If we hit the {@link OutOfOrderEvaluationException} for + * some kinds of access paths quads mode access paths, then we + * might need to look at the {@link DataSetJoin} in more depth + * and the way in which named graph and default graph joins are + * being executed for both local and scale-out deployments. One + * fall back position is to feed the input solutions in one at a + * time in different running queries. This will give us the + * exact output cardinality for a given source solution while + * preserving parallel evaluation over some chunk of source + * solutions. However, this approach can do too much work and + * will incur more overhead than injecting a rowid column into + * the source solutions. */ private static EdgeSample runComplexJoin(// final QueryEngine queryEngine,// @@ -1212,14 +1129,6 @@ final PipelineJoinStats joinStats = (PipelineJoinStats) runningQuery .getStats().get(joinOp.getId()); - /* - * TODO It would be interesting to see the stats on each operator in the - * plan for each join sampled. We would be able to observe that in the - * EXPLAIN view if we attached the IRunningQuery for the cutoff - * evaluation to the parent query. However, this should only be enabled - * in a mode for gruesome detail since we evaluate a LOT of cutoff joins - * when running the RTO on a single join graph. - */ if (log.isTraceEnabled()) log.trace(//Arrays.toString(BOpUtility.getPredIds(predicates)) + ": "+ "join::" + joinStats); Modified: branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/eval/rto/AbstractRTOTestCase.java =================================================================== --- branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/eval/rto/AbstractRTOTestCase.java 2014-01-10 17:25:56 UTC (rev 7760) +++ branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/eval/rto/AbstractRTOTestCase.java 2014-01-10 21:10:45 UTC (rev 7761) @@ -91,6 +91,25 @@ * * TODO Add some govtrack queries. Those queries use quads mode and have a * lot of interesting query constructions. + * + * TODO TESTS: Provide test coverage for running queries with complex + * FILTERs (triples mode is Ok). + * + * TODO TESTS: Test with FILTERs that can not run until after all joins. + * Such filters are only attached when the [pathIsComplete] flag is set. + * This might only occur when we have FILTERs that depend on variables that + * are only "maybe" bound by an OPTIONAL join. + * + * TODO TESTS: Quads mode tests. We need to look in depth at how the quads + * mode access paths are evaluated. There are several different conditions. + * We need to look at each condition and at whether and how it can be made + * compatible with cutoff evaluation. (This is somewhat similar to the old + * scan+filter versus nested query debate on quads mode joins.) + * + * TODO TESTS: Scale-out tests. For scale-out, we need to either mark the + * join's evaluation context based on whether or not the access path is + * local or remote (and whether the index is key-range distributed or hash + * partitioned). */ public class AbstractRTOTestCase extends AbstractDataDrivenSPARQLTestCase { Modified: branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/eval/rto/BSBM-Q4.rq =================================================================== --- branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/eval/rto/BSBM-Q4.rq 2014-01-10 17:25:56 UTC (rev 7760) +++ branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/eval/rto/BSBM-Q4.rq 2014-01-10 21:10:45 UTC (rev 7761) @@ -8,13 +8,17 @@ SELECT DISTINCT ?product ?label ?propertyTextual WHERE { + { + + # Note: The RTO is only applied to one of the join groups to make the + # test code simpler. + # # Control all RTO parameters for repeatable behavior. - hint:Query hint:optimizer "Runtime". - hint:Query hint:RTO-sampleType "DENSE". - hint:Query hint:RTO-limit "100". - hint:Query hint:RTO-nedges "1". + hint:Group hint:optimizer "Runtime". + hint:Group hint:RTO-sampleType "DENSE". + hint:Group hint:RTO-limit "100". + hint:Group hint:RTO-nedges "1". - { ?product rdfs:label ?label . ?product rdf:type <http://www4.wiwiss.fu-berlin.de/bizer/bsbm/v01/instances/ProductType19> . ?product bsbm:productFeature <http://www4.wiwiss.fu-berlin.de/bizer/bsbm/v01/instances/ProductFeature158> . Modified: branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/eval/rto/TestAll.java =================================================================== --- branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/eval/rto/TestAll.java 2014-01-10 17:25:56 UTC (rev 7760) +++ branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/eval/rto/TestAll.java 2014-01-10 21:10:45 UTC (rev 7761) @@ -64,12 +64,20 @@ // LUBM test suite. suite.addTestSuite(TestRTO_LUBM.class); - // BSBM test suite. + // BSBM test suite: TODO Add BSBM BI tests. suite.addTestSuite(TestRTO_BSBM.class); - // 'barData' test suite. + // 'barData' test suite (quads mode). suite.addTestSuite(TestRTO_BAR.class); + /* + * FOAF test suite (quads mode). + * + * TODO This test suite is disabled since queries are not complex enough + * to run the RTO (we need at least required joins). + */ +// suite.addTestSuite(TestRTO_FOAF.class); + return suite; } Modified: branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/eval/rto/TestRTO_BSBM.java =================================================================== --- branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/eval/rto/TestRTO_BSBM.java 2014-01-10 17:25:56 UTC (rev 7760) +++ branches/BIGDATA_RELEASE_1_3_0/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/eval/rto/TestRTO_BSBM.java 2014-01-10 21:10:45 UTC (rev 7761) @@ -180,9 +180,16 @@ /** * BSBM Q4 against pc100. + * <p> + * Note: This query has TWO join groups that are sufficiently complex to run + * the RTO. However, only one of the join groups is marked for RTO + * optimization in order to keep the test harness simple. The test harness + * assumes that there is a single JOIN group that is optimized by the RTO + * and then verifies the join ordering within that join group. The test + * harness breaks if there is more than one join group optimized by the RTO. */ public void test_BSBM_Q4_pc100() throws Exception { - + final TestHelper helper = new TestHelper(// "rto/BSBM-Q4", // testURI, "rto/BSBM-Q4.rq",// queryFileURL @@ -193,15 +200,8 @@ /* * Verify that the runtime optimizer produced the expected join path. */ - final int[] expected = new int[] { 3, 4, 5, 1, 2, 6, 7, 8, 9, 10, 11, 12 }; + final int[] expected = new int[] { 9, 6, 7, 8, 10, 11 }; - /* - * FIXME This fails because there are actually TWO JoinGraph instances - * and we are using getOnly() to extract just ONE. It looks like one of - * those instances might not even run based on conditional routing in - * the query plan. This is probably because the query is a UNION of two - * complex join groups and one of them is probably failing the FILTER. - */ assertSameJoinOrder(expected, helper); } This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |