You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jena.apache.org by rv...@apache.org on 2013/06/20 23:49:37 UTC

svn commit: r1495205 - in /jena/trunk/jena-arq/src: main/java/com/hp/hpl/jena/query/ 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...

Author: rvesse
Date: Thu Jun 20 21:49:37 2013
New Revision: 1495205

URL: http://svn.apache.org/r1495205
Log:
Initial implementation of implicit join and implicit left join optimizers (JENA-473)

While there are some test cases for these the test coverage is not yet great and experimentation has shown that there
are some kinks to be worked out.  Currently these optimizations are only used if explicitly enabled
i.e. they use context.isTrue() rather than context.isTrueOrUndef() to determine whether they should be applied

Added:
    jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterImplicitJoin.java
    jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformImplicitLeftJoin.java
    jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestSemanticEquivalence.java
Modified:
    jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/query/ARQ.java
    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/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/query/ARQ.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/query/ARQ.java?rev=1495205&r1=1495204&r2=1495205&view=diff
==============================================================================
--- jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/query/ARQ.java (original)
+++ jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/query/ARQ.java Thu Jun 20 21:49:37 2013
@@ -308,6 +308,18 @@ public class ARQ
      *  there is a potential risk of changing query semantics. 
      */  
     public static final Symbol optFilterEquality = ARQConstants.allocSymbol("optFilterEquality") ;
+    
+    /**
+     * Context key controlling whether the standard optimizer applies optimizations to implicit joins in FILTERs.
+     * This optimization is conservative - it does not take place if there is a potential risk of changing query semantics.
+     */
+    public static final Symbol optFilterImplicitJoin = ARQConstants.allocSymbol("optFilterImplicitJoin");
+    
+    /**
+     * Context key controlling whether the standard optimizer applies optimizations to implicit left joins.
+     * This optimization is conservative - it does not take place if there is a potential risk of changing query semantics.
+     */
+    public static final Symbol optImplicitLeftJoin = ARQConstants.allocSymbol("optImplicitLeftJoin");
 
     /** 
      *  Context key for a declaration that xsd:strings and simple literals are

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=1495205&r1=1495204&r2=1495205&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 Thu Jun 20 21:49:37 2013
@@ -18,11 +18,15 @@
 
 package com.hp.hpl.jena.sparql.algebra ;
 
+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 org.apache.jena.atlas.lib.Tuple;
+
 import com.hp.hpl.jena.graph.Node ;
 import com.hp.hpl.jena.graph.Triple ;
 import com.hp.hpl.jena.query.SortCondition ;
@@ -59,6 +63,30 @@ public class OpVars
         OpVarsPattern visitor = new OpVarsPattern(acc, true) ;
         OpWalker.walk(new WalkerVisitorVisible(visitor, acc), op) ;
     }
+    
+    public static Tuple<Set<Var>> mentionedVarsByPosition(Op op) {
+        Set<Var> graphAcc = collector() ;
+        Set<Var> subjAcc = collector() ;
+        Set<Var> predAcc = collector() ;
+        Set<Var> objAcc = collector() ;
+        Set<Var> unknownAcc = collector() ;
+        OpVarsPatternWithPositions visitor = new OpVarsPatternWithPositions(graphAcc, subjAcc, predAcc, objAcc, unknownAcc, false);
+        OpWalker.walk(op, visitor);
+        return Tuple.create(graphAcc, subjAcc, predAcc, objAcc, unknownAcc);
+    }
+    
+    public static Tuple<Set<Var>> mentionedVarsByPosition(Op... ops) {
+        Set<Var> graphAcc = collector() ;
+        Set<Var> subjAcc = collector() ;
+        Set<Var> predAcc = collector() ;
+        Set<Var> objAcc = collector() ;
+        Set<Var> unknownAcc = collector() ;
+        OpVarsPatternWithPositions visitor = new OpVarsPatternWithPositions(graphAcc, subjAcc, predAcc, objAcc, unknownAcc, false);
+        for (Op op : ops) {
+            OpWalker.walk(op, visitor);
+        }
+        return Tuple.create(graphAcc, subjAcc, predAcc, objAcc, unknownAcc);
+    }
 
     // All mentioned variables regardless of scope/visibility.
     public static Collection<Var> mentionedVars(Op op) {
@@ -84,7 +112,7 @@ public class OpVars
             addVarsFromTriple(acc, triple) ;
     }
 
-    /** Do project and don't walk into it. MINUS vars aren't visiible either */
+    /** Do project and don't walk into it. MINUS vars aren't visible either */
     private static class WalkerVisitorVisible extends WalkerVisitor
     {
         private final Collection<Var> acc ;
@@ -208,6 +236,134 @@ public class OpVars
         }
 
     }
+    
+    private static class OpVarsPatternWithPositions extends OpVisitorBase
+    {
+        // The possibly-set-vars
+        protected Set<Var> graphAcc, subjAcc, predAcc, objAcc, unknownAcc ;
+        final boolean      visibleOnly ;
+
+        OpVarsPatternWithPositions(Set<Var> graphAcc, Set<Var> subjAcc, Set<Var> predAcc, Set<Var> objAcc, Set<Var> unknownAcc, boolean visibleOnly) {
+            this.graphAcc = graphAcc;
+            this.subjAcc = subjAcc;
+            this.predAcc = predAcc;
+            this.objAcc = objAcc;
+            this.unknownAcc = unknownAcc;
+            this.visibleOnly = visibleOnly ;
+        }
+
+        @Override
+        public void visit(OpBGP opBGP) {
+            vars(opBGP.getPattern()) ;
+        }
+
+        @Override
+        public void visit(OpPath opPath) {
+            addVar(subjAcc, opPath.getTriplePath().getSubject()) ;
+            addVar(objAcc, opPath.getTriplePath().getObject()) ;
+        }
+
+        @Override
+        public void visit(OpQuadPattern quadPattern) {
+            addVar(graphAcc, quadPattern.getGraphNode()) ;
+            vars(quadPattern.getBasicPattern()) ;
+        }
+
+        @Override
+        public void visit(OpGraph opGraph) {
+            addVar(graphAcc, opGraph.getNode()) ;
+        }
+
+        @Override
+        public void visit(OpDatasetNames dsNames) {
+            addVar(graphAcc, dsNames.getGraphNode()) ;
+        }
+
+        @Override
+        public void visit(OpTable opTable) {
+            // Only the variables with values in the tables
+            // (When building, undefs didn't get into bindings so no variable
+            // mentioned)
+            Table t = opTable.getTable() ;
+            // Treat as unknown position
+            unknownAcc.addAll(t.getVars()) ;
+        }
+
+        @Override
+        public void visit(OpProject opProject) {
+            // The walker (WalkerVisitorVisible) handles this
+            // for visible variables, not mentioned variable collecting.
+            // The visibleOnly/clear is simply to be as general as possible.
+            List<Var> vs = opProject.getVars();
+            if (visibleOnly) {
+                clear(graphAcc, vs);
+                clear(subjAcc, vs);
+                clear(predAcc, vs);
+                clear(objAcc, vs);
+                
+            }
+            for (Var v : vs) {
+                if (!graphAcc.contains(v) && !subjAcc.contains(v) && !predAcc.contains(v) && !objAcc.contains(v)) {
+                    addVar(unknownAcc, v);
+                }
+            }
+        }
+
+        @Override
+        public void visit(OpAssign opAssign) {
+            // Unknown position
+            unknownAcc.addAll(opAssign.getVarExprList().getVars()) ;
+        }
+
+        @Override
+        public void visit(OpExtend opExtend) {
+            // Unknown position
+            unknownAcc.addAll(opExtend.getVarExprList().getVars()) ;
+        }
+
+        @Override
+        public void visit(OpPropFunc opPropFunc) {
+            addvars(subjAcc, opPropFunc.getSubjectArgs()) ;
+            addvars(objAcc, opPropFunc.getObjectArgs()) ;
+        }
+
+        private void addvars(Set<Var> acc, PropFuncArg pfArg) {
+            if (pfArg.isNode()) {
+                addVar(acc, pfArg.getArg()) ;
+                return ;
+            }
+            for (Node n : pfArg.getArgList())
+                addVar(acc, n) ;
+        }
+
+        @Override
+        public void visit(OpProcedure opProc) {
+            unknownAcc.addAll(OpVars.mentionedVars(opProc));
+        }
+        
+        private void vars(BasicPattern bp) {
+            for (Triple t : bp.getList())
+            {
+                addVar(subjAcc, t.getSubject());
+                addVar(predAcc, t.getPredicate());
+                addVar(objAcc, t.getObject());
+            }
+        }
+        
+        private void clear(Set<Var> acc, List<Var> visible) {
+            List<Var> toRemove = new ArrayList<Var>();
+            for (Var found : acc)
+            {
+                if (!visible.contains(found)) {
+                    toRemove.add(found);
+                }
+            }
+            for (Var v : toRemove) {
+                acc.remove(v);
+            }
+        }
+
+    }
 
     private static class OpVarsMentioned extends OpVarsPattern
     {
@@ -236,6 +392,7 @@ public class OpVars
         addVar(acc, t.getObject()) ;
     }
 
