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 2015/07/06 16:49:12 UTC

[1/3] jena git commit: Further work on eliminating/inlining assignments (JENA-780)

Repository: jena
Updated Branches:
  refs/heads/eliminate-assignments 57cf5dd9a -> d6f516dec


Further work on eliminating/inlining assignments (JENA-780)

- Support elimination/inlining to OpGroup and OpTopN
- Only eliminate/inline when expressions are stable
- Only inline into sort conditions if the expression is constant
- Expand test coverage


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

Branch: refs/heads/eliminate-assignments
Commit: bdcf8a6056092b04bf644607b6176afd0e834544
Parents: 57cf5dd
Author: Rob Vesse <rv...@apache.org>
Authored: Mon Jul 6 14:37:58 2015 +0100
Committer: Rob Vesse <rv...@apache.org>
Committed: Mon Jul 6 14:39:44 2015 +0100

----------------------------------------------------------------------
 .../optimize/TransformEliminateAssignments.java | 193 ++++++++++++++++---
 .../optimize/TransformRemoveAssignment.java     |  18 ++
 .../algebra/optimize/VariableUsagePopper.java   |  18 ++
 .../TestTransformEliminateAssignments.java      | 134 ++++++++++---
 4 files changed, 317 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/bdcf8a60/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformEliminateAssignments.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformEliminateAssignments.java b/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformEliminateAssignments.java
index c9bdb7c..91dc435 100644
--- a/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformEliminateAssignments.java
+++ b/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformEliminateAssignments.java
@@ -44,45 +44,95 @@ import com.hp.hpl.jena.sparql.algebra.op.OpTopN;
 import com.hp.hpl.jena.sparql.core.Var;
 import com.hp.hpl.jena.sparql.core.VarExprList;
 import com.hp.hpl.jena.sparql.expr.Expr;
+import com.hp.hpl.jena.sparql.expr.ExprAggregator;
+import com.hp.hpl.jena.sparql.expr.ExprLib;
 import com.hp.hpl.jena.sparql.expr.ExprList;
+import com.hp.hpl.jena.sparql.expr.ExprTransform;
 import com.hp.hpl.jena.sparql.expr.ExprTransformSubstitute;
 import com.hp.hpl.jena.sparql.expr.ExprTransformer;
 import com.hp.hpl.jena.sparql.expr.ExprVars;
