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 2014/09/12 11:32:10 UTC

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

Author: andy
Date: Fri Sep 12 09:32:09 2014
New Revision: 1624485

URL: http://svn.apache.org/r1624485
Log:
JENA-779 : Rework TransformFilterPlacement.processExtendAssign.

This is the JENA-779-filter-extend_distinct.patch. Includes filter movement through distinct and reduced operators.

Includes refactoring code to remove duplication for a class of "op1" processing.



Added:
    jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/op/OpDistinctReduced.java   (with props)
Modified:
    jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/op/OpDistinct.java
    jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/op/OpReduced.java
    jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement.java
    jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformFilterPlacement.java

Modified: jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/op/OpDistinct.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/op/OpDistinct.java?rev=1624485&r1=1624484&r2=1624485&view=diff
==============================================================================
--- jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/op/OpDistinct.java (original)
+++ jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/op/OpDistinct.java Fri Sep 12 09:32:09 2014
@@ -24,7 +24,7 @@ import com.hp.hpl.jena.sparql.algebra.Tr
 import com.hp.hpl.jena.sparql.sse.Tags ;
 import com.hp.hpl.jena.sparql.util.NodeIsomorphismMap ;
 
-public class OpDistinct extends OpModifier
+public class OpDistinct extends OpDistinctReduced
 {
     public static Op create(Op op)
     {

Added: jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/op/OpDistinctReduced.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/op/OpDistinctReduced.java?rev=1624485&view=auto
==============================================================================
--- jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/op/OpDistinctReduced.java (added)
+++ jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/op/OpDistinctReduced.java Fri Sep 12 09:32:09 2014
@@ -0,0 +1,28 @@
+/*
+ * 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.op;
+
+import com.hp.hpl.jena.sparql.algebra.Op ;
+
+public abstract class OpDistinctReduced extends OpModifier
+{
+    public OpDistinctReduced(Op subOp) {
+        super(subOp) ;
+    }
+}

Propchange: jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/op/OpDistinctReduced.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/op/OpReduced.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/op/OpReduced.java?rev=1624485&r1=1624484&r2=1624485&view=diff
==============================================================================
--- jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/op/OpReduced.java (original)
+++ jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/op/OpReduced.java Fri Sep 12 09:32:09 2014
@@ -24,7 +24,7 @@ import com.hp.hpl.jena.sparql.algebra.Tr
 import com.hp.hpl.jena.sparql.sse.Tags ;
 import com.hp.hpl.jena.sparql.util.NodeIsomorphismMap ;
 
-public class OpReduced extends OpModifier
+public class OpReduced extends OpDistinctReduced
 {
     public static Op create(Op op)
     {

Modified: jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement.java?rev=1624485&r1=1624484&r2=1624485&view=diff
==============================================================================
--- jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement.java (original)
+++ jena/trunk/jena-arq/src/main/java/com/hp/hpl/jena/sparql/algebra/optimize/TransformFilterPlacement.java Fri Sep 12 09:32:09 2014
@@ -31,7 +31,9 @@ import com.hp.hpl.jena.sparql.algebra.Tr
 import com.hp.hpl.jena.sparql.algebra.op.* ;
 import com.hp.hpl.jena.sparql.core.BasicPattern ;
 import com.hp.hpl.jena.sparql.core.Var ;
-import com.hp.hpl.jena.sparql.expr.* ;
+import com.hp.hpl.jena.sparql.expr.Expr ;
+import com.hp.hpl.jena.sparql.expr.ExprLib ;
+import com.hp.hpl.jena.sparql.expr.ExprList ;
 import com.hp.hpl.jena.sparql.util.VarUtils ;
 
 /**
@@ -145,9 +147,10 @@ public class TransformFilterPlacement ex
         return buildFilter(placement) ;
     }
 
+    /** Transform and return a placement */ 
     private Placement transform(ExprList exprs, Op input) {
         // Dispatch by visitor??
-        Placement placement = null ;
+        Placement placement = noChangePlacement ;
 
         if ( input instanceof OpBGP )
             placement = placeOrWrapBGP(exprs, (OpBGP)input) ;
@@ -175,8 +178,24 @@ public class TransformFilterPlacement ex
             placement = placeExtend(exprs, (OpExtend)input) ;
         else if ( input instanceof OpAssign )
             placement = placeAssign(exprs, (OpAssign)input) ;
+        // Modifiers
+//        else if ( input instanceof OpGroup ) {
+//            // TODO
+//            placement = noChangePlacement ;
+//        }
+//        else if ( input instanceof OpSlice ) {
+//            // Not sure what he best choice is here.
+//            placement = noChangePlacement ;
+//        }
+//        else if ( input instanceof OpTopN ) {
+//            // Not sure what he best choice is here.
+//            placement = noChangePlacement ;
+//        }
+
         else if ( input instanceof OpProject )
             placement = placeProject(exprs, (OpProject)input) ;
+        else if ( input instanceof OpDistinctReduced ) 
+            placement = placeDistinctReduced(exprs, (OpDistinctReduced)input) ;
         else if ( input instanceof OpTable )
             placement = placeTable(exprs, (OpTable)input) ;
 
@@ -207,11 +226,6 @@ public class TransformFilterPlacement ex
         else
             return wrapBGP(exprsIn, pattern) ;
     }
-    
-    public static Placement debugPlaceBGP(ExprList exprsIn, BasicPattern pattern) {
-        return placeBGP(exprsIn, pattern) ;
-    }
-    
     // See also placeQuadPattern.
     // 
     // An improvement might be to put any filters that apply to exactly one triple
@@ -322,7 +336,6 @@ public class TransformFilterPlacement ex
         }
         return result(op, exprs) ;
     }
