You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jena.apache.org by an...@apache.org on 2022/03/23 17:14:34 UTC

[jena] branch main updated: JENA-2317: Reorder BGPs before filter placement

This is an automated email from the ASF dual-hosted git repository.

andy pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/jena.git


The following commit(s) were added to refs/heads/main by this push:
     new a002611  JENA-2317: Reorder BGPs before filter placement
     new 6e4837e  Merge pull request #1230 from afs/bgp-reorder
a002611 is described below

commit a00261120bb2706883a9109404514361072a421d
Author: Andy Seaborne <an...@apache.org>
AuthorDate: Sat Mar 19 17:12:55 2022 +0000

    JENA-2317: Reorder BGPs before filter placement
---
 .../src/main/java/org/apache/jena/query/ARQ.java   |   9 +-
 .../jena/sparql/algebra/optimize/OptimizerStd.java |  21 ++-
 .../optimize/TransformFilterDisjunction.java       | 201 +++++++++------------
 .../engine/main/StageGeneratorGenericStar.java     |  28 ---
 .../engine/optimizer/reorder/ReorderFixed.java     |  10 +-
 .../reorder/ReorderTransformationSubstitution.java |  32 ++--
 .../sparql/algebra/optimize/TestOptimizer.java     |  49 +++--
 .../sparql/algebra/optimize/TestReorderBGP.java    |   2 +-
 8 files changed, 164 insertions(+), 188 deletions(-)

diff --git a/jena-arq/src/main/java/org/apache/jena/query/ARQ.java b/jena-arq/src/main/java/org/apache/jena/query/ARQ.java
index 8817e0f..2624520 100644
--- a/jena-arq/src/main/java/org/apache/jena/query/ARQ.java
+++ b/jena-arq/src/main/java/org/apache/jena/query/ARQ.java
@@ -358,7 +358,7 @@ public class ARQ
 
     /**
      *  Context key controlling whether to do filter placement within BGP and quad blocks.
-     *  Modies the effect of optFilterPlacement.
+     *  Modifies the effect of optFilterPlacement.
      *  Default is "true" - filter placement is pushed into BGPs.
      */
     public static final Symbol optFilterPlacementBGP = SystemARQ.allocSymbol("optFilterPlacementBGP") ;
@@ -497,10 +497,11 @@ public class ARQ
     /**
      * Context key controlling whether the standard optimizer applies the optimization
      * to reorder basic graph patterns.
-     * <p>By default, this transformation is NOT applied.
-     * It is left to the specific engines to decide.
+     * This is an algebra optimizer step. Because it interacts with filter placement,
+     * it is worth doing even though there are later reorderings.
      */
