You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jena.apache.org by an...@apache.org on 2013/11/22 11:32:57 UTC

svn commit: r1544468 - in /jena/trunk/jena-arq/src: main/java/com/hp/hpl/jena/sparql/algebra/ main/java/com/hp/hpl/jena/sparql/algebra/optimize/ test/java/com/hp/hpl/jena/sparql/algebra/ test/java/com/hp/hpl/jena/sparql/algebra/optimize/

Author: andy
Date: Fri Nov 22 10:32:57 2013
New Revision: 1544468

URL: http://svn.apache.org/r1544468
Log:
JENA-595 New filter placement code.

Added:
    jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement_Old.java
      - copied, changed from r1542836, jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement.java
    jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/TestOpVars.java
      - copied, changed from r1542836, jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/TestOpVar.java
    jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformFilterPlacement.java
Removed:
    jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/TestOpVar.java
Modified:
    jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/OpVars.java
    jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/Optimize.java
    jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement.java
    jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/TS_Algebra.java
    jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformFilters.java

Modified: jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/OpVars.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/OpVars.java?rev=1544468&r1=1544467&r2=1544468&view=diff
==============================================================================
--- jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/OpVars.java (original)
+++ jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/OpVars.java Fri Nov 22 10:32:57 2013
@@ -18,16 +18,13 @@
 
 package com.hp.hpl.jena.sparql.algebra ;
 
-import static com.hp.hpl.jena.sparql.core.Vars.* ;
+import static com.hp.hpl.jena.sparql.core.Vars.addVar ;
+import static com.hp.hpl.jena.sparql.core.Vars.addVarsFromTriple ;
 
-import java.util.ArrayList;
-import java.util.Collection ;
-import java.util.Iterator ;
-import java.util.LinkedHashSet ;
-import java.util.List;
-import java.util.Set ;
+import java.util.* ;
 
-import org.apache.jena.atlas.lib.Tuple;
+import org.apache.jena.atlas.lib.SetUtils ;
+import org.apache.jena.atlas.lib.Tuple ;
 
 import com.hp.hpl.jena.graph.Node ;
 import com.hp.hpl.jena.graph.Triple ;
@@ -37,6 +34,7 @@ import com.hp.hpl.jena.sparql.algebra.op
 import com.hp.hpl.jena.sparql.core.BasicPattern ;
 import com.hp.hpl.jena.sparql.core.Var ;
 import com.hp.hpl.jena.sparql.pfunction.PropFuncArg ;
+import com.hp.hpl.jena.sparql.util.VarUtils ;
 
 /** Get vars for a pattern */
 
@@ -65,6 +63,18 @@ public class OpVars
         OpWalker.walk(new WalkerVisitorVisible(visitor, acc), op) ;
     }
     
+    /** The set of variables that wil be in every solution of this Op */
+    public static Set<Var> fixedVars(Op op) {
+        Set<Var> acc = collector() ;
+        fixedVars(op, acc) ;
+        return acc ;
+    }
+
+    public static void fixedVars(Op op, Set<Var> acc) {
+        OpVarsPattern visitor = new OpVarsPattern(acc, true) ;
+        OpWalker.walk(new WalkerVisitorFixed(visitor, acc), op) ;
+    }
+    
     @SuppressWarnings("unchecked")
     public static Tuple<Set<Var>> mentionedVarsByPosition(Op op) {
         Set<Var> graphAcc = collector() ;
@@ -103,13 +113,16 @@ public class OpVars
         OpWalker.walk(op, visitor) ;
     }
 
+    /** @deprecate Use VarUtils.addVars */
+    @Deprecated
     public static Collection<Var> vars(BasicPattern pattern) {
         Set<Var> acc = collector() ;
         vars(pattern, acc) ;
         return acc ;
     }
 
-    public static void vars(BasicPattern pattern, Collection<Var> acc) {
+    /** @deprecate Use VarUtils.addVars */
+    @Deprecated    public static void vars(BasicPattern pattern, Collection<Var> acc) {
         for (Triple triple : pattern)
             addVarsFromTriple(acc, triple) ;
     }
@@ -145,6 +158,56 @@ public class OpVars
         }
     }
 