-
     
     /** Wrap the Graph node, Basic Pattern with any applicable expressions from the ExprList
      *  but do not break up the BasicPattern in any way.
@@ -489,13 +502,12 @@ public class TransformFilterPlacement ex
             Op right = input.getRight() ;
             Placement pRight = transform(exprs, right) ;
             
-            // JENA-652 Temporary fix
             if ( pLeft != null && ! pLeft.unplaced.isEmpty() )
                 return noChangePlacement ;
             if ( pRight != null && ! pRight.unplaced.isEmpty() )
                 return noChangePlacement ;
 
-            // Old, buggy if not guarded by the above.
+            // Musrt be guarded by the above.
             left = transformOpAlways(exprs, left) ;
             right = transformOpAlways(exprs, right) ;
             
@@ -562,46 +574,90 @@ public class TransformFilterPlacement ex
     
     private Placement placeAssign(ExprList exprs, OpAssign input) {
         return processExtendAssign(exprs, input) ;
-        
+    }
+    
+    /* Complete processing for an Op1. 
+     * Having split expressions into pushed an dunpsuehd at thispoint,
+     * try to push "psuehd" down further into the subOp.
+     */  
+    private Placement processSubOp1(ExprList pushed, ExprList unpushed, Op1 input) {
+        Op opSub = input.getSubOp() ;
+        Placement subPlacement = transform(pushed, opSub) ;
+        if ( subPlacement == null ) {
+            // (Same as if a placement of the exprlist and op passed in is given).
+            // Didn't make any changes below, so add a filter for the 'pushed' and
+            // return a placement for the unpushed. 
+            Op op1 = input.getSubOp() ;
+            if ( pushed != null &&! pushed.isEmpty() )
+                op1 = OpFilter.filter(pushed, op1) ;
+            Op op2 = input.copy(op1) ;
+            return result(op2, unpushed) ;
+        }
+        // Did make changes below.  Add filter for these (which includes the "pushed" at this level,
+        // now in the p.op or left in p.unplaced.
+        Op op_a = OpFilter.filter(subPlacement.unplaced, subPlacement.op) ;
+        op_a =  input.copy(op_a) ;
+       return result(op_a, unpushed) ;
     }
 
     private Placement processExtendAssign(ExprList exprs, OpExtendAssign input) {
-        // Could break up the VarExprList
-        Collection<Var> vars1 = input.getVarExprList().getVars() ;
+        // We assume that each (extend) and (assign) is in simple form - always one 
+        // assignment.  We cope with more but do not attempt reordering of assignments.
+        List<Var> vars1 = input.getVarExprList().getVars() ;
         ExprList pushed = new ExprList() ;
         ExprList unpushed = new ExprList() ;
+        Op subOp = input.getSubOp() ;
+        Set<Var> subVars = OpVars.fixedVars(subOp) ;
+        
+//        // ----
+//        // Backwards.
+//        for ( int i = vars1.size()-1; i >= 0 i-- ) {
+//            Var v =  
+//        }
+//        
+//        Set<Var> assignmentVars = new HashSet<>() ; 
+//        ExprList pushed2 = new ExprList() ;
+//        ExprList unpushed2 = new ExprList() ;
+//        for ( Var v : vars1 ) {
+//            assignmentVars.add(v) ;
+//            Expr e = input.getVarExprList().getExpr(v) ;
+//            for ( Expr expr : exprs ) {
+//                Set<Var> exprVars = expr.getVarsMentioned() ;
+//                if ( disjoint(vars1, exprVars) && subVars.containsAll(exprVars) ) 
+//                    System.out.ptinln(
+//                else
+//                    unpushed2.add(expr) ;
+//            }
+//            
+//            
+//            
+//            if ( ! assignmentVars.contains(v) && subVars.containsAll(exprVars) ) {
+//                
+//            }
+//            
+//            
+//        }
+
+        
+        
+        // ----
+        
         
         for ( Expr expr : exprs ) {
             Set<Var> exprVars = expr.getVarsMentioned() ;
-            if ( disjoint(vars1, exprVars) )
+            if ( disjoint(vars1, exprVars) && subVars.containsAll(exprVars) ) 
                 pushed.add(expr);
             else
                 unpushed.add(expr) ;
         }
                 
-        if ( pushed.isEmpty() ) 
+        if ( pushed.isEmpty() )
             return resultNoChange(input) ;
         
         // (filter ... (extend ... ))
         //   ===>
         // (extend ... (filter ... ))
-        Op opSub = input.getSubOp() ;
-        
-        // And try down the expressions
-        Placement p = transform(pushed, opSub) ;
-
-        if ( p == null ) {
-            // Couldn't place an filter expressions.  Do nothing.
-            return null ;
-        }
-        
-        if ( ! p.unplaced.isEmpty() )
-            // Some placed, not all.
-            // Pass back out all untouched expressions.
-            unpushed.addAll(p.unplaced) ; 
-        Op op1 = input.copy(p.op) ;
-        
-        return result(op1, unpushed) ;
+        return processSubOp1(pushed, unpushed, input) ;
     }
 
     private Placement placeProject(ExprList exprs, OpProject input) {
@@ -619,16 +675,27 @@ public class TransformFilterPlacement ex
         if ( pushed.isEmpty() ) 
             return resultNoChange(input) ;
         // (filter (project ...)) ===> (project (filter ...)) 
-        Op opSub = input.getSubOp() ;
-        Placement p = transform(pushed, opSub) ;
-        if ( p == null ) {
-            Op op1 = OpFilter.filter(pushed, opSub) ;
-            Op op2 = input.copy(op1) ;
-            return result(op2, unpushed) ;
-        }
-        Op op1 = OpFilter.filter(p.unplaced, p.op) ;
-        Op op2 = input.copy(op1) ;
-        return result(op2, unpushed) ;
+        return processSubOp1(pushed, unpushed, input) ;
+    }
+    
+    // For a  modifer without expressions (distinct, reduced), we push filters at least inside
+    // the modifier itself because does not affect scope.  It may enable parallel execution.
+    private Placement placeDistinctReduced(ExprList exprs, OpDistinctReduced input) {
+        Op subOp = input.getSubOp() ;
+        Placement p = transform(exprs, subOp) ;
+        
+//        if ( p == null )
+//            // If push in IFF it makes a difference further in.
+//            return resultNoChange(input) ;
+        
+        // Always push in.
+        // This is safe even if the filter contains vars not defined by the subOp
+        // OpDistinctReduced has the same scope inside and outside.
+        Op op = ( p == null )
+                ? OpFilter.filter(exprs, subOp)
+                : OpFilter.filter(p.unplaced, p.op) ;
+        op = input.copy(op) ;
+        return result(op, emptyList) ;
     }
     
     private Placement placeTable(ExprList exprs, OpTable input) {

Modified: jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformFilterPlacement.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformFilterPlacement.java?rev=1624485&r1=1624484&r2=1624485&view=diff
==============================================================================
--- jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformFilterPlacement.java (original)
+++ jena/trunk/jena-arq/src/test/java/com/hp/hpl/jena/sparql/algebra/optimize/TestTransformFilterPlacement.java Fri Sep 12 09:32:09 2014
@@ -319,6 +319,37 @@ public class TestTransformFilterPlacemen
         test("(filter (= ?x 123) (project (?x) (bgp (?s ?p ?x) (?s ?p ?z) ) ))",
              "(project (?x) (sequence (filter (= ?x 123) (bgp (?s ?p ?x)) ) (bgp (?s ?p ?z))) )") ;
     }