-    // However, StageGeneratorGeneric does reorder based on partial results.
+    // StageGeneratorGeneric does reorder based on partial results.
+    // TDB reorders based on stats when the input binding is known.
     public static final Symbol optReorderBGP = SystemARQ.allocSymbol("optReorderBGP");
 
     /**
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/algebra/optimize/OptimizerStd.java b/jena-arq/src/main/java/org/apache/jena/sparql/algebra/optimize/OptimizerStd.java
index 76fdb52..98c9ffb 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/algebra/optimize/OptimizerStd.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/algebra/optimize/OptimizerStd.java
@@ -133,11 +133,22 @@ public class OptimizerStd implements Rewrite
         if ( context.isTrueOrUndef(ARQ.optIndexJoinStrategy) )
             op = transformJoinStrategy(op) ;
 
-        // Place filters close to where their dependency variables are defined.
+        // Do a basic reordering so that triples with more defined terms go first.
+        if ( context.isTrueOrUndef(ARQ.optReorderBGP) )
+            op = transformReorder(op) ;
+
+        // Place filters close to where their input variables are defined.
         // This prunes the output of that step as early as possible.
+        //
+        // This is done after BGP reordering because inserting the filters breaks up BGPs,
+        // and would make transformReorder complicated, and also because a two-term triple pattern
+        // is (probably) more specific than many filters.
+        //
+        // Filters in involving equality are done separately.
+
         // If done before TransformJoinStrategy, you can get two applications
-        // of a filter in a (sequence) from each half of a (join).  This is harmless,
-        // because filters are generally cheap, but it looks a bit bad.
+        // of a filter in a (sequence) from each half of a (join).
+        // This is harmless but it looks a bit odd.
         if ( context.isTrueOrUndef(ARQ.optFilterPlacement) )
             op = transformFilterPlacement(op) ;
 
@@ -162,10 +173,6 @@ public class OptimizerStd implements Rewrite
         if ( context.isTrueOrUndef(ARQ.optMergeBGPs) )
             op = transformMergeBGPs(op) ;
 
-        // Normally, leave to the specific engines.
-        if ( context.isTrue(ARQ.optReorderBGP) )
-            op = transformReorder(op) ;
-
         // Merge (extend) and (assign) stacks
         if ( context.isTrueOrUndef(ARQ.optMergeExtends) )
             op = transformExtendCombine(op) ;
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/algebra/optimize/TransformFilterDisjunction.java b/jena-arq/src/main/java/org/apache/jena/sparql/algebra/optimize/TransformFilterDisjunction.java
index 66c03da..429e282 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/algebra/optimize/TransformFilterDisjunction.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/algebra/optimize/TransformFilterDisjunction.java
@@ -32,156 +32,131 @@ import org.apache.jena.sparql.expr.E_LogicalOr ;
 import org.apache.jena.sparql.expr.Expr ;
 import org.apache.jena.sparql.expr.ExprList ;
 
-/**Filter disjunction.
- * Merge with TransformFilterImprove
+/**
+ * Filter disjunction. This covers the case of
+ * <pre>
+ *  (filter (|| expr1 expr2) pattern)</pre>
+ * where either or both of {@code expr1} and {@code expr2} are equalities that help
+ * ground the pattern. This includes {@code ?x IN (....)} so this optimization can a
+ * significant improvement.
  */
 