+    @SuppressWarnings("unused")
     private static void addVarsFromQuad(Collection<Var> acc, Quad q) {
         addVar(acc, q.getSubject()) ;
         addVar(acc, q.getPredicate()) ;

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=1495205&r1=1495204&r2=1495205&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 Thu Jun 20 21:49:37 2013
@@ -171,6 +171,14 @@ public class Optimize implements Rewrite
         // Either, do filter placement and other sequence generating transformations.
         // or improve to place in a sequence (latter is better?)
         
+        // Currently implicit join optimizations must be explicitly enabled
+        
+        if ( context.isTrue(ARQ.optFilterImplicitJoin) )
+            op = apply("Filter Implicit Join", new TransformFilterImplicitJoin(), op);
+        
+        if ( context.isTrue(ARQ.optImplicitLeftJoin) )
+            op = apply("Implicit Left Join", new TransformImplicitLeftJoin(), op);
+        
         if ( context.isTrueOrUndef(ARQ.optFilterEquality) )
         {
             //boolean termStrings = context.isDefined(ARQ.optTermStrings) ;

Added: jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterImplicitJoin.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterImplicitJoin.java?rev=1495205&view=auto
==============================================================================
--- jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterImplicitJoin.java (added)
+++ jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterImplicitJoin.java Thu Jun 20 21:49:37 2013
@@ -0,0 +1,414 @@
+/*
+ * 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 static org.apache.jena.atlas.lib.CollectionUtils.disjoint;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.jena.atlas.lib.Pair;
+import org.apache.jena.atlas.lib.Tuple;
+
+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.*;
+import com.hp.hpl.jena.sparql.core.Substitute;
+import com.hp.hpl.jena.sparql.core.Var;
+import com.hp.hpl.jena.sparql.core.VarExprList;
+import com.hp.hpl.jena.sparql.expr.*;
+
+/**
+ * <p>
+ * Optimizer for transforming implicit joins. These covers queries like the
+ * following:
+ * </p>
+ * 
+ * <pre>
+ * SELECT *
+ * WHERE
+ * {
+ *   ?s a ?type1 .
+ *   ?t a ?type2 .
+ *   FILTER(?s = ?t)
+ * }
+ * </pre>
+ * <p>
+ * Clearly this is a trivial example but doing this optimization can have big
+ * performance gains since it can completely eliminate cross products that we
+ * would otherwise be required to evaluate. The optimization where applicable
+ * results in a query of the following form:
+ * </p>
+ * 
+ * <pre>
+ * SELECT *
+ * WHERE
+ * {
+ *   ?s a ?type1 .
+ *   ?s a ?type1 .
+ *   BIND(?s AS ?t)
+ * }
+ * </pre>
+ * <p>
+ * The optimizer does not cover the implicit left join case, for that see {@link TransformImplicitLeftJoin}
+ * </p>
+ * <h3>Applicability</h3>
+ * <p>
+ * This optimization aims to eliminate implicit joins of the form
+ * {@code ?x = ?y} or {@code SAMETERM(?x, ?y)}, the latter can almost always be
+ * safely eliminated while the former may only be eliminated in the case where
+ * we can guarantee that at least one of the variables is a non-literal e.g. it
+ * occurs in the subject/predicate position. In the case where this is not true
+ * the optimization may not be made since we cannot assume that we can map value
+ * equality to term equality by making the optimization.
+ * </p>
+ * <h3>Known Limitations/To Do</h3>
+ * <ul>
+ * <li>Application to implicit joins may block the sequence transform which means the potential benefits of the optimization are negated</li>
+ * </ul>
+ */
+public class TransformFilterImplicitJoin extends TransformCopy {
+
+    @Override
+    public Op transform(OpFilter opFilter, Op subOp) {
+        Op op = apply(opFilter.getExprs(), subOp);
+        if (op == null)
+            return super.transform(opFilter, subOp);
+        return op;
+    }
+
+    private static Op apply(ExprList exprs, Op subOp) {
+        // ---- Find and extract any implicit join filters.
+        Pair<List<Pair<Var, Var>>, ExprList> p = preprocessFilterImplicitJoin(subOp, exprs);
+        if (p == null || p.getLeft().size() == 0)
+            return null;
+
+        List<Pair<Var, Var>> joins = p.getLeft();
+        Collection<Var> varsMentioned = varsMentionedInImplictJoins(joins);
+        ExprList remaining = p.getRight();
+
+        // ---- Check if the subOp is the right shape to transform.
+        Op op = subOp;
+
+        // Special case : deduce that the filter will always "eval unbound"
+        // hence eliminate all rows. Return the empty table.
+        if (testSpecialCaseUnused(subOp, joins, remaining))
+            return OpTable.empty();
+
+        // Special case: the deep left op of a OpConditional/OpLeftJoin is unit
+        // table.
+        // This is { OPTIONAL{P1} OPTIONAL{P2} ... FILTER(?x = :x) }
+        if (testSpecialCase1(subOp, joins, remaining)) {
+            // Find backbone of ops
+            List<Op> ops = extractOptionals(subOp);
+            ops = processSpecialCase1(ops, joins);
+            // Put back together
+            op = rebuild((Op2) subOp, ops);
+            // Put all filters - either we optimized, or we left alone.
+            // Either way, the complete set of filter expressions.
+            op = OpFilter.filter(exprs, op);
+            return op;
+        }
+
+        // ---- Transform
+
+        if (!safeToTransform(joins, varsMentioned, op))
+            return null;
+        for (Pair<Var, Var> implicitJoin : joins)
+            op = processFilterWorker(op, implicitJoin.getLeft(), implicitJoin.getRight());
+
+        // ---- Place any filter expressions around the processed sub op.
+        if (remaining.size() > 0)
+            op = OpFilter.filter(remaining, op);
+        return op;
+    }
+
+    // --- find and extract
+    private static Pair<List<Pair<Var, Var>>, ExprList> preprocessFilterImplicitJoin(Op subOp, ExprList exprs) {
+        List<Pair<Var, Var>> exprsJoins = new ArrayList<Pair<Var, Var>>();
+        ExprList exprsOther = new ExprList();
+        for (Expr e : exprs.getList()) {
+            Pair<Var, Var> p = preprocess(subOp, e);
+            if (p != null) {
+                exprsJoins.add(p);
+            } else {
+                exprsOther.add(e);
+            }
+        }
+        if (exprsJoins.size() == 0)
+            return null;
+        return Pair.create(exprsJoins, exprsOther);
+    }
+
+    private static Pair<Var, Var> preprocess(Op subOp, Expr e) {
+        if (!(e instanceof E_Equals) && !(e instanceof E_SameTerm))
+            return null;
+
+        ExprFunction2 eq = (ExprFunction2) e;
+        Expr left = eq.getArg1();
+        Expr right = eq.getArg2();
+
+        if (!left.isVariable() || !right.isVariable()) {
+            return null;
+        }
+        if (left.equals(right)) {
+            return null;
+        }
+
+        if (e instanceof E_Equals) {
+            // Is a safe equals for this optimization?
+            Tuple<Set<Var>> varsByPosition = OpVars.mentionedVarsByPosition(subOp);
+
+            if (!isSafeEquals(varsByPosition, left.asVar(), right.asVar()))
+                return null;
+        }
+
+        return Pair.create(left.asVar(), right.asVar());
+    }
+
+    private static boolean isSafeEquals(Tuple<Set<Var>> varsByPosition, Var left, Var right) {
+        // For equality based joins ensure at least one variable must be
+        // used in graph/subject/predicate position thus guaranteeing it to
+        // not be a literal so replacing with term equality by ways of
+        // substitution will be safe
+
+        // Should get 5 sets
+        if (varsByPosition.size() != 5)
+            return false;
+
+        // If anything is used in the object/unknown position then we
+        // potentially have an issue unless it is also used in a safe
+        // position
+        Set<Var> safeVars = new HashSet<Var>();
+        safeVars.addAll(varsByPosition.get(0));
+        safeVars.addAll(varsByPosition.get(1));
+        safeVars.addAll(varsByPosition.get(2));
+        Set<Var> unsafeVars = new HashSet<Var>();
+        unsafeVars.addAll(varsByPosition.get(3));
+        unsafeVars.addAll(varsByPosition.get(4));
+        boolean lhsSafe = true, rhsSafe = true;
+        if (unsafeVars.size() > 0) {
+            if (unsafeVars.contains(left)) {
+                // LHS Variable occurred in unsafe position
+                if (!safeVars.contains(left)) {
+                    // LHS Variable is unsafe
+                    lhsSafe = false;
+                }
+            }
+            if (unsafeVars.contains(right)) {
+                // RHS Variable occurred in unsafe position
+                if (!safeVars.contains(right)) {
+                    // RHS Variable is unsafe
+                    rhsSafe = false;
+                }
+            }
+        }
+
+        // At least one variable must be safe or this equality expression is
+        // not an implicit join that can be safely optimized
+        return lhsSafe || rhsSafe;
+    }
+
+    private static Collection<Var> varsMentionedInImplictJoins(List<Pair<Var, Var>> joins) {
+        Set<Var> vars = new HashSet<Var>();
+        for (Pair<Var, Var> p : joins) {
+            vars.add(p.getLeft());
+            vars.add(p.getRight());
+        }
+        return vars;
+    }
+
+    private static boolean safeToTransform(List<Pair<Var, Var>> joins, Collection<Var> varsEquality, Op op) {
+        // Structure as a visitor?
+        if (op instanceof OpBGP || op instanceof OpQuadPattern)
+            return true;
+
+        if (op instanceof OpFilter) {
+            OpFilter opf = (OpFilter) op;
+            return safeToTransform(joins, varsEquality, opf.getSubOp());
+        }
+
+        // This will be applied also in sub-calls of the Transform but queries
+        // are very rarely so deep that it matters.
+        if (op instanceof OpSequence) {
+            OpN opN = (OpN) op;
+            for (Op subOp : opN.getElements()) {
+                if (!safeToTransform(joins, varsEquality, subOp))
+                    return false;
+            }
+            return true;
+        }
+
+        if (op instanceof OpJoin || op instanceof OpUnion) {
+            Op2 op2 = (Op2) op;
+            return safeToTransform(joins, varsEquality, op2.getLeft()) && safeToTransform(joins, varsEquality, op2.getRight());
+        }
+
+        // Not safe unless filter variables are mentioned on the LHS.
+        if (op instanceof OpConditional || op instanceof OpLeftJoin) {
+            Op2 opleftjoin = (Op2) op;
+
+            if (!safeToTransform(joins, varsEquality, opleftjoin.getLeft())
+                    || !safeToTransform(joins, varsEquality, opleftjoin.getRight()))
+                return false;
+
+            // Not only must the left and right be safe to transform,
+            // but the equality variable must be known to be always set.
+
+            // If the varsLeft are disjoint from assigned vars,
+            // we may be able to push assign down right
+            // (this generalises the unit table case specialcase1)
+            // Needs more investigation.
+
+            Op opLeft = opleftjoin.getLeft();
+            Set<Var> varsLeft = OpVars.visibleVars(opLeft);
+            if (varsLeft.containsAll(varsEquality))
+                return true;
+            return false;
+        }
+
+        if (op instanceof OpGraph) {
+            OpGraph opg = (OpGraph) op;
+            return safeToTransform(joins, varsEquality, opg.getSubOp());
+        }
+
+        // Subquery - assume scope rewriting has already been applied.
+        if (op instanceof OpModifier) {
+            // ORDER BY?
+            OpModifier opMod = (OpModifier) op;
+            if (opMod instanceof OpProject) {
+                OpProject opProject = (OpProject) op;
+                // Writing "SELECT ?var" for "?var" -> a value would need
+                // AS-ification.
+                for (Var v : opProject.getVars()) {
+                    if (varsEquality.contains(v))
+                        return false;
+                }
+            }
+            return safeToTransform(joins, varsEquality, opMod.getSubOp());
+        }
+
+        if (op instanceof OpGroup) {
+            OpGroup opGroup = (OpGroup) op;
+            VarExprList varExprList = opGroup.getGroupVars();
+            return safeToTransform(varsEquality, varExprList) && safeToTransform(joins, varsEquality, opGroup.getSubOp());
+        }
+
+        if (op instanceof OpTable) {
+            OpTable opTable = (OpTable) op;
+            if (opTable.isJoinIdentity())
+                return true;
+        }
+
+        // Op1 - OpGroup
+        // Op1 - OpOrder
+        // Op1 - OpAssign, OpExtend
+        // Op1 - OpFilter - done.
+        // Op1 - OpLabel - easy
+        // Op1 - OpService - no.
+
+        return false;
+    }
+
+    private static boolean safeToTransform(Collection<Var> varsEquality, VarExprList varsExprList) {
+        // If the named variable is used, unsafe to rewrite.
+        return disjoint(varsExprList.getVars(), varsEquality);
+    }
+
+    // -- A special case
+
+    private static boolean testSpecialCaseUnused(Op op, List<Pair<Var, Var>> joins, ExprList remaining) {
+        // If the op does not contain one of the vars at all, then the implicit
+        // join will be "eval unbound" i.e. false.
+        // We can return empty table.
+        Set<Var> patternVars = OpVars.visibleVars(op);
+        for (Pair<Var, Var> p : joins) {
+            if (!patternVars.contains(p.getLeft()) || !patternVars.contains(p.getRight()))
+                return true;
+        }
+        return false;
+    }
+
+    // If a sequence of OPTIONALS, and nothing prior to the first, we end up
+    // with a unit table on the left side of a next of LeftJoin/conditionals.
+
+    private static boolean testSpecialCase1(Op op, List<Pair<Var, Var>> joins, ExprList remaining) {
+        while (op instanceof OpConditional || op instanceof OpLeftJoin) {
+            Op2 opleftjoin2 = (Op2) op;
+            op = opleftjoin2.getLeft();
+        }
+        return isTableUnit(op);
+    }
+
+    private static List<Op> extractOptionals(Op op) {
+        List<Op> chain = new ArrayList<Op>();
+        while (op instanceof OpConditional || op instanceof OpLeftJoin) {
+            Op2 opleftjoin2 = (Op2) op;
+            chain.add(opleftjoin2.getRight());
+            op = opleftjoin2.getLeft();
+        }
+        return chain;
+    }
+
+    private static List<Op> processSpecialCase1(List<Op> ops, List<Pair<Var, Var>> joins) {
+        List<Op> ops2 = new ArrayList<Op>();
+        Collection<Var> vars = varsMentionedInImplictJoins(joins);
+
+        for (Op op : ops) {
+            Op op2 = op;
+            if (safeToTransform(joins, vars, op)) {
+                for (Pair<Var, Var> p : joins)
+                    op2 = processFilterWorker(op, p.getLeft(), p.getRight());
+            }
+            ops2.add(op2);
+        }
+        return ops2;
+    }
+
+    private static Op rebuild(Op2 subOp, List<Op> ops) {
+        Op chain = OpTable.unit();
+        for (Op op : ops) {
+            chain = subOp.copy(chain, op);
+        }
+        return chain;
+    }
+
+    private static boolean isTableUnit(Op op) {
+        if (op instanceof OpTable) {
+            if (((OpTable) op).isJoinIdentity())
+                return true;
+        }
+        return false;
+    }
+
+    // ---- Transformation
+
+    private static Op processFilterWorker(Op op, Var find, Var replace) {
+        return subst(op, find, replace);
+    }
+
+    private static Op subst(Op subOp, Var find, Var replace) {
+        Op op = Substitute.substitute(subOp, find, replace.asNode());
+        return OpAssign.assign(op, find, new ExprVar(replace));
+    }
+}

Added: jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformImplicitLeftJoin.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformImplicitLeftJoin.java?rev=1495205&view=auto
==============================================================================
--- jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformImplicitLeftJoin.java (added)
+++ jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformImplicitLeftJoin.java Thu Jun 20 21:49:37 2013
@@ -0,0 +1,417 @@
+/*
+ * 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 static org.apache.jena.atlas.lib.CollectionUtils.disjoint;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.jena.atlas.lib.Pair;
+import org.apache.jena.atlas.lib.Tuple;
+
+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.*;
+import com.hp.hpl.jena.sparql.core.Substitute;
+import com.hp.hpl.jena.sparql.core.Var;
+import com.hp.hpl.jena.sparql.core.VarExprList;
+import com.hp.hpl.jena.sparql.expr.*;
+
+/**
+ * <p>
+ * Optimizer for transforming implicit joins. These covers queries like the
+ * following:
+ * </p>
+ * 
+ * <pre>
+ * SELECT *
+ * WHERE
+ * {
+ *   ?s a ?type1 .
+ *   OPTIONAL
+ *   {
+ *     ?t a ?type2 .
+ *     FILTER(?s = ?t)
+ *   }
+ * }
+ * </pre>
+ * <p>
+ * Clearly this is a trivial example but doing this optimization can have big
+ * performance gains since it can completely eliminate cross products that we
+ * would otherwise be required to evaluate. The optimization where applicable
+ * results in a query of the following form:
+ * </p>
+ * 
+ * <pre>
+ * SELECT *
+ * WHERE
+ * {
+ *   ?s a ?type1 .
+ *   OPTIONAL
+ *   {
+ *     ?s a ?type1 .
+ *     BIND(?s AS ?t)
+ *   }
+ * }
+ * </pre>
+ * <p>
+ * This does not handle the simpler case of implicit joins where
+ * {@code OPTIONAL} is not involved, for that see
+ * {@link TransformFilterImplicitJoin}
+ * </p>
+ * <h3>Applicability</h3>
+ * <p>
+ * This optimization aims to eliminate implicit left joins of the form
+ * {@code ?x = ?y} or {@code SAMETERM(?x, ?y)}, the latter can almost always be
+ * safely eliminated while the former may only be eliminated in the case where
+ * we can guarantee that at least one of the variables is a non-literal e.g. it
+ * occurs in the subject/predicate position. In the case where this is not true
+ * the optimization may not be made since we cannot assume that we can map value
+ * equality to term equality by making the optimization.
+ * </p>
+ * <h3>Known Limitations/To Do</h3>
+ * <p>
+ * Currently there are several issues with enabling this optimizer:
+ * <p>
+ * <ul>
+ * <li>Application to implicit left joins blocks the conditional transform which
+ * means the potential benefits of the optimization are negated</li>
+ * </ul>
+ */
+public class TransformImplicitLeftJoin extends TransformCopy {
+
+    @Override
+    public Op transform(OpLeftJoin opLeftJoin, Op left, Op right) {
+        // Must have associated expressions to be eligible
+        if (opLeftJoin.getExprs() == null)
+            return super.transform(opLeftJoin, left, right);
+
+        // Try and apply this optimization
+        Op op = apply(opLeftJoin, left, right);
+        if (op == null)
+            return super.transform(opLeftJoin, left, right);
+        return op;
+    }
+
+    private static Op apply(OpLeftJoin opLeftJoin, Op left, Op right) {
+        Pair<List<Pair<Var, Var>>, ExprList> p = preprocessFilterImplicitJoin(left, right, opLeftJoin.getExprs());
+        if (p == null || p.getLeft().size() == 0)
+            return null;
+
+        List<Pair<Var, Var>> joins = p.getLeft();
+        Collection<Var> varsMentioned = varsMentionedInImplictJoins(joins);
+        ExprList remaining = p.getRight();
+
+        // ---- Check if the subOp is the right shape to transform.
+        Op op = right;
+        if (!safeToTransform(joins, varsMentioned, op))
+            return null;
+
+        // We apply substitution only to the RHS
+        // This is because applying to both sides could change the join
+        // semantics of the Left Join
+        Collection<Var> lhsVars = OpVars.visibleVars(left);
+        Collection<Var> rhsVars = OpVars.visibleVars(right);
+        for (Pair<Var, Var> implicitJoin : joins) {
+            // Which variable do we want to substitute out?
+            // We don't need to deal with the case of neither variable being on
+            // the RHS
+            Var lVar = implicitJoin.getLeft();
+            Var rVar = implicitJoin.getRight();
+
+            if (lhsVars.contains(lVar) && lhsVars.contains(rVar)) {
+                // Both vars are on LHS
+
+                if (rhsVars.contains(lVar) && rhsVars.contains(rVar)) {
+                    // Both vars are also on RHS
+                    // Order of substitution doesn't matter
+                    op = processFilterWorker(op, lVar, rVar);
+                } else if (rhsVars.contains(lVar)) {
+                    // Substitute left variable for right variable
+                    op = processFilterWorker(op, lVar, rVar);
+                } else {
+                    // Substitute right variable for left variable
+                    op = processFilterWorker(op, rVar, lVar);
+                }
+            } else if (lhsVars.contains(lVar)) {
+                // Only left variable on RHS
+
+                if (rhsVars.contains(rVar)) {
+                    // Substitute right variable for left variable
+                    op = processFilterWorker(op, rVar, lVar);
+                } else {
+                    // May be hit if trying to apply a sequence of substitutions
+                    return null;
+                }
+            } else if (lhsVars.contains(rVar)) {
+                // Only right variable on LHS
+
+                if (rhsVars.contains(lVar)) {
+                    // Substitute left variable for right variable
+                    op = processFilterWorker(op, lVar, rVar);
+                } else {
+                    // May be hit if trying to apply a sequence of substitutions
+                    return null;
+                }
+            } else {
+                // May be hit if trying to apply a sequence of substitutions
+                return null;
+            }
+
+            // Re-compute visible RHS vars after each substitution as it may
+            // affect subsequent substitutions
+            rhsVars = OpVars.visibleVars(op);
+        }
+
+        return OpLeftJoin.create(left, op, remaining);
+    }
+
+    private static Pair<List<Pair<Var, Var>>, ExprList> preprocessFilterImplicitJoin(Op left, Op right, ExprList exprs) {
+        List<Pair<Var, Var>> exprsJoins = new ArrayList<Pair<Var, Var>>();
+        ExprList exprsOther = new ExprList();
+        for (Expr e : exprs.getList()) {
+            Pair<Var, Var> p = preprocess(left, right, e, exprsOther);
+            if (p != null) {
+                exprsJoins.add(p);
+            } else {
+                exprsOther.add(e);
+            }
+        }
+        if (exprsJoins.size() == 0)
+            return null;
+        return Pair.create(exprsJoins, exprsOther);
+    }
+
+    private static Pair<Var, Var> preprocess(Op opLeft, Op opRight, Expr e, ExprList exprsOther) {
+        if (!(e instanceof E_Equals) && !(e instanceof E_SameTerm) && !(e instanceof E_LogicalAnd))
+            return null;
+
+        ExprFunction2 eq = (ExprFunction2) e;
+        if (e instanceof E_LogicalAnd) {
+            // Is LHS of the && an implicit join?
+            if (eq.getArg1() instanceof E_Equals || eq.getArg1() instanceof E_SameTerm) {
+                Pair<Var, Var> p = preprocess(opLeft, opRight, eq.getArg1(), exprsOther);
+                if (p != null) {
+                    exprsOther.add(eq.getArg2());
+                    return p;
+                }
+            }
+            // Is RHS of the && an implicit join?
+            if (eq.getArg2() instanceof E_Equals || eq.getArg2() instanceof E_SameTerm) {
+                Pair<Var, Var> p = preprocess(opLeft, opRight, eq.getArg2(), exprsOther);
+                if (p != null) {
+                    exprsOther.add(eq.getArg1());
+                    return p;
+                }
+            }
+            return null;
+        } else {
+            // An equals or same term implicit join
+            Expr left = eq.getArg1();
+            Expr right = eq.getArg2();
+
+            if (!left.isVariable() || !right.isVariable()) {
+                return null;
+            }
+            if (left.equals(right)) {
+                return null;
+            }
+
+            // If neither variable is visible in RHS optimization does not apply
+            Collection<Var> rhsVars = OpVars.visibleVars(opRight);
+            if (!rhsVars.contains(left.asVar()) && !rhsVars.contains(right.asVar()))
+                return null;
+
+            if (e instanceof E_Equals) {
+                // Is a safe equals for this optimization?
+                Tuple<Set<Var>> varsByPosition = OpVars.mentionedVarsByPosition(opLeft, opRight);
+
+                if (!isSafeEquals(varsByPosition, left.asVar(), right.asVar()))
+                    return null;
+            }
+
+            return Pair.create(left.asVar(), right.asVar());
+        }
+    }
+
+    private static boolean isSafeEquals(Tuple<Set<Var>> varsByPosition, Var left, Var right) {
+        // For equality based joins ensure at least one variable must be
+        // used in graph/subject/predicate position thus guaranteeing it to
+        // not be a literal so replacing with term equality by ways of
+        // substitution will be safe
+
+        // Should get 5 sets
+        if (varsByPosition.size() != 5)
+            return false;
+
+        // If anything is used in the object/unknown position then we
+        // potentially have an issue unless it is also used in a safe
+        // position
+        Set<Var> safeVars = new HashSet<Var>();
+        safeVars.addAll(varsByPosition.get(0));
+        safeVars.addAll(varsByPosition.get(1));
+        safeVars.addAll(varsByPosition.get(2));
+        Set<Var> unsafeVars = new HashSet<Var>();
+        unsafeVars.addAll(varsByPosition.get(3));
+        unsafeVars.addAll(varsByPosition.get(4));
+        boolean lhsSafe = true, rhsSafe = true;
+        if (unsafeVars.size() > 0) {
+            if (unsafeVars.contains(left)) {
+                // LHS Variable occurred in unsafe position
+                if (!safeVars.contains(left)) {
+                    // LHS Variable is unsafe
+                    lhsSafe = false;
+                }
+            }
+            if (unsafeVars.contains(right)) {
+                // RHS Variable occurred in unsafe position
+                if (!safeVars.contains(right)) {
+                    // RHS Variable is unsafe
+                    rhsSafe = false;
+                }
+            }
+        }
+
+        // At least one variable must be safe or this equality expression is
+        // not an implicit join that can be safely optimized
+        return lhsSafe || rhsSafe;
+    }
+
+    private static Collection<Var> varsMentionedInImplictJoins(List<Pair<Var, Var>> joins) {
+        Set<Var> vars = new HashSet<Var>();
+        for (Pair<Var, Var> p : joins) {
+            vars.add(p.getLeft());
+            vars.add(p.getRight());
+        }
+        return vars;
+    }
+
+    private static boolean safeToTransform(List<Pair<Var, Var>> joins, Collection<Var> varsEquality, Op op) {
+        // Structure as a visitor?
+        if (op instanceof OpBGP || op instanceof OpQuadPattern)
+            return true;
+
+        if (op instanceof OpFilter) {
+            OpFilter opf = (OpFilter) op;
+            return safeToTransform(joins, varsEquality, opf.getSubOp());
+        }
+
+        // This will be applied also in sub-calls of the Transform but queries
+        // are very rarely so deep that it matters.
+        if (op instanceof OpSequence) {
+            OpN opN = (OpN) op;
+            for (Op subOp : opN.getElements()) {
+                if (!safeToTransform(joins, varsEquality, subOp))
+                    return false;
+            }
+            return true;
+        }
+
+        if (op instanceof OpJoin || op instanceof OpUnion) {
+            Op2 op2 = (Op2) op;
+            return safeToTransform(joins, varsEquality, op2.getLeft()) && safeToTransform(joins, varsEquality, op2.getRight());
+        }
+
+        // Not safe unless filter variables are mentioned on the LHS.
+        if (op instanceof OpConditional || op instanceof OpLeftJoin) {
+            Op2 opleftjoin = (Op2) op;
+
+            if (!safeToTransform(joins, varsEquality, opleftjoin.getLeft())
+                    || !safeToTransform(joins, varsEquality, opleftjoin.getRight()))
+                return false;
+
+            // Not only must the left and right be safe to transform,
+            // but the equality variable must be known to be always set.
+
+            // If the varsLeft are disjoint from assigned vars,
+            // we may be able to push assign down right
+            // (this generalises the unit table case specialcase1)
+            // Needs more investigation.
+
+            Op opLeft = opleftjoin.getLeft();
+            Set<Var> varsLeft = OpVars.visibleVars(opLeft);
+            if (varsLeft.containsAll(varsEquality))
+                return true;
+            return false;
+        }
+
+        if (op instanceof OpGraph) {
+            OpGraph opg = (OpGraph) op;
+            return safeToTransform(joins, varsEquality, opg.getSubOp());
+        }
+
+        // Subquery - assume scope rewriting has already been applied.
+        if (op instanceof OpModifier) {
+            // ORDER BY?
+            OpModifier opMod = (OpModifier) op;
+            if (opMod instanceof OpProject) {
+                OpProject opProject = (OpProject) op;
+                // Writing "SELECT ?var" for "?var" -> a value would need
+                // AS-ification.
+                for (Var v : opProject.getVars()) {
+                    if (varsEquality.contains(v))
+                        return false;
+                }
+            }
+            return safeToTransform(joins, varsEquality, opMod.getSubOp());
+        }
+
+        if (op instanceof OpGroup) {
+            OpGroup opGroup = (OpGroup) op;
+            VarExprList varExprList = opGroup.getGroupVars();
+            return safeToTransform(varsEquality, varExprList) && safeToTransform(joins, varsEquality, opGroup.getSubOp());
+        }
+
+        if (op instanceof OpTable) {
+            OpTable opTable = (OpTable) op;
+            if (opTable.isJoinIdentity())
+                return true;
+        }
+
+        // Op1 - OpGroup
+        // Op1 - OpOrder
+        // Op1 - OpAssign, OpExtend
+        // Op1 - OpFilter - done.
+        // Op1 - OpLabel - easy
+        // Op1 - OpService - no.
+
+        return false;
+    }
+
+    private static boolean safeToTransform(Collection<Var> varsEquality, VarExprList varsExprList) {
+        // If the named variable is used, unsafe to rewrite.
+        return disjoint(varsExprList.getVars(), varsEquality);
+    }
+
+    // ---- Transformation
+
+    private static Op processFilterWorker(Op op, Var find, Var replace) {
+        return subst(op, find, replace);
+    }
+
+    private static Op subst(Op subOp, Var find, Var replace) {
+        Op op = Substitute.substitute(subOp, find, replace.asNode());
+        return OpAssign.assign(op, find, new ExprVar(replace));
+    }
+}

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=1495205&r1=1495204&r2=1495205&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 Thu Jun 20 21:49:37 2013
@@ -19,6 +19,7 @@
 package com.hp.hpl.jena.sparql.algebra;
 
 
+import com.hp.hpl.jena.sparql.algebra.optimize.TestSemanticEquivalence;
 import com.hp.hpl.jena.sparql.algebra.optimize.TestTransformFilters ;
 import com.hp.hpl.jena.sparql.algebra.optimize.TestOptimizer ;
 import com.hp.hpl.jena.sparql.algebra.optimize.TestTransformMergeBGPs ;
@@ -35,6 +36,7 @@ import org.junit.runners.Suite ;
     , TestClassify.class
     , TestTransformFilters.class
     , TestTransformQuads.class
+    , TestSemanticEquivalence.class
 
     , TestVarRename.class
     , TestOptimizer.class

Added: jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestSemanticEquivalence.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestSemanticEquivalence.java?rev=1495205&view=auto
==============================================================================
--- jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestSemanticEquivalence.java (added)
+++ jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestSemanticEquivalence.java Thu Jun 20 21:49:37 2013
@@ -0,0 +1,251 @@
+/*
+ * Copyright 2013 YarcData LLC All Rights Reserved.
+ */
+
+package com.hp.hpl.jena.sparql.algebra.optimize;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.query.ARQ;
+import com.hp.hpl.jena.query.Dataset;
+import com.hp.hpl.jena.query.DatasetFactory;
+import com.hp.hpl.jena.query.Query;
+import com.hp.hpl.jena.query.QueryExecution;
+import com.hp.hpl.jena.query.QueryExecutionFactory;
+import com.hp.hpl.jena.query.QueryFactory;
+import com.hp.hpl.jena.query.ResultSetFactory;
+import com.hp.hpl.jena.query.ResultSetRewindable;
+import com.hp.hpl.jena.rdf.model.ModelFactory;
+import com.hp.hpl.jena.sparql.algebra.Op;
+import com.hp.hpl.jena.sparql.algebra.OpVars;
+import com.hp.hpl.jena.sparql.core.DatasetGraph;
+import com.hp.hpl.jena.sparql.core.Quad;
+import com.hp.hpl.jena.sparql.core.Var;
+import com.hp.hpl.jena.sparql.engine.QueryIterator;
+import com.hp.hpl.jena.sparql.engine.ResultSetStream;
+import com.hp.hpl.jena.sparql.engine.binding.BindingFactory;
+import com.hp.hpl.jena.sparql.engine.main.QueryEngineMain;
+import com.hp.hpl.jena.sparql.resultset.ResultSetCompare;
+import com.hp.hpl.jena.sparql.sse.SSE;
+import com.hp.hpl.jena.sparql.util.Symbol;
+
+/**
+ * Tests for verifying that a query returns the same results both with and
+ * without a given optimizer enabled
+ * 
+ */
+public class TestSemanticEquivalence {
+
+    private static Dataset implJoin;
+
+    @BeforeClass
+    public static void setup() {
+        // Dataset for implicit join tests
+        implJoin = DatasetFactory.createMem();
+
+        Node a = NodeFactory.createURI("http://a");
+        Node b = NodeFactory.createURI("http://b");
+        Node c = NodeFactory.createURI("http://c");
+        Node p1 = NodeFactory.createURI("http://p1");
+        Node p2 = NodeFactory.createURI("http://p2");
+        Node o = NodeFactory.createLiteral("object");
+
+        DatasetGraph dsg = implJoin.asDatasetGraph();
+        dsg.add(Quad.defaultGraphNodeGenerated, a, p1, o);
+        dsg.add(Quad.defaultGraphNodeGenerated, a, p2, o);
+        dsg.add(Quad.defaultGraphNodeGenerated, b, p1, o);
+        dsg.add(Quad.defaultGraphNodeGenerated, b, p2, o);
+        dsg.add(Quad.defaultGraphNodeGenerated, c, p1, o);
+        
+        // Currently these optimizations are off by default
+        Assert.assertFalse(ARQ.isFalse(ARQ.optFilterImplicitJoin));
+        Assert.assertFalse(ARQ.isFalse(ARQ.optImplicitLeftJoin));
+    }
+
+    @AfterClass
+    public static void teardown() {
+        if (implJoin != null) {
+            implJoin.close();
+            implJoin = null;
+        }
+        
+        // Currently these optimizations are off by default
+        Assert.assertFalse(ARQ.isFalse(ARQ.optFilterImplicitJoin));
+        Assert.assertFalse(ARQ.isFalse(ARQ.optImplicitLeftJoin));
+    }
+
+    @Test
+    public void implicitJoinEvaluation1() {
+        String query = "SELECT * WHERE { ?x <http://p1> ?o1 . ?y <http://p2> ?o2 . FILTER(?x = ?y) }";
+        test(query, implJoin, ARQ.optFilterImplicitJoin, 2);
+
+        String alg1 = "(filter (= ?x ?y) (bgp (?x <http://p1> ?o1)(?y <http://p2> ?o2)))";
+        testAsAlgebra(alg1, implJoin, ARQ.optFilterImplicitJoin, 2);
+
+        String alg2 = "(filter (= ?y ?x) (bgp (?x <http://p1> ?o1)(?y <http://p2> ?o2)))";
+        testAsAlgebra(alg2, implJoin, ARQ.optFilterImplicitJoin, 2);
+    }
+
+    @Test
+    public void implicitJoinEvaluation2() {
+        String query = "SELECT * WHERE { ?x <http://p1> ?o1 . ?y <http://noSuchPredicate> ?o2 . FILTER(?x = ?y) }";
+        test(query, implJoin, ARQ.optFilterImplicitJoin, 0);
+
+        String alg1 = "(filter (= ?x ?y) (bgp (?x <http://p1> ?o1)(?y <http://noSuchPredicate> ?o2)))";
+        testAsAlgebra(alg1, implJoin, ARQ.optFilterImplicitJoin, 0);
+
+        String alg2 = "(filter (= ?y ?x) (bgp (?x <http://p1> ?o1)(?y <http://noSuchPredicate> ?o2)))";
+        testAsAlgebra(alg1, implJoin, ARQ.optFilterImplicitJoin, 0);
+    }
+
+    @Test
+    public void implicitJoinEvaluation3() {
+        String query = "SELECT * WHERE { ?x <http://p1> ?o1 . FILTER(?x = ?y) }";
+        test(query, implJoin, ARQ.optFilterImplicitJoin, 0);
+
+        String alg1 = "(filter (= ?x ?y) (bgp (?x <http://p1> ?o1)))";
+        testAsAlgebra(alg1, implJoin, ARQ.optFilterImplicitJoin, 0);
+
+        String alg2 = "(filter (= ?y ?x) (bgp (?x <http://p1> ?o1)))";
+        testAsAlgebra(alg1, implJoin, ARQ.optFilterImplicitJoin, 0);
+    }
+
+    @Test
+    public void implicitLeftJoinEvaluation1() {
+        String query = "SELECT * WHERE { ?x <http://p1> ?o1 . OPTIONAL { ?y <http://p2> ?o2 . FILTER(?x = ?y) } }";
+        test(query, implJoin, ARQ.optFilterImplicitJoin, 3);
+
+        String alg1 = "(leftjoin (bgp (?x <http://p1> ?o1)) (bgp (?y <http://p2> ?o2)) (= ?x ?y))";
+        testAsAlgebra(alg1, implJoin, ARQ.optFilterImplicitJoin, 3);
+
+        String alg2 = "(leftjoin (bgp (?x <http://p1> ?o1)) (bgp (?y <http://p2> ?o2)) (= ?y ?x))";
+        testAsAlgebra(alg2, implJoin, ARQ.optFilterImplicitJoin, 3);
+    }
+
+    @Test
+    public void implicitLeftJoinEvaluation2() {
+        String query = "SELECT * WHERE { ?x <http://p1> ?o1 . OPTIONAL { ?y <http://p2> ?o2 . FILTER(?x = ?y && ?o1 >= ?o2) } }";
+        test(query, implJoin, ARQ.optFilterImplicitJoin, 3);
+
+        String alg1 = "(leftjoin (bgp (?x <http://p1> ?o1)) (bgp (?y <http://p2> ?o2)) (&& (= ?x ?y)(> ?o1 ?o2)))";
+        testAsAlgebra(alg1, implJoin, ARQ.optFilterImplicitJoin, 3);
+
+        String alg2 = "(leftjoin (bgp (?x <http://p1> ?o1)) (bgp (?y <http://p2> ?o2)) (&& (= ?y ?x)(> ?o1 ?o2)))";
+        testAsAlgebra(alg2, implJoin, ARQ.optFilterImplicitJoin, 3);
+    }
+
+    /**
+     * Tests whether a query gives the same results when run both with and
+     * without a given optimizer
+     * 
+     * @param queryStr
+     *            Query
+     * @param ds
+     *            Dataset
+     * @param opt
+     *            Optimizer
+     * @param expected
+     *            Expected number of results
+     */
+    public static void test(String queryStr, Dataset ds, Symbol opt, int expected) {
+        Query q = QueryFactory.create(queryStr);
+
+        if (!q.isSelectType())
+            Assert.fail("Only SELECT queries are testable with this method");
+
+        // Track current state
+        boolean isEnabled = ARQ.isTrue(opt);
+        boolean isDisabled = ARQ.isFalse(opt);
+
+        try {
+            // Run first without optimization
+            ARQ.set(opt, false);
+            QueryExecution qe = QueryExecutionFactory.create(q, ds);
+            ResultSetRewindable rs = ResultSetFactory.makeRewindable(qe.execSelect());
+            Assert.assertEquals(expected, rs.size());
+            qe.close();
+
+            // Run with optimization
+            ARQ.set(opt, true);
+            QueryExecution qeOpt = QueryExecutionFactory.create(q, ds);
+            ResultSetRewindable rsOpt = ResultSetFactory.makeRewindable(qeOpt.execSelect());
+            Assert.assertEquals(expected, rsOpt.size());
+            qeOpt.close();
+
+            Assert.assertTrue(ResultSetCompare.isomorphic(rs, rsOpt));
+        } finally {
+            // Restore previous state
+            if (isEnabled) {
+                ARQ.set(opt, true);
+            } else if (isDisabled) {
+                ARQ.set(opt, false);
+            } else {
+                ARQ.unset(opt);
+            }
+        }
+    }
+
+    /**
+     * Tests whether an algebra expression gives the same results when run both
+     * with and without a given optimizer
+     * 
+     * @param algStr
+     *            Algebra
+     * @param ds
+     *            Dataset
+     * @param opt
+     *            Optimizer
+     * @param expected
+     *            Expected number of results
+     */
+    public static void testAsAlgebra(String algStr, Dataset ds, Symbol opt, int expected) {
+        Op op = SSE.parseOp(algStr);
+        List<String> vars = new ArrayList<String>();
+        for (Var v : OpVars.visibleVars(op)) {
+            vars.add(v.getName());
+        }
+
+        // Track current state
+        boolean isEnabled = ARQ.isTrue(opt);
+        boolean isDisabled = ARQ.isFalse(opt);
+
+        try {
+            // Run first without optimization
+            ARQ.set(opt, false);
+            QueryEngineMain engine = new QueryEngineMain(op, ds.asDatasetGraph(), BindingFactory.binding(), ARQ.getContext());
+            QueryIterator iter = engine.eval(op, ds.asDatasetGraph(), BindingFactory.binding(), ARQ.getContext());
+            ResultSetRewindable rs = ResultSetFactory.makeRewindable(new ResultSetStream(vars, ModelFactory.createDefaultModel(),
+                    iter));
+            Assert.assertEquals(expected, rs.size());
+            iter.close();
+
+            // Run with optimization
+            ARQ.set(opt, true);
+            engine = new QueryEngineMain(op, ds.asDatasetGraph(), BindingFactory.binding(), ARQ.getContext());
+            QueryIterator iterOpt = engine.eval(op, ds.asDatasetGraph(), BindingFactory.binding(), ARQ.getContext());
+            ResultSetRewindable rsOpt = ResultSetFactory.makeRewindable(new ResultSetStream(vars, ModelFactory
+                    .createDefaultModel(), iterOpt));
+            Assert.assertEquals(expected, rsOpt.size());
+            iterOpt.close();
+
+            Assert.assertTrue(ResultSetCompare.isomorphic(rs, rsOpt));
+        } finally {
+            // Restore previous state
+            if (isEnabled) {
+                ARQ.set(opt, true);
+            } else if (isDisabled) {
+                ARQ.set(opt, false);
+            } else {
+                ARQ.unset(opt);
+            }
+        }
+    }
+}

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=1495205&r1=1495204&r2=1495205&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 Thu Jun 20 21:49:37 2013
@@ -46,6 +46,8 @@ public class TestTransformFilters
     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() ;
     
     @Test public void equality01()
     {
@@ -531,7 +533,206 @@ public class TestTransformFilters
              t_expandOneOf,
              "(filter (exprlist (!= ?x <x>) (!= ?x 2) (!= ?x 3)) (bgp (?s ?p ?x)))") ;
     }
-
+    
+    @Test public void implicitJoin1()
+    {
+        test(
+             "(filter (= ?x ?y) (bgp (?x ?p ?o)(?y ?p1 ?o1)))",
+             t_implicitJoin,
+             "(assign ((?x ?y)) (bgp (?y ?p ?o)(?y ?p1 ?o1)))");
+    }
+    
+    @Test public void implicitJoin2()
+    {
+        test(
+             "(filter (= ?x ?y) (bgp (?x ?p ?o)))",
+             t_implicitJoin,
+             "(table empty)");
+    }
+    
+    @Test public void implicitJoin3()
+    {
+        // Still safe to transform as at least one is guaranteed non-literal
+        test(
+             "(filter (= ?x ?y) (bgp (?x ?p ?o)(?a ?b ?y)))",
+             t_implicitJoin,
+             "(assign ((?x ?y)) (bgp (?y ?p ?o)(?a ?b ?y)))");
+    }
+    
+    @Test public void implicitJoin4()
+    {
+        // Not safe to transform as both may be literals
+        test(
+             "(filter (= ?x ?y) (bgp (?a ?b ?x)(?c ?d ?y)))",
+             t_implicitJoin,
+             "(filter (= ?x ?y) (bgp (?a ?b ?x)(?c ?d ?y)))");
+    }
+    
+    @Test public void implicitJoin5()
+    {
+        // Safe to transform as although both may be literals we are using sameTerm so already relying on term equality
+        test(
+             "(filter (sameTerm ?x ?y) (bgp (?a ?b ?x)(?c ?d ?y)))",
+             t_implicitJoin,
+             "(assign ((?x ?y)) (bgp (?a ?b ?y)(?c ?d ?y)))");
+    }
+    
+    @Test public void implicitJoin6()
+    {
+        // Not safe to transform as equality on same variable
+        test(
+             "(filter (= ?x ?x) (bgp (?x ?p ?o)(?y ?p1 ?o1)))",
+             t_implicitJoin,
+             null);
+    }
+        
+    @Test public void implicitLeftJoin1()
+    {
+        // Possible to optimize some cases where it's an implicit left join
+        
+        // Covers the case with one variable on left and other on right
+        test(
+             "(leftjoin (bgp (?x ?p ?o)) (bgp (?y ?p1 ?o1)) ((= ?x ?y)))",
+             t_implicitLeftJoin,
+             "(leftjoin (bgp (?x ?p ?o)) (assign ((?y ?x)) (bgp (?x ?p1 ?o1))))");
+        
+        // Swapping the order of the equality expression should make no difference
+        test(
+                "(leftjoin (bgp (?x ?p ?o)) (bgp (?y ?p1 ?o1)) ((= ?y ?x)))",
+                t_implicitLeftJoin,
+                "(leftjoin (bgp (?x ?p ?o)) (assign ((?y ?x)) (bgp (?x ?p1 ?o1))))");
+    }
+    
+    @Test public void implicitLeftJoin2()
+    {
+        // Possible to optimize some cases where it's an implicit left join
+        
+        // Covers the case with one variable on left and other on right
+        test(
+             "(leftjoin (bgp (?y ?p ?o)) (bgp (?x ?p1 ?o1)) ((= ?x ?y)))",
+             t_implicitLeftJoin,
+             "(leftjoin (bgp (?y ?p ?o)) (assign ((?x ?y)) (bgp (?y ?p1 ?o1))))");
+        
+        // Swapping the order of the equality expression should make no difference
+        test(
+                "(leftjoin (bgp (?y ?p ?o)) (bgp (?x ?p1 ?o1)) ((= ?y ?x)))",
+                t_implicitLeftJoin,
+                "(leftjoin (bgp (?y ?p ?o)) (assign ((?x ?y)) (bgp (?y ?p1 ?o1))))");
+    }
+    
+    @Test public void implicitLeftJoin3()
+    {
+        // Possible to optimize some cases where it's an implicit left join
+        
+        // Covers the case with one variable on left and both on right
+        test(
+             "(leftjoin (bgp (?x ?p ?o)) (bgp (?x <http://type> ?type)(?y ?p1 ?o1)) ((= ?x ?y)))",
+             t_implicitLeftJoin,
+             "(leftjoin (bgp (?x ?p ?o)) (assign ((?y ?x)) (bgp (?x <http://type> ?type)(?x ?p1 ?o1))))");
+        
+        // Swapping the order of the equality expression should make no difference
+        test(
+                "(leftjoin (bgp (?x ?p ?o)) (bgp (?x <http://type> ?type)(?y ?p1 ?o1)) ((= ?y ?x)))",
+                t_implicitLeftJoin,
+                "(leftjoin (bgp (?x ?p ?o)) (assign ((?y ?x)) (bgp (?x <http://type> ?type)(?x ?p1 ?o1))))");
+    }
+    
+    @Test public void implicitLeftJoin4()
+    {
+        // Possible to optimize some cases where it's an implicit left join
+        
+        // Covers the case with one variable on left and both on right
+        test(
+             "(leftjoin (bgp (?y ?p ?o)) (bgp (?x <http://type> ?type)(?y ?p1 ?o1)) ((= ?x ?y)))",
+             t_implicitLeftJoin,
+             "(leftjoin (bgp (?y ?p ?o)) (assign ((?x ?y)) (bgp (?y <http://type> ?type)(?y ?p1 ?o1))))");
+        
+        // Swapping the order of the equality expression should make no difference
+        test(
+                "(leftjoin (bgp (?y ?p ?o)) (bgp (?x <http://type> ?type)(?y ?p1 ?o1)) ((= ?y ?x)))",
+                t_implicitLeftJoin,
+                "(leftjoin (bgp (?y ?p ?o)) (assign ((?x ?y)) (bgp (?y <http://type> ?type)(?y ?p1 ?o1))))");
+    }
+    
+    @Test public void implicitLeftJoin5()
+    {
+        // Possible to optimize some cases where it's an implicit left join
+        
+        // Covers the case of both variables on left and only one on right
+        test(
+             "(leftjoin (bgp (?x ?p ?o)(?x <http://pred> ?y)) (bgp (?y ?p1 ?o1)) ((= ?x ?y)))",
+             t_implicitLeftJoin,
+             "(leftjoin (bgp (?x ?p ?o)(?x <http://pred> ?y)) (assign ((?y ?x)) (bgp (?x ?p1 ?o1))))");
+        
+        // Swapping the order of the equality expression should make no difference
+        test(
+                "(leftjoin (bgp (?x ?p ?o)(?x <http://pred> ?y)) (bgp (?y ?p1 ?o1)) ((= ?y ?x)))",
+                t_implicitLeftJoin,
+                "(leftjoin (bgp (?x ?p ?o)(?x <http://pred> ?y)) (assign ((?y ?x)) (bgp (?x ?p1 ?o1))))");
+    }
+    
+    @Test public void implicitLeftJoin6()
+    {
+        // Possible to optimize some cases where it's an implicit left join
+        
+        // Covers the case of both variables on left and only one on right
+        test(
+             "(leftjoin (bgp (?x ?p ?o)(?x <http://pred> ?y)) (bgp (?x ?p1 ?o1)) ((= ?x ?y)))",
+             t_implicitLeftJoin,
+             "(leftjoin (bgp (?x ?p ?o)(?x <http://pred> ?y)) (assign ((?x ?y)) (bgp (?y ?p1 ?o1))))");
+        
+        // Swapping the order of the equality expression should make no difference
+        test(
+                "(leftjoin (bgp (?x ?p ?o)(?x <http://pred> ?y)) (bgp (?x ?p1 ?o1)) ((= ?y ?x)))",
+                t_implicitLeftJoin,
+                "(leftjoin (bgp (?x ?p ?o)(?x <http://pred> ?y)) (assign ((?x ?y)) (bgp (?y ?p1 ?o1))))");
+    }
+    
+    @Test public void implicitLeftJoin7()
+    {
+        // Possible to optimize some cases where it's an implicit left join
+        
+        // Covers the case of both variables on both sides
+        test(
+             "(leftjoin (bgp (?x ?p ?o)(?x <http://pred> ?y)) (bgp (?x <http://type> ?type)(?y ?p1 ?o1)) ((= ?x ?y)))",
+             t_implicitLeftJoin,
+             "(leftjoin (bgp (?x ?p ?o)(?x <http://pred> ?y)) (assign ((?x ?y)) (bgp (?y <http://type> ?type)(?y ?p1 ?o1))))");
+        
+        // Swapping the order of the equality expression will make a difference in this case
+        test(
+                "(leftjoin (bgp (?x ?p ?o)(?x <http://pred> ?y)) (bgp (?x <http://type> ?type)(?y ?p1 ?o1)) ((= ?y ?x)))",
+                t_implicitLeftJoin,
+                "(leftjoin (bgp (?x ?p ?o)(?x <http://pred> ?y)) (assign ((?y ?x)) (bgp (?x <http://type> ?type)(?x ?p1 ?o1))))");
+    }
+    
+    @Test public void implicitJoin8()
+    {
+        // We don't currently optimize the case where the filter will evaluate to false
+        // for all solutions because neither variable in on the RHS
+        test(
+             "(leftjoin (bgp (?x ?p ?o)(?x <http://pred> ?y)) (bgp (?a ?b ?c)) ((= ?x ?y)))",
+             t_implicitLeftJoin,
+             null);
+    }
+    
+    @Test public void implicitLeftJoin9()
+    {
+        // && means both conditions must hold so can optimize out the implicit join
+        test(
+             "(leftjoin (bgp (?x ?p ?o)) (bgp (?y ?p1 ?o1)) (&& (= ?x ?y) (> ?o1 ?o2)))",
+             t_implicitLeftJoin,
+             "(leftjoin (bgp (?x ?p ?o)) (assign ((?y ?x)) (bgp (?x ?p1 ?o1))) (> ?o1 ?o2))");
+    }
+    
+    @Test public void implicitLeftJoin10()
+    {
+        // Unsafe to optimize
+        test(
+             "(leftjoin (bgp (?x ?p ?o)) (bgp (?y ?p1 ?o1)) (|| (= ?x ?y) (> ?o1 ?o2)))",
+             t_implicitLeftJoin,
+             null);
+    }
+        
     public static void test(String input, Transform transform, String... output)
     {
         Op op1 = SSE.parseOp(input) ;
@@ -546,5 +747,5 @@ public class TestTransformFilters
         Op op3 = SSE.parseOp(StrUtils.strjoinNL(output)) ;
         Assert.assertEquals(op3, op2) ;
     }
-    
+        
 }