+    // Only consider variables that are visible and definitely defined.
+    // OPTIONAL (2 forms) and UNION are the interesting cases.
+    private static class WalkerVisitorFixed extends WalkerVisitor
+    {
+        private final Collection<Var> acc ;
+
+        public WalkerVisitorFixed(OpVarsPattern visitor, Collection<Var> acc) {
+            super(visitor) ;
+            this.acc = acc ;
+        }
+        
+        @Override
+        public void visit(OpLeftJoin x) {
+            x.getLeft().visit(this);
+        }
+
+        @Override
+        public void visit(OpConditional x) {
+            x.getLeft().visit(this);
+        }
+
+        @Override
+        public void visit(OpUnion x) {
+            Set<Var> left = fixedVars(x.getLeft()) ;
+            Set<Var> right = fixedVars(x.getRight()) ;
+            Set<Var> r = SetUtils.intersection(left,  right) ;
+            acc.addAll(r) ;
+        }
+        
+        @Override
+        public void visit(OpProject op) {
+            before(op) ;
+            // Skip Project subop.
+            acc.addAll(op.getVars()) ;
+            after(op) ;
+        }
+
+        @Override
+        public void visit(OpMinus op) {
+            before(op) ;
+            if (op.getLeft() != null)
+                op.getLeft().visit(this) ;
+            // Skip right.
+            // if ( op.getRight() != null ) op.getRight().visit(this) ;
+            if (visitor != null)
+                op.visit(visitor) ;
+            after(op) ;
+        }
+    }
+
     private static class OpVarsPattern extends OpVisitorBase
     {
         // The possibly-set-vars
@@ -158,7 +221,7 @@ public class OpVars
 
         @Override
         public void visit(OpBGP opBGP) {
-            vars(opBGP.getPattern(), acc) ;
+            VarUtils.addVars(acc, opBGP.getPattern()) ;
         }
 
         @Override
@@ -170,7 +233,7 @@ public class OpVars
         @Override
         public void visit(OpQuadPattern quadPattern) {
             addVar(acc, quadPattern.getGraphNode()) ;
-            vars(quadPattern.getBasicPattern(), acc) ;
+            VarUtils.addVars(acc, quadPattern.getBasicPattern()) ;
 //            // Pure quading
 //            for (Iterator<Quad> iter = quadPattern.getQuads().iterator(); iter.hasNext();) {
 //                Quad quad = iter.next() ;

Modified: jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/Optimize.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/Optimize.java?rev=1544468&r1=1544467&r2=1544468&view=diff
==============================================================================
--- jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/Optimize.java (original)
+++ jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/Optimize.java Fri Nov 22 10:32:57 2013
@@ -187,10 +187,6 @@ public class Optimize implements Rewrite
         if ( context.isTrueOrUndef(ARQ.optFilterDisjunction) )
             op = apply("Filter Disjunction", new TransformFilterDisjunction(), op) ;
         
-        if ( context.isTrueOrUndef(ARQ.optFilterPlacement) )
-            // This can be done too early (breaks up BGPs).
-            op = apply("Filter Placement", new TransformFilterPlacement(), op) ;
-        
         if ( context.isTrueOrUndef(ARQ.optTopNSorting) )
         	op = apply("TopN Sorting", new TransformTopN(), op) ;
         
@@ -209,11 +205,20 @@ public class Optimize implements Rewrite
         // Convert paths to triple patterns. 
         // Also done in the AlgebraGenerator so this transform step catches programmatically built op expressions 
         op = apply("Path flattening", new TransformPathFlattern(), op) ;
-        
+
         // Find joins/leftJoin that can be done by index joins (generally preferred as fixed memory overhead).
         if ( context.isTrueOrUndef(ARQ.optIndexJoinStrategy) )
             op = apply("Index Join strategy", new TransformJoinStrategy(), op) ;
         
+        // Place filters close to where their dependency variabnes are defined.
+        // This prunes the output of that step as early as possible.
+        // If done before TransformJoinStrategy, you can get two applications
+        // of a filter in a (sequence) from eachhalf of a (join).  This is harmless,
+        // because filters are generally cheap, but it looks a bit bad.
+        if ( context.isTrueOrUndef(ARQ.optFilterPlacement) )
+            // This can be done too early (breaks up BGPs).
+            op = apply("Filter Placement", new TransformFilterPlacement(), op) ;
+
         // Merge adjacent BGPs
         if ( context.isTrueOrUndef(ARQ.optMergeBGPs) )
             op = apply("Merge BGPs", new TransformMergeBGPs(), op) ;

Modified: jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement.java?rev=1544468&r1=1544467&r2=1544468&view=diff
==============================================================================
--- jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement.java (original)
+++ jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement.java Fri Nov 22 10:32:57 2013
@@ -16,167 +16,190 @@
  * limitations under the License.
  */
 
-/* Contains code submitted in email to jena-user@incubator.apache.org 
- * so software grant and relicensed under the Apache Software License. 
- *    transformFilterConditional
- */
-
-package com.hp.hpl.jena.sparql.algebra.optimize;
+package com.hp.hpl.jena.sparql.algebra.optimize ;
 
-import java.util.HashSet ;
+import java.util.Collection ;
 import java.util.Iterator ;
 import java.util.List ;
 import java.util.Set ;
 
+import org.apache.jena.atlas.lib.CollectionUtils ;
+import org.apache.jena.atlas.lib.DS ;
+
 import com.hp.hpl.jena.graph.Node ;
 import com.hp.hpl.jena.graph.Triple ;
 import com.hp.hpl.jena.sparql.algebra.Op ;
 import com.hp.hpl.jena.sparql.algebra.OpVars ;
 import com.hp.hpl.jena.sparql.algebra.TransformCopy ;
-import com.hp.hpl.jena.sparql.algebra.op.OpBGP ;
-import com.hp.hpl.jena.sparql.algebra.op.OpConditional ;
-import com.hp.hpl.jena.sparql.algebra.op.OpFilter ;
-import com.hp.hpl.jena.sparql.algebra.op.OpQuadPattern ;
-import com.hp.hpl.jena.sparql.algebra.op.OpSequence ;
-import com.hp.hpl.jena.sparql.algebra.op.OpTable ;
+import com.hp.hpl.jena.sparql.algebra.op.* ;
 import com.hp.hpl.jena.sparql.core.BasicPattern ;
 import com.hp.hpl.jena.sparql.core.Var ;
 import com.hp.hpl.jena.sparql.expr.Expr ;
 import com.hp.hpl.jena.sparql.expr.ExprList ;
 import com.hp.hpl.jena.sparql.util.VarUtils ;
 
-/** Rewrite an algebra expression to put filters as close to their bound variables in a BGP.
- *  Works on (filter (BGP ...) )
- *  Could be made to work on a wider class of forms.
- */  
-
-public class TransformFilterPlacement extends TransformCopy
-{
-    static boolean doFilterPlacement = true ;
-    
-    public static Op transform(ExprList exprs, BasicPattern bgp)
-    {
-        if ( ! doFilterPlacement )
-            return OpFilter.filter(exprs, new OpBGP(bgp)) ;
-        // Mutated
-        ExprList exprs2 = new ExprList(exprs) ;
-        Op op = transformFilterBGP(exprs2, new HashSet<Var>(), bgp) ;
-        // Remaining filters? e.g. ones mentioning var s not used anywhere. 
-        op = buildFilter(exprs2, op) ;
-        return op ;
+/**
+ * Rewrite an algebra expression to put filters as close to their bound
+ * variables.
+ * <p>Process BGP (whether triples or quads) is left as a separate step (but after this transform)
+ * because it can desirable to reorder the BGP before placing filters,
+ * or afterwards.   
+ */
+
+public class TransformFilterPlacement extends TransformCopy {
+    static class Placement {
+        final Op op ;
+        final ExprList unplaced ; 
+        Placement(Op op, ExprList remaining) { this.op = op ; this.unplaced = remaining ; }
     }
     
-    public static Op transform(ExprList exprs, Node graphNode, BasicPattern bgp)
-    {
-        if ( ! doFilterPlacement )
-            return OpFilter.filter(exprs, new OpQuadPattern(graphNode, bgp)) ;
-        // Mutated
-        ExprList exprs2 = new ExprList(exprs) ;
-        Op op =  transformFilterQuadPattern(exprs2, new HashSet<Var>(), graphNode, bgp);
-        op = buildFilter(exprs2, op) ;
+    static final ExprList emptyList = new ExprList() ;
+    static final Placement noChangePlacement = null ; //new Placement(null, null) ;
+    
+    private static Placement result(Op op, ExprList remaining) { 
+        if ( op == null )
+            return null ;
+        return new Placement(op, remaining) ; 
+    }
+    
+    private static Placement resultNoChange(Op original) { 
+        return noChangePlacement ;
+    }
+    private static boolean isNoChange(Placement placement) { 
+        return placement == noChangePlacement ;
+    }
+
+    public static Op transform(ExprList exprs, BasicPattern bgp) {
+        Placement placement = placeBGP(exprs, bgp) ;
+        Op op = ( placement == null ) ? new OpBGP(bgp) : placement.op ;
+        if ( placement != null )
+            op = buildFilter(placement.unplaced, op) ;
         return op ;
     }
 
-    public TransformFilterPlacement()
-    { }
-    
+    public static Op transform(ExprList exprs, Node graphNode, BasicPattern bgp) {
+        Placement placement = placeQuadPattern(exprs, graphNode, bgp) ;
+        Op op = ( placement == null ) ? new OpQuadPattern(graphNode, bgp) : placement.op ;
+        if ( placement != null )
+            op = buildFilter(placement.unplaced, op) ;
+        return op ;
+    }
+
+    public TransformFilterPlacement() {}
+
     @Override
-    public Op transform(OpFilter opFilter, Op x)
-    {
-        if ( ! doFilterPlacement )
-            return super.transform(opFilter, x) ;
-        
-        // Destructive use of exprs - copy it.
-        ExprList exprs = new ExprList(opFilter.getExprs()) ;
-        Set<Var> varsScope = new HashSet<Var>() ;
-        
-        Op op = transform(exprs, varsScope, x) ;
-        if ( op == x )
+    public Op transform(OpFilter opFilter, Op x) {
+        ExprList exprs = opFilter.getExprs() ;
+        Placement placement = transform(exprs, x) ;
+        if ( placement == null || placement.op == x )
             // Didn't do anything.
             return super.transform(opFilter, x) ;
-        
-        // Remaining exprs
-        op = buildFilter(exprs, op) ;
+        Op op = buildFilter(placement) ;
         return op ;
     }
-    
-    private static Op transform(ExprList exprs, Set<Var> varsScope, Op x)
-    {
-        // OpAssign/OpExtend could be done if the assignment and exprs are independent.
-        //  Dispatch by visitor??
-        if ( x instanceof OpBGP )
-            return transformFilterBGP(exprs, varsScope, (OpBGP)x) ;
 
-        if ( x instanceof OpSequence )
-            return transformFilterSequence(exprs, varsScope, (OpSequence)x) ;
-        
-        if ( x instanceof OpQuadPattern )
-            return transformFilterQuadPattern(exprs, varsScope, (OpQuadPattern)x) ;
-        
-        if ( x instanceof OpSequence )
-            return transformFilterSequence(exprs, varsScope, (OpSequence)x) ;
-        
-        if ( x instanceof OpConditional )
-            return transformFilterConditional(exprs, varsScope, (OpConditional)x) ;
-        
-        // Not special - advance the variable scope tracking. 
-        OpVars.visibleVars(x, varsScope) ;
-        return x ;
-    }
-    
-    // == The transformFilter* modify the exprs and patternVarsScope arguments 
-    
-    private static Op transformFilterBGP(ExprList exprs, Set<Var> patternVarsScope, OpBGP x)
-    {
-        return  transformFilterBGP(exprs, patternVarsScope, x.getPattern()) ;
+    // Recurse
+    private static Op transformOp(ExprList exprs, Op x) {
+        Placement placement = transform(exprs, x) ;
+        Op op = buildFilter(placement) ;
+        return op ;
     }
 
-    // Mutates exprs
-    private static Op transformFilterBGP(ExprList exprs, Set<Var> patternVarsScope, BasicPattern pattern)
-    {
-        // Any filters that depend on no variables. 
-        Op op = insertAnyFilter(exprs, patternVarsScope, null) ;
-        
-        for ( Triple triple : pattern )
-        {
+    private static Placement transform(ExprList exprs, Op input) {
+        // Dispatch by visitor??
+        Placement placement = null ;
+
+        if ( input instanceof OpBGP )
+            placement = placeBGP(exprs, (OpBGP)input) ;
+        else if ( input instanceof OpSequence )
+            placement = placeSequence(exprs, (OpSequence)input) ;
+        else if ( input instanceof OpQuadPattern )
+            placement = placeQuadPattern(exprs, (OpQuadPattern)input) ;
+        else if ( input instanceof OpJoin )
+            placement = placeJoin(exprs, (OpJoin)input) ;
+        else if ( input instanceof OpConditional )
+            placement = placeConditional(exprs, (OpConditional)input) ;
+        else if ( input instanceof OpLeftJoin )
+            placement = placeLeftJoin(exprs, (OpLeftJoin)input) ;
+        else if ( input instanceof OpFilter )
+            placement = placeFilter(exprs, (OpFilter)input) ;
+        else if ( input instanceof OpUnion )
+            placement = placeUnion(exprs, (OpUnion)input) ;
+        
+        
+        // These are operations where chnaging the order of operations
+        // does not in itself make a differencebut enables expressions
+        // to be pushed own down to where they might make a difference.
+        // Otherwise these would blockers.
+        
+        else if ( input instanceof OpExtend )
+            placement = placeExtend(exprs, (OpExtend)input) ;
+        else if ( input instanceof OpAssign )
+            placement = placeAssign(exprs, (OpAssign)input) ;
+        else if ( input instanceof OpProject )
+            placement = placeProject(exprs, (OpProject)input) ;
+
+        return placement ;
+    }
+    
+    private static Placement x_placeNoOp(ExprList exprs, Op op) {
+        return result(op, exprs) ;
+    }
+
+    private static Placement placeFilter(ExprList exprs, OpFilter input) {
+        Placement p = transform(exprs, input.getSubOp()) ;
+        if ( p == null )
+            p = new Placement(input.getSubOp(), exprs) ;
+        // These would have already been placed (bottom up conversion)
+        // so no point in including in the transform() call above.
+        p.unplaced.addAll(input.getExprs());
+        return p ;
+    }
+
+    private static Placement placeBGP(ExprList exprs, OpBGP x) {
+        return placeBGP(exprs, x.getPattern()) ;
+    }
+
+    private static Placement placeBGP(ExprList exprsIn, BasicPattern pattern) {
+        ExprList exprs = new ExprList(exprsIn) ;
+        Set<Var> patternVarsScope = DS.set() ;
+        // Any filters that depend on no variables.
+        Op op = null ;
+
+        for (Triple triple : pattern) {
+            // Place any filters that are now covered.
+            op = insertAnyFilter(exprs, patternVarsScope, op) ;
+            // Consider this triple.
+            // Get BGP that is accumulating triples.
             OpBGP opBGP = getBGP(op) ;
-            if ( opBGP == null )
-            {
+            if ( opBGP == null ) {
                 // Last thing was not a BGP (so it likely to be a filter)
                 // Need to pass the results from that into the next triple.
-                // Which is a join and sequence is a special case of join
-                // which always evaluates by passing results of the early
-                // part into the next element of the sequence.
-                
-                opBGP = new OpBGP() ;    
+                opBGP = new OpBGP() ;
                 op = OpSequence.create(op, opBGP) ;
             }
-            
+
             opBGP.getPattern().add(triple) ;
             // Update variables in scope.
             VarUtils.addVarsFromTriple(patternVarsScope, triple) ;
-            
-            // Attempt to place any filters
-            op = insertAnyFilter(exprs, patternVarsScope, op) ;
-        } 
-        // Leave any remaining filter expressions - don't wrap up any as something else may take them.
-        return op ;
+        }
+        
+        // Place any filters this whole BGP covers. 
+        op = insertAnyFilter(exprs, patternVarsScope, op) ;
+        return result(op, exprs) ;
     }
-    
-    /** Find the current OpBGP, or return null. */ 
-    private static OpBGP getBGP(Op op)
-    {
+
+    /** Find the current OpBGP, or return null. */
+    private static OpBGP getBGP(Op op) {
         if ( op instanceof OpBGP )
             return (OpBGP)op ;
-        
-        if ( op instanceof OpSequence )
-        {
+
+        if ( op instanceof OpSequence ) {
             // Is last in OpSequence an BGP?
             OpSequence opSeq = (OpSequence)op ;
             List<Op> x = opSeq.getElements() ;
-            if ( x.size() > 0 )
-            {                
-                Op opTop = x.get(x.size()-1) ;
+            if ( x.size() > 0 ) {
+                Op opTop = x.get(x.size() - 1) ;
                 if ( opTop instanceof OpBGP )
                     return (OpBGP)opTop ;
                 // Drop through
@@ -185,134 +208,297 @@ public class TransformFilterPlacement ex
         // Can't find.
         return null ;
     }
-    
-    private static Op transformFilterQuadPattern(ExprList exprs, Set<Var> patternVarsScope, OpQuadPattern pattern)
-    {
-        return transformFilterQuadPattern(exprs, patternVarsScope, pattern.getGraphNode(), pattern.getBasicPattern()) ;
+
+    private static Placement placeQuadPattern(ExprList exprs, OpQuadPattern pattern) {
+        return placeQuadPattern(exprs, pattern.getGraphNode(), pattern.getBasicPattern()) ;
     }
-    
-    private static Op transformFilterQuadPattern(ExprList exprs, Set<Var> patternVarsScope, Node graphNode, BasicPattern pattern) 
-    {
-        // Any filters that depend on no variables. 
-        Op op = insertAnyFilter(exprs, patternVarsScope, null) ;
-        if ( Var.isVar(graphNode) )
-        {
+
+    private static Placement placeQuadPattern(ExprList exprsIn, Node graphNode, BasicPattern pattern) {
+        ExprList exprs = new ExprList(exprsIn) ;
+        Set<Var> patternVarsScope = DS.set() ;
+        // Any filters that depend on no variables.
+
+        if ( Var.isVar(graphNode) ) {
             // Add in the graph node of the quad block.
-            // It's picked up after the first triple is processed.
             VarUtils.addVar(patternVarsScope, Var.alloc(graphNode)) ;
         }
+        Op op = null ;
         
-        for ( Triple triple : pattern )
-        {
+        for (Triple triple : pattern) {
+            op = insertAnyFilter(exprs, patternVarsScope, op) ;
             OpQuadPattern opQuad = getQuads(op) ;
-            if ( opQuad == null )
-            {
-                opQuad = new OpQuadPattern(graphNode, new BasicPattern()) ;    
+            if ( opQuad == null ) {
+                opQuad = new OpQuadPattern(graphNode, new BasicPattern()) ;
                 op = OpSequence.create(op, opQuad) ;
             }
-            
+
             opQuad.getBasicPattern().add(triple) ;
             // Update variables in scope.
             VarUtils.addVarsFromTriple(patternVarsScope, triple) ;
-
-            // Attempt to place any filters
-            op = insertAnyFilter(exprs, patternVarsScope, op) ;
         }
-        
-        
-        return op ;
+        // Place any filters this whole quad block covers. 
+        op = insertAnyFilter(exprs, patternVarsScope, op) ;
+        return result(op, exprs) ;
     }
-    
-    /** Find the current OpQuadPattern, or return null. */ 
-    private static OpQuadPattern getQuads(Op op)
-    {
+
+    /** Find the current OpQuadPattern, or return null. */
+    private static OpQuadPattern getQuads(Op op) {
         if ( op instanceof OpQuadPattern )
             return (OpQuadPattern)op ;
-        
-        if ( op instanceof OpSequence )
-        {
+
+        if ( op instanceof OpSequence ) {
             // Is last in OpSequence an BGP?
             OpSequence opSeq = (OpSequence)op ;
             List<Op> x = opSeq.getElements() ;
-            if ( x.size() > 0 )
-            {                
-                Op opTop = x.get(x.size()-1) ;
+            if ( x.size() > 0 ) {
+                Op opTop = x.get(x.size() - 1) ;
                 if ( opTop instanceof OpQuadPattern )
                     return (OpQuadPattern)opTop ;
                 // Drop through
             }
         }
-        // Can't find.
         return null ;
     }
 
-    private static Op transformFilterSequence(ExprList exprs, Set<Var> varScope, OpSequence opSequence)
-    {
+    /*
+     * A Sequence is a number of joins where scoping means the LHS can be
+     * substituted into the right, i.e. there are no scoping issues. Assuming a
+     * substitution join is going to be done, filtering once as soon as the
+     * accumulated variables cover the filter is a good thing to do. It is
+     * effectively pusing on teh left side only - the right side, by
+     * substitution, will never see the variables. The variable can not be
+     * reintroduced (it will have been renamed away if it's the same name,
+     * different scope, which is a different variable with the same name in the
+     * orginal query.
+     */
+
+    private static Placement placeSequence(ExprList exprsIn, OpSequence opSequence) {
+        ExprList exprs = new ExprList(exprsIn) ;
+        Set<Var> varScope = DS.set() ;
         List<Op> ops = opSequence.getElements() ;
-        
-        // Any filters that depend on no variables. 
-        Op op = insertAnyFilter(exprs, varScope, null) ;
-        
-        for ( Iterator<Op> iter = ops.iterator() ; iter.hasNext() ; )
-        {
-            Op seqElt = iter.next() ;
-            // Process the sequence element.  This may insert filters (sequence or BGP)
-            seqElt = transform(exprs, varScope, seqElt) ;
-            // Merge into sequence.
-            op = OpSequence.create(op, seqElt) ;
-            // Place any filters now ready.
+
+        Op op = null ;
+        // No point placing on the last element as that is the same as filtering the entire expression.
+        for (int i = 0 ; i < ops.size() ; i++ ) {
             op = insertAnyFilter(exprs, varScope, op) ;
+            Op seqElt = ops.get(i) ;
+            if ( i != ops.size()-1 ) {
+                Placement p = transform(exprs, seqElt) ;
+                if ( p != null ) {
+                    exprs = p.unplaced ;
+                    seqElt = p.op ;
+                }
+                varScope.addAll(fixedVars(seqElt)) ;
+            }
+            op = OpSequence.create(op, seqElt) ;
         }
-        return op ;
+        return result(op, exprs) ;
     }
     
-    // Modularize.
-    private static Op transformFilterConditional(ExprList exprs, Set<Var> varScope, OpConditional opConditional)
-    {
-        // Any filters that depend on no variables. 
-        Op op = insertAnyFilter(exprs, varScope, null) ;
-        Op left = opConditional.getLeft();
-        left = transform(exprs, varScope, left);
-        Op right = opConditional.getRight();
-        op = new OpConditional(left, right);
-        op = insertAnyFilter(exprs, varScope, op);
-        return op;
-     }
+    // Whether to push a covered filter into the RHS even if pushed into the LHS.
+    // If this is run after join->sequence, then this is good to do.
+    static boolean pushRightAsWellAsLeft = true ; 
+    
+    private static Placement placeJoin(ExprList exprs, OpJoin opJoin) {
+        Op left = opJoin.getLeft() ;
+        Op right = opJoin.getRight() ;
+        Collection<Var> leftVars = fixedVars(left) ;
+        Collection<Var> rightVars = fixedVars(right) ;
+        ExprList unpushed = new ExprList() ;
+        ExprList pushLeft = new ExprList() ;
+        ExprList pushRight = new ExprList() ;
+
+        for (Expr expr : exprs) {
+            Set<Var> vars = expr.getVarsMentioned() ;
+            boolean pushed = false ;
+
+            if ( leftVars.containsAll(vars) ) {
+                pushLeft.add(expr) ;
+                pushed = true ;
+            }
+            
+            if ( pushed && ! pushRightAsWellAsLeft )
+                continue ;
+            // If left only, make this "else if" of left test, remove "continue" 
+            if ( rightVars.containsAll(vars) ) {
+                // Push right
+                pushRight.add(expr) ;
+                pushed = true ;
+            }
+
+            if ( !pushed )
+                unpushed.add(expr) ;
+        }
+
+        if ( pushLeft.isEmpty() && pushRight.isEmpty() )
+            return null ;
+
+        Op opLeftNew = left ;
+        if ( !pushLeft.isEmpty() )
+            opLeftNew = transformOp(pushLeft, opLeftNew) ;
+
+        Op opRightNew = right ;
+        if ( !pushRight.isEmpty() )
+            opRightNew = transformOp(pushRight, opRightNew) ;
+
+        Op op = OpJoin.create(opLeftNew, opRightNew) ;
+        return result(op, unpushed) ;
+    }
+
+    /* A conditional is left join without scoping complications. */
     
-    // ---- Utilities
+    private static Placement placeConditional(ExprList exprs, OpConditional opConditional) {
+        Op left = opConditional.getLeft() ;
+        Op right = opConditional.getRight() ;
+        Placement nLeft = transform(exprs, left) ;
+        if ( nLeft == null )
+            return result(opConditional, exprs) ;
+        Op op = new OpConditional(nLeft.op, right) ;
+        return result(op, nLeft.unplaced) ;
+    }
+
+    private static Placement placeLeftJoin(ExprList exprs, OpLeftJoin opLeftJoin) {
+        // Push LHS only.  RHS may result in no matches - is that safe to push into? 
+        Op left = opLeftJoin.getLeft() ;
+        Op right = opLeftJoin.getRight() ;
+        Placement nLeft = transform(exprs, left) ;
+        if ( nLeft == null )
+            return result(opLeftJoin, exprs) ;
+        Op op = OpLeftJoin.create(nLeft.op, right, opLeftJoin.getExprs()) ;
+        return result(op, nLeft.unplaced) ;
+    }
     
+    private static Placement placeUnion(ExprList exprs, OpUnion input) {
+        // Unsubtle - push into both sides.
+        // Neater - for all unpushed put outside the union. 
+        Op left = input.getLeft() ;
+        Placement pLeft = transform(exprs, left) ;
+        left = buildFilter(pLeft) ;
+        
+        Op right = input.getRight() ;
+        Placement pRight = transform(exprs, right) ;
+        right = buildFilter(pRight) ;
+        
+        Op op2 = OpUnion.create(left, right) ;
+        return result(op2, emptyList) ;
+    }
+
+    /** Try to optimize (filter (extend ...)) */
+    private static Placement placeExtend(ExprList exprs, OpExtend input) {
+        return processExtendAssign(exprs, input) ;
+    }
+    
+    private static Placement placeAssign(ExprList exprs, OpAssign input) {
+        return processExtendAssign(exprs, input) ;
+        
+    }
+
+    private static Placement processExtendAssign(ExprList exprs, OpExtendAssign input) {
+        // Could break up the VarExprList
+        Collection<Var> vars1 = input.getVarExprList().getVars() ;
+        ExprList pushed = new ExprList() ;
+        ExprList unpushed = new ExprList() ;
+        
+        for ( Expr expr : exprs ) {
+            Set<Var> exprVars = expr.getVarsMentioned() ;
+            if ( disjoint(vars1, exprVars) )
+                pushed.add(expr);
+            else
+                unpushed.add(expr) ;
+        }
+                
+        if ( pushed.isEmpty() ) 
+            return resultNoChange(input) ;
+        
+        // (filter ... (extend ... ))
+        //   ===>
+        // (extend ... (filter ... ))
+        Op opSub = input.getSubOp() ;
+        
+        // And try down the expressions
+        Placement p = transform(pushed, opSub) ;
+        
+        if ( p == null ) {
+            Op op1 = OpFilter.filter(pushed, opSub) ;
+            Op op2 = input.copy(op1) ; //, input.getVarExprList()) ; //OpExtend.extend(op1, input.getVarExprList()) ;
+            return result(op2, unpushed) ;
+        }
+        Op op1 = OpFilter.filter(p.unplaced, p.op) ;
+        Op op2 = input.copy(op1) ;
+        return result(op2, unpushed) ;
+    }
+
+    private static Placement placeProject(ExprList exprs, OpProject input) {
+        Collection<Var> varsProject = input.getVars() ;
+        ExprList pushed = new ExprList() ;
+        ExprList unpushed = new ExprList() ;
+        
+        for ( Expr expr : exprs ) {
+            Set<Var> exprVars = expr.getVarsMentioned() ;
+            if ( varsProject.containsAll(exprVars) )
+                pushed.add(expr);
+            else
+                unpushed.add(expr) ;
+        }
+        if ( pushed.isEmpty() ) 
+            return resultNoChange(input) ;
+        // (filter (project ...)) ===> (project (filter ...)) 
+        Op opSub = input.getSubOp() ;
+        Placement p = transform(pushed, opSub) ;
+        if ( p == null ) {
+            Op op1 = OpFilter.filter(pushed, opSub) ;
+            Op op2 = input.copy(op1) ;
+            return result(op2, unpushed) ;
+        }
+        Op op1 = OpFilter.filter(p.unplaced, p.op) ;
+        Op op2 = input.copy(op1) ;
+        return result(op2, unpushed) ;
+    }
+    
+    private static Set<Var> fixedVars(Op op) {
+        return OpVars.fixedVars(op) ;
+    }
+
     /** For any expression now in scope, wrap the op with a filter */
-    private static Op insertAnyFilter(ExprList exprs, Set<Var> patternVarsScope, Op op)
-    {
-        for ( Iterator<Expr> iter = exprs.iterator() ; iter.hasNext() ; )
-        {
+    private static Op insertAnyFilter(ExprList exprs, Set<Var> patternVarsScope, Op op) {
+        for (Iterator<Expr> iter = exprs.iterator(); iter.hasNext();) {
             Expr expr = iter.next() ;
             // Cache
             Set<Var> exprVars = expr.getVarsMentioned() ;
-            if ( patternVarsScope.containsAll(exprVars) )
-            {
+            if ( patternVarsScope.containsAll(exprVars) ) {
                 if ( op == null )
                     op = OpTable.unit() ;
                 op = OpFilter.filter(expr, op) ;
                 iter.remove() ;
+                // Record expr.
             }
         }
         return op ;
     }
+
+    private static <T> boolean disjoint(Collection<T> collection, Collection<T> possibleElts) {
+        return CollectionUtils.disjoint(collection, possibleElts) ;
+    }
+
+    /** Place expressions around an Op */
+    private static Op buildFilter(Placement placement) {
+        if ( placement == null )
+            return null ;
+        if ( placement.unplaced.isEmpty() )
+            return placement.op ;
+        return buildFilter(placement.unplaced, placement.op) ;
+    }
     
-    /** Place expressions around an Op */ 
-    private static Op buildFilter(ExprList exprs, Op op)
-    {
-        if ( exprs.isEmpty() )
+    private static Op buildFilter(ExprList exprs, Op op) {
+        if ( exprs == null || exprs.isEmpty() )
             return op ;
-    
-        for ( Iterator<Expr> iter = exprs.iterator() ; iter.hasNext() ; )
-        {
+
+        for (Iterator<Expr> iter = exprs.iterator(); iter.hasNext();) {
             Expr expr = iter.next() ;
             if ( op == null )
                 op = OpTable.unit() ;
             op = OpFilter.filter(expr, op) ;
-            iter.remove();
+            iter.remove() ;
         }
         return op ;
     }

Copied: jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement_Old.java (from r1542836, jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement.java)
URL: http://svn.apache.org/viewvc/jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement_Old.java?p2=jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement_Old.java&p1=jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement.java&r1=1542836&r2=1544468&rev=1544468&view=diff
==============================================================================
--- jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement.java (original)
+++ jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement_Old.java Fri Nov 22 10:32:57 2013
@@ -45,12 +45,15 @@ import com.hp.hpl.jena.sparql.expr.Expr 
 import com.hp.hpl.jena.sparql.expr.ExprList ;
 import com.hp.hpl.jena.sparql.util.VarUtils ;
 
+// The old (to jena 2.11.0) filter placement engine.
+// Remove if version > 2.11.2 
+
 /** Rewrite an algebra expression to put filters as close to their bound variables in a BGP.
  *  Works on (filter (BGP ...) )
  *  Could be made to work on a wider class of forms.
  */  
 
-public class TransformFilterPlacement extends TransformCopy
+public class TransformFilterPlacement_Old extends TransformCopy
 {
     static boolean doFilterPlacement = true ;
     
@@ -77,7 +80,7 @@ public class TransformFilterPlacement ex
         return op ;
     }
 
-    public TransformFilterPlacement()
+    public TransformFilterPlacement_Old()
     { }
     
     @Override

Modified: jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/TS_Algebra.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/TS_Algebra.java?rev=1544468&r1=1544467&r2=1544468&view=diff
==============================================================================
--- jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/TS_Algebra.java (original)
+++ jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/TS_Algebra.java Fri Nov 22 10:32:57 2013
@@ -27,10 +27,12 @@ import org.junit.runners.Suite ;
 
 @RunWith(Suite.class)
 @Suite.SuiteClasses( {
-    TestVarFinder.class
+    TestOpVars.class
+    , TestVarFinder.class
     , TestAlgebraTranslate.class
     , TestClassify.class
     , TestTransformFilters.class
+    , TestTransformFilterPlacement.class
     , TestTransformQuads.class
     , TestSemanticEquivalence.class
 

Copied: jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/TestOpVars.java (from r1542836, jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/TestOpVar.java)
URL: http://svn.apache.org/viewvc/jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/TestOpVars.java?p2=jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/TestOpVars.java&p1=jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/TestOpVar.java&r1=1542836&r2=1544468&rev=1544468&view=diff
==============================================================================
--- jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/TestOpVar.java (original)
+++ jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/TestOpVars.java Fri Nov 22 10:32:57 2013
@@ -29,22 +29,40 @@ import org.junit.Test ;
 import com.hp.hpl.jena.sparql.core.Var ;
 import com.hp.hpl.jena.sparql.sse.SSE ;
 
-public class TestOpVar extends BaseTest
+public class TestOpVars extends BaseTest
 {
-    @Test public void opvar_01() { opVarPattern("(bgp (?s :p ?o))", "s", "o") ; }
-    @Test public void opvar_02() { opVarPattern("(leftjoin (bgp (?s :p ?o)) (bgp (?s1 :p ?o1)) )", "s1", "o1", "s", "o") ; }
-    @Test public void opvar_03() { opVarPattern("(leftjoin (bgp (?s :p ?o)) (bgp (?s :p ?o)) )", "s", "o") ; }
-    
-    @Test public void opvar_04() { opVarPattern("(project (?s) (bgp(?s :p ?o)))", "s") ; }
-    @Test public void opvar_05() { opVarPattern("(minus (bgp (?s :p ?o)) (bgp (?s1 :p ?o1)) )", "s", "o") ; }
-    @Test public void opvar_06() { opVarPattern("(join (project (?x) (bgp(?x :p ?z)))  (bgp(?s :p 1)) )", "x", "s") ; }
+    @Test public void opvars_01() { visible("(bgp (?s :p ?o))", "s", "o") ; }
+    @Test public void opvars_02() { visible("(leftjoin (bgp (?s :p ?o)) (bgp (?s1 :p ?o1)) )", "s1", "o1", "s", "o") ; }
+    @Test public void opvars_03() { visible("(leftjoin (bgp (?s :p ?o)) (bgp (?s :p ?o)) )", "s", "o") ; }
     
-    private static void opVarPattern(String string, String... vars)
+    @Test public void opvars_04() { visible("(project (?s) (bgp(?s :p ?o)))", "s") ; }
+    @Test public void opvars_05() { visible("(minus (bgp (?s :p ?o)) (bgp (?s1 :p ?o1)) )", "s", "o") ; }
+    @Test public void opvars_06() { visible("(join (project (?x) (bgp(?x :p ?z)))  (bgp(?s :p 1)) )", "x", "s") ; }
+    
+    
+    @Test public void opvars_10() { fixed("(bgp (?s :p ?o))", "s", "o") ; }
+    @Test public void opvars_11() { fixed("(leftjoin (bgp (?s :p ?o)) (bgp (?s1 :p ?o1)) )", "s", "o") ; }
+    @Test public void opvars_12() { fixed("(leftjoin (bgp (?s :p ?o)) (bgp (?s :p ?o)) )", "s", "o") ; }
+    
+    @Test public void opvars_13() { fixed("(union (bgp (?s :p ?o1)) (bgp (?s :p ?o2)) )", "s") ; }
+    @Test public void opvars_14() { fixed("(minus (bgp (?s :p ?o)) (bgp (?s1 :p ?o1)) )", "s", "o") ; }
+    @Test public void opvars_15() { fixed("(join (project (?x) (bgp(?x :p ?z)))  (bgp(?s :p 1)) )", "x", "s") ; }
+    
+    
+    private static void visible(String string, String... vars)
     {
         Op op = SSE.parseOp(string) ;
         Collection<Var> c = OpVars.visibleVars(op) ;
         check(vars, c) ;
     }
+    
+    private static void fixed(String string, String... vars)
+    {
+        Op op = SSE.parseOp(string) ;
+        Collection<Var> c = OpVars.fixedVars(op) ;
+        check(vars, c) ;
+    }
+    
 
     private static void check(String[] varsExpected, Collection<Var> varsFound)
     {

Added: jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformFilterPlacement.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformFilterPlacement.java?rev=1544468&view=auto
==============================================================================
--- jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformFilterPlacement.java (added)
+++ jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformFilterPlacement.java Fri Nov 22 10:32:57 2013
@@ -0,0 +1,267 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.hp.hpl.jena.sparql.algebra.optimize ;
+
+import org.apache.jena.atlas.junit.BaseTest ;
+import org.apache.jena.atlas.lib.StrUtils ;
+import org.junit.Assert ;
+import org.junit.Test ;
+
+import com.hp.hpl.jena.sparql.algebra.Op ;
+import com.hp.hpl.jena.sparql.algebra.Transform ;
+import com.hp.hpl.jena.sparql.algebra.Transformer ;
+import com.hp.hpl.jena.sparql.sse.SSE ;
+
+public class TestTransformFilterPlacement extends BaseTest { //extends AbstractTestTransform {
+    
+    // ** Filter
+    
+    @Test public void place_bgp_01() {
+        test("(filter (= ?x 1) (bgp ( ?s ?p ?x)))", "(filter (= ?x 1) (bgp ( ?s ?p ?x)))") ;
+    }
+
+    @Test public void place_bgp_02() {
+        test("(filter (= ?x 1) (bgp (?s ?p ?x) (?s1 ?p1 ?x1) ))",
+             "(sequence (filter (= ?x 1) (bgp ( ?s ?p ?x))) (bgp (?s1 ?p1 ?x1)))") ;
+    }
+
+    @Test public void place_bgp_03() {
+        test("(filter (= ?x 1) (bgp (?s ?p ?x) (?s1 ?p1 ?x) ))",
+             "(sequence (filter (= ?x 1) (bgp ( ?s ?p ?x))) (bgp (?s1 ?p1 ?x)))") ;
+    }
+
+    @Test public void place_bgp_04() {
+        test("(filter (= ?XX 1) (bgp (?s ?p ?x) (?s1 ?p1 ?XX) ))", "(filter (= ?XX 1) (bgp (?s ?p ?x) (?s1 ?p1 ?XX) ))") ;
+    }
+
+    @Test public void place_no_match_01() {
+        // Unbound
+        test("(filter (= ?x ?unbound) (bgp (?s ?p ?x)))", null) ;
+    }
+
+    @Test public void place_no_match_02() {
+        test("(filter (= ?x ?unbound) (bgp (?s ?p ?x) (?s ?p ?x)))", null) ;
+    }
+
+    @Test public void place_no_match_03() {
+        test("(filter (= ?x ?unbound) (bgp (?s ?p ?x) (?s1 ?p1 ?XX)))", null) ;
+    }
+
+    @Test public void place_sequence_01() {
+        test("(filter (= ?x 123) (sequence (bgp (?s ?p ?x)) (bgp (?s ?p ?z)) ))",
+             "(sequence (filter (= ?x 123) (bgp (?s ?p ?x))) (bgp (?s ?p ?z)) )") ;
+    }
+
+    @Test public void place_sequence_02() {
+        // Given the sequence flows left into right, only need to filter in the
+        // LHS.  The RHS can't introduce ?x because it would not be a legal sequence
+        // if, for example, it had a BIND in it.
+        test("(filter (= ?x 123) (sequence (bgp (?s ?p ?x)) (bgp (?s ?p ?x)) ))",
+             "(sequence (filter (= ?x 123) (bgp (?s ?p ?x))) (bgp (?s ?p ?x)) )") ;
+    }
+
+    @Test public void place_sequence_03() {
+        test("(filter (= ?x 123) (sequence  (bgp (?s ?p ?x)) (bgp (?s ?p ?x1)) (bgp (?s ?p ?x2)) ))",
+             "(sequence (filter (= ?x 123) (bgp (?s ?p ?x))) (bgp (?s ?p ?x1)) (bgp (?s ?p ?x2)) )") ;
+    }
+
+    @Test public void place_sequence_04() {
+        test("(filter (= ?x 123) (sequence (bgp (?s ?p ?x1)) (bgp (?s ?p ?x)) (bgp (?s ?p ?x2)) ))",
+             "(sequence (bgp (?s ?p ?x1)) (filter (= ?x 123) (bgp (?s ?p ?x))) (bgp (?s ?p ?x2)) )") ;
+    }
+
+    @Test public void place_sequence_05() {
+        test("(filter (= ?x 123) (sequence (bgp (?s ?p ?x1) (?s ?p ?x)) (bgp (?s ?p ?x2)) ))",
+            "(sequence (filter (= ?x 123) (bgp (?s ?p ?x1) (?s ?p ?x))) (bgp (?s ?p ?x2)) )") ;
+    }
+
+    @Test public void place_sequence_06() {
+        test("(filter (= ?x 123) (sequence (bgp (?s ?p ?x1) (?s ?p ?x2)) (bgp (?s ?p ?x)) ))",
+             // If push filter to last element ... which is of no benefit
+            //"(sequence (bgp (?s ?p ?x1) (?s ?p ?x2)) (filter (= ?x 123) (bgp (?s ?p ?x))) )") ;
+             null) ;
+    }
+
+    @Test public void place_sequence_07() {
+        test("(filter (= ?A 123) (sequence (bgp (?s ?p ?x)) (bgp (?s ?p ?z)) ))",
+             null) ;
+    }
+
+    @Test public void place_sequence_08() {
+        test("(sequence (bgp (?s ?p ?x)) (filter (= ?z 123) (bgp (?s ?p ?z))) )",
+             null) ;
+    }
+
+    // Join : one sided push.
+    @Test public void place_join_01() {
+        test("(filter (= ?x 123) (join (bgp (?s ?p ?x)) (bgp (?s ?p ?z)) ))",
+             "(join (filter (= ?x 123) (bgp (?s ?p ?x))) (bgp (?s ?p ?z)) )") ;
+    }
+
+    // Join : two side push
+    @Test public void place_join_02() {
+        test("(filter (= ?x 123) (join (bgp (?s ?p ?x)) (bgp (?s ?p ?x)) ))",
+             "(join  (filter (= ?x 123) (bgp (?s ?p ?x))) (filter (= ?x 123) (bgp (?s ?p ?x))) )") ;
+    }
+
+    @Test public void place_join_03() {
+        String x = StrUtils.strjoinNL
+            ("(filter ((= 13 14) (> ?o1 12) (< ?o 56) (< (+ ?o ?o1) 999))",
+             "   (join", 
+             "      (bgp (triple ?s ?p ?o))" ,
+             "      (bgp (triple ?s ?p1 ?o1))))") ;
+
+        // Everything pushed down once. 
+        String y = StrUtils.strjoinNL
+            ("(filter (< (+ ?o ?o1) 999)",
+             "  (join",
+             "    (filter ((= 13 14) (< ?o 56))", 
+             "      (bgp (triple ?s ?p ?o)))", 
+             "    (filter ((= 13 14) (> ?o1 12))", 
+             "      (bgp (triple ?s ?p1 ?o1)))))") ;
+        // Recursive push in - causes (= 13 14) to go into BGP
+        String y1 = StrUtils.strjoinNL
+            ("(filter (< (+ ?o ?o1) 999)",
+             "  (join",
+             "  (filter (< ?o 56)",
+             "    (sequence",
+             "      (filter (= 13 14)",
+             "        (table unit))",
+             "      (bgp (triple ?s ?p ?o))))",
+             "  (filter (> ?o1 12)",
+             "    (sequence",
+             "      (filter (= 13 14)",
+             "        (table unit))",
+             "      (bgp (triple ?s ?p1 ?o1))))",
+             "   ))") ;
+        test(x, y1) ;
+    }
+
+
+    @Test public void place_conditional_01() {
+        test("(filter (= ?x 123) (conditional (bgp (?s ?p ?x)) (bgp (?s ?p ?z)) ))",
+             "(conditional (filter (= ?x 123) (bgp (?s ?p ?x))) (bgp (?s ?p ?z)) )") ;
+    }
+
+    @Test public void place_conditional_02() {
+        test("(filter (= ?z 123) (conditional (bgp (?s ?p ?x)) (bgp (?s ?p ?z)) ))",
+             "(filter (= ?z 123) (conditional (bgp (?s ?p ?x)) (bgp (?s ?p ?z)) ))") ;
+    }
+
+    @Test public void place_conditional_03() {
+        test("(filter (= ?x 123) (conditional (bgp (?s ?p ?x)) (bgp (?s ?p ?x)) ))",
+             "(conditional (filter (= ?x 123) (bgp (?s ?p ?x))) (bgp (?s ?p ?x)) )") ;
+    }
+
+    @Test public void place_leftjoin_01() {
+        // conditional
+        test("(filter (= ?x 123) (leftjoin (bgp (?s ?p ?x)) (bgp (?s ?p ?z)) ))",
+             "(leftjoin (filter (= ?x 123) (bgp (?s ?p ?x))) (bgp (?s ?p ?z)) )") ;
+    }
+
+    @Test public void place_leftjoin_02() {
+        // conditional
+        test("(filter (= ?z 123) (leftjoin (bgp (?s ?p ?x)) (bgp (?s ?p ?z)) ))",
+             "(filter (= ?z 123) (leftjoin (bgp (?s ?p ?x)) (bgp (?s ?p ?z)) ))") ;
+    }
+
+    @Test public void place_leftjoin_03() {
+        // conditional
+        test("(filter (= ?x 123) (leftjoin (bgp (?s ?p ?x)) (bgp (?s ?p ?x)) ))",
+             "(leftjoin (filter (= ?x 123) (bgp (?s ?p ?x))) (bgp (?s ?p ?x)) )") ;
+    }
+
+    @Test public void place_project_01() {
+        test("(filter (= ?x 123) (project (?x) (bgp (?s ?p ?x)) ))",
+             "(project (?x) (filter (= ?x 123) (bgp (?s ?p ?x)) ))") ;
+    }
+
+    @Test public void place_project_02() {
+        test("(filter (= ?x 123) (project (?s) (bgp (?s ?p ?x)) ))",
+             null) ;
+    }
+    
+    @Test public void place_project_03() {
+        test("(filter (= ?x 123) (project (?x) (bgp (?s ?p ?x) (?s ?p ?z) ) ))",
+             "(project (?x) (sequence (filter (= ?x 123) (bgp (?s ?p ?x)) ) (bgp (?s ?p ?z))) )") ;
+    }
+
+    @Test public void place_extend_01() {
+        test("(filter (= ?x 123) (extend ((?z 123)) (bgp (?s ?p ?x)) ))",
+             "(extend ((?z 123)) (filter (= ?x 123) (bgp (?s ?p ?x)) ))") ;
+    }
+    
+    @Test public void place_extend_02() { // Blocked
+        test("(filter (= ?x 123) (extend ((?x 123)) (bgp (?s ?p ?z)) ))",
+             null) ;
+    }
+
+    @Test public void place_extend_03() {
+        test("(filter (= ?x 123) (extend ((?x1 123)) (filter (< ?x 456) (bgp (?s ?p ?x) (?s ?p ?z))) ))",
+             "(extend (?x1 123) (sequence (filter ((= ?x 123) (< ?x 456)) (bgp (?s ?p ?x))) (bgp (?s ?p ?z))) )") ;
+    }
+
+    @Test public void place_assign_01() {
+        test("(filter (= ?x 123) (assign ((?z 123)) (bgp (?s ?p ?x)) ))",
+             "(assign ((?z 123)) (filter (= ?x 123) (bgp (?s ?p ?x)) ))") ;
+    }
+    
+    @Test public void place_assign_02() { // Blocked
+        test("(filter (= ?x 123) (assign ((?x 123)) (bgp (?s ?p ?z)) ))",
+             null) ;
+    }
+
+    @Test public void place_assign_03() {
+        // Caution - OpFilter equality is sensitive to the order of expressions 
+        test("(filter (= ?x 123) (assign ((?x1 123)) (filter (< ?x 456) (bgp (?s ?p ?x) (?s ?p ?z))) ))",
+             "(assign (?x1 123) (sequence (filter ((= ?x 123) (< ?x 456)) (bgp (?s ?p ?x))) (bgp (?s ?p ?z))) )") ;
+    }
+
+    @Test public void place_filter_01() {
+        test("(filter (= ?x 123) (filter (= ?y 456) (bgp (?s ?p ?x) (?s ?p ?y)) ))" , 
+             "(filter (= ?y 456) (sequence (filter (= ?x 123) (bgp (?s ?p ?x))) (bgp (?s ?p ?y)) ))" ) ;
+    }
+
+    @Test public void place_filter_02() {
+        test("(filter (= ?x 123) (filter (= ?y 456) (bgp (?s ?p ?x) (?s ?p ?y) (?s ?p ?z) )))" , 
+             "(sequence (filter (= ?y 456) (sequence (filter (= ?x 123) (bgp (?s ?p ?x))) (bgp (?s ?p ?y)))) (bgp (?s ?p ?z)))") ;
+    }
+
+    @Test public void place_union_01() {
+        test("(filter (= ?x 123) (union (bgp (?s ?p ?x) (?s ?p ?y)) (bgp (?s ?p ?z)  (?s1 ?p1 ?x)) ))",
+             "(union  (sequence (filter (= ?x 123) (bgp (?s ?p ?x))) (bgp (?s ?p ?y))) "+
+                      "(filter (= ?x 123) (bgp (?s ?p ?z)  (?s1 ?p1 ?x)) ))") ;
+    }
+
+        
+        public static void test(String input, String output) {
+        Transform t_placement = new TransformFilterPlacement() ;
+        Op op1 = SSE.parseOp(input) ;
+        Op op2 = Transformer.transform(t_placement, op1) ;
+        if ( output == null ) {
+            // No transformation.
+            Assert.assertEquals(op1, op2) ;
+            return ;
+        }
+
+        Op op3 = SSE.parseOp(output) ;
+        Assert.assertEquals(op3, op2) ;
+    }
+
+}

Modified: jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformFilters.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformFilters.java?rev=1544468&r1=1544467&r2=1544468&view=diff
==============================================================================
--- jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformFilters.java (original)
+++ jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformFilters.java Fri Nov 22 10:32:57 2013
@@ -19,21 +19,15 @@
 package com.hp.hpl.jena.sparql.algebra.optimize;
 
 import org.apache.jena.atlas.lib.StrUtils ;
-import org.junit.Assert ;
 import org.junit.Test ;
 
-import com.hp.hpl.jena.sparql.algebra.Op ;
 import com.hp.hpl.jena.sparql.algebra.Transform ;
-import com.hp.hpl.jena.sparql.algebra.op.OpBGP ;
-import com.hp.hpl.jena.sparql.algebra.op.OpFilter ;
-import com.hp.hpl.jena.sparql.sse.SSE ;
 
 /** Tests of transforms related to filters */
 public class TestTransformFilters extends AbstractTestTransform
 {
     private Transform t_equality    = new TransformFilterEquality() ;
     private Transform t_disjunction = new TransformFilterDisjunction() ;
-    private Transform t_placement   = new TransformFilterPlacement() ;
     private Transform t_expandOneOf = new TransformExpandOneOf() ;
     private Transform t_implicitJoin = new TransformFilterImplicitJoin() ;
     private Transform t_implicitLeftJoin = new TransformImplicitLeftJoin() ;
@@ -207,11 +201,11 @@ public class TestTransformFilters extend
             , "}") ; 
         
         String ops = StrUtils.strjoinNL
-            ("(filter (= ?test <http://localhost/t1>)"
-            ,"    (sequence"
-            ,"      (bgp (triple ?test ?p1 ?X))"
-            ,"      (project (?s1 ?test)"
-            ,"        (bgp (triple ?test ?/p2 ?/o2)))))"
+            ("(sequence"
+            ,"   (filter (= ?test <http://localhost/t1>)"
+            ,"     (bgp (triple ?test ?p1 ?X)))"
+            ,"   (project (?s1 ?test)"
+            ,"     (bgp (triple ?test ?/p2 ?/o2))))"
             ) ;
         TestOptimizer.check(qs, ops) ;
     }
@@ -424,88 +418,6 @@ public class TestTransformFilters extend
         ) ;
     }
 
-    @Test public void placement01()
-    {
-        testOp("(filter (= ?x 1) (bgp ( ?s ?p ?x)))",
-             t_placement,
-             "(filter (= ?x 1) (bgp ( ?s ?p ?x)))") ;
-        	
-    }
-    
-    @Test public void placement02()
-    {
-        testOp("(filter (= ?x 1) (bgp (?s ?p ?x) (?s1 ?p1 ?x1) ))",
-             t_placement,
-             "(sequence (filter (= ?x 1) (bgp ( ?s ?p ?x))) (bgp (?s1 ?p1 ?x1)))") ;
-            
-    }
-
-    @Test public void placement03()
-    {
-        testOp("(filter (= ?x 1) (bgp (?s ?p ?x) (?s1 ?p1 ?x) ))",
-             t_placement,
-             "(sequence (filter (= ?x 1) (bgp ( ?s ?p ?x))) (bgp (?s1 ?p1 ?x)))") ;
-    }
-
-    @Test public void placement04()
-    {
-        testOp("(filter (= ?XX 1) (bgp (?s ?p ?x) (?s1 ?p1 ?XX) ))",
-             t_placement,
-             "(filter (= ?XX 1) (bgp (?s ?p ?x) (?s1 ?p1 ?XX) ))") ;
-    }
-    
-    @Test public void placement10()
-    {
-        // Unbound
-        testOp("(filter (= ?x ?unbound) (bgp (?s ?p ?x)))",
-             t_placement,
-             "(filter (= ?x ?unbound) (bgp (?s ?p ?x)))") ;
-    }
-    
-    @Test public void placement11()
-    {
-        Op op1 = SSE.parseOp("(filter (= ?x ?unbound) (bgp (?s ?p ?x)))") ;
-        OpFilter f = (OpFilter)op1 ;
-        Op op2 = TransformFilterPlacement.transform(f.getExprs(), ((OpBGP)f.getSubOp()).getPattern()) ;
-        Op op3 = SSE.parseOp("(filter (= ?x ?unbound) (bgp (?s ?p ?x)))") ;
-        Assert.assertEquals(op3, op2) ;
-    }
-
-    @Test public void placement12()
-    {
-        Op op1 = SSE.parseOp("(filter (= ?x ?unbound) (bgp (?s ?p ?x) (?s1 ?p1 ?XX)))") ;
-        OpFilter f = (OpFilter)op1 ;
-        Op op2 = TransformFilterPlacement.transform(f.getExprs(), ((OpBGP)f.getSubOp()).getPattern()) ;
-        Op op3 = SSE.parseOp("(filter (= ?x ?unbound) (bgp (?s ?p ?x) (?s1 ?p1 ?XX)))") ;
-        Assert.assertEquals(op3, op2) ;
-    }
-    
-    
-    @Test public void placement20()
-    {
-        // conditional
-        testOp("(filter (= ?x 123) (conditional (bgp (?s ?p ?x)) (bgp (?s ?p ?z)) ))",
-             t_placement,
-             "(conditional (filter (= ?x 123) (bgp (?s ?p ?x))) (bgp (?s ?p ?z)) )") ;
-    }
-
-    @Test public void placement21()
-    {
-        // conditional
-        testOp("(filter (= ?z 123) (conditional (bgp (?s ?p ?x)) (bgp (?s ?p ?z)) ))",
-             t_placement,
-             "(filter (= ?z 123) (conditional (bgp (?s ?p ?x)) (bgp (?s ?p ?z)) ))") ;
-    }
-
-    @Test public void placement22()
-    {
-        // conditional
-        testOp("(filter (= ?x 123) (conditional (bgp (?s ?p ?x)) (bgp (?s ?p ?x)) ))",
-             t_placement,
-             "(conditional (filter (= ?x 123) (bgp (?s ?p ?x))) (bgp (?s ?p ?x)) )") ;
-    }
-
-    
     @Test public void oneOf1()
     {
         testOp(