-public class TransformFilterDisjunction extends TransformCopy
-{
+public class TransformFilterDisjunction extends TransformCopy {
     public TransformFilterDisjunction() {}
-    
+
     @Override
-    public Op transform(OpFilter opFilter, final Op subOp)
-    {
-        ExprList exprList = opFilter.getExprs() ;
-        
+    public Op transform(OpFilter opFilter, final Op subOp) {
+        ExprList exprList = opFilter.getExprs();
+
         // First pass - any disjunctions at all?
-        boolean processDisjunction = false ;
-        for ( Expr expr : exprList )
-        {
-            if ( isDisjunction(expr) )
-            {
-                processDisjunction = true ;
-                break ;
+        boolean processDisjunction = false;
+        for ( Expr expr : exprList ) {
+            if ( isDisjunction(expr) ) {
+                processDisjunction = true;
+                break;
             }
         }
-        
-        // Still may be a disjunction in a form we don't optimize. 
-        if ( ! processDisjunction )
-            return super.transform(opFilter, subOp) ;
-        
-        ExprList exprList2 = new ExprList() ;
-        Op newOp = subOp ;
-        // remember what's been seen so that FILTER(?x = <x> || ?x = <x> ) does not result in two transforms. 
-        Set<Expr> doneSoFar = new HashSet<>() ;
-        
-        for ( Expr expr : exprList )
-        {
-            if ( ! isDisjunction(expr) )
-            {
-                // Assignment there?
-                exprList2.add(expr) ;
-                continue ;
+
+        // Still may be a disjunction in a form we don't optimize.
+        if ( !processDisjunction )
+            return super.transform(opFilter, subOp);
+
+        ExprList exprList2 = new ExprList();
+        Op newOp = subOp;
+        // remember what's been seen so that FILTER(?x = <x> || ?x = <x> ) does not
+        // result in two transforms.
+        Set<Expr> doneSoFar = new HashSet<>();
+
+        for ( Expr expr : exprList ) {
+            if ( !isDisjunction(expr) ) {
+                // not for this transform.
+                exprList2.add(expr);
+                continue;
             }
-            
+
 //            // Relies on expression equality.
 //            if ( doneSoFar.contains(expr) )
 //                continue ;
 //            // Must be canonical: ?x = <x> is the same as <x> = ?x
 //            doneSoFar.add(expr) ;
-            
-            Op op2 = expandDisjunction(expr, newOp) ;
-            
+
+            Op op2 = expandDisjunction(expr, newOp);
             if ( op2 != null )
-                newOp = op2 ;
+                newOp = op2;
         }
 
         if ( exprList2.isEmpty() )
-            return newOp ;
+            return newOp;
 
         // There should have been at least on disjunction.
         if ( newOp == subOp ) {
-            Log.warn(this, "FilterDisjunction assumption failure: didn't find a disjunction after all") ;
-            return super.transform(opFilter, subOp) ;
+            Log.warn(this, "FilterDisjunction assumption failure: didn't find a disjunction after all");
+            return super.transform(opFilter, subOp);
         }
-            
-
-        // Failed.  These a was one or more expressions we couldn't handle.
-        // So the full pattern is going to be executed anyway. 
-        //return super.transform(super.transform(opFilter, subOp)) ;
-        
-        
-        // Put the non-disjunctions outside the disjunction and the pattern rewrite. 
-        Op opOther = OpFilter.filterBy(exprList2, newOp) ;
-        if ( opOther instanceof OpFilter) {
-            return opOther ;
+
+        // Put the non-disjunctions outside the disjunction and the pattern rewrite.
+        Op opOther = OpFilter.filterBy(exprList2, newOp);
+        if ( opOther instanceof OpFilter ) {
+            return opOther;
         }
-            
-        // opOther is not a filter any more - should not happen but to isolate from future changes ...
-        Log.warn(this, "FilterDisjunction assumption failure: not a filter after processing disjunction/other mix") ;
-        return super.transform(opFilter, subOp) ;
+
+        // opOther is not a filter any more - should not happen but to isolate from
+        // future changes ...
+        Log.warn(this, "FilterDisjunction assumption failure: not a filter after processing disjunction/other mix");
+        return super.transform(opFilter, subOp);
     }
-    
-    private boolean isDisjunction(Expr expr)
-    {
-        return ( expr instanceof E_LogicalOr ) ; 
+
+    private boolean isDisjunction(Expr expr) {
+        return (expr instanceof E_LogicalOr);
     }
 
-    // Todo:
-    // 1 - convert TransformEqualityFilter to use ExprLib for testing.
-    // 2 - Scan for safe equality filters in disjunction.
-    
-    public static Op expandDisjunction(Expr expr, Op subOp)
-    {
-//        if ( !( expr instanceof E_LogicalOr ) )
-//            return null ;
-
-        List<Expr> exprList = explodeDisjunction(new ArrayList<Expr>(), expr) ;
-        
-        // All disjunctions - some can be done efficiently via assignments, some can not.
-        // Really should only do if every disjunction can turned into a assign-grounded pattern
-        // otherwise the full is done anyway. 
-        
-        List<Expr> exprList2 = null ;
-        Op op = null ;
-        for ( Expr e : exprList )
-        {
-            Op op2 = TransformFilterEquality.processFilter(e, subOp) ;
-            if ( op2 == null )
-            {
+    public static Op expandDisjunction(Expr expr, Op subOp) {
+        List<Expr> exprList = explodeDisjunction(new ArrayList<Expr>(), expr);
+
+        // All disjunctions - some can be done efficiently via assignments,
+        // some can not (value tests).
+        List<Expr> exprList2 = null;
+        Op op = null;
+        for ( Expr e : exprList ) {
+            Op op2 = TransformFilterEquality.processFilter(e, subOp);
+            if ( op2 == null ) {
                 // Not done.
                 if ( exprList2 == null )
-                    exprList2 = new ArrayList<>() ;
-                exprList2.add(e) ;
-                //continue ;
-                // Can't do one so don't do any as the original pattern is still executed. 
+                    exprList2 = new ArrayList<>();
+                exprList2.add(e);
             }
 
-            op = OpDisjunction.create(op, op2) ;
+            op = OpDisjunction.create(op, op2);
         }
-        
-        if ( exprList2 != null && !exprList2.isEmpty() )
-        {
+
+        if ( exprList2 != null && !exprList2.isEmpty() ) {
             // These are left as disjunctions.
-            Expr eOther = null ;
-            for ( Expr e : exprList2 )
-            {
+            Expr eOther = null;
+            for ( Expr e : exprList2 ) {
                 if ( eOther == null )
-                    eOther = e ;
+                    eOther = e;
                 else
-                    eOther = new E_LogicalOr(eOther, e) ;
+                    eOther = new E_LogicalOr(eOther, e);
             }
-            Op opOther = OpFilter.filter(eOther, subOp) ;
-            op = OpDisjunction.create(op, opOther) ;
+            Op opOther = OpFilter.filter(eOther, subOp);
+            op = OpDisjunction.create(op, opOther);
         }
-        
-        return op ;
+
+        return op;
     }
 
-    /** Explode a expr into a list of disjunctions */
-    private static List<Expr> explodeDisjunction(List<Expr> exprList, Expr expr)
-    {
-        if ( !( expr instanceof E_LogicalOr ) )
-        {
-            exprList.add(expr) ;
-            return exprList ;
+    /** Explode an expr into a list of disjunctions */
+    private static List<Expr> explodeDisjunction(List<Expr> exprList, Expr expr) {
+        if ( !(expr instanceof E_LogicalOr) ) {
+            exprList.add(expr);
+            return exprList;
         }
-        
-        E_LogicalOr exprOr = (E_LogicalOr)expr ;
-        Expr e1 =  exprOr.getArg1() ;
-        Expr e2 =  exprOr.getArg2() ;
-        explodeDisjunction(exprList, e1) ; 
-        explodeDisjunction(exprList, e2) ;
-        return exprList ;
+
+        E_LogicalOr exprOr = (E_LogicalOr)expr;
+        Expr e1 = exprOr.getArg1();
+        Expr e2 = exprOr.getArg2();
+        explodeDisjunction(exprList, e1);
+        explodeDisjunction(exprList, e2);
+        return exprList;
     }
 }
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/StageGeneratorGenericStar.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/StageGeneratorGenericStar.java
deleted file mode 100644
index 9bb2a7a..0000000
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/StageGeneratorGenericStar.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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 org.apache.jena.sparql.engine.main;
-
-/**
- * Generic - always works - StageGenerator.
- * @deprecated Use {@link StageGeneratorGeneric}
- */
-@Deprecated
-public class StageGeneratorGenericStar extends StageGeneratorGeneric {
-    public StageGeneratorGenericStar() {}
-}
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/optimizer/reorder/ReorderFixed.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/optimizer/reorder/ReorderFixed.java
index 89a1c46..fd1351e 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/optimizer/reorder/ReorderFixed.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/optimizer/reorder/ReorderFixed.java
@@ -73,23 +73,23 @@ public class ReorderFixed extends ReorderTransformationSubstitution {
     static { init() ; }
 
     private static void init() {
-        // Set constant late.
-
         // rdf:type can be a bad choice e.g rdf:type rdf:Resource
         // with inference enabled.
+        // Use two weighing matchers, one for rdf:type and done for otherwise.
         // Weight use of rdf:type worse then the general pattern
         // that would also match by using two matchers.
 
-        // Numbers chosen as an approximation ratios for a graph of 100 triples
-
-        // 1 : TERM type TERM is builtin (SPO).
+        // Weight 1 / TERM type TERM is builtin because it must be one.
         // matcherRdfType.addPattern(new Pattern(1, TERM, TERM, TERM)) ;
+
+        // matcher for rdf:type.
         matcherRdfType.addPattern(new Pattern(5, VAR, rdfType, TERM)) ;
         matcherRdfType.addPattern(new Pattern(50, VAR, rdfType, VAR)) ;
 
         // SPO - built-in - not needed as a rule
         // matcher.addPattern(new Pattern(1, TERM, TERM, TERM)) ;
 
+        // matcher for P != rdf:type
         matcher.addPattern(new Pattern(2, TERM, TERM, VAR)) ;                   // SP?
         matcher.addPattern(new Pattern(3, VAR, TERM, TERM)) ;                   // ?PO
         matcher.addPattern(new Pattern(2, TERM, VAR, TERM)) ;                   // S?O
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/optimizer/reorder/ReorderTransformationSubstitution.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/optimizer/reorder/ReorderTransformationSubstitution.java
index 6cdbaff..66ce774 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/optimizer/reorder/ReorderTransformationSubstitution.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/optimizer/reorder/ReorderTransformationSubstitution.java
@@ -38,17 +38,19 @@ import org.slf4j.LoggerFactory ;
 /** Machinery.
  * This code implements the connectiveness assumed by execution based on substitution (index joins).
  * i.e. if <code>{ ?x :p ?v . ?x :q ?w }</code> then <code>?x</code> is <code>TERM</code>
- * at the second triple. 
+ * at the second triple.
  */
 public abstract class ReorderTransformationSubstitution implements ReorderTransformation
 {
     static public final Logger log = LoggerFactory.getLogger(ReorderTransformationSubstitution.class) ;
-    static private final boolean DEBUG = false ;  
-    
+    static private final boolean DEBUG = false ;
+
     public ReorderTransformationSubstitution() {}
-    
+
     @Override
-    public BasicPattern reorder(BasicPattern pattern) {
+    public  final BasicPattern reorder(BasicPattern pattern) {
+        if ( pattern.size() <= 1 )
+            return pattern;
         return reorderIndexes(pattern).reorder(pattern) ;
     }
 
@@ -78,7 +80,7 @@ public abstract class ReorderTransformationSubstitution implements ReorderTransf
     private String formatted(List<PatternTriple> components) {
         return components.stream().map(c->"(" + printAbbrev(c.toString()) + ")").collect(Collectors.joining(" "));
     }
-    
+
     protected ReorderProc reorder(List<Triple> triples, List<PatternTriple> components) {
         int N = components.size() ;
         int numReorder = N ;        // Maybe choose 4, say, and copy over the rest.
@@ -110,8 +112,8 @@ public abstract class ReorderTransformationSubstitution implements ReorderTransf
         ReorderProc proc = new ReorderProcIndexes(indexes) ;
 
         return proc ;
-    }    
-    
+    }
+
     /** Return index of next pattern triple */
     protected int chooseNext(List<PatternTriple> pTriples) {
         if ( DEBUG ) {
@@ -171,7 +173,7 @@ public abstract class ReorderTransformationSubstitution implements ReorderTransf
                             x = 0.01 ;
                             break ;
                         case LAST :
-                            // Default action : 
+                            // Default action :
                             break ;
                         case ZERO :
                             x = 0 ;
@@ -181,7 +183,7 @@ public abstract class ReorderTransformationSubstitution implements ReorderTransf
                             break ;
                     }
                 }
-                
+
                 // Not found. No default action.
                 // Make sure something is returned but otherwise ignore this pattern (goes
                 // last).
@@ -214,18 +216,18 @@ public abstract class ReorderTransformationSubstitution implements ReorderTransf
 
     /** Return the weight of the pattern, or -1 if no knowledge for it */
     protected abstract double weight(PatternTriple pt) ;
-    
+
     protected enum DefaultChoice { ZERO, LAST, FIRST , NUMERIC ; }
     /** What to do if the {@link weight} comes back as "not found".
      * Choices are:
-     *    ZERO      Assume the weight is zero (the rules were complete over the data so this is a pattern that will not match the data. 
+     *    ZERO      Assume the weight is zero (the rules were complete over the data so this is a pattern that will not match the data.
      *    LAST      Place after all explicitly weighted triple patterns
      *    FIRST     Place before all explicitly weighted triple patterns
      *    NUMERIC   Use value returned by {@link defaultWeight}
-     * The default, default choice is LAST.   
+     * The default, default choice is LAST.
      */
     protected DefaultChoice defaultChoice(PatternTriple pt) { return null ; } // return DefaultChoice.LAST ; }
-    
+
     protected double defaultWeight(PatternTriple pt) { return -1 ; }
 
     /** Update components to note any variables from triple */
@@ -251,7 +253,7 @@ public abstract class ReorderTransformationSubstitution implements ReorderTransf
                 elt.object = PatternElements.TERM ;
         }
     }
-    
+
     /** Update based on a variable/value (c.f. Substitute.substitute) */
     protected static void update(Var var, Node value, List<PatternTriple> components) {
         for ( PatternTriple elt : components )
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/algebra/optimize/TestOptimizer.java b/jena-arq/src/test/java/org/apache/jena/sparql/algebra/optimize/TestOptimizer.java
index 8a2779c..62e5891 100644
--- a/jena-arq/src/test/java/org/apache/jena/sparql/algebra/optimize/TestOptimizer.java
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/algebra/optimize/TestOptimizer.java
@@ -242,23 +242,42 @@ public class TestOptimizer extends AbstractTestTransform
              ,"      (triple ?var2 :p2 ?var3)"
              ,"    ))") ;
 
-        String out = StrUtils.strjoinNL
+        // Answer when  reorder BGPs before general filter placements.
+        String expected = StrUtils.strjoinNL
             ("(filter (!= ?VAR 123)"
-             ," (disjunction"
-             ,"  (assign ((?var3 'ABC'))"
-             ,"    (sequence"
-             ,"      (filter (regex ?var4 'pat1')"
-             ,"        (bgp (triple ?var2 :p1 ?var4)))"
-             ,"      (bgp (triple ?var2 :p2 'ABC'))))"
-             ,"  (assign ((?var3 'XYZ'))"
-             ,"    (sequence"
-             ,"      (filter (regex ?var4 'pat1')"
-             ,"        (bgp (triple ?var2 :p1 ?var4)))"
-             ,"      (bgp (triple ?var2 :p2 'XYZ'))))))"
-             ) ;
-        checkAlgebra(in, out) ;
-    }
+            ,"  (disjunction"
+            ,"      (assign ((?var3 'ABC'))"
+            ,"        (filter (regex ?var4 'pat1')"
+            ,"          (bgp"
+            ,"            (triple ?var2 <http://example/p2> 'ABC')"
+            ,"            (triple ?var2 <http://example/p1> ?var4)"
+            ,"          )))"
+            ,"      (assign ((?var3 'XYZ'))"
+            ,"        (filter (regex ?var4 'pat1')"
+            ,"          (bgp"
+            ,"           (triple ?var2 <http://example/p2> 'XYZ')"
+            ,"            (triple ?var2 <http://example/p1> ?var4)"
+            ,"         )))))"
+            );
 
+        checkAlgebra(in, expected) ;
+
+        // Before JENA-2317 when BGP reordering was done in the algebra optimization phase.
+//        String out = StrUtils.strjoinNL
+//                ("(filter (!= ?VAR 123)"
+//                 ," (disjunction"
+//                 ,"   (assign ((?var3 'ABC'))"
+//                 ,"     (sequence"
+//                 ,"       (filter (regex ?var4 'pat1')"
+//                 ,"         (bgp (triple ?var2 :p1 ?var4)))"
+//                 ,"       (bgp (triple ?var2 :p2 'ABC'))))"
+//                 ,"   (assign ((?var3 'XYZ'))"
+//                 ,"     (sequence"
+//                 ,"       (filter (regex ?var4 'pat1')"
+//                 ,"         (bgp (triple ?var2 :p1 ?var4)))"
+//                 ,"       (bgp (triple ?var2 :p2 'XYZ'))))))"
+//                 ) ;
+    }
 
     @Test public void combine_extend_01()
     {
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/algebra/optimize/TestReorderBGP.java b/jena-arq/src/test/java/org/apache/jena/sparql/algebra/optimize/TestReorderBGP.java
index e0667c3..c6345ff 100644
--- a/jena-arq/src/test/java/org/apache/jena/sparql/algebra/optimize/TestReorderBGP.java
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/algebra/optimize/TestReorderBGP.java
@@ -52,7 +52,7 @@ public class TestReorderBGP extends AbstractTestTransform {
         testOp("(bgp (?s rdf:type :T) (?s :p 123) )", t_reorder, "(bgp  (?s :p 123) (?s rdf:type :T))") ;
     }
 
-    private static String expected3 = "(bgp  (?s :p 123)  (?s rdf:type :T) (?s :p ?o) )" ;
+    private static String expected3 = "(bgp  (?s :p 123) (?s rdf:type :T) (?s :p ?o) )" ;
 
     @Test public void reorderbgp_3_1() {
         testOp("(bgp (?s rdf:type :T) (?s :p ?o) (?s :p 123) )",