+import com.hp.hpl.jena.sparql.expr.NodeValue;
 
 /**
- * A transform that tries to remove unecessary assignments
+ * A transform that tries to in-line/eliminate assignments
  * <p>
- * There are two classes of assignments that we can try and remove:
+ * There are two classes of assignments that we can try and in-line/eliminate:
  * </p>
  * <ol>
  * <li>Assignments where the assigned variable is used only once in a subsequent
- * assignment</li>
- * <li>Assignments where the assigned value is never used elsewhere</li>
+ * assignment can be in-lined</li>
+ * <li>Assignments where the assigned value is never used elsewhere can be
+ * eliminated</li>
  * </ol>
  * <p>
  * Both of these changes can only happen inside of projections as otherwise we
  * have to assume that the user may need the resulting variable and thus we
- * leave the assignment alone.
+ * leave the assignment alone. Assignments to be in-lined must also be
+ * deterministic i.e. moving their placement in the query and thus the possible
+ * solutions they might operate must not change their outputs. Whether an
+ * expression is deterministic is defined by {@link ExprLib#isStable(Expr)}.
+ * </p>
+ * <p>
+ * Assignments may be in-lined in the following places:
+ * </p>
+ * <ul>
+ * <li>Filter Expressions</li>
+ * <li>Bind and Select Expressions</li>
+ * <li>Group By Expressions</li>
+ * <li>Order By Expressions if aggressive in-lining is enabled</li>
+ * </ul>
+ * <p>
+ * In the case of order by we only in-line assignments when aggressive mode is
+ * set as the realities of order by are that expressions may be recomputed
+ * multiple times and so in-lining may actually hurt performance in those cases
+ * unless the expression to be in-lined is itself a constant.
  * </p>
- * 
  */
 public class TransformEliminateAssignments extends TransformCopy {
 
     public static Op eliminate(Op op) {
+        return eliminate(op, false);
+    }
+
+    public static Op eliminate(Op op, boolean aggressive) {
         AssignmentTracker tracker = new AssignmentTracker();
         AssignmentPusher pusher = new AssignmentPusher(tracker);
         AssignmentPopper popper = new AssignmentPopper(tracker);
-        Transform transform = new TransformEliminateAssignments(tracker, pusher, popper);
+        Transform transform = new TransformEliminateAssignments(tracker, pusher, popper, aggressive);
 
         return Transformer.transform(transform, op, pusher, popper);
     }
 
-    private OpVisitor before, after;
-    private AssignmentTracker tracker;
+    private final OpVisitor before, after;
+    private final AssignmentTracker tracker;
+    private final boolean aggressive;
 
     private TransformEliminateAssignments(AssignmentTracker tracker, OpVisitor before, OpVisitor after) {
+        this(tracker, before, after, false);
+    }
+
+    private TransformEliminateAssignments(AssignmentTracker tracker, OpVisitor before, OpVisitor after,
+            boolean aggressive) {
         this.tracker = tracker;
         this.before = before;
+        this.after = after;
+        this.aggressive = aggressive;
+    }
+
+    protected boolean canInline(Expr e) {
+        return ExprLib.isStable(e);
+    }
+
+    protected boolean shouldInline(Expr e) {
+        // Inline everything when being aggressive
+        if (this.aggressive)
+            return true;
+
+        if (e == null)
+            return false;
+
+        // If not being aggressive only inline if the expression is a constant
+        return e.isConstant() || e instanceof NodeValue;
     }
 
     protected boolean isApplicable() {
@@ -121,13 +171,12 @@ public class TransformEliminateAssignments extends TransformCopy {
             // Usage count will be 2 if we can eliminate the assignment
             // First usage is when it is introduced by the assignment and the
             // second is when it is used now in this filter
-            if (this.tracker.getUsageCount(var) == 2 && this.tracker.getAssignments().containsKey(var)) {
+            Expr e = getAssignExpr(var);
+            if (this.tracker.getUsageCount(var) == 2 && hasAssignment(var) && canInline(e)) {
                 // Can go back and eliminate that assignment
-                subOp = Transformer.transform(
-                        new TransformRemoveAssignment(var, this.tracker.getAssignments().get(var)), subOp);
+                subOp = eliminateAssignment(subOp, var);
                 // Replace the variable usage with the expression
-                exprs = ExprTransformer.transform(
-                        new ExprTransformSubstitute(var, this.tracker.getAssignments().get(var)), exprs);
+                exprs = ExprTransformer.transform(new ExprTransformSubstitute(var, e), exprs);
                 this.tracker.getAssignments().remove(var);
                 modified = true;
             }
@@ -141,6 +190,14 @@ public class TransformEliminateAssignments extends TransformCopy {
         return super.transform(opFilter, subOp);
     }
 
+    private boolean hasAssignment(Var var) {
+        return this.tracker.getAssignments().containsKey(var);
+    }
+
+    private Expr getAssignExpr(Var var) {
+        return this.tracker.getAssignments().get(var);
+    }
+
     @Override
     public Op transform(OpExtend opExtend, Op subOp) {
         // No point tracking assignments if not in a projection as we can't
@@ -152,6 +209,9 @@ public class TransformEliminateAssignments extends TransformCopy {
         // Track the assignments for future reference
         this.tracker.putAssignments(opExtend.getVarExprList());
 
+        // TODO Could also eliminate assignments where the value is only used in
+        // a subsequent assignment
+
         // See if there are any assignments we can eliminate entirely i.e. those
         // where the assigned value is never used
         VarExprList assignments = processUnused(opExtend.getVarExprList());
@@ -203,11 +263,12 @@ public class TransformEliminateAssignments extends TransformCopy {
             // Usage count will be 2 if we can eliminate the assignment
             // First usage is when it is introduced by the assignment and the
             // second is when it is used now in this filter
-            if (this.tracker.getUsageCount(var) == 2 && this.tracker.getAssignments().containsKey(var)) {
+            Expr e = getAssignExpr(var);
+            if (this.tracker.getUsageCount(var) == 2 && hasAssignment(var) && canInline(e) && shouldInline(e)) {
                 // Can go back and eliminate that assignment
-                subOp = Transformer.transform(
-                        new TransformRemoveAssignment(var, this.tracker.getAssignments().get(var)), subOp);
-                // Replace the variable usage with the expression within the sort conditions
+                subOp = eliminateAssignment(subOp, var);
+                // Replace the variable usage with the expression within the
+                // sort conditions
                 conditions = processConditions(opOrder.getConditions(), conditions, var);
                 this.tracker.getAssignments().remove(var);
             }
@@ -228,25 +289,113 @@ public class TransformEliminateAssignments extends TransformCopy {
 
         for (SortCondition cond : inputConditions) {
             Expr e = cond.getExpression();
-            e = ExprTransformer.transform(new ExprTransformSubstitute(var, this.tracker.getAssignments().get(var)), e);
+            e = ExprTransformer.transform(new ExprTransformSubstitute(var, getAssignExpr(var)), e);
             outputConditions.add(new SortCondition(e, cond.getDirection()));
         }
-       
+
         return outputConditions;
     }
 
     @Override
     public Op transform(OpTopN opTop, Op subOp) {
-        // TODO Auto-generated method stub
+        if (!this.isApplicable())
+            return super.transform(opTop, subOp);
+
+        // See what vars are used in the sort conditions
+        Collection<Var> vars = new ArrayList<>();
+        for (SortCondition cond : opTop.getConditions()) {
+            ExprVars.varsMentioned(vars, cond.getExpression());
+        }
+
+        // Are any of these vars single usage?
+        List<SortCondition> conditions = null;
+        for (Var var : vars) {
+            // Usage count will be 2 if we can eliminate the assignment
+            // First usage is when it is introduced by the assignment and the
+            // second is when it is used now in this filter
+            Expr e = getAssignExpr(var);
+            if (this.tracker.getUsageCount(var) == 2 && hasAssignment(var) && canInline(e) && shouldInline(e)) {
+                // Can go back and eliminate that assignment
+                subOp = eliminateAssignment(subOp, var);
+                // Replace the variable usage with the expression within the
+                // sort conditions
+                conditions = processConditions(opTop.getConditions(), conditions, var);
+                this.tracker.getAssignments().remove(var);
+            }
+        }
+
+        // Create a new order if we've substituted any expressions
+        if (conditions != null) {
+            return new OpTopN(subOp, opTop.getLimit(), conditions);
+        }
+
         return super.transform(opTop, subOp);
     }
 
     @Override
     public Op transform(OpGroup opGroup, Op subOp) {
-        // TODO Auto-generated method stub
+        if (!this.isApplicable())
+            return super.transform(opGroup, subOp);
+
+        // See what vars are used in the filter
+        Collection<Var> vars = new ArrayList<>();
+        VarExprList exprs = new VarExprList(opGroup.getGroupVars());
+        List<ExprAggregator> aggs = new ArrayList<ExprAggregator>(opGroup.getAggregators());
+        for (Expr expr : exprs.getExprs().values()) {
+            ExprVars.varsMentioned(vars, expr);
+        }
+
+        // Are any of these vars single usage?
+        boolean modified = false;
+        for (Var var : vars) {
+            // Usage count will be 2 if we can eliminate the assignment
+            // First usage is when it is introduced by the assignment and the
+            // second is when it is used now in this group by
+            Expr e = getAssignExpr(var);
+            if (this.tracker.getUsageCount(var) == 2 && hasAssignment(var) && canInline(e)) {
+                // Can go back and eliminate that assignment
+                subOp = eliminateAssignment(subOp, var);
+                // Replace the variable usage with the expression in both the
+                // expressions and the aggregators
+                ExprTransform transform = new ExprTransformSubstitute(var, e);
+                exprs = processVarExprList(exprs, transform);
+                aggs = processAggregators(aggs, transform);
+                this.tracker.getAssignments().remove(var);
+                modified = true;
+            }
+        }
+
+        // Create a new group by if we've substituted any expressions
+        if (modified) {
+            return new OpGroup(subOp, exprs, aggs);
+        }
+
         return super.transform(opGroup, subOp);
     }
 
+    private Op eliminateAssignment(Op subOp, Var var) {
+        return Transformer.transform(new TransformRemoveAssignment(var, getAssignExpr(var)), subOp);
+    }
+
+    private VarExprList processVarExprList(VarExprList exprs, ExprTransform transform) {
+        VarExprList newExprs = new VarExprList();
+        for (Var v : exprs.getVars()) {
+            Expr e = exprs.getExpr(v);
+            Expr e2 = ExprTransformer.transform(transform, e);
+            newExprs.add(v, e2);
+        }
+        return newExprs;
+    }
+
+    private List<ExprAggregator> processAggregators(List<ExprAggregator> aggs, ExprTransform transform) {
+        List<ExprAggregator> newAggs = new ArrayList<ExprAggregator>();
+        for (ExprAggregator agg : aggs) {
+            ExprAggregator e2 = (ExprAggregator) ExprTransformer.transform(transform, agg);
+            newAggs.add(e2);
+        }
+        return newAggs;
+    }
+
     private static class AssignmentTracker extends VariableUsageTracker {
 
         private Map<Var, Expr> assignments = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/jena/blob/bdcf8a60/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformRemoveAssignment.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformRemoveAssignment.java b/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformRemoveAssignment.java
index dba9271..d7c08d4 100644
--- a/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformRemoveAssignment.java
+++ b/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformRemoveAssignment.java
@@ -1,3 +1,21 @@
+/*
+ * 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 com.hp.hpl.jena.sparql.algebra.Op;

http://git-wip-us.apache.org/repos/asf/jena/blob/bdcf8a60/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/VariableUsagePopper.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/VariableUsagePopper.java b/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/VariableUsagePopper.java
index e73bfee..73e7ec9 100644
--- a/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/VariableUsagePopper.java
+++ b/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/VariableUsagePopper.java
@@ -1,3 +1,21 @@
+/*
+ * 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 java.util.Collection;

http://git-wip-us.apache.org/repos/asf/jena/blob/bdcf8a60/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformEliminateAssignments.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformEliminateAssignments.java b/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformEliminateAssignments.java
index 2f2ced9..163ce8c 100644
--- a/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformEliminateAssignments.java
+++ b/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformEliminateAssignments.java
@@ -32,13 +32,17 @@ import com.hp.hpl.jena.sparql.sse.SSE;
 public class TestTransformEliminateAssignments {
 
     private void test(String input, String... output) {
+        test(input, false, output);
+    }
+
+    private void test(String input, boolean aggressive, String... output) {
         Op original = SSE.parseOp(input);
-        test(original, output);
+        test(original, aggressive, output);
     }
 
-    private void test(Op original, String... output) {
+    private void test(Op original, boolean aggressive, String... output) {
         // Transform
-        Op actual = TransformEliminateAssignments.eliminate(original);
+        Op actual = TransformEliminateAssignments.eliminate(original, aggressive);
 
         // Check results
         if (output == null) {
@@ -51,17 +55,12 @@ public class TestTransformEliminateAssignments {
         }
     }
 
-    @SuppressWarnings("unused")
-    private void testNoChange(String input) {
-        test(input, (String[]) null);
-    }
-
     private void testNoChange(String... input) {
         test(StrUtils.strjoinNL(input), (String[]) null);
     }
 
     @Test
-    public void eliminate_single_use_extend_01() {
+    public void single_use_extend_01() {
         // Assigned variable used only once can substitute expression for the
         // later usage of the variable
         // However we must be inside a projection as otherwise the assigned
@@ -78,7 +77,7 @@ public class TestTransformEliminateAssignments {
     }
 
     @Test
-    public void eliminate_single_use_extend_02() {
+    public void single_use_extend_02() {
         // Assignment for ?y can be removed because it is never used
         // However we must be inside a projection as otherwise the assigned
         // variable would be visible and we couldn't eliminate the assignment
@@ -92,9 +91,9 @@ public class TestTransformEliminateAssignments {
              "    (table unit)))");
         //@formatter:on
     }
-    
+
     @Test
-    public void eliminate_single_use_extend_03() {
+    public void single_use_extend_03() {
         // Assigned variable used only once can substitute expression for the
         // later usage of the variable
         // However we must be inside a projection as otherwise the assigned
@@ -111,7 +110,92 @@ public class TestTransformEliminateAssignments {
     }
 
     @Test
-    public void single_use_extend_unchanged_01() {
+    public void single_use_extend_complex_01() {
+        // Assigned variable used only once can substitute expression for the
+        // later usage of the variable
+        // BUT we won't do this by default for complex expressions where they
+        // are used in a place where they could be evaluated multiple times
+        //@formatter:off
+        testNoChange(StrUtils.strjoinNL("(project (?y)",
+                                        "  (order (?x)",
+                                        "    (extend (?x (contains 'foo' 'bar'))",
+                                        "      (table unit))))"));
+        //@formatter:on
+    }
+
+    @Test
+    public void single_use_extend_complex_02() {
+        // Assigned variable used only once can substitute expression for the
+        // later usage of the variable
+        // BUT we won't do this by default for complex expressions where they
+        // are used in a place where they could be evaluated multiple times
+        // EXCEPT if we are doing aggressive in-lining
+        //@formatter:off
+        test(StrUtils.strjoinNL("(project (?y)",
+                                "  (order (?x)",
+                                "    (extend (?x (contains 'foo' 'bar'))",
+                                "      (table unit))))"),
+             true,
+             "(project (?y)",
+             "  (order ((contains 'foo' 'bar'))",
+             "    (table unit)))");
+        //@formatter:on
+    }
+    
+    @Test
+    public void single_use_extend_unstable_01() {
+        // Assigned variable used only once can substitute expression for the
+        // later usage of the variable
+        // EXCEPT if the expression is unstable in which case we leave it alone
+        //@formatter:off
+        testNoChange(StrUtils.strjoinNL("(project (?y)",
+                                        "  (filter (exprlist ?x)",
+                                        "    (extend (?x (rand))",
+                                        "      (table unit))))"));
+        //@formatter:on
+    }
+    
+    @Test
+    public void single_use_extend_unstable_02() {
+        // Assigned variable used only once can substitute expression for the
+        // later usage of the variable
+        // EXCEPT if the expression is unstable in which case we leave it alone
+        //@formatter:off
+        testNoChange(StrUtils.strjoinNL("(project (?y)",
+                                        "  (filter (exprlist ?x)",
+                                        "    (extend (?x (uuid))",
+                                        "      (table unit))))"));
+        //@formatter:on
+    }
+    
+    @Test
+    public void single_use_extend_unstable_03() {
+        // Assigned variable used only once can substitute expression for the
+        // later usage of the variable
+        // EXCEPT if the expression is unstable in which case we leave it alone
+        //@formatter:off
+        testNoChange(StrUtils.strjoinNL("(project (?y)",
+                                        "  (filter (exprlist ?x)",
+                                        "    (extend (?x (struuid))",
+                                        "      (table unit))))"));
+        //@formatter:on
+    }
+    
+    @Test
+    public void single_use_extend_unstable_04() {
+        // Assigned variable used only once can substitute expression for the
+        // later usage of the variable
+        // EXCEPT if the expression is unstable in which case we leave it alone
+        //@formatter:off
+        testNoChange(StrUtils.strjoinNL("(project (?y)",
+                                        "  (filter (exprlist ?x)",
+                                        "    (extend (?x (bnode))",
+                                        "      (table unit))))"));
+        //@formatter:on
+    }
+
+    @Test
+    public void single_use_extend_outside_projection_01() {
         // Cannot eliminate as there is no projection so the assigned variable
         // is visible even though in the algebra given it is used only once
         //@formatter:off
@@ -122,7 +206,7 @@ public class TestTransformEliminateAssignments {
     }
 
     @Test
-    public void single_use_extend_unchanged_02() {
+    public void single_use_extend_outside_projection_02() {
         // Cannot eliminate as there is no projection so the assigned variable
         // is visible even though in the algebra given it is used only once
         //@formatter:off
@@ -133,26 +217,28 @@ public class TestTransformEliminateAssignments {
     }
 
     @Test
-    public void multi_use_extend_unchanged_01() {
+    public void multi_use_extend_01() {
         // As the assigned variable is used multiple times we leave the
         // assignment alone
         //@formatter:off
-        testNoChange("(filter (> (* ?x ?x) 16)",
-                     "  (extend (?x 3)",
-                     "    (table unit)))");
+        testNoChange("(project (?y)",
+                     "  (filter (> (* ?x ?x) 16)",
+                     "    (extend (?x 3)",
+                     "      (table unit))))");
         //@formatter:on
     }
 
     @Test
-    public void multi_use_extend_unchanged_02() {
+    public void multi_use_extend_02() {
         // Because the value of the assignment is used in multiple places we
         // leave the assignment alone
         //@formatter:off
-        testNoChange("(filter (exprlist ?x)",
-                     "  (join",
-                     "    (extend (?x true)",
-                     "      (table unit))",
-                     "    (bgp (triple ?x ?y ?z))))");
+        testNoChange("(project (?y)",
+                     "  (filter (exprlist ?x)",
+                     "    (join",
+                     "      (extend (?x true)",
+                     "        (table unit))",
+                     "      (bgp (triple ?x ?y ?z)))))");
         //@formatter:on
     }
 


[3/3] jena git commit: More bug fixes to inlining assignments in extend (JENA-780)

Posted by rv...@apache.org.
More bug fixes to inlining assignments in extend (JENA-780)

This commit adds additional test cases and some bug fixes that cover the
case where there are single use assignments that can be inlined where
the value is only used in the extend we are currently processing which
requires some extra work to make sure we eliminate unused assignments
after we inline them.  It also ensures that subsequent assignments which
can also have assignments inlined into them refer to the correct
expression taking into account previous inlinings that have happened.


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

Branch: refs/heads/eliminate-assignments
Commit: d6f516decedd40e0339012279a4e392239865cf0
Parents: 985b995
Author: Rob Vesse <rv...@apache.org>
Authored: Mon Jul 6 15:46:20 2015 +0100
Committer: Rob Vesse <rv...@apache.org>
Committed: Mon Jul 6 15:46:20 2015 +0100

----------------------------------------------------------------------
 .../optimize/TransformEliminateAssignments.java | 104 +++++++++++--------
 .../TestTransformEliminateAssignments.java      |  32 ++++++
 2 files changed, 93 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/d6f516de/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformEliminateAssignments.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformEliminateAssignments.java b/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformEliminateAssignments.java
index 4d59fc3..59e77c2 100644
--- a/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformEliminateAssignments.java
+++ b/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformEliminateAssignments.java
@@ -78,7 +78,6 @@ import com.hp.hpl.jena.sparql.expr.NodeValue;
  * <ul>
  * <li>Filter Expressions</li>
  * <li>Bind and Select Expressions</li>
- * <li>Group By Expressions</li>
  * <li>Order By Expressions if aggressive in-lining is enabled</li>
  * </ul>
  * <p>
@@ -211,11 +210,7 @@ public class TransformEliminateAssignments extends TransformCopy {
         // Track the assignments for future reference
         this.tracker.putAssignments(opExtend.getVarExprList());
 
-        // TODO Could also eliminate assignments where the value is only used in
-        // a subsequent assignment
-
-        // See if there are any assignments we can eliminate entirely i.e. those
-        // where the assigned value is never used
+        // Eliminate and inline assignments
         VarExprList unusedAssignments = processUnused(opExtend.getVarExprList());
         VarExprList newAssignments = new VarExprList();
         for (Var assignVar : opExtend.getVarExprList().getVars()) {
@@ -229,6 +224,7 @@ public class TransformEliminateAssignments extends TransformCopy {
             Collection<Var> vars = new ArrayList<>();
             ExprVars.varsMentioned(vars, currExpr);
 
+            // See if we can inline anything
             for (Var var : vars) {
                 // Usage count will be 2 if we can eliminate the assignment
                 // First usage is when it is introduced by the assignment and
@@ -242,6 +238,10 @@ public class TransformEliminateAssignments extends TransformCopy {
                     currExpr = ExprTransformer.transform(new ExprTransformSubstitute(var, e), currExpr);
                     this.tracker.getAssignments().remove(var);
 
+                    // Need to update any assignments we may be tracking that
+                    // refer to the variable we just inlined
+                    this.tracker.updateAssignments(var, e);
+
                     // If the assignment to be eliminated was introduced by the
                     // extend we are processing need to remove it from the
                     // VarExprList we are currently building
@@ -271,7 +271,7 @@ public class TransformEliminateAssignments extends TransformCopy {
             if (this.tracker.getUsageCount(var) == 1)
                 singleUse.add(var, assignments.getExpr(var));
         }
-        
+
         // If nothing is single use
         if (singleUse.size() == 0)
             return null;
@@ -367,49 +367,57 @@ public class TransformEliminateAssignments extends TransformCopy {
 
     @Override
     public Op transform(OpGroup opGroup, Op subOp) {
-        if (!this.isApplicable())
-            return super.transform(opGroup, subOp);
-
-        // See what vars are used in the filter
-        Collection<Var> vars = new ArrayList<>();
-        VarExprList exprs = new VarExprList(opGroup.getGroupVars());
-        List<ExprAggregator> aggs = new ArrayList<ExprAggregator>(opGroup.getAggregators());
-        for (Expr expr : exprs.getExprs().values()) {
-            ExprVars.varsMentioned(vars, expr);
-        }
-
-        // Are any of these vars single usage?
-        boolean modified = false;
-        for (Var var : vars) {
-            // Usage count will be 2 if we can eliminate the assignment
-            // First usage is when it is introduced by the assignment and the
-            // second is when it is used now in this group by
-            Expr e = getAssignExpr(var);
-            if (this.tracker.getUsageCount(var) == 2 && hasAssignment(var) && canInline(e)) {
-                // Can go back and eliminate that assignment
-                subOp = eliminateAssignment(subOp, var);
-                // Replace the variable usage with the expression in both the
-                // expressions and the aggregators
-                ExprTransform transform = new ExprTransformSubstitute(var, e);
-                exprs = processVarExprList(exprs, transform);
-                aggs = processAggregators(aggs, transform);
-                this.tracker.getAssignments().remove(var);
-                modified = true;
-            }
-        }
-
-        // Create a new group by if we've substituted any expressions
-        if (modified) {
-            return new OpGroup(subOp, exprs, aggs);
-        }
-
         return super.transform(opGroup, subOp);
+
+        // TODO Unclear if this will work properly or not because group can
+        // introduce new assignments as well as evaluate expressions
+
+        //@formatter:off
+//        if (!this.isApplicable())
+//            return super.transform(opGroup, subOp);
+//
+//        // See what vars are used in the filter
+//        Collection<Var> vars = new ArrayList<>();
+//        VarExprList exprs = new VarExprList(opGroup.getGroupVars());
+//        List<ExprAggregator> aggs = new ArrayList<ExprAggregator>(opGroup.getAggregators());
+//        for (Expr expr : exprs.getExprs().values()) {
+//            ExprVars.varsMentioned(vars, expr);
+//        }
+//
+//        // Are any of these vars single usage?
+//        boolean modified = false;
+//        for (Var var : vars) {
+//            // Usage count will be 2 if we can eliminate the assignment
+//            // First usage is when it is introduced by the assignment and the
+//            // second is when it is used now in this group by
+//            Expr e = getAssignExpr(var);
+//            if (this.tracker.getUsageCount(var) == 2 && hasAssignment(var) && canInline(e)) {
+//                // Can go back and eliminate that assignment
+//                subOp = eliminateAssignment(subOp, var);
+//                // Replace the variable usage with the expression in both the
+//                // expressions and the aggregators
+//                ExprTransform transform = new ExprTransformSubstitute(var, e);
+//                exprs = processVarExprList(exprs, transform);
+//                aggs = processAggregators(aggs, transform);
+//                this.tracker.getAssignments().remove(var);
+//                modified = true;
+//            }
+//        }
+//
+//        // Create a new group by if we've substituted any expressions
+//        if (modified) {
+//            return new OpGroup(subOp, exprs, aggs);
+//        }
+//
+//        return super.transform(opGroup, subOp);
+        //@formatter:on
     }
 
     private Op eliminateAssignment(Op subOp, Var var) {
         return Transformer.transform(new TransformRemoveAssignment(var, getAssignExpr(var)), subOp);
     }
 
+    @SuppressWarnings("unused")
     private VarExprList processVarExprList(VarExprList exprs, ExprTransform transform) {
         VarExprList newExprs = new VarExprList();
         for (Var v : exprs.getVars()) {
@@ -420,6 +428,7 @@ public class TransformEliminateAssignments extends TransformCopy {
         return newExprs;
     }
 
+    @SuppressWarnings("unused")
     private List<ExprAggregator> processAggregators(List<ExprAggregator> aggs, ExprTransform transform) {
         List<ExprAggregator> newAggs = new ArrayList<ExprAggregator>();
         for (ExprAggregator agg : aggs) {
@@ -459,6 +468,15 @@ public class TransformEliminateAssignments extends TransformCopy {
             }
         }
 
+        public void updateAssignments(Var v, Expr e) {
+            ExprTransformSubstitute transform = new ExprTransformSubstitute(v, e);
+            for (Var assignVar : this.assignments.keySet()) {
+                Expr assignExpr = this.assignments.get(assignVar);
+                assignExpr = ExprTransformer.transform(transform, assignExpr);
+                this.assignments.put(assignVar, assignExpr);
+            }
+        }
+
         public void incrementDepth() {
             this.depth++;
         }

http://git-wip-us.apache.org/repos/asf/jena/blob/d6f516de/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformEliminateAssignments.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformEliminateAssignments.java b/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformEliminateAssignments.java
index cd0904b..7d6cf40 100644
--- a/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformEliminateAssignments.java
+++ b/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformEliminateAssignments.java
@@ -145,6 +145,38 @@ public class TestTransformEliminateAssignments {
              "    (table unit)))");
         //@formatter:on
     }
+    
+    @Test
+    public void extend_02() {
+        // Assigned variable used only once can substitute expression for the
+        // later usage of the variable
+        // However we must be inside a projection as otherwise the assigned
+        // variable would be visible and we couldn't eliminate the assignment
+        //@formatter:off
+        test(StrUtils.strjoinNL("(project (?z)",
+                                "  (extend ((?x true) (?y ?x) (?z ?y))",
+                                "    (table unit)))"),
+             "(project (?z)",
+             "  (extend (?z true)",
+             "    (table unit)))");
+        //@formatter:on
+    }
+    
+    @Test
+    public void extend_03() {
+        // Assigned variable used only once can substitute expression for the
+        // later usage of the variable
+        // However we must be inside a projection as otherwise the assigned
+        // variable would be visible and we couldn't eliminate the assignment
+        //@formatter:off
+        test(StrUtils.strjoinNL("(project (?z)",
+                                "  (extend ((?a true) (?b ?a) (?c false) (?d ?c) (?z (|| ?b ?d)))",
+                                "    (table unit)))"),
+             "(project (?z)",
+             "  (extend (?z (|| true false))",
+             "    (table unit)))");
+        //@formatter:on
+    }
 
     @Test
     public void orderby_01() {


[2/3] jena git commit: Improve assignment inlining (JENA-780)

Posted by rv...@apache.org.
Improve assignment inlining (JENA-780)

Fixes a bug in how assignments were eliminated/inlined from usages in
extend and generally improves how we cope with extend such that we can
inline expressions where they are defined in the same extend as they are
used.


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

Branch: refs/heads/eliminate-assignments
Commit: 985b995be1f21c7ee0627a00326866b27babfd03
Parents: bdcf8a6
Author: Rob Vesse <rv...@apache.org>
Authored: Mon Jul 6 15:29:22 2015 +0100
Committer: Rob Vesse <rv...@apache.org>
Committed: Mon Jul 6 15:29:22 2015 +0100

----------------------------------------------------------------------
 .../optimize/TransformEliminateAssignments.java |  73 ++++++---
 .../TestTransformEliminateAssignments.java      | 160 ++++++++++++++++---
 2 files changed, 193 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/985b995b/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformEliminateAssignments.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformEliminateAssignments.java b/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformEliminateAssignments.java
index 91dc435..4d59fc3 100644
--- a/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformEliminateAssignments.java
+++ b/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformEliminateAssignments.java
@@ -120,17 +120,19 @@ public class TransformEliminateAssignments extends TransformCopy {
     }
 
     protected boolean canInline(Expr e) {
+        if (e == null)
+            return false;
         return ExprLib.isStable(e);
     }
 
     protected boolean shouldInline(Expr e) {
+        if (e == null)
+            return false;
+
         // Inline everything when being aggressive
         if (this.aggressive)
             return true;
 
-        if (e == null)
-            return false;
-
         // If not being aggressive only inline if the expression is a constant
         return e.isConstant() || e instanceof NodeValue;
     }
@@ -214,13 +216,47 @@ public class TransformEliminateAssignments extends TransformCopy {
 
         // See if there are any assignments we can eliminate entirely i.e. those
         // where the assigned value is never used
-        VarExprList assignments = processUnused(opExtend.getVarExprList());
-        if (assignments == null)
-            return super.transform(opExtend, subOp);
+        VarExprList unusedAssignments = processUnused(opExtend.getVarExprList());
+        VarExprList newAssignments = new VarExprList();
+        for (Var assignVar : opExtend.getVarExprList().getVars()) {
+            // If unused eliminate
+            if (unusedAssignments != null && unusedAssignments.contains(assignVar))
+                continue;
+
+            Expr currExpr = opExtend.getVarExprList().getExpr(assignVar);
+
+            // See what vars are used in the current expression
+            Collection<Var> vars = new ArrayList<>();
+            ExprVars.varsMentioned(vars, currExpr);
+
+            for (Var var : vars) {
+                // Usage count will be 2 if we can eliminate the assignment
+                // First usage is when it is introduced by the assignment and
+                // the second is when it is used now used in another assignment
+                Expr e = getAssignExpr(var);
+                if (this.tracker.getUsageCount(var) == 2 && hasAssignment(var) && canInline(e)) {
+                    // Can go back and eliminate that assignment
+                    subOp = eliminateAssignment(subOp, var);
+                    // Replace the variable usage with the expression within
+                    // expression
+                    currExpr = ExprTransformer.transform(new ExprTransformSubstitute(var, e), currExpr);
+                    this.tracker.getAssignments().remove(var);
+
+                    // If the assignment to be eliminated was introduced by the
+                    // extend we are processing need to remove it from the
+                    // VarExprList we are currently building
+                    if (newAssignments.contains(var) && newAssignments.getExpr(var).equals(e)) {
+                        newAssignments.getVars().remove(var);
+                        newAssignments.getExprs().remove(var);
+                    }
+                }
+            }
+            newAssignments.add(assignVar, currExpr);
+        }
 
-        // Can eliminate some assignments entirely
-        if (assignments.size() > 0) {
-            return OpExtend.extend(subOp, assignments);
+        // May be able to eliminate the extend entirely in some cases
+        if (newAssignments.size() > 0) {
+            return OpExtend.extend(subOp, newAssignments);
         } else {
             return subOp;
         }
@@ -230,20 +266,17 @@ public class TransformEliminateAssignments extends TransformCopy {
         if (CollectionUtils.disjoint(assignments.getVars(), this.tracker.getAssignments().keySet()))
             return null;
 
-        VarExprList modified = new VarExprList();
+        VarExprList singleUse = new VarExprList();
         for (Var var : assignments.getVars()) {
-            // If an assignment is used more than once then it must be preserved
-            // for now
-            if (this.tracker.getUsageCount(var) > 1)
-                modified.add(var, assignments.getExpr(var));
+            if (this.tracker.getUsageCount(var) == 1)
+                singleUse.add(var, assignments.getExpr(var));
         }
-
-        // If all assignments are used more than once then there are no changes
-        // and we return null
-        if (modified.size() == assignments.size())
+        
+        // If nothing is single use
+        if (singleUse.size() == 0)
             return null;
 
-        return modified;
+        return singleUse;
     }
 
     @Override
@@ -262,7 +295,7 @@ public class TransformEliminateAssignments extends TransformCopy {
         for (Var var : vars) {
             // Usage count will be 2 if we can eliminate the assignment
             // First usage is when it is introduced by the assignment and the
-            // second is when it is used now in this filter
+            // second is when it is used now in this order expression
             Expr e = getAssignExpr(var);
             if (this.tracker.getUsageCount(var) == 2 && hasAssignment(var) && canInline(e) && shouldInline(e)) {
                 // Can go back and eliminate that assignment

http://git-wip-us.apache.org/repos/asf/jena/blob/985b995b/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformEliminateAssignments.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformEliminateAssignments.java b/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformEliminateAssignments.java
index 163ce8c..cd0904b 100644
--- a/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformEliminateAssignments.java
+++ b/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformEliminateAssignments.java
@@ -56,11 +56,48 @@ public class TestTransformEliminateAssignments {
     }
 
     private void testNoChange(String... input) {
-        test(StrUtils.strjoinNL(input), (String[]) null);
+        testNoChange(false, input);
+    }
+
+    private void testNoChangeAggressive(String... input) {
+        testNoChange(true, input);
+    }
+
+    private void testNoChange(boolean aggressive, String... input) {
+        test(StrUtils.strjoinNL(input), aggressive, (String[]) null);
+    }
+
+    @Test
+    public void unused_01() {
+        // Assignments never used can be eliminated
+        // However we must be inside a projection as otherwise the assigned
+        // variable would be visible and we couldn't eliminate the assignment
+        //@formatter:off
+        test(StrUtils.strjoinNL("(project (?y)",
+                                "  (extend (?x true)",
+                                "    (table unit)))"),
+             "(project (?y)",
+             "  (table unit))");
+        //@formatter:on
+    }
+
+    @Test
+    public void unused_02() {
+        // Assignments never used can be eliminated
+        // However we must be inside a projection as otherwise the assigned
+        // variable would be visible and we couldn't eliminate the assignment
+        //@formatter:off
+        test(StrUtils.strjoinNL("(project (?y)",
+                                "  (extend ((?x true) (?y false))",
+                                "    (table unit)))"),
+             "(project (?y)",
+             "  (extend (?y false)",
+             "    (table unit)))");
+        //@formatter:on
     }
 
     @Test
-    public void single_use_extend_01() {
+    public void filter_01() {
         // Assigned variable used only once can substitute expression for the
         // later usage of the variable
         // However we must be inside a projection as otherwise the assigned
@@ -77,8 +114,9 @@ public class TestTransformEliminateAssignments {
     }
 
     @Test
-    public void single_use_extend_02() {
+    public void filter_02() {
         // Assignment for ?y can be removed because it is never used
+        // Assignment for ?x can be in-lined
         // However we must be inside a projection as otherwise the assigned
         // variable would be visible and we couldn't eliminate the assignment
         //@formatter:off
@@ -93,7 +131,23 @@ public class TestTransformEliminateAssignments {
     }
 
     @Test
-    public void single_use_extend_03() {
+    public void extend_01() {
+        // Assigned variable used only once can substitute expression for the
+        // later usage of the variable
+        // However we must be inside a projection as otherwise the assigned
+        // variable would be visible and we couldn't eliminate the assignment
+        //@formatter:off
+        test(StrUtils.strjoinNL("(project (?y)",
+                                "  (extend ((?x true) (?y ?x))",
+                                "    (table unit)))"),
+             "(project (?y)",
+             "  (extend (?y true)",
+             "    (table unit)))");
+        //@formatter:on
+    }
+
+    @Test
+    public void orderby_01() {
         // Assigned variable used only once can substitute expression for the
         // later usage of the variable
         // However we must be inside a projection as otherwise the assigned
@@ -110,7 +164,7 @@ public class TestTransformEliminateAssignments {
     }
 
     @Test
-    public void single_use_extend_complex_01() {
+    public void orderby_02() {
         // Assigned variable used only once can substitute expression for the
         // later usage of the variable
         // BUT we won't do this by default for complex expressions where they
@@ -124,7 +178,7 @@ public class TestTransformEliminateAssignments {
     }
 
     @Test
-    public void single_use_extend_complex_02() {
+    public void orderby_03() {
         // Assigned variable used only once can substitute expression for the
         // later usage of the variable
         // BUT we won't do this by default for complex expressions where they
@@ -141,9 +195,9 @@ public class TestTransformEliminateAssignments {
              "    (table unit)))");
         //@formatter:on
     }
-    
+
     @Test
-    public void single_use_extend_unstable_01() {
+    public void filter_unstable_01() {
         // Assigned variable used only once can substitute expression for the
         // later usage of the variable
         // EXCEPT if the expression is unstable in which case we leave it alone
@@ -154,9 +208,9 @@ public class TestTransformEliminateAssignments {
                                         "      (table unit))))"));
         //@formatter:on
     }
-    
+
     @Test
-    public void single_use_extend_unstable_02() {
+    public void filter_unstable_02() {
         // Assigned variable used only once can substitute expression for the
         // later usage of the variable
         // EXCEPT if the expression is unstable in which case we leave it alone
@@ -167,9 +221,9 @@ public class TestTransformEliminateAssignments {
                                         "      (table unit))))"));
         //@formatter:on
     }
-    
+
     @Test
-    public void single_use_extend_unstable_03() {
+    public void filter_unstable_03() {
         // Assigned variable used only once can substitute expression for the
         // later usage of the variable
         // EXCEPT if the expression is unstable in which case we leave it alone
@@ -180,9 +234,9 @@ public class TestTransformEliminateAssignments {
                                         "      (table unit))))"));
         //@formatter:on
     }
-    
+
     @Test
-    public void single_use_extend_unstable_04() {
+    public void filter_unstable_04() {
         // Assigned variable used only once can substitute expression for the
         // later usage of the variable
         // EXCEPT if the expression is unstable in which case we leave it alone
@@ -195,7 +249,59 @@ public class TestTransformEliminateAssignments {
     }
 
     @Test
-    public void single_use_extend_outside_projection_01() {
+    public void orderby_unstable_01() {
+        // Assigned variable used only once can substitute expression for the
+        // later usage of the variable
+        // EXCEPT if the expression is unstable in which case we leave it alone
+        //@formatter:off
+        testNoChangeAggressive(StrUtils.strjoinNL("(project (?y)",
+                                                  "  (order (?x)",
+                                                  "    (extend (?x (rand))",
+                                                  "      (table unit))))"));
+        //@formatter:on
+    }
+
+    @Test
+    public void orderby_unstable_02() {
+        // Assigned variable used only once can substitute expression for the
+        // later usage of the variable
+        // EXCEPT if the expression is unstable in which case we leave it alone
+        //@formatter:off
+        testNoChangeAggressive(StrUtils.strjoinNL("(project (?y)",
+                                                  "  (order (?x)",
+                                                  "    (extend (?x (uuid))",
+                                            "      (table unit))))"));
+        //@formatter:on
+    }
+
+    @Test
+    public void orderby_unstable_03() {
+        // Assigned variable used only once can substitute expression for the
+        // later usage of the variable
+        // EXCEPT if the expression is unstable in which case we leave it alone
+        //@formatter:off
+        testNoChangeAggressive(StrUtils.strjoinNL("(project (?y)",
+                                                  "  (order (?x)",
+                                                  "    (extend (?x (struuid))",
+                                                  "      (table unit))))"));
+        //@formatter:on
+    }
+
+    @Test
+    public void orderby_unstable_04() {
+        // Assigned variable used only once can substitute expression for the
+        // later usage of the variable
+        // EXCEPT if the expression is unstable in which case we leave it alone
+        //@formatter:off
+        testNoChangeAggressive(StrUtils.strjoinNL("(project (?y)",
+                                                  "  (order (?x)",
+                                                  "    (extend (?x (bnode))",
+                                                  "      (table unit))))"));
+        //@formatter:on
+    }
+
+    @Test
+    public void ineligible_01() {
         // Cannot eliminate as there is no projection so the assigned variable
         // is visible even though in the algebra given it is used only once
         //@formatter:off
@@ -206,7 +312,7 @@ public class TestTransformEliminateAssignments {
     }
 
     @Test
-    public void single_use_extend_outside_projection_02() {
+    public void ineligible_02() {
         // Cannot eliminate as there is no projection so the assigned variable
         // is visible even though in the algebra given it is used only once
         //@formatter:off
@@ -217,7 +323,7 @@ public class TestTransformEliminateAssignments {
     }
 
     @Test
-    public void multi_use_extend_01() {
+    public void ineligible_03() {
         // As the assigned variable is used multiple times we leave the
         // assignment alone
         //@formatter:off
@@ -229,7 +335,7 @@ public class TestTransformEliminateAssignments {
     }
 
     @Test
-    public void multi_use_extend_02() {
+    public void ineligible_04() {
         // Because the value of the assignment is used in multiple places we
         // leave the assignment alone
         //@formatter:off
@@ -243,7 +349,21 @@ public class TestTransformEliminateAssignments {
     }
 
     @Test
-    public void scoped_use_extend_01() {
+    public void scope_01() {
+        // If the assignment is out of scope by the time it is used in the outer
+        // scope then we can't substitute it out there
+        // In this test the outer ?x is technically different from the inner ?x
+        // anyway because of the projection
+        //@formatter:off
+        testNoChange(StrUtils.strjoinNL("(filter (exprlist ?x)",
+                                        "  (project (?x ?y)",
+                                        "    (extend (?x true)",
+                                        "      (table unit))))"));
+        //@formatter:on
+    }
+
+    @Test
+    public void scope_02() {
         // If the assignment is out of scope by the time it is used in the outer
         // scope then we can't substitute it out there
         // However if the scoping means the value is never used we can instead
@@ -260,7 +380,7 @@ public class TestTransformEliminateAssignments {
     }
 
     @Test
-    public void scoped_use_extend_02() {
+    public void scope_03() {
         // If the assignment is out of scope by the time it is used in the outer
         // scope then we can't substitute it out there
         // However in this case we can substitute it in the inner scope