+    
+    @Test public void place_distinct_01() {
+        test("(filter (= ?x 123) (distinct (bgp (?s ?p ?x)) ))",
+             "(distinct (filter (= ?x 123) (bgp (?s ?p ?x)) ))") ;
+    }
+
+    @Test public void place_distinct_02() {
+        test("(filter (= ?x 123) (distinct (bgp (?s ?p ?o)) ))",
+             "(distinct (filter (= ?x 123) (bgp (?s ?p ?o)) ))") ;
+    }
+    
+    @Test public void place_distinct_03() {
+        test("(filter (= ?x 123) (reduced (extend ((?x 123)) (bgp (?s ?p ?o)) )))",
+             "(reduced (filter (= ?x 123) (extend ((?x 123)) (bgp (?s ?p ?o)) )))") ;
+    }
+
+    @Test public void place_reduced_01() {
+        test("(filter (= ?x 123) (reduced (bgp (?s ?p ?x)) ))",
+             "(reduced (filter (= ?x 123) (bgp (?s ?p ?x)) ))") ;
+    }
+
+    @Test public void place_reduced_02() {
+        test("(filter (= ?x 123) (reduced (bgp (?s ?p ?o)) ))",
+             "(reduced (filter (= ?x 123) (bgp (?s ?p ?o)) ))") ;
+    }
+    
+    @Test public void place_reduced_03() {
+        test("(filter (= ?x 123) (distinct (extend ((?x 123)) (bgp (?s ?p ?o)) )))",
+             "(distinct (filter (= ?x 123) (extend ((?x 123)) (bgp (?s ?p ?o)) )))") ;
+    }
+
 
     @Test public void place_extend_01() {
         test("(filter (= ?x 123) (extend ((?z 123)) (bgp (?s ?p ?x)) ))",
@@ -351,6 +382,60 @@ public class TestTransformFilterPlacemen
         test("(filter (= ?z 1) (join (extend (?x1 123) (bgp (?s ?p ?x))) (bgp (?s ?p ?z))))" ,
              "(join (extend (?x1 123) (bgp (?s ?p ?x))) (filter (= ?z 1) (bgp (?s ?p ?z))) )") ;
     }
+    
+    @Test public void place_extend_07() {
+        // Push filters through extend where the extend itself is over an
+        // extend that can have filters pushed into it.
+        String x1 = StrUtils.strjoinNL 
+            ("(filter ( (= ?s 5) (= ?w 6) (= ?s1 7) )"
+             ,"  (extend ((?w 2))"
+             ,"    (extend ((?s 1))"
+             ,"      (table (vars ?s1)"
+             ,"        (row [?s1 1])"
+             ,"))))") ;
+        String x2 = StrUtils.strjoinNL 
+            ("(filter (= ?w 6)"
+            ,"  (extend ((?w 2))"
+            ,"    (filter (= ?s 5)"
+            ,"      (extend ((?s 1))"
+            ,"        (filter (= ?s1 7)"
+            ,"          (table (vars ?s1)"
+            ,"            (row [?s1 1])"
+            ,"          ))))))") ;
+        test(x1, x2) ;
+    }
+
+    @Test public void place_extend_08() {
+        // Push filters through extend where the extend itself is over an
+        // extend that can have filters pushed into it.
+        String x1 = StrUtils.strjoinNL 
+            ("(filter ( (= ?s 'S') (= ?w 'W') (= ?s1 'S1') (= ?a 'A') (= ?b 'B'))"
+             ,"  (extend ((?w 2))"
+             ,"    (extend ((?s 1))"
+             ,"      (distinct"
+             ,"        (extend ((?a 2))"
+             ,"          (extend ((?b 1))"
+             ,"            (table (vars ?s1)"
+             ,"              (row [?s1 1])"
+             ,")))))))"
+             ) ;
+        String x2 = StrUtils.strjoinNL
+            ("(filter (= ?w 'W')"
+            ,"  (extend ((?w 2))"
+            ,"    (filter (= ?s 'S')"
+            ,"      (extend ((?s 1))"
+            ,"        (distinct"
+            ,"          (filter (= ?a 'A')"
+            ,"            (extend ((?a 2))"
+            ,"              (filter (= ?b 'B')"
+            ,"                (extend ((?b 1))"
+            ,"                  (filter (= ?s1 'S1')"
+            ,"                    (table (vars ?s1)"
+            ,"                      (row [?s1 1])"
+            ,"                    )))))))))))"
+            ) ;
+        test(x1, x2) ;
+    }
 
     @Test public void place_assign_01() {
         test("(filter (= ?x 123) (assign ((?z 123)) (bgp (?s ?p ?x)) ))",
@@ -359,7 +444,9 @@ public class TestTransformFilterPlacemen
     
     @Test public void place_assign_02() {
         test("(filter ((= ?x1 123) (= ?x2 456)) (assign (?z 789) (bgp (?s ?p ?x1)) ))",
-             "(filter (= ?x2 456) (assign (?z 789) (filter (= ?x1 123) (bgp (?s ?p ?x1)) )))") ;
+             "(filter (= ?x2 456) (assign (?z 789) (filter (= ?x1 123) (bgp (?s ?p ?x1)) )))"
+             ) ;
+             
     }
     
     @Test public void place_assign_03() { // Blocked
@@ -387,7 +474,61 @@ public class TestTransformFilterPlacemen
     @Test public void place_assign_06a() {
         // With No BGP we won't break up the BGP but we will still push the filter down
         testNoBGP("(filter (= ?x 123) (assign ((?z 123)) (bgp (?s ?p ?x) (?s ?p ?x1) )))",
-             "(assign ((?z 123)) (filter (= ?x 123) (bgp (?s ?p ?x) (?s ?p ?x1)) ) )") ;
+                  "(assign ((?z 123)) (filter (= ?x 123) (bgp (?s ?p ?x) (?s ?p ?x1)) ) )") ;
+    }
+
+    @Test public void place_assign_07() {
+        // Push filters through assign where the assign itself is over an
+        // assign that can have filters pushed into it.
+        String x1 = StrUtils.strjoinNL 
+            ("(filter ( (= ?s 5) (= ?w 6) (= ?s1 7) )"
+             ,"  (assign ((?w 2))"
+             ,"    (assign ((?s 1))"
+             ,"      (table (vars ?s1)"
+             ,"        (row [?s1 1])"
+             ,"))))") ;
+        String x2 = StrUtils.strjoinNL 
+            ("(filter (= ?w 6)"
+            ,"  (assign ((?w 2))"
+            ,"    (filter (= ?s 5)"
+            ,"      (assign ((?s 1))"
+            ,"        (filter (= ?s1 7)"
+            ,"          (table (vars ?s1)"
+            ,"            (row [?s1 1])"
+            ,"          ))))))") ;
+        test(x1, x2) ;
+    }
+
+    @Test public void place_assign_08() {
+        // Push filters through assign where the assign itself is over an
+        // assign that can have filters pushed into it.
+        String x1 = StrUtils.strjoinNL 
+            ("(filter ( (= ?s 'S') (= ?w 'W') (= ?s1 'S1') (= ?a 'A') (= ?b 'B'))"
+             ,"  (assign ((?w 2))"
+             ,"    (assign ((?s 1))"
+             ,"      (distinct"
+             ,"        (assign ((?a 2))"
+             ,"          (assign ((?b 1))"
+             ,"            (table (vars ?s1)"
+             ,"              (row [?s1 1])"
+             ,")))))))"
+             ) ;
+        String x2 = StrUtils.strjoinNL
+            ("(filter (= ?w 'W')"
+            ,"  (assign ((?w 2))"
+            ,"    (filter (= ?s 'S')"
+            ,"      (assign ((?s 1))"
+            ,"        (distinct"
+            ,"          (filter (= ?a 'A')"
+            ,"            (assign ((?a 2))"
+            ,"              (filter (= ?b 'B')"
+            ,"                (assign ((?b 1))"
+            ,"                  (filter (= ?s1 'S1')"
+            ,"                    (table (vars ?s1)"
+            ,"                      (row [?s1 1])"
+            ,"                    )))))))))))"
+            ) ;
+        test(x1, x2) ;
     }
 
     @Test public void place_filter_01() {
@@ -673,6 +814,5 @@ public class TestTransformFilterPlacemen
 
         Op op3 = SSE.parseOp(output) ;
         Assert.assertEquals(op3, op2) ;
-        
     }
 }