You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@vxquery.apache.org by pr...@apache.org on 2015/06/03 23:24:12 UTC

[1/8] vxquery git commit: Fixed the rewrite rules to work with descendent path step.

Repository: vxquery
Updated Branches:
  refs/heads/prestonc/xmark d7fcc0e76 -> c14424972


Fixed the rewrite rules to work with descendent path step.


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

Branch: refs/heads/prestonc/xmark
Commit: 01a3bf1e48d40a6c561ceb386c61957769e86e30
Parents: 82fda88
Author: Eldon Carman <ec...@ucr.edu>
Authored: Thu May 28 19:54:15 2015 -0700
Committer: Eldon Carman <ec...@ucr.edu>
Committed: Wed Jun 3 14:23:47 2015 -0700

----------------------------------------------------------------------
 .../compiler/rewriter/RewriteRuleset.java       |  10 +-
 .../AbstractUsedVariablesProcessingRule.java    |   6 +-
 .../rewriter/rules/IntroduceCollectionRule.java |  16 +--
 .../rules/PushChildIntoDataScanRule.java        |  10 +-
 .../rules/RemoveUnusedUnnestIterateRule.java    | 102 +++++++++++++++++++
 .../step/AbstractForwardAxisPathStep.java       |   4 +-
 ...DescendantOrSelfPathStepScalarEvaluator.java |  74 +++-----------
 .../step/DescendantOrSelfPathStepUnnesting.java |  29 +++---
 .../step/DescendantPathStepScalarEvaluator.java |  33 +++---
 9 files changed, 174 insertions(+), 110 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/vxquery/blob/01a3bf1e/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
index d551932..008083d 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
@@ -20,7 +20,6 @@ import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.vxquery.compiler.rewriter.rules.ConsolidateAssignAggregateRule;
-import org.apache.vxquery.compiler.rewriter.rules.ConsolidateUnnestsRule;
 import org.apache.vxquery.compiler.rewriter.rules.ConvertAssignToUnnestRule;
 import org.apache.vxquery.compiler.rewriter.rules.ConvertFromAlgebricksExpressionsRule;
 import org.apache.vxquery.compiler.rewriter.rules.ConvertToAlgebricksExpressionsRule;
@@ -39,6 +38,7 @@ import org.apache.vxquery.compiler.rewriter.rules.RemoveRedundantPromoteExpressi
 import org.apache.vxquery.compiler.rewriter.rules.RemoveRedundantTreatExpressionsRule;
 import org.apache.vxquery.compiler.rewriter.rules.RemoveUnusedSortDistinctNodesRule;
 import org.apache.vxquery.compiler.rewriter.rules.RemoveUnusedTreatRule;
+import org.apache.vxquery.compiler.rewriter.rules.RemoveUnusedUnnestIterateRule;
 import org.apache.vxquery.compiler.rewriter.rules.SetCollectionDataSourceRule;
 import org.apache.vxquery.compiler.rewriter.rules.SetVariableIdContextRule;
 
@@ -117,7 +117,9 @@ public class RewriteRuleset {
         normalization.add(new RemoveUnusedAssignAndAggregateRule());
 
         // Adds child steps to the data source scan.
-        normalization.add(new ConsolidateUnnestsRule());
+        // TODO Replace consolidate with a new child function that takes multiple paths.
+        //        normalization.add(new ConsolidateUnnestsRule());
+        normalization.add(new RemoveUnusedUnnestIterateRule());
         normalization.add(new PushChildIntoDataScanRule());
 
         // Improvement for scalar child expressions
@@ -269,8 +271,8 @@ public class RewriteRuleset {
         prepareForJobGenRewrites.add(new IsolateHyracksOperatorsRule(
                 HeuristicOptimizer.hyraxOperatorsBelowWhichJobGenIsDisabled));
         prepareForJobGenRewrites.add(new ExtractCommonOperatorsRule());
-//        prepareForJobGenRewrites.add(new VXQueryExtractCommonOperatorsRule());
-//        prepareForJobGenRewrites.add(new DelayMaterializationForJoinProbeRule());
+        //        prepareForJobGenRewrites.add(new VXQueryExtractCommonOperatorsRule());
+        //        prepareForJobGenRewrites.add(new DelayMaterializationForJoinProbeRule());
         // Re-infer all types, so that, e.g., the effect of not-is-null is
         // propagated.
         prepareForJobGenRewrites.add(new PushProjectIntoDataSourceScanRule());

http://git-wip-us.apache.org/repos/asf/vxquery/blob/01a3bf1e/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractUsedVariablesProcessingRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractUsedVariablesProcessingRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractUsedVariablesProcessingRule.java
index 7cbe503..e80597b 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractUsedVariablesProcessingRule.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractUsedVariablesProcessingRule.java
@@ -42,6 +42,7 @@ public abstract class AbstractUsedVariablesProcessingRule implements IAlgebraicR
 
     protected List<LogicalVariable> usedVariables = new ArrayList<LogicalVariable>();
     protected boolean hasRun = false;
+    Mutable<ILogicalOperator> firstOpRef;
 
     @Override
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
@@ -51,8 +52,11 @@ public abstract class AbstractUsedVariablesProcessingRule implements IAlgebraicR
 
     @Override
     public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-        if (hasRun) {
+        opRef.getValue().getOperatorTag();
+        if (hasRun && !firstOpRef.equals(opRef)) {
             return false;
+        } else {
+            firstOpRef = opRef;
         }
         boolean modified = false;
         boolean modified_last_pass;

http://git-wip-us.apache.org/repos/asf/vxquery/blob/01a3bf1e/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/IntroduceCollectionRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/IntroduceCollectionRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/IntroduceCollectionRule.java
index a5cdfe7..c599876 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/IntroduceCollectionRule.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/IntroduceCollectionRule.java
@@ -41,7 +41,7 @@ import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOpera
  * Before
  * 
  *   plan__parent
- *   UNNEST( $v2 : iterate( $v1 ) )
+ *   UNNEST( $v2 : exp($v1) )
  *   ASSIGN( $v1 : collection( $source ) )
  *   plan__child
  *   
@@ -54,12 +54,13 @@ import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOpera
  * After 
  * 
  *   plan__parent
- *   DATASCAN( collection( $source ) , $v2 )
+ *   UNNEST( $v2 : exp($v1) )
+ *   DATASCAN( collection( $source ) , $v1 )
  *   plan__child
  *   
  *   Where DATASCAN operator is configured to use the collection( $source) for 
- *   data represented by the “constant” and $v2 represents the xml document 
- *   node.
+ *   data represented by the "constant" and $v1 represents the xml document 
+ *   nodes from the collection.
  * </pre>
  * 
  * @author prestonc
@@ -81,12 +82,13 @@ public class IntroduceCollectionRule extends AbstractCollectionRule {
                 // Known to be true because of collection name.
                 AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
                 UnnestOperator unnest = (UnnestOperator) op;
-                AbstractLogicalOperator op2 = (AbstractLogicalOperator) unnest.getInputs().get(0).getValue();
+                Mutable<ILogicalOperator> opRef2 = unnest.getInputs().get(0);
+                AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
                 AssignOperator assign = (AssignOperator) op2;
 
-                DataSourceScanOperator opNew = new DataSourceScanOperator(unnest.getVariables(), ds);
+                DataSourceScanOperator opNew = new DataSourceScanOperator(assign.getVariables(), ds);
                 opNew.getInputs().addAll(assign.getInputs());
-                opRef.setValue(opNew);
+                opRef2.setValue(opNew);
                 return true;
             }
         }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/01a3bf1e/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/PushChildIntoDataScanRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/PushChildIntoDataScanRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/PushChildIntoDataScanRule.java
index d49d696..3c23731 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/PushChildIntoDataScanRule.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/PushChildIntoDataScanRule.java
@@ -78,12 +78,7 @@ public class PushChildIntoDataScanRule extends AbstractUsedVariablesProcessingRu
 
     protected boolean processOperator(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
-        AbstractLogicalOperator op0 = (AbstractLogicalOperator) opRef.getValue();
-        if (op0.getInputs().isEmpty()) {
-            return false;
-        }
-
-        AbstractLogicalOperator op1 = (AbstractLogicalOperator) op0.getInputs().get(0).getValue();
+        AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();;
         if (op1.getOperatorTag() != LogicalOperatorTag.UNNEST) {
             return false;
         }
@@ -107,8 +102,7 @@ public class PushChildIntoDataScanRule extends AbstractUsedVariablesProcessingRu
                     datascan.getVariables().get(0));
             AssignOperator noOp = new AssignOperator(unnest.getVariable(), varExp);
             noOp.getInputs().addAll(unnest.getInputs());
-            op0.getInputs().clear();
-            op0.getInputs().add(new MutableObject<ILogicalOperator>(noOp));
+            opRef.setValue(noOp);
             return true;
         }
         return false;

http://git-wip-us.apache.org/repos/asf/vxquery/blob/01a3bf1e/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedUnnestIterateRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedUnnestIterateRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedUnnestIterateRule.java
new file mode 100644
index 0000000..d960735
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedUnnestIterateRule.java
@@ -0,0 +1,102 @@
+/*
+ * 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.vxquery.compiler.rewriter.rules;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.vxquery.functions.BuiltinOperators;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+
+/**
+ * The rule searches for an unnest operator (1) immediately following an unnest 
+ * or data scan operator (2). If the variable is only used in unnest (1) then
+ * unnest (1) can be removed.
+ * 
+ * <pre>
+ * Before 
+ * 
+ *   plan__parent
+ *   UNNEST( $v2 : iterate( $v1 ) )
+ *   UNNEST( $v1 : $v )
+ *   plan__child
+ *   
+ *   Where $v1 is not used in plan__parent.
+ *   
+ * After
+ * 
+ *   plan__parent
+ *   UNNEST( $v2 : $v )
+ *   plan__child
+ * </pre>
+ * 
+ * @author prestonc
+ */
+public class RemoveUnusedUnnestIterateRule extends AbstractUsedVariablesProcessingRule {
+
+    protected boolean processOperator(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.UNNEST) {
+            return false;
+        }
+        UnnestOperator unnest = (UnnestOperator) op;
+
+        // Check to see if the expression is a function and iterate.
+        ILogicalExpression logicalExpressionUnnest1 = (ILogicalExpression) unnest.getExpressionRef().getValue();
+        if (logicalExpressionUnnest1.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        AbstractFunctionCallExpression functionCallUnnest1 = (AbstractFunctionCallExpression) logicalExpressionUnnest1;
+        if (!functionCallUnnest1.getFunctionIdentifier().equals(BuiltinOperators.ITERATE.getFunctionIdentifier())) {
+            return false;
+        }
+
+        // Check to see if the expression is a variable.
+        ILogicalExpression logicalExpressionUnnest2 = (ILogicalExpression) functionCallUnnest1.getArguments().get(0)
+                .getValue();
+        if (logicalExpressionUnnest2.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+            return false;
+        }
+        VariableReferenceExpression vre2 = (VariableReferenceExpression) logicalExpressionUnnest2;
+        LogicalVariable unnestInput = vre2.getVariableReference();
+
+        // Check if the input is an DATASCAN or UNNEST operator..
+        Mutable<ILogicalOperator> opRef2 = unnest.getInputs().get(0);
+        AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
+        if (op2.getOperatorTag() == LogicalOperatorTag.UNNEST || op2.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+            AbstractScanOperator aso = (AbstractScanOperator) op2;
+            if (aso.getVariables().size() == 1 && aso.getVariables().contains(unnestInput)) {
+                aso.setVariables(unnest.getVariables());
+                opRef.setValue(aso);
+                return true;
+            }
+        }
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/01a3bf1e/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractForwardAxisPathStep.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractForwardAxisPathStep.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractForwardAxisPathStep.java
index e4d25cf..e0c7931 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractForwardAxisPathStep.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractForwardAxisPathStep.java
@@ -38,6 +38,7 @@ public abstract class AbstractForwardAxisPathStep {
     protected final PointablePool pp;
     protected final NodeTreePointable ntp = (NodeTreePointable) NodeTreePointable.FACTORY.createPointable();
     private final ArrayBackedValueStorage nodeAbvs = new ArrayBackedValueStorage();
+    private final TaggedValuePointable tvpConvert = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
     private final DocumentNodePointable dnp = (DocumentNodePointable) DocumentNodePointable.FACTORY.createPointable();
     private final ElementNodePointable enp = (ElementNodePointable) ElementNodePointable.FACTORY.createPointable();
     private final NodeSubTreeBuilder nstb = new NodeSubTreeBuilder();
@@ -61,7 +62,8 @@ public abstract class AbstractForwardAxisPathStep {
                     return;
                 }
         }
-        XDMConstants.setEmptySequence(seqp);
+        XDMConstants.setEmptySequence(tvpConvert);
+        tvpConvert.getValue(seqp);
     }
 
     protected void setNodeToResult(TaggedValuePointable tvpItem, IPointable result) throws IOException {

http://git-wip-us.apache.org/repos/asf/vxquery/blob/01a3bf1e/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepScalarEvaluator.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepScalarEvaluator.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepScalarEvaluator.java
index 5572166..4640dc8 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepScalarEvaluator.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepScalarEvaluator.java
@@ -18,61 +18,44 @@ package org.apache.vxquery.runtime.functions.step;
 
 import java.io.IOException;
 
-import org.apache.vxquery.datamodel.accessors.SequencePointable;
 import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
-import org.apache.vxquery.datamodel.values.ValueTag;
+import org.apache.vxquery.datamodel.builders.sequence.SequenceBuilder;
 import org.apache.vxquery.exceptions.ErrorCode;
 import org.apache.vxquery.exceptions.SystemException;
-import org.apache.vxquery.types.DocumentType;
-import org.apache.vxquery.types.ElementType;
-import org.apache.vxquery.types.Quantifier;
-import org.apache.vxquery.types.SequenceType;
 
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.data.std.api.IPointable;
-import edu.uci.ics.hyracks.data.std.primitive.VoidPointable;
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 
 public class DescendantOrSelfPathStepScalarEvaluator extends AbstractDescendantPathStepScalarEvaluator {
-    private final TaggedValuePointable rootTVP;
-    private final ArrayBackedValueStorage seqAbvs;
-    private final SequencePointable seqp = (SequencePointable) SequencePointable.FACTORY.createPointable();
-    private final VoidPointable p = (VoidPointable) VoidPointable.FACTORY.createPointable();
-    private final TaggedValuePointable tvp = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+    private final SequenceBuilder seqb = new SequenceBuilder();
+
+    private final ArrayBackedValueStorage seqAbvs = new ArrayBackedValueStorage();
+
+    private final DescendantOrSelfPathStepUnnesting descendentOrSelfPathStep;
+
+    private final TaggedValuePointable itemTvp = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
 
     public DescendantOrSelfPathStepScalarEvaluator(IScalarEvaluator[] args, IHyracksTaskContext ctx) {
         super(args, ctx);
-        rootTVP = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
-        seqAbvs = new ArrayBackedValueStorage();
+        descendentOrSelfPathStep = new DescendantOrSelfPathStepUnnesting(ctx, ppool, true);
     }
 
     @Override
     protected final void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException {
         try {
-            // Set up the result sequence.
+            descendentOrSelfPathStep.init(args);
             seqAbvs.reset();
             seqb.reset(seqAbvs);
-
-            // Check the argument passed in as sequence or node tree.
-            if (args[0].getTag() == ValueTag.SEQUENCE_TAG) {
-                args[0].getValue(seqp);
-                for (int j = 0; j < seqp.getEntryCount(); ++j) {
-                    seqp.getEntry(j, p);
-                    tvp.set(p.getByteArray(), p.getStartOffset(), p.getLength());
-                    if (tvp.getTag() != ValueTag.NODE_TREE_TAG) {
-                        throw new SystemException(ErrorCode.SYSE0001);
-                    }
-                    tvp.getValue(ntp);
-                    processNodeTree(rootTVP);
+            try {
+                while (descendentOrSelfPathStep.step(itemTvp)) {
+                    seqb.addItem(itemTvp);
                 }
-            } else if (args[0].getTag() == ValueTag.NODE_TREE_TAG) {
-                args[0].getValue(ntp);
-                processNodeTree(rootTVP);
-            } else {
-                throw new SystemException(ErrorCode.SYSE0001);
+            } catch (AlgebricksException e) {
+                throw new SystemException(ErrorCode.SYSE0001, e);
             }
-
             seqb.finish();
             result.set(seqAbvs);
         } catch (IOException e) {
@@ -80,29 +63,4 @@ public class DescendantOrSelfPathStepScalarEvaluator extends AbstractDescendantP
         }
     }
 
-    private void processNodeTree(TaggedValuePointable rootTVP) throws SystemException {
-        ntp.getRootNode(rootTVP);
-
-        // Solve for self.
-        switch (rootTVP.getTag()) {
-            case ValueTag.DOCUMENT_NODE_TAG:
-                setNodeTest(SequenceType.create(DocumentType.ANYDOCUMENT, Quantifier.QUANT_ONE));
-                break;
-            case ValueTag.ELEMENT_NODE_TAG:
-                setNodeTest(SequenceType.create(ElementType.ANYELEMENT, Quantifier.QUANT_ONE));
-                break;
-            default:
-                throw new SystemException(ErrorCode.SYSE0001);
-        }
-        itemTvp.set(rootTVP);
-        try {
-            appendNodeToResult();
-        } catch (IOException e) {
-            throw new SystemException(ErrorCode.SYSE0001, e);
-        }
-
-        // Solve for descendants.
-        searchSubtree(rootTVP);
-    }
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/01a3bf1e/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepUnnesting.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepUnnesting.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepUnnesting.java
index 57ab9e7..3fd1daf 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepUnnesting.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepUnnesting.java
@@ -33,10 +33,11 @@ import edu.uci.ics.hyracks.data.std.api.IPointable;
 
 public class DescendantOrSelfPathStepUnnesting extends AbstractForwardAxisPathStep {
     private boolean testSelf;
+    private boolean printSelf;
     private int indexSeqArgs;
     private int seqArgsLength;
     private List<Integer> indexSequence = new ArrayList<Integer>();
-    private List<Boolean> checkSelf = new ArrayList<Boolean>();
+    private List<Integer> printedSequence = new ArrayList<Integer>();
 
     private final SequencePointable seqNtp = (SequencePointable) SequencePointable.FACTORY.createPointable();
     private final TaggedValuePointable tvpItem = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
@@ -49,9 +50,10 @@ public class DescendantOrSelfPathStepUnnesting extends AbstractForwardAxisPathSt
     }
 
     protected void init(TaggedValuePointable[] args) throws SystemException {
-        checkSelf.add(true);
+        printSelf = true;
         indexSeqArgs = 0;
         indexSequence.add(0);
+        printedSequence.add(0);
 
         // Check the argument passed in as sequence or node tree.
         if (args[0].getTag() == ValueTag.SEQUENCE_TAG) {
@@ -80,7 +82,7 @@ public class DescendantOrSelfPathStepUnnesting extends AbstractForwardAxisPathSt
                 }
                 // Next node tree in sequence.
                 indexSeqArgs++;
-                checkSelf.set(0, true);
+                printSelf = true;
             }
         } else {
             // Single node tree input.
@@ -93,8 +95,8 @@ public class DescendantOrSelfPathStepUnnesting extends AbstractForwardAxisPathSt
     }
 
     private boolean processNodeTree(TaggedValuePointable rootTVP, IPointable result) throws AlgebricksException {
-        if (testSelf && checkSelf.get(0)) {
-            checkSelf.set(0, false);
+        if (testSelf && printSelf) {
+            printSelf = false;
             tvpItem.set(rootTVP);
             try {
                 setNodeToResult(tvpItem, result);
@@ -118,11 +120,9 @@ public class DescendantOrSelfPathStepUnnesting extends AbstractForwardAxisPathSt
     protected boolean stepNodeTree(TaggedValuePointable tvpInput, int level, IPointable result)
             throws AlgebricksException {
         // Set up next level tracking.
-        if (level + 1 >= indexSequence.size()) {
+        if (level + 1 > indexSequence.size()) {
             indexSequence.add(0);
-        }
-        if (level + 1 >= checkSelf.size()) {
-            checkSelf.add(true);
+            printedSequence.add(0);
         }
 
         SequencePointable seqItem = pp.takeOne(SequencePointable.class);
@@ -134,16 +134,13 @@ public class DescendantOrSelfPathStepUnnesting extends AbstractForwardAxisPathSt
                 seqItem.getEntry(indexSequence.get(level), tvpItem);
 
                 // Check current node
-                if (checkSelf.get(level)) {
-                    checkSelf.set(level, false);
+                if (indexSequence.get(level) == printedSequence.get(level)) {
+                   printedSequence.set(level, printedSequence.get(level) + 1);
                     setNodeToResult(tvpItem, result);
                     return true;
                 }
                 // Check children nodes
-                if (level + 1 < indexSequence.size()) {
-                    if (level + 1 < checkSelf.size()) {
-                        checkSelf.set(level + 1, true);
-                    }
+                if (level + 1 <= indexSequence.size()) {
                     if (stepNodeTree(tvpItem, level + 1, result)) {
                         return true;
                     }
@@ -153,8 +150,10 @@ public class DescendantOrSelfPathStepUnnesting extends AbstractForwardAxisPathSt
             // Reset for next node tree.
             if (level == 0) {
                 indexSequence.set(level, 0);
+                printedSequence.set(level, 0);
             } else {
                 indexSequence.remove(level);
+                printedSequence.remove(level);
             }
             return false;
         } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/vxquery/blob/01a3bf1e/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantPathStepScalarEvaluator.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantPathStepScalarEvaluator.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantPathStepScalarEvaluator.java
index c415f33..7672d37 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantPathStepScalarEvaluator.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantPathStepScalarEvaluator.java
@@ -19,42 +19,43 @@ package org.apache.vxquery.runtime.functions.step;
 import java.io.IOException;
 
 import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
-import org.apache.vxquery.datamodel.values.ValueTag;
+import org.apache.vxquery.datamodel.builders.sequence.SequenceBuilder;
 import org.apache.vxquery.exceptions.ErrorCode;
 import org.apache.vxquery.exceptions.SystemException;
 
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.data.std.api.IPointable;
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 
 public class DescendantPathStepScalarEvaluator extends AbstractDescendantPathStepScalarEvaluator {
-    private final TaggedValuePointable rootTVP;
+    private final SequenceBuilder seqb = new SequenceBuilder();
 
-    private final ArrayBackedValueStorage seqAbvs;
+    private final ArrayBackedValueStorage seqAbvs = new ArrayBackedValueStorage();
+
+    private final DescendantOrSelfPathStepUnnesting descendentOrSelfPathStep;
+
+    private final TaggedValuePointable itemTvp = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
 
     public DescendantPathStepScalarEvaluator(IScalarEvaluator[] args, IHyracksTaskContext ctx) {
         super(args, ctx);
-        rootTVP = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
-        seqAbvs = new ArrayBackedValueStorage();
+        descendentOrSelfPathStep = new DescendantOrSelfPathStepUnnesting(ctx, ppool, false);
     }
 
     @Override
     protected final void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException {
         try {
-            if (args[0].getTag() != ValueTag.NODE_TREE_TAG) {
-                throw new SystemException(ErrorCode.SYSE0001);
-            }
-            args[0].getValue(ntp);
-
-            // Set up the result sequence and get the root node.
+            descendentOrSelfPathStep.init(args);
             seqAbvs.reset();
             seqb.reset(seqAbvs);
-            ntp.getRootNode(rootTVP);
-
-            // Solve for descendants.
-            searchSubtree(rootTVP);
-
+            try {
+                while (descendentOrSelfPathStep.step(itemTvp)) {
+                    seqb.addItem(itemTvp);
+                }
+            } catch (AlgebricksException e) {
+                throw new SystemException(ErrorCode.SYSE0001, e);
+            }
             seqb.finish();
             result.set(seqAbvs);
         } catch (IOException e) {


[5/8] vxquery git commit: apt documentation file for cluster installation

Posted by pr...@apache.org.
apt documentation file for cluster installation


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

Branch: refs/heads/prestonc/xmark
Commit: 82fda88dbd91c744b5a1109698bea5a7f4c2ff8e
Parents: 6d13ad4
Author: kaltirim <ka...@lab.grid.auth.gr>
Authored: Wed May 6 16:22:28 2015 +0300
Committer: Eldon Carman <ec...@ucr.edu>
Committed: Wed Jun 3 14:23:47 2015 -0700

----------------------------------------------------------------------
 src/site/apt/cluster_installation.apt | 149 +++++++++++++++++++++++++++++
 src/site/site.xml                     |   3 +
 2 files changed, 152 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/vxquery/blob/82fda88d/src/site/apt/cluster_installation.apt
----------------------------------------------------------------------
diff --git a/src/site/apt/cluster_installation.apt b/src/site/apt/cluster_installation.apt
new file mode 100644
index 0000000..1342908
--- /dev/null
+++ b/src/site/apt/cluster_installation.apt
@@ -0,0 +1,149 @@
+~~ 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.
+
+Cluster Installation
+
+
+* Requirements
+
+  * Apache VXQuery\x99 source archive (apache-vxquery-X.Y-source-release.zip)
+
+  * JDK >= 1.7
+
+  * Apache Maven >= 3.2
+
+
+* Steps
+
+  * <<Export JAVA_HOME>>
+  
+---
+$ export JAVA_HOME=/usr/java/latest
+---
+
+
+  * <<Unzip and build VXQuery>>
+
+--- 
+$ unzip apache-vxquery-X.Y-source-release.zip
+$ cd apache-vxquery-X.Y
+$ mvn package -DskipTests
+$ cd ..
+---
+
+
+  * <<Create configuration file>>
+    
+    Create a configuration xml file containing the information of the vxquery cluster.Here is an example of a VXQuery configuration file for a cluster with 1 master and 3 slaves.
+
+---
+    <cluster xmlns="cluster">
+      <name>local</name>
+      <username>joe</username>
+      <master_node>
+          <id>master</id>
+          <client_ip>128.195.52.177</client_ip>
+          <cluster_ip>192.168.100.0</cluster_ip>
+      </master_node>
+      <node>
+          <id>nodeA</id>
+          <cluster_ip>192.168.100.1</cluster_ip>
+      </node>
+      <node>
+          <id>nodeB</id>
+          <cluster_ip>192.168.100.2</cluster_ip>
+      </node>
+      <node>
+          <id>nodeC</id>
+          <cluster_ip>192.168.100.3</cluster_ip>
+      </node>
+  </cluster>
+---
+
+    * Fields that are required:
+
+      * name : name of the cluster
+      
+      * username : user that will execute commands in all the machines of the cluster. Preferably a user that has passwordless ssh access to the machines.
+
+      * id : hostname of the node
+
+      * cluster_ip : ip of the host in the cluster
+
+      * client_ip : ip of the master
+
+    * Some optional fields:
+
+      * CCPORT : port for the Cluster Controller
+
+      * J_OPTS : define the java options you want, for Cluster Controller and Node Controller
+
+
+  * <<Deploy cluster>>
+
+    To deploy the cluster you need to execute this command in the vxquery installation directory
+    
+---
+$python cluster_cli.py -c ../conf/cluster.xml -a deploy -d /apache-vxquery/vxquery-server
+---
+
+    * Arguments: 
+
+      * -c : path to the configuration file you created
+
+      * -a : action you want to perform
+
+      * -d : directory in the system to deploy the cluster
+
+
+  * <<Start cluster>>
+
+    The command to start the cluster is 
+
+---
+$python cluster_cli.py -c ../conf/cluster.xml -a start
+---
+  
+
+  * <<Stop cluster>>
+
+    The command to stop the cluster is 
+
+---
+$python cluster_cli.py -c ../conf/cluster.xml -a stop
+---
+
+
+  * <<Check process status for Cluster Controller>>
+
+  You can try these commands to check on the status of the processes
+
+---
+$ps -ef|grep ${USER}|grep java|grep 'Dapp.name=vxquerycc'
+---
+
+
+ * <<Check process status for Node Controller>>
+
+---
+$ps -ef|grep ${USER}|grep java|grep 'Dapp.name=vxquerync'
+---
+
+
+ * <<Check process status for hyracks process>>
+
+---
+$ps -ef|grep ${USER}|grep java|grep 'hyracks'
+---
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/82fda88d/src/site/site.xml
----------------------------------------------------------------------
diff --git a/src/site/site.xml b/src/site/site.xml
index 3d10b94..0ab3acd 100644
--- a/src/site/site.xml
+++ b/src/site/site.xml
@@ -63,6 +63,9 @@ limitations under the License.
                 name="Installation"
                 href="user_installation.html" />
             <item
+                name="Cluster Installation"
+                href="cluster_installation.html" />
+            <item
                 name="Executing a Query"
                 href="user_query.html" />
             <item


[2/8] vxquery git commit: New set of passing tests after patch.

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/vxquery/blob/61f68a3e/vxquery-xtest/results/xqts.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/results/xqts.txt b/vxquery-xtest/results/xqts.txt
index 110b241..816b4e5 100644
--- a/vxquery-xtest/results/xqts.txt
+++ b/vxquery-xtest/results/xqts.txt
@@ -1320,7 +1320,7 @@ Expressions/FLWORExpr/LetExpr//LetExpr002, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/FLWORExpr/LetExpr//LetExpr003, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/FLWORExpr/LetExpr//LetExpr004, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/FLWORExpr/LetExpr//LetExpr005, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/FLWORExpr/LetExpr//LetExpr006, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/LetExpr//LetExpr006, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/FLWORExpr/LetExpr//LetExpr007, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/FLWORExpr/LetExpr//LetExpr008, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/FLWORExpr/LetExpr//LetExpr009, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -1457,23 +1457,23 @@ Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-37, EXPECTED_RES
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-38, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-39, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-40, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-41, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-42, EXPECTED_RESULT_GOT_ERROR
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-40, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-41, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-42, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-43, EXPECTED_RESULT_GOT_ERROR
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-44, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-45, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-46, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-47, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-49, EXPECTED_RESULT_GOT_ERROR
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-45, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-46, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-47, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-49, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-50, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-51, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-52, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-55, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-56, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-57, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-59, EXPECTED_RESULT_GOT_ERROR
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-50, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-51, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-52, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-55, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-56, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-57, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-59, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-6, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-60, EXPECTED_ERROR_GOT_RESULT
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWith//orderbylocal-7, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
@@ -1497,23 +1497,23 @@ Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-22, EXPECTE
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-23, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-24, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-25, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-26, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-27, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-28, EXPECTED_RESULT_GOT_ERROR
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-26, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-27, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-28, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-29, EXPECTED_RESULT_GOT_ERROR
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-30, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-31, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-32, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-33, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-34, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-35, EXPECTED_RESULT_GOT_ERROR
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-30, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-31, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-32, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-33, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-34, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-35, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-36, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-37, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-38, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-39, EXPECTED_RESULT_GOT_ERROR
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-37, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-38, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-39, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-40, EXPECTED_RESULT_GOT_ERROR
+Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-40, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-6, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/OrderbyExpr/OrderbyExprWithout//orderbywithout-7, EXPECTED_RESULT_GOT_ERROR
@@ -2265,7 +2265,7 @@ Expressions/Operators/ArithExpr/DurationArith/DayTimeDurationDivide//op-divide-d
 Expressions/Operators/ArithExpr/DurationArith/DayTimeDurationDivide//op-divide-dayTimeDuration2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/DurationArith/DayTimeDurationDivide//op-divide-dayTimeDuration2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/DurationArith/DayTimeDurationDivideDTD//K-DayTimeDurationDivideDTD-1, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/DurationArith/DayTimeDurationDivideDTD//op-divide-dayTimeDuration-by-dTD-1, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/DurationArith/DayTimeDurationDivideDTD//op-divide-dayTimeDuration-by-dTD-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/Operators/ArithExpr/DurationArith/DayTimeDurationDivideDTD//op-divide-dayTimeDuration-by-dTD-10, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/DurationArith/DayTimeDurationDivideDTD//op-divide-dayTimeDuration-by-dTD-11, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/DurationArith/DayTimeDurationDivideDTD//op-divide-dayTimeDuration-by-dTD-12, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -2691,18 +2691,18 @@ Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-add-derived-2,
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-add-derived-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-add-derived-4, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-add-derived-5, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-adddbl2args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-adddbl2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-adddbl2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-adddbl2args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-adddbl2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-adddbl2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-adddec2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-adddec2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-adddec2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-adddec2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addflt2args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addflt2args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addflt2args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addflt2args-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addflt2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addflt2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addflt2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addflt2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addint2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addint2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addint2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -2718,10 +2718,10 @@ Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addlng2args-4,
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addmix2args-1, EXPECTED_ERROR_GOT_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addmix2args-2, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addmix2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addmix2args-4, EXPECTED_RESULT_GOT_ERROR
-Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addmix2args-5, EXPECTED_RESULT_GOT_ERROR
+Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addmix2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addmix2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addmix2args-6, EXPECTED_ERROR_GOT_SAME_ERROR
-Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addmix2args-7, EXPECTED_RESULT_GOT_ERROR
+Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addmix2args-7, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addmix2args-8, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addmix2args-9, EXPECTED_ERROR_GOT_FAILURE
 Expressions/Operators/ArithExpr/NumericOpr/NumericAdd//op-numeric-addnint2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -2809,15 +2809,15 @@ Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividedec2a
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividedec2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividedec2args-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividedec2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideflt2args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideflt2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideflt2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideflt2args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideflt2args-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideflt2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideflt2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideint2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideint2args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideint2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideint2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideint2args-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideint2args-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideint2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideint2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideintg2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideintg2args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-divideintg2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -2832,10 +2832,10 @@ Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividelng2a
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividemix2args-1, EXPECTED_ERROR_GOT_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividemix2args-2, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividemix2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividemix2args-4, EXPECTED_RESULT_GOT_ERROR
-Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividemix2args-5, EXPECTED_RESULT_GOT_ERROR
+Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividemix2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividemix2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividemix2args-6, EXPECTED_ERROR_GOT_SAME_ERROR
-Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividemix2args-7, EXPECTED_RESULT_GOT_ERROR
+Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividemix2args-7, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividemix2args-8, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividemix2args-9, EXPECTED_ERROR_GOT_FAILURE
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividenint2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -2855,7 +2855,7 @@ Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividepint2
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividepint2args-4, EXPECTED_RESULT_GOT_ERROR
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividepint2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividesht2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividesht2args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividesht2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividesht2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividesht2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericDivide//op-numeric-dividesht2args-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
@@ -2951,10 +2951,10 @@ Expressions/Operators/ArithExpr/NumericOpr/NumericIntegerDivide//op-numeric-inte
 Expressions/Operators/ArithExpr/NumericOpr/NumericIntegerDivide//op-numeric-integer-dividemix2args-10, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 Expressions/Operators/ArithExpr/NumericOpr/NumericIntegerDivide//op-numeric-integer-dividemix2args-2, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/Operators/ArithExpr/NumericOpr/NumericIntegerDivide//op-numeric-integer-dividemix2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericIntegerDivide//op-numeric-integer-dividemix2args-4, EXPECTED_RESULT_GOT_ERROR
-Expressions/Operators/ArithExpr/NumericOpr/NumericIntegerDivide//op-numeric-integer-dividemix2args-5, EXPECTED_RESULT_GOT_ERROR
+Expressions/Operators/ArithExpr/NumericOpr/NumericIntegerDivide//op-numeric-integer-dividemix2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericIntegerDivide//op-numeric-integer-dividemix2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericIntegerDivide//op-numeric-integer-dividemix2args-6, EXPECTED_ERROR_GOT_SAME_ERROR
-Expressions/Operators/ArithExpr/NumericOpr/NumericIntegerDivide//op-numeric-integer-dividemix2args-7, EXPECTED_RESULT_GOT_ERROR
+Expressions/Operators/ArithExpr/NumericOpr/NumericIntegerDivide//op-numeric-integer-dividemix2args-7, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericIntegerDivide//op-numeric-integer-dividemix2args-8, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/Operators/ArithExpr/NumericOpr/NumericIntegerDivide//op-numeric-integer-dividemix2args-9, EXPECTED_ERROR_GOT_FAILURE
 Expressions/Operators/ArithExpr/NumericOpr/NumericIntegerDivide//op-numeric-integer-dividenint2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -3015,7 +3015,7 @@ Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-mod-1, EXPECTE
 Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-moddbl2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-moddec2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-moddec2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modflt2args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modflt2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modint2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modintg2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modintg2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -3023,10 +3023,10 @@ Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modlng2args-1,
 Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modmix2args-1, EXPECTED_ERROR_GOT_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modmix2args-2, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modmix2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modmix2args-4, EXPECTED_RESULT_GOT_ERROR
-Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modmix2args-5, EXPECTED_RESULT_GOT_ERROR
+Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modmix2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modmix2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modmix2args-6, EXPECTED_ERROR_GOT_SAME_ERROR
-Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modmix2args-7, EXPECTED_RESULT_GOT_ERROR
+Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modmix2args-7, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modmix2args-8, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modmix2args-9, EXPECTED_ERROR_GOT_FAILURE
 Expressions/Operators/ArithExpr/NumericOpr/NumericMod//op-numeric-modnni2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -3075,28 +3075,28 @@ Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//K-NumericMultiply-7,
 Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//K-NumericMultiply-8, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//K-NumericMultiply-9, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiply-1, EXPECTED_RESULT_GOT_FAILURE
-Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplydbl2args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplydbl2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplydbl2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplydbl2args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplydbl2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplydbl2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplydbl2args-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplydbl2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplydec2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplydec2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplydec2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplydec2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplydec2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplyflt2args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplyflt2args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplyflt2args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplyflt2args-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplyflt2args-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplyflt2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplyflt2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplyflt2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplyflt2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplyflt2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplymix2args-1, EXPECTED_ERROR_GOT_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplymix2args-2, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplymix2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplymix2args-4, EXPECTED_RESULT_GOT_ERROR
-Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplymix2args-5, EXPECTED_RESULT_GOT_ERROR
+Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplymix2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplymix2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplymix2args-6, EXPECTED_ERROR_GOT_SAME_ERROR
-Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplymix2args-7, EXPECTED_RESULT_GOT_ERROR
+Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplymix2args-7, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplymix2args-8, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplymix2args-9, EXPECTED_ERROR_GOT_FAILURE
 Expressions/Operators/ArithExpr/NumericOpr/NumericMultiply//op-numeric-multiplynni2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -3161,20 +3161,20 @@ Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//K-NumericSubtract-8,
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//K-NumericSubtract-9, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtract-1, EXPECTED_RESULT_GOT_FAILURE
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractdbl2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractdbl2args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractdbl2args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractdbl2args-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractdbl2args-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractdbl2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractdbl2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractdbl2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractdbl2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractdec2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractdec2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractdec2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractdec2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractdec2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractflt2args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractflt2args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractflt2args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractflt2args-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractflt2args-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractflt2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractflt2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractflt2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractflt2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractflt2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractint2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractint2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractint2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -3193,10 +3193,10 @@ Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractl
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractmix2args-1, EXPECTED_ERROR_GOT_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractmix2args-2, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractmix2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractmix2args-4, EXPECTED_RESULT_GOT_ERROR
-Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractmix2args-5, EXPECTED_RESULT_GOT_ERROR
+Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractmix2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractmix2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractmix2args-6, EXPECTED_ERROR_GOT_SAME_ERROR
-Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractmix2args-7, EXPECTED_RESULT_GOT_ERROR
+Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractmix2args-7, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractmix2args-8, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractmix2args-9, EXPECTED_ERROR_GOT_FAILURE
 Expressions/Operators/ArithExpr/NumericOpr/NumericSubtract//op-numeric-subtractnint2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -3243,15 +3243,15 @@ Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//K-NumericUnaryMinu
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//K-NumericUnaryMinus-8, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//K-NumericUnaryMinus-9, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//op-numeric-unary-minus-1, EXPECTED_RESULT_GOT_FAILURE
-Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//op-numeric-unary-minusdbl1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//op-numeric-unary-minusdbl1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//op-numeric-unary-minusdbl1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//op-numeric-unary-minusdbl1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//op-numeric-unary-minusdbl1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//op-numeric-unary-minusdec1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//op-numeric-unary-minusdec1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//op-numeric-unary-minusdec1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//op-numeric-unary-minusflt1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//op-numeric-unary-minusflt1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//op-numeric-unary-minusflt1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//op-numeric-unary-minusflt1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//op-numeric-unary-minusflt1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//op-numeric-unary-minusint1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//op-numeric-unary-minusint1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryMinus//op-numeric-unary-minusint1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -3295,15 +3295,15 @@ Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//K-NumericUnaryPlus-
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//K-NumericUnaryPlus-8, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//K-NumericUnaryPlus-9, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//op-numeric-unary-plus-1, EXPECTED_RESULT_GOT_FAILURE
-Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//op-numeric-unary-plusdbl1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//op-numeric-unary-plusdbl1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//op-numeric-unary-plusdbl1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//op-numeric-unary-plusdbl1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//op-numeric-unary-plusdbl1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//op-numeric-unary-plusdec1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//op-numeric-unary-plusdec1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//op-numeric-unary-plusdec1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//op-numeric-unary-plusflt1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//op-numeric-unary-plusflt1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//op-numeric-unary-plusflt1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//op-numeric-unary-plusflt1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//op-numeric-unary-plusflt1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//op-numeric-unary-plusint1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//op-numeric-unary-plusint1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/ArithExpr/NumericOpr/NumericUnaryPlus//op-numeric-unary-plusint1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -7316,8 +7316,8 @@ Expressions/exprSeqTypes/SeqExprCast//CastAs005, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs006, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs007, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs008, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/exprSeqTypes/SeqExprCast//CastAs009, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/exprSeqTypes/SeqExprCast//CastAs010, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/exprSeqTypes/SeqExprCast//CastAs009, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/exprSeqTypes/SeqExprCast//CastAs010, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs011, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/exprSeqTypes/SeqExprCast//CastAs012, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs013, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -7398,8 +7398,8 @@ Expressions/exprSeqTypes/SeqExprCast//CastAs087, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs088, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs089, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs090, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/exprSeqTypes/SeqExprCast//CastAs091, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/exprSeqTypes/SeqExprCast//CastAs092, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/exprSeqTypes/SeqExprCast//CastAs091, EXPECTED_RESULT_GOT_SAME_RESULT
+Expressions/exprSeqTypes/SeqExprCast//CastAs092, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs093, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/exprSeqTypes/SeqExprCast//CastAs094, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs095, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -7482,7 +7482,7 @@ Expressions/exprSeqTypes/SeqExprCast//CastAs171, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs172, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs173, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs174, EXPECTED_RESULT_GOT_ERROR
-Expressions/exprSeqTypes/SeqExprCast//CastAs175, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/exprSeqTypes/SeqExprCast//CastAs175, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs176, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs177, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs178, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -7508,17 +7508,17 @@ Expressions/exprSeqTypes/SeqExprCast//CastAs197, EXPECTED_ERROR_GOT_DIFFERENT_ER
 Expressions/exprSeqTypes/SeqExprCast//CastAs198, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 Expressions/exprSeqTypes/SeqExprCast//CastAs199, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs200, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/exprSeqTypes/SeqExprCast//CastAs201, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/exprSeqTypes/SeqExprCast//CastAs201, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs202, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs203, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/exprSeqTypes/SeqExprCast//CastAs204, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
+Expressions/exprSeqTypes/SeqExprCast//CastAs204, EXPECTED_ERROR_GOT_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs205, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs206, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs207, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs208, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs209, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs210, EXPECTED_RESULT_GOT_ERROR
-Expressions/exprSeqTypes/SeqExprCast//CastAs211, EXPECTED_RESULT_GOT_ERROR
+Expressions/exprSeqTypes/SeqExprCast//CastAs211, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs212, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//CastAs213, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/exprSeqTypes/SeqExprCast//CastAs214, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -8851,7 +8851,7 @@ Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-437, EXPECTED_ERROR_GOT_FAIL
 Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-438, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-439, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-44, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-440, EXPECTED_RESULT_GOT_ERROR
+Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-440, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-441, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-442, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-443, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -8903,7 +8903,7 @@ Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-484, EXPECTED_ERROR_GOT_DIFF
 Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-485, EXPECTED_ERROR_GOT_FAILURE
 Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-486, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-487, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-488, EXPECTED_RESULT_GOT_ERROR
+Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-488, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-489, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-49, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCast//K-SeqExprCast-490, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -9780,14 +9780,14 @@ Expressions/exprSeqTypes/SeqExprCastable//CastableAs200, EXPECTED_RESULT_GOT_SAM
 Expressions/exprSeqTypes/SeqExprCastable//CastableAs201, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCastable//CastableAs202, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCastable//CastableAs203, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/exprSeqTypes/SeqExprCastable//CastableAs204, EXPECTED_RESULT_GOT_ERROR
+Expressions/exprSeqTypes/SeqExprCastable//CastableAs204, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCastable//CastableAs205, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCastable//CastableAs206, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCastable//CastableAs207, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCastable//CastableAs208, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCastable//CastableAs209, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCastable//CastableAs210, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/exprSeqTypes/SeqExprCastable//CastableAs211, EXPECTED_RESULT_GOT_ERROR
+Expressions/exprSeqTypes/SeqExprCastable//CastableAs211, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCastable//CastableAs212, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCastable//CastableAs213, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/exprSeqTypes/SeqExprCastable//CastableAs214, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -10924,8 +10924,8 @@ FunctX/functx-fn-false//functx-fn-false-1, EXPECTED_RESULT_GOT_SAME_RESULT
 FunctX/functx-fn-floor//functx-fn-floor-1, EXPECTED_RESULT_GOT_SAME_RESULT
 FunctX/functx-fn-floor//functx-fn-floor-2, EXPECTED_RESULT_GOT_SAME_RESULT
 FunctX/functx-fn-floor//functx-fn-floor-3, EXPECTED_RESULT_GOT_SAME_RESULT
-FunctX/functx-fn-floor//functx-fn-floor-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-FunctX/functx-fn-floor//functx-fn-floor-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+FunctX/functx-fn-floor//functx-fn-floor-4, EXPECTED_RESULT_GOT_SAME_RESULT
+FunctX/functx-fn-floor//functx-fn-floor-5, EXPECTED_RESULT_GOT_SAME_RESULT
 FunctX/functx-fn-floor//functx-fn-floor-6, EXPECTED_RESULT_GOT_FAILURE
 FunctX/functx-fn-floor//functx-fn-floor-all, EXPECTED_RESULT_GOT_FAILURE
 FunctX/functx-fn-hours-from-dateTime//functx-fn-hours-from-dateTime-1, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -11077,7 +11077,7 @@ FunctX/functx-fn-not//functx-fn-not-7, EXPECTED_RESULT_GOT_ERROR
 FunctX/functx-fn-not//functx-fn-not-all, EXPECTED_RESULT_GOT_FAILURE
 FunctX/functx-fn-number//functx-fn-number-1, EXPECTED_RESULT_GOT_FAILURE
 FunctX/functx-fn-number//functx-fn-number-2, EXPECTED_RESULT_GOT_FAILURE
-FunctX/functx-fn-number//functx-fn-number-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+FunctX/functx-fn-number//functx-fn-number-3, EXPECTED_RESULT_GOT_SAME_RESULT
 FunctX/functx-fn-number//functx-fn-number-4, EXPECTED_RESULT_GOT_SAME_RESULT
 FunctX/functx-fn-number//functx-fn-number-5, EXPECTED_RESULT_GOT_SAME_RESULT
 FunctX/functx-fn-number//functx-fn-number-6, EXPECTED_RESULT_GOT_FAILURE
@@ -11137,7 +11137,7 @@ FunctX/functx-fn-reverse//functx-fn-reverse-all, EXPECTED_RESULT_GOT_DIFFERENT_R
 FunctX/functx-fn-root//functx-fn-root-1, EXPECTED_RESULT_GOT_FAILURE
 FunctX/functx-fn-root//functx-fn-root-2, EXPECTED_RESULT_GOT_ERROR
 FunctX/functx-fn-root//functx-fn-root-all, EXPECTED_RESULT_GOT_FAILURE
-FunctX/functx-fn-round-half-to-even//functx-fn-round-half-to-even-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+FunctX/functx-fn-round-half-to-even//functx-fn-round-half-to-even-1, EXPECTED_RESULT_GOT_SAME_RESULT
 FunctX/functx-fn-round-half-to-even//functx-fn-round-half-to-even-2, EXPECTED_RESULT_GOT_SAME_RESULT
 FunctX/functx-fn-round-half-to-even//functx-fn-round-half-to-even-3, EXPECTED_RESULT_GOT_SAME_RESULT
 FunctX/functx-fn-round-half-to-even//functx-fn-round-half-to-even-4, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -11968,15 +11968,15 @@ Functions/AccessorFunc/DataFunc//K-DataFunc-4, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AccessorFunc/DataFunc//K2-DataFunc-1, EXPECTED_ERROR_GOT_SAME_ERROR
 Functions/AccessorFunc/DataFunc//K2-DataFunc-2, EXPECTED_ERROR_GOT_SAME_ERROR
 Functions/AccessorFunc/DataFunc//fn-datacomplextype-1, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-Functions/AccessorFunc/DataFunc//fn-datadbl1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/AccessorFunc/DataFunc//fn-datadbl1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AccessorFunc/DataFunc//fn-datadbl1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/AccessorFunc/DataFunc//fn-datadbl1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/AccessorFunc/DataFunc//fn-datadbl1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AccessorFunc/DataFunc//fn-datadec1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AccessorFunc/DataFunc//fn-datadec1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AccessorFunc/DataFunc//fn-datadec1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/AccessorFunc/DataFunc//fn-dataflt1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/AccessorFunc/DataFunc//fn-dataflt1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AccessorFunc/DataFunc//fn-dataflt1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/AccessorFunc/DataFunc//fn-dataflt1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/AccessorFunc/DataFunc//fn-dataflt1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AccessorFunc/DataFunc//fn-dataint1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AccessorFunc/DataFunc//fn-dataint1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AccessorFunc/DataFunc//fn-dataint1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -12124,15 +12124,15 @@ Functions/AccessorFunc/StringFunc//K-StringFunc-6, EXPECTED_ERROR_GOT_DIFFERENT_
 Functions/AccessorFunc/StringFunc//fn-string-1, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 Functions/AccessorFunc/StringFunc//fn-string-2, EXPECTED_RESULT_GOT_ERROR
 Functions/AccessorFunc/StringFunc//fn-string-3, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-Functions/AccessorFunc/StringFunc//fn-stringdbl1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/AccessorFunc/StringFunc//fn-stringdbl1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AccessorFunc/StringFunc//fn-stringdbl1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/AccessorFunc/StringFunc//fn-stringdbl1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/AccessorFunc/StringFunc//fn-stringdbl1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AccessorFunc/StringFunc//fn-stringdec1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AccessorFunc/StringFunc//fn-stringdec1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AccessorFunc/StringFunc//fn-stringdec1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/AccessorFunc/StringFunc//fn-stringflt1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/AccessorFunc/StringFunc//fn-stringflt1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AccessorFunc/StringFunc//fn-stringflt1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/AccessorFunc/StringFunc//fn-stringflt1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/AccessorFunc/StringFunc//fn-stringflt1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AccessorFunc/StringFunc//fn-stringint1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AccessorFunc/StringFunc//fn-stringint1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AccessorFunc/StringFunc//fn-stringint1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -12446,21 +12446,21 @@ Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concat-6, EXPECTED_RESU
 Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concat-7, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concat-8, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concat-9, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatdbl2args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatdbl2args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatdbl2args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatdbl2args-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatdbl2args-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatdbl2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatdbl2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatdbl2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatdbl2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatdbl2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatdec2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatdec2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatdec2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatdec2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatdec2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatflt2args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatflt2args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatflt2args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatflt2args-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatflt2args-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatflt2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatflt2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatflt2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatflt2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatflt2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatint2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatint2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/AllStringFunc/GeneralStringFunc/ConcatFunc//fn-concatint2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -14582,36 +14582,36 @@ Functions/NodeFunc/NodeNumberFunc//fn-number-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-number-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-number-3, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 Functions/NodeFunc/NodeNumberFunc//fn-number-4, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-Functions/NodeFunc/NodeNumberFunc//fn-numberdbl1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NodeFunc/NodeNumberFunc//fn-numberdbl1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numberdbl1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NodeFunc/NodeNumberFunc//fn-numberdbl1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/NodeFunc/NodeNumberFunc//fn-numberdec1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NodeFunc/NodeNumberFunc//fn-numberdbl1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/NodeFunc/NodeNumberFunc//fn-numberdec1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numberdec1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NodeFunc/NodeNumberFunc//fn-numberdec1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/NodeFunc/NodeNumberFunc//fn-numberflt1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NodeFunc/NodeNumberFunc//fn-numberdec1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/NodeFunc/NodeNumberFunc//fn-numberflt1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numberflt1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NodeFunc/NodeNumberFunc//fn-numberflt1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NodeFunc/NodeNumberFunc//fn-numberflt1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numberint1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numberint1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numberint1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NodeFunc/NodeNumberFunc//fn-numberintg1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NodeFunc/NodeNumberFunc//fn-numberintg1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numberintg1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NodeFunc/NodeNumberFunc//fn-numberintg1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NodeFunc/NodeNumberFunc//fn-numberintg1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numberlng1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numberlng1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numberlng1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NodeFunc/NodeNumberFunc//fn-numbernint1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NodeFunc/NodeNumberFunc//fn-numbernint1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numbernint1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numbernint1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numbernni1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numbernni1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NodeFunc/NodeNumberFunc//fn-numbernni1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/NodeFunc/NodeNumberFunc//fn-numbernpi1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NodeFunc/NodeNumberFunc//fn-numbernni1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/NodeFunc/NodeNumberFunc//fn-numbernpi1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numbernpi1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numbernpi1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numberpint1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numberpint1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NodeFunc/NodeNumberFunc//fn-numberpint1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NodeFunc/NodeNumberFunc//fn-numberpint1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numbersht1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numbersht1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NodeFunc/NodeNumberFunc//fn-numbersht1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -14897,15 +14897,15 @@ Functions/NumericFunc/ABSFunc//fn-abs-more-args-083, EXPECTED_ERROR_GOT_SAME_ERR
 Functions/NumericFunc/ABSFunc//fn-abs-more-args-084, EXPECTED_ERROR_GOT_SAME_ERROR
 Functions/NumericFunc/ABSFunc//fn-abs-more-args-085, EXPECTED_ERROR_GOT_SAME_ERROR
 Functions/NumericFunc/ABSFunc//fn-abs-more-args-086, EXPECTED_ERROR_GOT_SAME_ERROR
-Functions/NumericFunc/ABSFunc//fn-absdbl1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NumericFunc/ABSFunc//fn-absdbl1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/ABSFunc//fn-absdbl1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NumericFunc/ABSFunc//fn-absdbl1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NumericFunc/ABSFunc//fn-absdbl1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/ABSFunc//fn-absdec1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/ABSFunc//fn-absdec1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/ABSFunc//fn-absdec1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NumericFunc/ABSFunc//fn-absflt1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NumericFunc/ABSFunc//fn-absflt1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/ABSFunc//fn-absflt1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NumericFunc/ABSFunc//fn-absflt1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NumericFunc/ABSFunc//fn-absflt1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/ABSFunc//fn-absint1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/ABSFunc//fn-absint1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/ABSFunc//fn-absint1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -14955,15 +14955,15 @@ Functions/NumericFunc/CeilingFunc//K2-CeilingFunc-7, EXPECTED_RESULT_GOT_ERROR
 Functions/NumericFunc/CeilingFunc//K2-CeilingFunc-8, EXPECTED_RESULT_GOT_ERROR
 Functions/NumericFunc/CeilingFunc//K2-CeilingFunc-9, EXPECTED_RESULT_GOT_ERROR
 Functions/NumericFunc/CeilingFunc//fn-ceiling-1, EXPECTED_RESULT_GOT_FAILURE
-Functions/NumericFunc/CeilingFunc//fn-ceilingdbl1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NumericFunc/CeilingFunc//fn-ceilingdbl1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/CeilingFunc//fn-ceilingdbl1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NumericFunc/CeilingFunc//fn-ceilingdbl1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/NumericFunc/CeilingFunc//fn-ceilingdec1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/NumericFunc/CeilingFunc//fn-ceilingdec1args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/NumericFunc/CeilingFunc//fn-ceilingdec1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/NumericFunc/CeilingFunc//fn-ceilingflt1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NumericFunc/CeilingFunc//fn-ceilingdbl1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/NumericFunc/CeilingFunc//fn-ceilingdec1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/NumericFunc/CeilingFunc//fn-ceilingdec1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/NumericFunc/CeilingFunc//fn-ceilingdec1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/NumericFunc/CeilingFunc//fn-ceilingflt1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/CeilingFunc//fn-ceilingflt1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NumericFunc/CeilingFunc//fn-ceilingflt1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NumericFunc/CeilingFunc//fn-ceilingflt1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/CeilingFunc//fn-ceilingint1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/CeilingFunc//fn-ceilingint1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/CeilingFunc//fn-ceilingint1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -14998,7 +14998,7 @@ Functions/NumericFunc/FloorFunc//K-FloorFunc-1, EXPECTED_ERROR_GOT_SAME_ERROR
 Functions/NumericFunc/FloorFunc//K-FloorFunc-2, EXPECTED_ERROR_GOT_SAME_ERROR
 Functions/NumericFunc/FloorFunc//K-FloorFunc-3, EXPECTED_RESULT_GOT_FAILURE
 Functions/NumericFunc/FloorFunc//K-FloorFunc-4, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NumericFunc/FloorFunc//K-FloorFunc-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NumericFunc/FloorFunc//K-FloorFunc-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/FloorFunc//K-FloorFunc-6, EXPECTED_ERROR_GOT_SAME_ERROR
 Functions/NumericFunc/FloorFunc//K2-FloorFunc-1, EXPECTED_RESULT_GOT_ERROR
 Functions/NumericFunc/FloorFunc//K2-FloorFunc-10, EXPECTED_RESULT_GOT_ERROR
@@ -15014,15 +15014,15 @@ Functions/NumericFunc/FloorFunc//K2-FloorFunc-7, EXPECTED_RESULT_GOT_ERROR
 Functions/NumericFunc/FloorFunc//K2-FloorFunc-8, EXPECTED_RESULT_GOT_ERROR
 Functions/NumericFunc/FloorFunc//K2-FloorFunc-9, EXPECTED_RESULT_GOT_ERROR
 Functions/NumericFunc/FloorFunc//fn-floor-1, EXPECTED_RESULT_GOT_FAILURE
-Functions/NumericFunc/FloorFunc//fn-floordbl1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NumericFunc/FloorFunc//fn-floordbl1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/FloorFunc//fn-floordbl1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NumericFunc/FloorFunc//fn-floordbl1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NumericFunc/FloorFunc//fn-floordbl1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/FloorFunc//fn-floordec1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/FloorFunc//fn-floordec1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/FloorFunc//fn-floordec1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NumericFunc/FloorFunc//fn-floorflt1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NumericFunc/FloorFunc//fn-floorflt1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/FloorFunc//fn-floorflt1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NumericFunc/FloorFunc//fn-floorflt1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NumericFunc/FloorFunc//fn-floorflt1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/FloorFunc//fn-floorint1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/FloorFunc//fn-floorint1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/FloorFunc//fn-floorint1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -15080,7 +15080,7 @@ Functions/NumericFunc/RoundEvenFunc//K2-RoundEvenFunc-26, EXPECTED_RESULT_GOT_ER
 Functions/NumericFunc/RoundEvenFunc//K2-RoundEvenFunc-27, EXPECTED_RESULT_GOT_ERROR
 Functions/NumericFunc/RoundEvenFunc//K2-RoundEvenFunc-28, EXPECTED_RESULT_GOT_ERROR
 Functions/NumericFunc/RoundEvenFunc//K2-RoundEvenFunc-3, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/NumericFunc/RoundEvenFunc//K2-RoundEvenFunc-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NumericFunc/RoundEvenFunc//K2-RoundEvenFunc-4, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/RoundEvenFunc//K2-RoundEvenFunc-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/RoundEvenFunc//K2-RoundEvenFunc-6, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/RoundEvenFunc//K2-RoundEvenFunc-7, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -15330,9 +15330,9 @@ Functions/NumericFunc/RoundFunc//fn-round-1, EXPECTED_RESULT_GOT_FAILURE
 Functions/NumericFunc/RoundFunc//fn-rounddbl1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Functions/NumericFunc/RoundFunc//fn-rounddbl1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/RoundFunc//fn-rounddbl1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/NumericFunc/RoundFunc//fn-rounddec1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/NumericFunc/RoundFunc//fn-rounddec1args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/NumericFunc/RoundFunc//fn-rounddec1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/NumericFunc/RoundFunc//fn-rounddec1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/NumericFunc/RoundFunc//fn-rounddec1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/NumericFunc/RoundFunc//fn-rounddec1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/RoundFunc//fn-roundflt1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Functions/NumericFunc/RoundFunc//fn-roundflt1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/NumericFunc/RoundFunc//fn-roundflt1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
@@ -15584,18 +15584,18 @@ Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-008, EXPECTED_RES
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-009, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-010, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-011, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-012, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-012, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-013, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-014, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-014, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-015, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-016, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-017, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-018, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-019, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-020, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-021, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-022, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-023, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-020, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-021, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-022, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-023, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-024, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-025, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-026, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -15639,9 +15639,9 @@ Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-063, EXPECTED_ERR
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-064, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-065, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avg-mix-args-066, EXPECTED_ERROR_GOT_FAILURE
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdbl1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdbl1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdbl1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdbl1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdbl1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdbl2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdbl2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdbl2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -15651,18 +15651,18 @@ Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdec1args-1, EXPECTED_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdec1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdec1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdec2args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdec2args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdec2args-2, EXPECTED_RESULT_GOT_ERROR
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdec2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdec2args-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdec2args-4, EXPECTED_RESULT_GOT_ERROR
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgdec2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgflt1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgflt1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgflt1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgflt1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgflt1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgflt2args-1, EXPECTED_ERROR_GOT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgflt2args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgflt2args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgflt2args-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgflt2args-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgflt2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgflt2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgflt2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgflt2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgint1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgint1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgint1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -15699,9 +15699,9 @@ Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgnni1args-1, EXPECTED_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgnni1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgnni1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgnni2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgnni2args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgnni2args-2, EXPECTED_RESULT_GOT_ERROR
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgnni2args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgnni2args-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgnni2args-4, EXPECTED_RESULT_GOT_ERROR
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgnni2args-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgnpi1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqAVGFunc//fn-avgnpi1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -15877,14 +15877,14 @@ Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-max-6, EXPECTED_RESULT_GOT_SAM
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-max-7, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-max-8, EXPECTED_ERROR_GOT_SAME_ERROR
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-max-9, EXPECTED_ERROR_GOT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdbl1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdbl1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdbl1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdbl1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdbl2args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdbl1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdbl2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdbl2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdbl2args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdbl2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdbl2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdbl2args-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdbl2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdec1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdec1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdec1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -15893,14 +15893,14 @@ Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdec2args-2, EXPECTED_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdec2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdec2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxdec2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxflt1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxflt1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxflt1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxflt1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxflt2args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxflt1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxflt2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxflt2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxflt2args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxflt2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxflt2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxflt2args-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxflt2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxint1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxint1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMAXFunc//fn-maxint1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -16057,14 +16057,14 @@ Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-min-6, EXPECTED_RESULT_GOT_SAM
 Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-min-7, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-min-8, EXPECTED_ERROR_GOT_SAME_ERROR
 Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-min-9, EXPECTED_ERROR_GOT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindbl1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindbl1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindbl1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindbl1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindbl2args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindbl2args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindbl2args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindbl2args-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindbl2args-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindbl1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindbl2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindbl2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindbl2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindbl2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindbl2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindec1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindec1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindec1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -16073,14 +16073,14 @@ Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindec2args-2, EXPECTED_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindec2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindec2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-mindec2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minflt1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minflt1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minflt1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minflt1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minflt2args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minflt2args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minflt2args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minflt2args-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minflt2args-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minflt1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minflt2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minflt2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minflt2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minflt2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minflt2args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minint1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minint1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqMINFunc//fn-minint1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -16208,17 +16208,17 @@ Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sum-6, EXPECTED_RESULT_GOT_SAM
 Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sum-7, EXPECTED_ERROR_GOT_SAME_ERROR
 Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sum-8, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sum-9, EXPECTED_ERROR_GOT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl2args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl2args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl3args-1, EXPECTED_ERROR_GOT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl3args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl3args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl3args-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl3args-2, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl3args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl3args-4, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl3args-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdbl3args-6, EXPECTED_ERROR_GOT_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdec1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -16232,17 +16232,17 @@ Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdec3args-1, EXPECTED_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdec3args-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdec3args-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumdec3args-4, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumflt1args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumflt1args-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumflt1args-2, EXPECTED_RESULT_GOT_SAME_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumflt1args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumflt2args-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumflt2args-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumflt2args-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumflt2args-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumflt1args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumflt2args-1, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumflt2args-2, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumflt2args-3, EXPECTED_RESULT_GOT_SAME_RESULT
+Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumflt2args-4, EXPECTED_RESULT_GOT_SAME_RESULT
 Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumflt3args-1, EXPECTED_ERROR_GOT_RESULT
-Functions/SeqFunc/AggregateSeqFunc/SeqSUMFunc//fn-sumflt3args-2, EXPECTED_

<TRUNCATED>

[7/8] vxquery git commit: Cleaned up the rewrite rules for adding collection.

Posted by pr...@apache.org.
Cleaned up the rewrite rules for adding collection.

 - Rules now remove unused expressions that have input from other operators.
 - Collection has been simplified to run only when the argument is a constant (previous item makes this possible).
 - Removed old treat rule that is now outdated.
 - Updated promote to work on sub-quantifiers.
 - Collection leaves the unnest operator above the data scan (Allows descendent path step).
 - New rule to remove unnest iterate when operating on a single tuple.


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

Branch: refs/heads/prestonc/xmark
Commit: 6f9c8f309d787db6fb381f73b1dc76919c6d7e57
Parents: 01a3bf1
Author: Eldon Carman <ec...@ucr.edu>
Authored: Sun May 31 12:40:35 2015 -0700
Committer: Eldon Carman <ec...@ucr.edu>
Committed: Wed Jun 3 14:23:48 2015 -0700

----------------------------------------------------------------------
 .../compiler/rewriter/RewriteRuleset.java       |  12 +-
 .../rewriter/rules/AbstractCollectionRule.java  |  46 ++----
 .../rewriter/rules/IntroduceCollectionRule.java |  13 +-
 .../RemoveRedundantPromoteExpressionsRule.java  |  16 +-
 .../rewriter/rules/RemoveUnusedTreatRule.java   | 147 -------------------
 .../rules/RemoveUnusedUnnestIterateRule.java    |  21 ++-
 .../rewriter/rules/util/ExpressionToolbox.java  |  23 +++
 .../rewriter/rules/util/OperatorToolbox.java    |   5 +-
 .../xmlquery/query/XMLQueryCompiler.java        |   2 +-
 9 files changed, 75 insertions(+), 210 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/vxquery/blob/6f9c8f30/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
index 008083d..8bda5b6 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
@@ -37,7 +37,6 @@ import org.apache.vxquery.compiler.rewriter.rules.RemoveRedundantDataExpressions
 import org.apache.vxquery.compiler.rewriter.rules.RemoveRedundantPromoteExpressionsRule;
 import org.apache.vxquery.compiler.rewriter.rules.RemoveRedundantTreatExpressionsRule;
 import org.apache.vxquery.compiler.rewriter.rules.RemoveUnusedSortDistinctNodesRule;
-import org.apache.vxquery.compiler.rewriter.rules.RemoveUnusedTreatRule;
 import org.apache.vxquery.compiler.rewriter.rules.RemoveUnusedUnnestIterateRule;
 import org.apache.vxquery.compiler.rewriter.rules.SetCollectionDataSourceRule;
 import org.apache.vxquery.compiler.rewriter.rules.SetVariableIdContextRule;
@@ -85,16 +84,17 @@ public class RewriteRuleset {
 
         // Remove unused functions.
         normalization.add(new RemoveUnusedSortDistinctNodesRule());
+        normalization.add(new RemoveRedundantTreatExpressionsRule());
+        normalization.add(new RemoveRedundantDataExpressionsRule());
+        normalization.add(new RemoveRedundantPromoteExpressionsRule());
+        normalization.add(new RemoveRedundantCastExpressionsRule());
+        normalization.add(new RemoveRedundantBooleanExpressionsRule());
         normalization.add(new RemoveRedundantVariablesRule());
         normalization.add(new RemoveUnusedAssignAndAggregateRule());
 
         // TODO Fix the group by operator before putting back in the rule set.
         //        normalization.add(new ConvertAssignSortDistinctNodesToOperatorsRule());
 
-        normalization.add(new RemoveUnusedTreatRule());
-        normalization.add(new RemoveRedundantVariablesRule());
-        normalization.add(new RemoveUnusedAssignAndAggregateRule());
-
         // Find unnest followed by aggregate in a subplan. 
         normalization.add(new EliminateUnnestAggregateSubplanRule());
         normalization.add(new RemoveRedundantVariablesRule());
@@ -154,7 +154,7 @@ public class RewriteRuleset {
     /**
      * Remove expressions known to be redundant.
      */
-    public final static List<IAlgebraicRewriteRule> buildRedundantExpressionNormalizationRuleCollection() {
+    public final static List<IAlgebraicRewriteRule> buildInlineRedundantExpressionNormalizationRuleCollection() {
         List<IAlgebraicRewriteRule> normalization = new LinkedList<IAlgebraicRewriteRule>();
         normalization.add(new InlineNestedVariablesRule());
         normalization.add(new RemoveRedundantTreatExpressionsRule());

http://git-wip-us.apache.org/repos/asf/vxquery/blob/6f9c8f30/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractCollectionRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractCollectionRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractCollectionRule.java
index a5ef4ba..717914a 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractCollectionRule.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractCollectionRule.java
@@ -23,10 +23,10 @@ import java.util.Arrays;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.vxquery.compiler.algebricks.VXQueryConstantValue;
+import org.apache.vxquery.compiler.rewriter.rules.util.OperatorToolbox;
 import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
 import org.apache.vxquery.datamodel.values.ValueTag;
 import org.apache.vxquery.functions.BuiltinFunctions;
-import org.apache.vxquery.functions.BuiltinOperators;
 import org.apache.vxquery.types.BuiltinTypeRegistry;
 import org.apache.vxquery.types.Quantifier;
 import org.apache.vxquery.types.SequenceType;
@@ -39,6 +39,7 @@ import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
@@ -81,45 +82,24 @@ public abstract class AbstractCollectionRule implements IAlgebraicRewriteRule {
             return null;
         }
 
+        ILogicalExpression logicalExpression2 = (ILogicalExpression) functionCall.getArguments().get(0).getValue();
+        if (logicalExpression2.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+            return null;
+        }
+        VariableReferenceExpression vre = (VariableReferenceExpression) logicalExpression2;
+        Mutable<ILogicalOperator> opRef3 = OperatorToolbox.findProducerOf(opRef, vre.getVariableReference());
+
         // Get the string assigned to the collection function.
-        AbstractLogicalOperator op3 = (AbstractLogicalOperator) assign.getInputs().get(0).getValue();
+        AbstractLogicalOperator op3 = (AbstractLogicalOperator) opRef3.getValue();
         if (op3.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
             AssignOperator assign2 = (AssignOperator) op3;
 
             // Check to see if the expression is a constant expression and type string.
-            ILogicalExpression logicalExpression2 = (ILogicalExpression) assign2.getExpressions().get(0).getValue();
-            if (logicalExpression2.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                return null;
-            }
-            ConstantExpression constantExpression = (ConstantExpression) logicalExpression2;
-            constantValue = (VXQueryConstantValue) constantExpression.getValue();
-            if (constantValue.getType() != SequenceType.create(BuiltinTypeRegistry.XS_STRING, Quantifier.QUANT_ONE)) {
-                return null;
-            }
-        } else if (op3.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
-            ILogicalExpression logicalExpression2 = (ILogicalExpression) functionCall.getArguments().get(0).getValue();
-            if (logicalExpression2.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-                return null;
-            }
-            AbstractFunctionCallExpression functionCall2 = (AbstractFunctionCallExpression) logicalExpression2;
-            if (!functionCall2.getFunctionIdentifier().equals(BuiltinOperators.PROMOTE.getFunctionIdentifier())) {
-                return null;
-            }
-
-            ILogicalExpression logicalExpression3 = (ILogicalExpression) functionCall2.getArguments().get(0).getValue();
-            if (logicalExpression3.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-                return null;
-            }
-            AbstractFunctionCallExpression functionCall3 = (AbstractFunctionCallExpression) logicalExpression3;
-            if (!functionCall3.getFunctionIdentifier().equals(BuiltinFunctions.FN_DATA_1.getFunctionIdentifier())) {
-                return null;
-            }
-
-            ILogicalExpression logicalExpression4 = (ILogicalExpression) functionCall3.getArguments().get(0).getValue();
-            if (logicalExpression4.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            ILogicalExpression logicalExpression3 = (ILogicalExpression) assign2.getExpressions().get(0).getValue();
+            if (logicalExpression3.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
                 return null;
             }
-            ConstantExpression constantExpression = (ConstantExpression) logicalExpression4;
+            ConstantExpression constantExpression = (ConstantExpression) logicalExpression3;
             constantValue = (VXQueryConstantValue) constantExpression.getValue();
             if (constantValue.getType() != SequenceType.create(BuiltinTypeRegistry.XS_STRING, Quantifier.QUANT_ONE)) {
                 return null;

http://git-wip-us.apache.org/repos/asf/vxquery/blob/6f9c8f30/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/IntroduceCollectionRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/IntroduceCollectionRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/IntroduceCollectionRule.java
index c599876..9daafed 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/IntroduceCollectionRule.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/IntroduceCollectionRule.java
@@ -42,23 +42,18 @@ import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOpera
  * 
  *   plan__parent
  *   UNNEST( $v2 : exp($v1) )
- *   ASSIGN( $v1 : collection( $source ) )
- *   plan__child
- *   
- *   Where $v1 is not used anywhere else in the plan and $source is:
- *   ASSIGN( $source : promote( data( constant ) ) )
- *    or
- *   ASSIGN( $source : promote( data( $v0 ) ) )
+ *   ASSIGN( $v1 : collection( $v0 ) )
  *   ASSIGN( $v0 : constant )
+ *   plan__child
  *   
  * After 
  * 
  *   plan__parent
  *   UNNEST( $v2 : exp($v1) )
- *   DATASCAN( collection( $source ) , $v1 )
+ *   DATASCAN( collection( $v0 ) , $v1 )
  *   plan__child
  *   
- *   Where DATASCAN operator is configured to use the collection( $source) for 
+ *   Where DATASCAN operator is configured to use the collection( $v0) for 
  *   data represented by the "constant" and $v1 represents the xml document 
  *   nodes from the collection.
  * </pre>

http://git-wip-us.apache.org/repos/asf/vxquery/blob/6f9c8f30/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantPromoteExpressionsRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantPromoteExpressionsRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantPromoteExpressionsRule.java
index 12865e0..3700abf 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantPromoteExpressionsRule.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantPromoteExpressionsRule.java
@@ -16,10 +16,7 @@
  */
 package org.apache.vxquery.compiler.rewriter.rules;
 
-import org.apache.vxquery.datamodel.values.ValueTag;
 import org.apache.vxquery.functions.BuiltinOperators;
-import org.apache.vxquery.runtime.functions.cast.CastToDoubleOperation;
-import org.apache.vxquery.runtime.functions.cast.CastToFloatOperation;
 import org.apache.vxquery.types.BuiltinTypeRegistry;
 import org.apache.vxquery.types.SequenceType;
 
@@ -64,9 +61,16 @@ public class RemoveRedundantPromoteExpressionsRule extends AbstractRemoveRedunda
                 // These types can not be promoted.
                 return true;
             }
-            if (sTypeOutput != null && sTypeOutput.equals(sTypeArg)) {
-                // Same type.
-                return true;
+            if (sTypeOutput != null) {
+                if (sTypeOutput.equals(sTypeArg)) {
+                    // Same type and quantifier.
+                    return true;
+                }
+                if (sTypeOutput.getItemType().equals(sTypeArg.getItemType())
+                        && sTypeArg.getQuantifier().isSubQuantifier(sTypeOutput.getQuantifier())) {
+                    // Same type and stronger quantifier.
+                    return true;
+                }
             }
         }
         return false;

http://git-wip-us.apache.org/repos/asf/vxquery/blob/6f9c8f30/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedTreatRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedTreatRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedTreatRule.java
deleted file mode 100644
index 4b14c50..0000000
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedTreatRule.java
+++ /dev/null
@@ -1,147 +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.vxquery.compiler.rewriter.rules;
-
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.vxquery.compiler.algebricks.VXQueryConstantValue;
-import org.apache.vxquery.context.RootStaticContextImpl;
-import org.apache.vxquery.context.StaticContextImpl;
-import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
-import org.apache.vxquery.functions.BuiltinOperators;
-import org.apache.vxquery.runtime.functions.type.SequenceTypeMatcher;
-import org.apache.vxquery.types.SequenceType;
-
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
-import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
-
-/**
- * The rule searches for where the xquery treat function is used and
- * determines if the treat is necessary. The plan is modified if
- * any of these items is not required.
- * 
- * <pre>
- * Before
- * 
- *   plan__parent
- *   ASSIGN( $v1 : treat( $v0, \@type_expression ) )
- *   ASSIGN( $v0 : $$constant )
- *   plan__child
- *   
- *   Where $$constant is of \@type_expression.
- *   
- * After 
- * 
- *   plan__parent
- *   ASSIGN( $v1 : $v0 )
- *   ASSIGN( $v0 : $$constant )
- *   plan__child
- * </pre>
- * 
- * @author prestonc
- */
-// TODO Replace with constant folding rule.
-public class RemoveUnusedTreatRule implements IAlgebraicRewriteRule {
-    final StaticContextImpl dCtx = new StaticContextImpl(RootStaticContextImpl.INSTANCE);
-
-    @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-        return false;
-    }
-
-    @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
-        boolean operatorChanged = false;
-        // Check if assign is for treat.
-        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-        if (op.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
-            return false;
-        }
-        AssignOperator assignTreat = (AssignOperator) op;
-
-        // Check to see if the expression is a function and treat.
-        ILogicalExpression logicalExpression11 = (ILogicalExpression) assignTreat.getExpressions().get(0).getValue();
-        if (logicalExpression11.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-            return false;
-        }
-        AbstractFunctionCallExpression functionTreat = (AbstractFunctionCallExpression) logicalExpression11;
-        if (!functionTreat.getFunctionIdentifier().equals(BuiltinOperators.TREAT.getFunctionIdentifier())) {
-            return false;
-        }
-
-        // Find the variable id used as the parameter.
-        ILogicalExpression treatArg1 = (ILogicalExpression) functionTreat.getArguments().get(0).getValue();
-        if (treatArg1.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
-            return false;
-        }
-        VariableReferenceExpression variableExpression = (VariableReferenceExpression) treatArg1;
-        int variableId = variableExpression.getVariableReference().getId();
-
-        // Get type to check against constant.
-        ILogicalExpression treatArg2 = (ILogicalExpression) functionTreat.getArguments().get(1).getValue();
-        if (treatArg2.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-            return false;
-        }
-        TaggedValuePointable tvp = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
-        getConstantAsPointable((ConstantExpression) treatArg2, tvp);
-
-        IntegerPointable pTypeCode = (IntegerPointable) IntegerPointable.FACTORY.createPointable();
-        tvp.getValue(pTypeCode);
-        SequenceType sType = dCtx.lookupSequenceType(pTypeCode.getInteger());
-
-        AbstractLogicalOperator op2 = (AbstractLogicalOperator) assignTreat.getInputs().get(0).getValue();
-        if (op2.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
-            return false;
-        }
-        AssignOperator assignConstant = (AssignOperator) op2;
-
-        if (variableId == assignConstant.getVariables().get(0).getId()) {
-            ILogicalExpression expressionConstant = (ILogicalExpression) assignConstant.getExpressions().get(0)
-                    .getValue();
-            if (expressionConstant.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-                // Check constant against type supplied.
-                TaggedValuePointable tvp2 = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
-                getConstantAsPointable((ConstantExpression) expressionConstant, tvp2);
-
-                SequenceTypeMatcher stm = new SequenceTypeMatcher();
-                stm.setSequenceType(sType);
-
-                if (stm.sequenceTypeMatch(tvp2)) {
-                    assignTreat.getExpressions().get(0).setValue(treatArg1);
-                    operatorChanged = true;
-                }
-            }
-        }
-
-        return operatorChanged;
-    }
-
-    private void getConstantAsPointable(ConstantExpression typeExpression, TaggedValuePointable tvp) {
-        VXQueryConstantValue treatTypeConstant = (VXQueryConstantValue) typeExpression.getValue();
-        tvp.set(treatTypeConstant.getValue(), 0, treatTypeConstant.getValue().length);
-    }
-}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/6f9c8f30/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedUnnestIterateRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedUnnestIterateRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedUnnestIterateRule.java
index d960735..ceab6a9 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedUnnestIterateRule.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedUnnestIterateRule.java
@@ -17,6 +17,8 @@
 package org.apache.vxquery.compiler.rewriter.rules;
 
 import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.vxquery.compiler.rewriter.rules.propagationpolicies.documentorder.DocumentOrder;
+import org.apache.vxquery.compiler.rewriter.rules.propagationpolicies.uniquenodes.UniqueNodes;
 import org.apache.vxquery.functions.BuiltinOperators;
 
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -30,11 +32,12 @@ import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionC
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
 
 /**
- * The rule searches for an unnest operator (1) immediately following an unnest 
+ * The rule searches for an unnest operator (1) immediately following an unnest
  * or data scan operator (2). If the variable is only used in unnest (1) then
  * unnest (1) can be removed.
  * 
@@ -51,7 +54,8 @@ import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOpera
  * After
  * 
  *   plan__parent
- *   UNNEST( $v2 : $v )
+ *   ASSIGN( $v2 : $v1 )
+ *   UNNEST( $v1 : $v )
  *   plan__child
  * </pre>
  * 
@@ -78,8 +82,8 @@ public class RemoveUnusedUnnestIterateRule extends AbstractUsedVariablesProcessi
         }
 
         // Check to see if the expression is a variable.
-        ILogicalExpression logicalExpressionUnnest2 = (ILogicalExpression) functionCallUnnest1.getArguments().get(0)
-                .getValue();
+        Mutable<ILogicalExpression> logicalExpressionUnnestRef2 = functionCallUnnest1.getArguments().get(0);
+        ILogicalExpression logicalExpressionUnnest2 = (ILogicalExpression) logicalExpressionUnnestRef2.getValue();
         if (logicalExpressionUnnest2.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
             return false;
         }
@@ -89,11 +93,14 @@ public class RemoveUnusedUnnestIterateRule extends AbstractUsedVariablesProcessi
         // Check if the input is an DATASCAN or UNNEST operator..
         Mutable<ILogicalOperator> opRef2 = unnest.getInputs().get(0);
         AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
-        if (op2.getOperatorTag() == LogicalOperatorTag.UNNEST || op2.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+        if (op2.getOperatorTag() == LogicalOperatorTag.UNNEST
+                || op2.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
             AbstractScanOperator aso = (AbstractScanOperator) op2;
             if (aso.getVariables().size() == 1 && aso.getVariables().contains(unnestInput)) {
-                aso.setVariables(unnest.getVariables());
-                opRef.setValue(aso);
+                // Add in a noop.
+                AssignOperator assign = new AssignOperator(unnest.getVariable(), logicalExpressionUnnestRef2);
+                assign.getInputs().addAll(unnest.getInputs());
+                opRef.setValue(assign);
                 return true;
             }
         }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/6f9c8f30/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/ExpressionToolbox.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/ExpressionToolbox.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/ExpressionToolbox.java
index 34efdd8..e76fd15 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/ExpressionToolbox.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/ExpressionToolbox.java
@@ -38,6 +38,8 @@ import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpressio
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 
@@ -226,6 +228,14 @@ public class ExpressionToolbox {
                     case UNNEST:
                         UnnestOperator unnest = (UnnestOperator) variableOp;
                         return getOutputSequenceType(variableProducer, unnest.getExpressionRef(), dCtx);
+                    case ASSIGN:
+                        AssignOperator assign = (AssignOperator) variableOp;
+                        for (int i = 0; i < assign.getVariables().size(); ++i) {
+                            if (variableId.equals(assign.getVariables().get(i))) {
+                                return getOutputSequenceType(variableProducer, assign.getExpressions().get(i), dCtx);
+                            }
+                        }
+                        return null;
                     default:
                         // TODO Consider support for other operators. i.e. Assign.
                         break;
@@ -234,4 +244,17 @@ public class ExpressionToolbox {
         }
         return null;
     }
+
+    public static boolean isFunctionExpression(Mutable<ILogicalExpression> mutableLe,
+            AbstractFunctionCallExpression afce) {
+        ILogicalExpression le = mutableLe.getValue();
+        if (le.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        AbstractFunctionCallExpression fc = (AbstractFunctionCallExpression) le;
+        if (!fc.getFunctionIdentifier().equals(afce)) {
+            return false;
+        }
+        return true;
+    }
 }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/6f9c8f30/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/OperatorToolbox.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/OperatorToolbox.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/OperatorToolbox.java
index 725a082..78cd80f 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/OperatorToolbox.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/OperatorToolbox.java
@@ -30,6 +30,7 @@ import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBin
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 
 public class OperatorToolbox {
@@ -144,8 +145,10 @@ public class OperatorToolbox {
                     }
                 }
                 break;
-            case EMPTYTUPLESOURCE:
             case NESTEDTUPLESOURCE:
+                NestedTupleSourceOperator nts = (NestedTupleSourceOperator) op;
+                return findProducerOf(nts.getDataSourceReference(), lv);
+            case EMPTYTUPLESOURCE:
                 return null;
             default:
                 // Skip operators and go look at input.

http://git-wip-us.apache.org/repos/asf/vxquery/blob/6f9c8f30/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryCompiler.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryCompiler.java b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryCompiler.java
index 3055c64..38272dd 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryCompiler.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryCompiler.java
@@ -234,7 +234,7 @@ public class XMLQueryCompiler {
         defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(priorityCtrl,
                 RewriteRuleset.buildXQueryNormalizationRuleCollection()));
         defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,
-                RewriteRuleset.buildRedundantExpressionNormalizationRuleCollection()));
+                RewriteRuleset.buildInlineRedundantExpressionNormalizationRuleCollection()));
         defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(priorityCtrl,
                 RewriteRuleset.buildNestedDataSourceRuleCollection()));
         defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,


[3/8] vxquery git commit: New set of passing tests after patch.

Posted by pr...@apache.org.
New set of passing tests after patch.


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

Branch: refs/heads/prestonc/xmark
Commit: 61f68a3e6849c5c37531571d09b2f0a4a518357c
Parents: c230b90
Author: Eldon Carman <ec...@ucr.edu>
Authored: Mon Apr 13 13:07:40 2015 -0700
Committer: Eldon Carman <ec...@ucr.edu>
Committed: Wed Jun 3 14:23:47 2015 -0700

----------------------------------------------------------------------
 vxquery-xtest/results/xqts.txt | 464 ++++++++++++++++++------------------
 1 file changed, 232 insertions(+), 232 deletions(-)
----------------------------------------------------------------------



[6/8] vxquery git commit: VXQUERY-144: fix doap.rdf

Posted by pr...@apache.org.
VXQUERY-144: fix doap.rdf


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

Branch: refs/heads/prestonc/xmark
Commit: 6d13ad403983feaf7f59216e54b00583445923d4
Parents: 61f68a3
Author: Till Westmann <we...@gmail.com>
Authored: Mon May 4 23:07:27 2015 -0700
Committer: Eldon Carman <ec...@ucr.edu>
Committed: Wed Jun 3 14:23:47 2015 -0700

----------------------------------------------------------------------
 doap.rdf | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/vxquery/blob/6d13ad40/doap.rdf
----------------------------------------------------------------------
diff --git a/doap.rdf b/doap.rdf
index d690c74..fad42f6 100644
--- a/doap.rdf
+++ b/doap.rdf
@@ -55,9 +55,9 @@
       </foaf:Person>
     </maintainer>
     <asfext:implements><asfext:Standard>
-      <asfext:title>XQuery 1.0</asfext:title>
+      <asfext:title>XQuery 1.0: An XML Query Language</asfext:title>
       <asfext:body>W3C</asfext:body>
-      <asfext:id></asfext:id>
+      <asfext:id>XQuery 1.0</asfext:id>
       <asfext:url rdf:resource="http://www.w3.org/TR/xquery/"/>
     </asfext:Standard></asfext:implements>
   </Project>


[8/8] vxquery git commit: Updated based on feedback and now include an update to the XQTS results.

Posted by pr...@apache.org.
Updated based on feedback and now include an update to the XQTS results.


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

Branch: refs/heads/prestonc/xmark
Commit: c14424972962aebf49d8dced3884910b153feea4
Parents: 6f9c8f3
Author: Eldon Carman <ec...@ucr.edu>
Authored: Tue Jun 2 13:21:29 2015 -0700
Committer: Eldon Carman <ec...@ucr.edu>
Committed: Wed Jun 3 14:23:48 2015 -0700

----------------------------------------------------------------------
 .../compiler/rewriter/RewriteRuleset.java       |   2 -
 .../AbstractUsedVariablesProcessingRule.java    |   1 -
 .../step/DescendantOrSelfPathStepUnnesting.java |  24 +--
 .../xmlquery/translator/XMLQueryTranslator.java |   2 +-
 vxquery-xtest/results/xqts.txt                  | 158 +++++++++----------
 5 files changed, 92 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/vxquery/blob/c1442497/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
index 8bda5b6..b67402b 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
@@ -271,8 +271,6 @@ public class RewriteRuleset {
         prepareForJobGenRewrites.add(new IsolateHyracksOperatorsRule(
                 HeuristicOptimizer.hyraxOperatorsBelowWhichJobGenIsDisabled));
         prepareForJobGenRewrites.add(new ExtractCommonOperatorsRule());
-        //        prepareForJobGenRewrites.add(new VXQueryExtractCommonOperatorsRule());
-        //        prepareForJobGenRewrites.add(new DelayMaterializationForJoinProbeRule());
         // Re-infer all types, so that, e.g., the effect of not-is-null is
         // propagated.
         prepareForJobGenRewrites.add(new PushProjectIntoDataSourceScanRule());

http://git-wip-us.apache.org/repos/asf/vxquery/blob/c1442497/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractUsedVariablesProcessingRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractUsedVariablesProcessingRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractUsedVariablesProcessingRule.java
index e80597b..5c95a0e 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractUsedVariablesProcessingRule.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractUsedVariablesProcessingRule.java
@@ -52,7 +52,6 @@ public abstract class AbstractUsedVariablesProcessingRule implements IAlgebraicR
 
     @Override
     public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-        opRef.getValue().getOperatorTag();
         if (hasRun && !firstOpRef.equals(opRef)) {
             return false;
         } else {

http://git-wip-us.apache.org/repos/asf/vxquery/blob/c1442497/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepUnnesting.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepUnnesting.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepUnnesting.java
index 3fd1daf..090655b 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepUnnesting.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepUnnesting.java
@@ -33,11 +33,11 @@ import edu.uci.ics.hyracks.data.std.api.IPointable;
 
 public class DescendantOrSelfPathStepUnnesting extends AbstractForwardAxisPathStep {
     private boolean testSelf;
-    private boolean printSelf;
+    private boolean returnSelf;
     private int indexSeqArgs;
     private int seqArgsLength;
     private List<Integer> indexSequence = new ArrayList<Integer>();
-    private List<Integer> printedSequence = new ArrayList<Integer>();
+    private List<Integer> returnSequence = new ArrayList<Integer>();
 
     private final SequencePointable seqNtp = (SequencePointable) SequencePointable.FACTORY.createPointable();
     private final TaggedValuePointable tvpItem = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
@@ -50,10 +50,10 @@ public class DescendantOrSelfPathStepUnnesting extends AbstractForwardAxisPathSt
     }
 
     protected void init(TaggedValuePointable[] args) throws SystemException {
-        printSelf = true;
+        returnSelf = true;
         indexSeqArgs = 0;
         indexSequence.add(0);
-        printedSequence.add(0);
+        returnSequence.add(0);
 
         // Check the argument passed in as sequence or node tree.
         if (args[0].getTag() == ValueTag.SEQUENCE_TAG) {
@@ -82,7 +82,7 @@ public class DescendantOrSelfPathStepUnnesting extends AbstractForwardAxisPathSt
                 }
                 // Next node tree in sequence.
                 indexSeqArgs++;
-                printSelf = true;
+                returnSelf = true;
             }
         } else {
             // Single node tree input.
@@ -95,8 +95,8 @@ public class DescendantOrSelfPathStepUnnesting extends AbstractForwardAxisPathSt
     }
 
     private boolean processNodeTree(TaggedValuePointable rootTVP, IPointable result) throws AlgebricksException {
-        if (testSelf && printSelf) {
-            printSelf = false;
+        if (testSelf && returnSelf) {
+            returnSelf = false;
             tvpItem.set(rootTVP);
             try {
                 setNodeToResult(tvpItem, result);
@@ -122,7 +122,7 @@ public class DescendantOrSelfPathStepUnnesting extends AbstractForwardAxisPathSt
         // Set up next level tracking.
         if (level + 1 > indexSequence.size()) {
             indexSequence.add(0);
-            printedSequence.add(0);
+            returnSequence.add(0);
         }
 
         SequencePointable seqItem = pp.takeOne(SequencePointable.class);
@@ -134,8 +134,8 @@ public class DescendantOrSelfPathStepUnnesting extends AbstractForwardAxisPathSt
                 seqItem.getEntry(indexSequence.get(level), tvpItem);
 
                 // Check current node
-                if (indexSequence.get(level) == printedSequence.get(level)) {
-                   printedSequence.set(level, printedSequence.get(level) + 1);
+                if (indexSequence.get(level) == returnSequence.get(level)) {
+                    returnSequence.set(level, returnSequence.get(level) + 1);
                     setNodeToResult(tvpItem, result);
                     return true;
                 }
@@ -150,10 +150,10 @@ public class DescendantOrSelfPathStepUnnesting extends AbstractForwardAxisPathSt
             // Reset for next node tree.
             if (level == 0) {
                 indexSequence.set(level, 0);
-                printedSequence.set(level, 0);
+                returnSequence.set(level, 0);
             } else {
                 indexSequence.remove(level);
-                printedSequence.remove(level);
+                returnSequence.remove(level);
             }
             return false;
         } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/vxquery/blob/c1442497/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java
index 6b443b6..d684730 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java
@@ -1048,7 +1048,7 @@ public class XMLQueryTranslator {
                     for (LetVarDeclNode lvdNode : lcNode.getVariables()) {
                         LogicalVariable seqVar = translateExpression(lvdNode.getSequence(), tCtx);
                         tCtx.pushVariableScope();
-                        SequenceType letVarType = SequenceType.create(AnyItemType.INSTANCE, Quantifier.QUANT_ONE);
+                        SequenceType letVarType = SequenceType.create(AnyItemType.INSTANCE, Quantifier.QUANT_STAR);
                         if (lvdNode.getType() != null) {
                             letVarType = createSequenceType(lvdNode.getType());
                         }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/c1442497/vxquery-xtest/results/xqts.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/results/xqts.txt b/vxquery-xtest/results/xqts.txt
index 816b4e5..acb34d6 100644
--- a/vxquery-xtest/results/xqts.txt
+++ b/vxquery-xtest/results/xqts.txt
@@ -117,7 +117,7 @@ Construct/ComputeCon/ComputeConAttr//K2-ComputeConAttr-58, EXPECTED_RESULT_GOT_E
 Construct/ComputeCon/ComputeConAttr//K2-ComputeConAttr-59, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 Construct/ComputeCon/ComputeConAttr//K2-ComputeConAttr-6, EXPECTED_ERROR_GOT_SAME_ERROR
 Construct/ComputeCon/ComputeConAttr//K2-ComputeConAttr-60, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-Construct/ComputeCon/ComputeConAttr//K2-ComputeConAttr-61, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
+Construct/ComputeCon/ComputeConAttr//K2-ComputeConAttr-61, EXPECTED_ERROR_GOT_SAME_ERROR
 Construct/ComputeCon/ComputeConAttr//K2-ComputeConAttr-7, EXPECTED_ERROR_GOT_SAME_ERROR
 Construct/ComputeCon/ComputeConAttr//K2-ComputeConAttr-8, EXPECTED_ERROR_GOT_SAME_ERROR
 Construct/ComputeCon/ComputeConAttr//K2-ComputeConAttr-9, EXPECTED_ERROR_GOT_SAME_ERROR
@@ -469,7 +469,7 @@ Construct/DirectConElem/DirectConElemNamespace//K2-DirectConElemNamespace-73, EX
 Construct/DirectConElem/DirectConElemNamespace//K2-DirectConElemNamespace-74, EXPECTED_ERROR_GOT_RESULT
 Construct/DirectConElem/DirectConElemNamespace//K2-DirectConElemNamespace-75, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 Construct/DirectConElem/DirectConElemNamespace//K2-DirectConElemNamespace-76, EXPECTED_ERROR_GOT_RESULT
-Construct/DirectConElem/DirectConElemNamespace//K2-DirectConElemNamespace-77, EXPECTED_RESULT_GOT_FAILURE
+Construct/DirectConElem/DirectConElemNamespace//K2-DirectConElemNamespace-77, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Construct/DirectConElem/DirectConElemNamespace//K2-DirectConElemNamespace-78, EXPECTED_RESULT_GOT_FAILURE
 Construct/DirectConElem/DirectConElemNamespace//K2-DirectConElemNamespace-79, EXPECTED_ERROR_GOT_FAILURE
 Construct/DirectConElem/DirectConElemNamespace//K2-DirectConElemNamespace-8, EXPECTED_ERROR_GOT_SAME_ERROR
@@ -575,7 +575,7 @@ ErrorsAndOptimization//errors-and-optimization-1, EXPECTED_RESULT_GOT_ERROR
 ErrorsAndOptimization//errors-and-optimization-2, EXPECTED_ERROR_GOT_SAME_ERROR
 ErrorsAndOptimization//errors-and-optimization-3, EXPECTED_RESULT_GOT_FAILURE
 ErrorsAndOptimization//errors-and-optimization-4, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-ErrorsAndOptimization//errors-and-optimization-5, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
+ErrorsAndOptimization//errors-and-optimization-5, EXPECTED_ERROR_GOT_SAME_ERROR
 ErrorsAndOptimization//errors-and-optimization-6, EXPECTED_ERROR_GOT_SAME_ERROR
 ErrorsAndOptimization//errors-and-optimization-7, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/CondExpr//CondExpr002, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
@@ -805,8 +805,8 @@ Expressions/Construct/ComputeCon/ConText//Constr-text-parent-1, EXPECTED_RESULT_
 Expressions/Construct/ComputeCon/ConText//Constr-text-string-1, EXPECTED_RESULT_GOT_ERROR
 Expressions/Construct/ConInScopeNamespace//Constr-inscope-1, EXPECTED_RESULT_GOT_FAILURE
 Expressions/Construct/ConInScopeNamespace//Constr-inscope-10, EXPECTED_RESULT_GOT_FAILURE
-Expressions/Construct/ConInScopeNamespace//Constr-inscope-11, EXPECTED_RESULT_GOT_FAILURE
-Expressions/Construct/ConInScopeNamespace//Constr-inscope-12, EXPECTED_RESULT_GOT_FAILURE
+Expressions/Construct/ConInScopeNamespace//Constr-inscope-11, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Construct/ConInScopeNamespace//Constr-inscope-12, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/Construct/ConInScopeNamespace//Constr-inscope-13, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Construct/ConInScopeNamespace//Constr-inscope-14, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/Construct/ConInScopeNamespace//Constr-inscope-15, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -816,12 +816,12 @@ Expressions/Construct/ConInScopeNamespace//Constr-inscope-18, EXPECTED_RESULT_GO
 Expressions/Construct/ConInScopeNamespace//Constr-inscope-19, EXPECTED_RESULT_GOT_FAILURE
 Expressions/Construct/ConInScopeNamespace//Constr-inscope-2, EXPECTED_RESULT_GOT_FAILURE
 Expressions/Construct/ConInScopeNamespace//Constr-inscope-20, EXPECTED_RESULT_GOT_FAILURE
-Expressions/Construct/ConInScopeNamespace//Constr-inscope-3, EXPECTED_RESULT_GOT_FAILURE
+Expressions/Construct/ConInScopeNamespace//Constr-inscope-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/Construct/ConInScopeNamespace//Constr-inscope-4, EXPECTED_RESULT_GOT_FAILURE
 Expressions/Construct/ConInScopeNamespace//Constr-inscope-5, EXPECTED_RESULT_GOT_FAILURE
 Expressions/Construct/ConInScopeNamespace//Constr-inscope-6, EXPECTED_RESULT_GOT_FAILURE
-Expressions/Construct/ConInScopeNamespace//Constr-inscope-7, EXPECTED_RESULT_GOT_FAILURE
-Expressions/Construct/ConInScopeNamespace//Constr-inscope-8, EXPECTED_RESULT_GOT_FAILURE
+Expressions/Construct/ConInScopeNamespace//Constr-inscope-7, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Construct/ConInScopeNamespace//Constr-inscope-8, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/Construct/ConInScopeNamespace//Constr-inscope-9, EXPECTED_RESULT_GOT_FAILURE
 Expressions/Construct/DirectConElem//Constr-elem-curlybr-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Construct/DirectConElem//Constr-elem-curlybr-2, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -883,7 +883,7 @@ Expressions/Construct/DirectConElem/DirectConElemAttr//Constr-attr-ws-5, EXPECTE
 Expressions/Construct/DirectConElem/DirectConElemContent//Constr-cont-adjtext-1, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Construct/DirectConElem/DirectConElemContent//Constr-cont-adjtext-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/Construct/DirectConElem/DirectConElemContent//Constr-cont-adjtext-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Construct/DirectConElem/DirectConElemContent//Constr-cont-adjtext-4, EXPECTED_RESULT_GOT_FAILURE
+Expressions/Construct/DirectConElem/DirectConElemContent//Constr-cont-adjtext-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/Construct/DirectConElem/DirectConElemContent//Constr-cont-attr-1, EXPECTED_ERROR_GOT_FAILURE
 Expressions/Construct/DirectConElem/DirectConElemContent//Constr-cont-attr-2, EXPECTED_ERROR_GOT_FAILURE
 Expressions/Construct/DirectConElem/DirectConElemContent//Constr-cont-attr-3, EXPECTED_RESULT_GOT_FAILURE
@@ -960,8 +960,8 @@ Expressions/Construct/DirectConElem/DirectConElemContent//Constr-cont-text-2, EX
 Expressions/Construct/DirectConElem/DirectConElemContent//Constr-cont-uripres-1, EXPECTED_RESULT_GOT_ERROR
 Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-1, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-10, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-11, EXPECTED_RESULT_GOT_FAILURE
-Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-12, EXPECTED_RESULT_GOT_FAILURE
+Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-11, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-12, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-13, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-14, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-15, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
@@ -970,10 +970,10 @@ Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-17,
 Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-18, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-19, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-20, EXPECTED_RESULT_GOT_FAILURE
+Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-20, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-21, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-22, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-23, EXPECTED_RESULT_GOT_FAILURE
+Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-23, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-24, EXPECTED_ERROR_GOT_RESULT
 Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-25, EXPECTED_ERROR_GOT_RESULT
 Expressions/Construct/DirectConElem/DirectConElemNamespace//Constr-namespace-26, EXPECTED_RESULT_GOT_FAILURE
@@ -1032,11 +1032,11 @@ Expressions/Construct/DirectConElem/DirectConElemWhitespace//Constr-ws-nobound-4
 Expressions/Construct/DirectConElem/DirectConElemWhitespace//Constr-ws-nobound-5, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Construct/DirectConElem/DirectConElemWhitespace//Constr-ws-nobound-6, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Construct/DirectConElem/DirectConElemWhitespace//Constr-ws-tag-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Construct/DirectConElem/DirectConElemWhitespace//Constr-ws-tag-10, EXPECTED_RESULT_GOT_FAILURE
+Expressions/Construct/DirectConElem/DirectConElemWhitespace//Constr-ws-tag-10, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/Construct/DirectConElem/DirectConElemWhitespace//Constr-ws-tag-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/Construct/DirectConElem/DirectConElemWhitespace//Constr-ws-tag-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/Construct/DirectConElem/DirectConElemWhitespace//Constr-ws-tag-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/Construct/DirectConElem/DirectConElemWhitespace//Constr-ws-tag-5, EXPECTED_RESULT_GOT_FAILURE
+Expressions/Construct/DirectConElem/DirectConElemWhitespace//Constr-ws-tag-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/Construct/DirectConElem/DirectConElemWhitespace//Constr-ws-tag-6, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Construct/DirectConElem/DirectConElemWhitespace//Constr-ws-tag-7, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Construct/DirectConElem/DirectConElemWhitespace//Constr-ws-tag-8, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -1324,23 +1324,23 @@ Expressions/FLWORExpr/LetExpr//LetExpr006, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/FLWORExpr/LetExpr//LetExpr007, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/FLWORExpr/LetExpr//LetExpr008, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/FLWORExpr/LetExpr//LetExpr009, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/FLWORExpr/LetExpr//LetExpr010, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
+Expressions/FLWORExpr/LetExpr//LetExpr010, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/FLWORExpr/LetExpr//LetExpr011, EXPECTED_RESULT_GOT_FAILURE
 Expressions/FLWORExpr/LetExpr//LetExpr012, EXPECTED_RESULT_GOT_FAILURE
 Expressions/FLWORExpr/LetExpr//LetExpr013, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/FLWORExpr/LetExpr//LetExpr014, EXPECTED_ERROR_GOT_SAME_ERROR
-Expressions/FLWORExpr/LetExpr//LetExpr015, EXPECTED_RESULT_GOT_ERROR
-Expressions/FLWORExpr/LetExpr//LetExpr016, EXPECTED_RESULT_GOT_ERROR
+Expressions/FLWORExpr/LetExpr//LetExpr015, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/FLWORExpr/LetExpr//LetExpr016, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/FLWORExpr/LetExpr//LetExpr017, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/LetExpr//LetExpr018, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 Expressions/FLWORExpr/LetExpr//LetExpr019, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/LetExpr//LetExpr020, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/FLWORExpr/LetExpr//LetExpr021, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/FLWORExpr/LetExpr//xquery10keywords, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/FLWORExpr/LetExpr//xquery10keywords2, EXPECTED_RESULT_GOT_FAILURE
+Expressions/FLWORExpr/LetExpr//xquery10keywords2, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/FLWORExpr/LetExpr//xquery10keywords3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/LetExpr//xquery11keywords, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/FLWORExpr/LetExpr//xquery11keywords2, EXPECTED_RESULT_GOT_FAILURE
+Expressions/FLWORExpr/LetExpr//xquery11keywords2, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/FLWORExpr/LetExpr//xquery11keywords3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/FLWORExpr/LetExpr/LetExprWithout//K-LetExprWithout-1, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/FLWORExpr/LetExpr/LetExprWithout//K-LetExprWithout-2, EXPECTED_ERROR_GOT_SAME_ERROR
@@ -4200,7 +4200,7 @@ Expressions/Operators/CompExpr/ValComp/BooleanOp/BooleanEqual//op-boolean-equal-
 Expressions/Operators/CompExpr/ValComp/BooleanOp/BooleanEqual//op-boolean-equal-14, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/CompExpr/ValComp/BooleanOp/BooleanEqual//op-boolean-equal-15, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/CompExpr/ValComp/BooleanOp/BooleanEqual//op-boolean-equal-16, EXPECTED_RESULT_GOT_SAME_RESULT
-Expressions/Operators/CompExpr/ValComp/BooleanOp/BooleanEqual//op-boolean-equal-17, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
+Expressions/Operators/CompExpr/ValComp/BooleanOp/BooleanEqual//op-boolean-equal-17, EXPECTED_ERROR_GOT_SAME_ERROR
 Expressions/Operators/CompExpr/ValComp/BooleanOp/BooleanEqual//op-boolean-equal-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/CompExpr/ValComp/BooleanOp/BooleanEqual//op-boolean-equal-3, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/Operators/CompExpr/ValComp/BooleanOp/BooleanEqual//op-boolean-equal-4, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -6490,28 +6490,28 @@ Expressions/PrologExpr/EmptyOrderProlog//K-EmptyOrderProlog-1, EXPECTED_RESULT_G
 Expressions/PrologExpr/EmptyOrderProlog//K-EmptyOrderProlog-2, EXPECTED_RESULT_GOT_SAME_RESULT
 Expressions/PrologExpr/EmptyOrderProlog//K-EmptyOrderProlog-3, EXPECTED_ERROR_GOT_RESULT
 Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-1, EXPECTED_ERROR_GOT_RESULT
-Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-10, EXPECTED_RESULT_GOT_FAILURE
-Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-11, EXPECTED_RESULT_GOT_FAILURE
+Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-10, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-11, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-12, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-13, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-14, EXPECTED_RESULT_GOT_FAILURE
-Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-15, EXPECTED_RESULT_GOT_FAILURE
+Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-14, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-15, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-16, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-17, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-18, EXPECTED_RESULT_GOT_FAILURE
-Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-19, EXPECTED_RESULT_GOT_FAILURE
-Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-2, EXPECTED_RESULT_GOT_FAILURE
+Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-18, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-19, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-2, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-20, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-21, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-22, EXPECTED_RESULT_GOT_FAILURE
+Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-22, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-23, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-24, EXPECTED_RESULT_GOT_FAILURE
+Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-24, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-25, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-26, EXPECTED_RESULT_GOT_FAILURE
+Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-26, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-27, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-28, EXPECTED_RESULT_GOT_FAILURE
+Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-28, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-29, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-3, EXPECTED_RESULT_GOT_FAILURE
+Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-3, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 Expressions/PrologExpr/EmptyOrderProlog//emptyorderdecl-6, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
@@ -10522,7 +10522,7 @@ FLWORExpr/ForExpr/ForExprPositionalVar//K2-ForExprPositionalVar-3, EXPECTED_RESU
 FLWORExpr/ForExpr/ForExprPositionalVar//K2-ForExprPositionalVar-4, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 FLWORExpr/ForExpr/ForExprWith//K2-ForExprWith-1, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 FLWORExpr/ForExpr/ForExprWithout//K2-ForExprWithout-1, EXPECTED_ERROR_GOT_SAME_ERROR
-FLWORExpr/ForExpr/ForExprWithout//K2-ForExprWithout-10, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
+FLWORExpr/ForExpr/ForExprWithout//K2-ForExprWithout-10, EXPECTED_ERROR_GOT_RESULT
 FLWORExpr/ForExpr/ForExprWithout//K2-ForExprWithout-11, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 FLWORExpr/ForExpr/ForExprWithout//K2-ForExprWithout-12, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 FLWORExpr/ForExpr/ForExprWithout//K2-ForExprWithout-13, EXPECTED_RESULT_GOT_FAILURE
@@ -10593,8 +10593,8 @@ FLWORExpr/LetExpr/LetExprWithout//K2-LetExprWithout-7, EXPECTED_ERROR_GOT_DIFFER
 FLWORExpr/LetExpr/LetExprWithout//K2-LetExprWithout-8, EXPECTED_ERROR_GOT_SAME_ERROR
 FLWORExpr/LetExpr/LetExprWithout//K2-LetExprWithout-9, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-1, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-10, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-11, EXPECTED_RESULT_GOT_FAILURE
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-10, EXPECTED_ERROR_GOT_RESULT
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-11, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-12, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-13, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-14, EXPECTED_RESULT_GOT_FAILURE
@@ -10602,45 +10602,45 @@ FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-15, EXPECTED_ERR
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-16, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-17, EXPECTED_RESULT_GOT_FAILURE
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-18, EXPECTED_RESULT_GOT_SAME_RESULT
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-19, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-19, EXPECTED_ERROR_GOT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-2, EXPECTED_ERROR_GOT_RESULT
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-20, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-21, EXPECTED_RESULT_GOT_FAILURE
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-20, EXPECTED_ERROR_GOT_RESULT
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-21, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-22, EXPECTED_ERROR_GOT_RESULT
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-23, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-24, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-25, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-26, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-27, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-28, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-23, EXPECTED_ERROR_GOT_RESULT
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-24, EXPECTED_ERROR_GOT_RESULT
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-25, EXPECTED_ERROR_GOT_RESULT
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-26, EXPECTED_ERROR_GOT_RESULT
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-27, EXPECTED_ERROR_GOT_RESULT
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-28, EXPECTED_ERROR_GOT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-29, EXPECTED_ERROR_GOT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-3, EXPECTED_ERROR_GOT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-30, EXPECTED_ERROR_GOT_RESULT
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-31, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-32, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-31, EXPECTED_ERROR_GOT_RESULT
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-32, EXPECTED_ERROR_GOT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-33, EXPECTED_ERROR_GOT_FAILURE
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-34, EXPECTED_ERROR_GOT_FAILURE
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-35, EXPECTED_ERROR_GOT_FAILURE
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-36, EXPECTED_ERROR_GOT_FAILURE
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-37, EXPECTED_ERROR_GOT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-38, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-39, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-39, EXPECTED_ERROR_GOT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-4, EXPECTED_ERROR_GOT_RESULT
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-40, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-40, EXPECTED_ERROR_GOT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-41, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-42, NO_RESULT_FILE
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-43, EXPECTED_RESULT_GOT_FAILURE
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-44, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-45, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-46, EXPECTED_RESULT_GOT_ERROR
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-47, EXPECTED_RESULT_GOT_ERROR
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-46, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-47, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-48, EXPECTED_RESULT_GOT_ERROR
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-49, EXPECTED_RESULT_GOT_ERROR
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-5, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-6, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-7, EXPECTED_ERROR_GOT_SAME_ERROR
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-8, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-9, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-8, EXPECTED_ERROR_GOT_RESULT
+FLWORExpr/OrderbyExpr/OrderbyExprWithout//K2-OrderbyExprWithout-9, EXPECTED_ERROR_GOT_RESULT
 FLWORExpr/ReturnExpr//K2-ReturnExpr-1, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 FLWORExpr/WhereExpr//K2-WhereExpr-1, EXPECTED_ERROR_GOT_SAME_ERROR
 FLWORExpr/WhereExpr//K2-WhereExpr-2, EXPECTED_ERROR_GOT_SAME_ERROR
@@ -14659,7 +14659,7 @@ Functions/NodeFunc/NodeRootFunc//fn-root-8, EXPECTED_RESULT_GOT_ERROR
 Functions/NodeFunc/NodeRootFunc//fn-root-9, EXPECTED_RESULT_GOT_ERROR
 Functions/NodeSeqFunc/SeqCollectionFunc//fn-collection-1, EXPECTED_ERROR_GOT_SAME_ERROR
 Functions/NodeSeqFunc/SeqCollectionFunc//fn-collection-10, EXPECTED_RESULT_GOT_ERROR
-Functions/NodeSeqFunc/SeqCollectionFunc//fn-collection-10d, EXPECTED_RESULT_GOT_ERROR
+Functions/NodeSeqFunc/SeqCollectionFunc//fn-collection-10d, EXPECTED_RESULT_GOT_FAILURE
 Functions/NodeSeqFunc/SeqCollectionFunc//fn-collection-2, EXPECTED_ERROR_GOT_FAILURE
 Functions/NodeSeqFunc/SeqCollectionFunc//fn-collection-3, EXPECTED_ERROR_GOT_FAILURE
 Functions/NodeSeqFunc/SeqCollectionFunc//fn-collection-4, EXPECTED_RESULT_GOT_ERROR
@@ -17495,7 +17495,7 @@ Operators/ArithExpr/NumericOpr/NumericDivide//K2-NumericDivide-13, EXPECTED_RESU
 Operators/ArithExpr/NumericOpr/NumericDivide//K2-NumericDivide-14, EXPECTED_RESULT_GOT_SAME_RESULT
 Operators/ArithExpr/NumericOpr/NumericDivide//K2-NumericDivide-15, EXPECTED_RESULT_GOT_SAME_RESULT
 Operators/ArithExpr/NumericOpr/NumericDivide//K2-NumericDivide-16, EXPECTED_RESULT_GOT_SAME_RESULT
-Operators/ArithExpr/NumericOpr/NumericDivide//K2-NumericDivide-17, EXPECTED_RESULT_GOT_FAILURE
+Operators/ArithExpr/NumericOpr/NumericDivide//K2-NumericDivide-17, EXPECTED_RESULT_GOT_SAME_RESULT
 Operators/ArithExpr/NumericOpr/NumericDivide//K2-NumericDivide-2, EXPECTED_ERROR_GOT_SAME_ERROR
 Operators/ArithExpr/NumericOpr/NumericDivide//K2-NumericDivide-3, EXPECTED_ERROR_GOT_SAME_ERROR
 Operators/ArithExpr/NumericOpr/NumericDivide//K2-NumericDivide-4, EXPECTED_ERROR_GOT_SAME_ERROR
@@ -17570,7 +17570,7 @@ Operators/ArithExpr/NumericOpr/NumericMod//K2-NumericMod-59, EXPECTED_ERROR_GOT_
 Operators/ArithExpr/NumericOpr/NumericMod//K2-NumericMod-6, EXPECTED_RESULT_GOT_ERROR
 Operators/ArithExpr/NumericOpr/NumericMod//K2-NumericMod-60, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 Operators/ArithExpr/NumericOpr/NumericMod//K2-NumericMod-61, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-Operators/ArithExpr/NumericOpr/NumericMod//K2-NumericMod-62, EXPECTED_RESULT_GOT_FAILURE
+Operators/ArithExpr/NumericOpr/NumericMod//K2-NumericMod-62, EXPECTED_RESULT_GOT_SAME_RESULT
 Operators/ArithExpr/NumericOpr/NumericMod//K2-NumericMod-7, EXPECTED_ERROR_GOT_SAME_ERROR
 Operators/ArithExpr/NumericOpr/NumericMod//K2-NumericMod-8, EXPECTED_ERROR_GOT_SAME_ERROR
 Operators/ArithExpr/NumericOpr/NumericMod//K2-NumericMod-9, EXPECTED_RESULT_GOT_SAME_RESULT
@@ -17905,11 +17905,11 @@ PathExpr/Steps//K2-Steps-1, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 PathExpr/Steps//K2-Steps-10, EXPECTED_ERROR_GOT_FAILURE
 PathExpr/Steps//K2-Steps-11, EXPECTED_ERROR_GOT_SAME_ERROR
 PathExpr/Steps//K2-Steps-12, EXPECTED_ERROR_GOT_FAILURE
-PathExpr/Steps//K2-Steps-13, EXPECTED_RESULT_GOT_FAILURE
+PathExpr/Steps//K2-Steps-13, EXPECTED_RESULT_GOT_SAME_RESULT
 PathExpr/Steps//K2-Steps-14, EXPECTED_ERROR_GOT_RESULT
-PathExpr/Steps//K2-Steps-15, EXPECTED_RESULT_GOT_ERROR
-PathExpr/Steps//K2-Steps-16, EXPECTED_RESULT_GOT_ERROR
-PathExpr/Steps//K2-Steps-17, EXPECTED_RESULT_GOT_ERROR
+PathExpr/Steps//K2-Steps-15, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+PathExpr/Steps//K2-Steps-16, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+PathExpr/Steps//K2-Steps-17, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 PathExpr/Steps//K2-Steps-18, EXPECTED_RESULT_GOT_FAILURE
 PathExpr/Steps//K2-Steps-19, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 PathExpr/Steps//K2-Steps-2, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
@@ -17931,7 +17931,7 @@ PathExpr/Steps//K2-Steps-33, EXPECTED_RESULT_GOT_ERROR
 PathExpr/Steps//K2-Steps-34, EXPECTED_RESULT_GOT_ERROR
 PathExpr/Steps//K2-Steps-35, EXPECTED_RESULT_GOT_FAILURE
 PathExpr/Steps//K2-Steps-4, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-PathExpr/Steps//K2-Steps-5, EXPECTED_RESULT_GOT_FAILURE
+PathExpr/Steps//K2-Steps-5, EXPECTED_RESULT_GOT_SAME_RESULT
 PathExpr/Steps//K2-Steps-6, EXPECTED_RESULT_GOT_SAME_RESULT
 PathExpr/Steps//K2-Steps-7, EXPECTED_RESULT_GOT_SAME_RESULT
 PathExpr/Steps//K2-Steps-8, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
@@ -18055,7 +18055,7 @@ PathExpr/Steps/Axes//K2-Axes-89, EXPECTED_RESULT_GOT_ERROR
 PathExpr/Steps/Axes//K2-Axes-9, EXPECTED_ERROR_GOT_SAME_ERROR
 PathExpr/Steps/Axes//K2-Axes-90, EXPECTED_ERROR_GOT_SAME_ERROR
 PathExpr/Steps/Axes//K2-Axes-91, EXPECTED_ERROR_GOT_SAME_ERROR
-PathExpr/Steps/Axes//K2-Axes-92, EXPECTED_ERROR_GOT_FAILURE
+PathExpr/Steps/Axes//K2-Axes-92, EXPECTED_ERROR_GOT_RESULT
 PathExpr/Steps/Axes//K2-Axes-93, EXPECTED_ERROR_GOT_SAME_ERROR
 PathExpr/Steps/Axes//K2-Axes-94, EXPECTED_ERROR_GOT_FAILURE
 PathExpr/Steps/Axes//K2-Axes-95, EXPECTED_ERROR_GOT_SAME_ERROR
@@ -18088,7 +18088,7 @@ PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-29, EXPECTED_ERROR_GOT_SAME
 PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-3, EXPECTED_ERROR_GOT_SAME_ERROR
 PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-30, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-31, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
-PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-32, EXPECTED_RESULT_GOT_FAILURE
+PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-32, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-33, EXPECTED_ERROR_GOT_SAME_ERROR
 PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-34, EXPECTED_ERROR_GOT_SAME_ERROR
 PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-35, EXPECTED_ERROR_GOT_FAILURE
@@ -18122,12 +18122,12 @@ PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-6, EXPECTED_ERROR_GOT_SAME_
 PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-60, EXPECTED_ERROR_GOT_FAILURE
 PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-61, EXPECTED_ERROR_GOT_FAILURE
 PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-62, EXPECTED_ERROR_GOT_FAILURE
-PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-63, EXPECTED_RESULT_GOT_FAILURE
-PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-64, EXPECTED_RESULT_GOT_FAILURE
-PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-65, EXPECTED_RESULT_GOT_FAILURE
+PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-63, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-64, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
+PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-65, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-66, EXPECTED_ERROR_GOT_SAME_ERROR
 PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-67, EXPECTED_ERROR_GOT_SAME_ERROR
-PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-68, EXPECTED_ERROR_GOT_FAILURE
+PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-68, EXPECTED_ERROR_GOT_RESULT
 PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-69, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-7, EXPECTED_ERROR_GOT_SAME_ERROR
 PathExpr/Steps/NodeTestSection/NameTest//K2-NameTest-70, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
@@ -18185,7 +18185,7 @@ PathExpr/Steps/NodeTestSection/NodeTest//K2-NodeTest-35, EXPECTED_ERROR_GOT_DIFF
 PathExpr/Steps/NodeTestSection/NodeTest//K2-NodeTest-36, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 PathExpr/Steps/NodeTestSection/NodeTest//K2-NodeTest-37, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 PathExpr/Steps/NodeTestSection/NodeTest//K2-NodeTest-38, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-PathExpr/Steps/NodeTestSection/NodeTest//K2-NodeTest-39, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
+PathExpr/Steps/NodeTestSection/NodeTest//K2-NodeTest-39, EXPECTED_ERROR_GOT_RESULT
 PathExpr/Steps/NodeTestSection/NodeTest//K2-NodeTest-4, EXPECTED_ERROR_GOT_SAME_ERROR
 PathExpr/Steps/NodeTestSection/NodeTest//K2-NodeTest-40, EXPECTED_ERROR_GOT_RESULT
 PathExpr/Steps/NodeTestSection/NodeTest//K2-NodeTest-41, EXPECTED_RESULT_GOT_FAILURE
@@ -18452,24 +18452,24 @@ StaticTyping/STFLWORExpr//statictyping-1, EXPECTED_ERROR_GOT_SAME_ERROR
 StaticTyping/STFLWORExpr//statictyping-10, EXPECTED_ERROR_GOT_SAME_ERROR
 StaticTyping/STFLWORExpr//statictyping-11, EXPECTED_ERROR_GOT_SAME_ERROR
 StaticTyping/STFLWORExpr//statictyping-12, EXPECTED_ERROR_GOT_SAME_ERROR
-StaticTyping/STFLWORExpr//statictyping-13, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-StaticTyping/STFLWORExpr//statictyping-14, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-StaticTyping/STFLWORExpr//statictyping-15, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-StaticTyping/STFLWORExpr//statictyping-16, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-StaticTyping/STFLWORExpr//statictyping-17, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-StaticTyping/STFLWORExpr//statictyping-18, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-StaticTyping/STFLWORExpr//statictyping-19, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
+StaticTyping/STFLWORExpr//statictyping-13, EXPECTED_ERROR_GOT_SAME_ERROR
+StaticTyping/STFLWORExpr//statictyping-14, EXPECTED_ERROR_GOT_SAME_ERROR
+StaticTyping/STFLWORExpr//statictyping-15, EXPECTED_ERROR_GOT_SAME_ERROR
+StaticTyping/STFLWORExpr//statictyping-16, EXPECTED_ERROR_GOT_SAME_ERROR
+StaticTyping/STFLWORExpr//statictyping-17, EXPECTED_ERROR_GOT_SAME_ERROR
+StaticTyping/STFLWORExpr//statictyping-18, EXPECTED_ERROR_GOT_SAME_ERROR
+StaticTyping/STFLWORExpr//statictyping-19, EXPECTED_ERROR_GOT_SAME_ERROR
 StaticTyping/STFLWORExpr//statictyping-2, EXPECTED_ERROR_GOT_SAME_ERROR
-StaticTyping/STFLWORExpr//statictyping-20, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
+StaticTyping/STFLWORExpr//statictyping-20, EXPECTED_ERROR_GOT_SAME_ERROR
 StaticTyping/STFLWORExpr//statictyping-21, EXPECTED_ERROR_GOT_SAME_ERROR
 StaticTyping/STFLWORExpr//statictyping-22, EXPECTED_ERROR_GOT_RESULT
 StaticTyping/STFLWORExpr//statictyping-23, EXPECTED_ERROR_GOT_SAME_ERROR
-StaticTyping/STFLWORExpr//statictyping-24, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
+StaticTyping/STFLWORExpr//statictyping-24, EXPECTED_ERROR_GOT_RESULT
 StaticTyping/STFLWORExpr//statictyping-3, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 StaticTyping/STFLWORExpr//statictyping-4, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 StaticTyping/STFLWORExpr//statictyping-5, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 StaticTyping/STFLWORExpr//statictyping-6, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-StaticTyping/STFLWORExpr//statictyping-7, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
+StaticTyping/STFLWORExpr//statictyping-7, EXPECTED_ERROR_GOT_RESULT
 StaticTyping/STFLWORExpr//statictyping-8, EXPECTED_ERROR_GOT_SAME_ERROR
 StaticTyping/STFLWORExpr//statictyping-9, EXPECTED_ERROR_GOT_SAME_ERROR
 StaticTyping/STFunctions//ST-Data001, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
@@ -18686,7 +18686,7 @@ exprSeqTypes/PrologExpr/NamespaceProlog//K2-NamespaceProlog-1, EXPECTED_ERROR_GO
 exprSeqTypes/PrologExpr/NamespaceProlog//K2-NamespaceProlog-10, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 exprSeqTypes/PrologExpr/NamespaceProlog//K2-NamespaceProlog-11, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
 exprSeqTypes/PrologExpr/NamespaceProlog//K2-NamespaceProlog-12, EXPECTED_ERROR_GOT_DIFFERENT_ERROR
-exprSeqTypes/PrologExpr/NamespaceProlog//K2-NamespaceProlog-13, EXPECTED_RESULT_GOT_FAILURE
+exprSeqTypes/PrologExpr/NamespaceProlog//K2-NamespaceProlog-13, EXPECTED_RESULT_GOT_DIFFERENT_RESULT
 exprSeqTypes/PrologExpr/NamespaceProlog//K2-NamespaceProlog-14, EXPECTED_ERROR_GOT_SAME_ERROR
 exprSeqTypes/PrologExpr/NamespaceProlog//K2-NamespaceProlog-15, EXPECTED_ERROR_GOT_RESULT
 exprSeqTypes/PrologExpr/NamespaceProlog//K2-NamespaceProlog-16, EXPECTED_ERROR_GOT_SAME_ERROR


[4/8] vxquery git commit: Shivani's patch for VXQUERY-141 and a fix XTest results when running multiple threads.

Posted by pr...@apache.org.
Shivani's patch for VXQUERY-141 and a fix XTest results when running multiple threads.


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

Branch: refs/heads/prestonc/xmark
Commit: c230b902b5bf77ea5baf409f3d5591e0707a1d3a
Parents: d7fcc0e
Author: Eldon Carman <ec...@ucr.edu>
Authored: Mon Apr 13 11:36:19 2015 -0700
Committer: Eldon Carman <ec...@ucr.edu>
Committed: Wed Jun 3 14:23:47 2015 -0700

----------------------------------------------------------------------
 .../functions/cast/CastToDecimalOperation.java  |   6 +-
 .../functions/cast/CastToDoubleOperation.java   |  12 +-
 .../functions/cast/CastToFloatOperation.java    | 101 ++-----
 .../functions/cast/CastToStringOperation.java   | 273 ++++---------------
 .../java/org/apache/vxquery/xtest/XTest.java    |   3 +
 5 files changed, 89 insertions(+), 306 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/vxquery/blob/c230b902/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDecimalOperation.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDecimalOperation.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDecimalOperation.java
index 03a1e14..f44a8e0 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDecimalOperation.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDecimalOperation.java
@@ -66,8 +66,12 @@ public class CastToDecimalOperation extends AbstractCastToOperation {
                 throw new SystemException(ErrorCode.FORG0001);
             }
         }
+        if (Double.isNaN(doublep.getDouble()) || Double.isInfinite(doublep.getDouble())) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
         abvsInner.reset();
-        castToString.convertDoubleCanonical(doublep, dOutInner);
+        dOutInner.write(ValueTag.XS_STRING_TAG);
+        dOutInner.writeUTF(Double.toString(doublep.getDouble()));
         stringp.set(abvsInner.getByteArray(), abvsInner.getStartOffset() + 1, abvsInner.getLength() - 1);
         convertStringExtra(stringp, dOut, true);
     }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/c230b902/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDoubleOperation.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDoubleOperation.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDoubleOperation.java
index 13bb1f0..43b06ff 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDoubleOperation.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDoubleOperation.java
@@ -92,14 +92,16 @@ public class CastToDoubleOperation extends AbstractCastToOperation {
             if (charIterator.next() != ICharacterIterator.EOS_CHAR) {
                 throw new SystemException(ErrorCode.FORG0001);
             } else if (c == Character.valueOf('I') && c2 == Character.valueOf('N') && c3 == Character.valueOf('F')) {
-                valueDouble = Double.POSITIVE_INFINITY;
+                if (negativeValue) {
+                    valueDouble = Double.NEGATIVE_INFINITY;
+                } else {
+                    valueDouble = Double.POSITIVE_INFINITY;
+                }
             } else if (c == Character.valueOf('N') && c2 == Character.valueOf('a') && c3 == Character.valueOf('N')) {
                 valueDouble = Double.NaN;
             } else {
                 throw new SystemException(ErrorCode.FORG0001);
             }
-            dOut.write(ValueTag.XS_DOUBLE_TAG);
-            dOut.writeDouble((negativeValue ? -valueDouble : valueDouble));
         } else {
             // We create an object to keep the conversion algorithm simple and improve precision.
             // While a better solution may be available this will hold us over until then.
@@ -110,9 +112,9 @@ public class CastToDoubleOperation extends AbstractCastToOperation {
             } catch (NumberFormatException e) {
                 throw new SystemException(ErrorCode.FORG0001);
             }
-            dOut.write(ValueTag.XS_DOUBLE_TAG);
-            dOut.writeDouble(valueDouble);
         }
+        dOut.write(ValueTag.XS_DOUBLE_TAG);
+        dOut.writeDouble(valueDouble);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/vxquery/blob/c230b902/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToFloatOperation.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToFloatOperation.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToFloatOperation.java
index d311c4c..6dbf917 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToFloatOperation.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToFloatOperation.java
@@ -40,10 +40,6 @@ public class CastToFloatOperation extends AbstractCastToOperation {
     /*
      * All the positive powers of 10 that can be represented exactly in float.
      */
-    private static final float powersOf10upTo10[] = { 1.0e0f, 1.0e1f, 1.0e2f, 1.0e3f, 1.0e4f, 1.0e5f, 1.0e6f, 1.0e7f,
-            1.0e8f, 1.0e9f, 1.0e10f };
-    private static final float powersOf10from20to30[] = { 1.0e20f, 1.0e21f, 1.0e22f, 1.0e23f, 1.0e24f, 1.0e25f,
-            1.0e26f, 1.0e27f, 1.0e28f, 1.0e29f, 1.0e30f };
 
     @Override
     public void convertBoolean(BooleanPointable boolp, DataOutput dOut) throws SystemException, IOException {
@@ -83,10 +79,8 @@ public class CastToFloatOperation extends AbstractCastToOperation {
     public void convertString(UTF8StringPointable stringp, DataOutput dOut) throws SystemException, IOException {
         ICharacterIterator charIterator = new UTF8StringCharacterIterator(stringp);
         charIterator.reset();
-        byte decimalPlace = 0;
-        long value = 0;
         float valueFloat;
-        boolean pastDecimal = false, negativeValue = false;
+        boolean negativeValue = false;
         int c = ICharacterIterator.EOS_CHAR;
         int c2 = ICharacterIterator.EOS_CHAR;
         int c3 = ICharacterIterator.EOS_CHAR;
@@ -104,94 +98,31 @@ public class CastToFloatOperation extends AbstractCastToOperation {
             if (charIterator.next() != ICharacterIterator.EOS_CHAR) {
                 throw new SystemException(ErrorCode.FORG0001);
             } else if (c == Character.valueOf('I') && c2 == Character.valueOf('N') && c3 == Character.valueOf('F')) {
-                valueFloat = Float.NEGATIVE_INFINITY;
+                if (negativeValue) {
+                    valueFloat = Float.NEGATIVE_INFINITY;
+                } else {
+                    valueFloat = Float.POSITIVE_INFINITY;
+                }
             } else if (c == Character.valueOf('N') && c2 == Character.valueOf('a') && c3 == Character.valueOf('N')) {
                 valueFloat = Float.NaN;
             } else {
                 throw new SystemException(ErrorCode.FORG0001);
             }
         } else {
-            // Read in the number.
-            do {
-                if (Character.isDigit(c)) {
-                    value = value * 10 - Character.getNumericValue(c);
-                    if (pastDecimal) {
-                        decimalPlace--;
-                    }
-                } else if (c == Character.valueOf('.') && pastDecimal == false) {
-                    pastDecimal = true;
-                } else if (c == Character.valueOf('E') || c == Character.valueOf('e')) {
-                    break;
-                } else {
-                    throw new SystemException(ErrorCode.FORG0001);
-                }
-            } while ((c = charIterator.next()) != ICharacterIterator.EOS_CHAR);
-
-            // Parse the exponent.
-            if (c == Character.valueOf('E') || c == Character.valueOf('e')) {
-                int moveOffset = 0;
-                boolean negativeOffset = false;
-                // Check for the negative sign.
-                c = charIterator.next();
-                if (c == Character.valueOf('-')) {
-                    negativeOffset = true;
-                    c = charIterator.next();
-                }
-                // Process the numeric value.
-                do {
-                    if (Character.isDigit(c)) {
-                        moveOffset = moveOffset * 10 + Character.getNumericValue(c);
-                    } else {
-                        throw new SystemException(ErrorCode.FORG0001);
-                    }
-                } while ((c = charIterator.next()) != ICharacterIterator.EOS_CHAR);
-                decimalPlace += (negativeOffset ? -moveOffset : moveOffset);
-            }
-
-            /*
-             * The following conditions to create the floating point value is using known valid float values.
-             * In addition, each one only needs one or two operations to get the float value, further minimizing
-             * possible errors. (Not perfect, but pretty good.)
-             */
-            valueFloat = (float) value;
-            if (decimalPlace == 0 || valueFloat == 0.0f) {
-                // No modification required to float value.
-            } else if (decimalPlace >= 0) {
-                if (decimalPlace <= 10) {
-                    valueFloat *= powersOf10upTo10[decimalPlace];
-                } else if (decimalPlace <= 20) {
-                    valueFloat *= powersOf10upTo10[10];
-                    valueFloat *= powersOf10upTo10[decimalPlace - 10];
-                } else if (decimalPlace <= 30) {
-                    valueFloat *= powersOf10from20to30[decimalPlace];
-                } else if (decimalPlace <= 38) {
-                    valueFloat *= powersOf10from20to30[10];
-                    valueFloat *= powersOf10upTo10[decimalPlace - 30];
-                }
-            } else {
-                if (decimalPlace >= -10) {
-                    valueFloat /= powersOf10upTo10[-decimalPlace];
-                } else if (decimalPlace >= -20) {
-                    valueFloat /= powersOf10upTo10[10];
-                    valueFloat /= powersOf10upTo10[-decimalPlace - 10];
-                } else if (decimalPlace >= -30) {
-                    valueFloat /= powersOf10from20to30[-decimalPlace];
-                } else if (decimalPlace >= -40) {
-                    valueFloat /= powersOf10from20to30[10];
-                    valueFloat /= powersOf10upTo10[-decimalPlace - 30];
-                } else if (decimalPlace >= -45) {
-                    valueFloat /= powersOf10from20to30[0];
-                    valueFloat /= powersOf10from20to30[-decimalPlace - 20];
-                }
+            // We create an object to keep the conversion algorithm simple and improve precision.
+            // While a better solution may be available this will hold us over until then.
+            StringBuilder sb = new StringBuilder();
+            stringp.toString(sb);
+            try {
+                valueFloat = Float.parseFloat(sb.toString());
+            } catch (NumberFormatException e) {
+                throw new SystemException(ErrorCode.FORG0001);
             }
         }
 
         dOut.write(ValueTag.XS_FLOAT_TAG);
-        if (valueFloat == 0.0f) {
-            dOut.writeFloat((negativeValue ? -0.0f : 0.0f));
-        } else {
-            dOut.writeFloat((negativeValue ? valueFloat : -valueFloat));
-        }
+        dOut.writeFloat(valueFloat);
+
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/vxquery/blob/c230b902/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToStringOperation.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToStringOperation.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToStringOperation.java
index 3cbcd5b..5f84682 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToStringOperation.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToStringOperation.java
@@ -49,16 +49,6 @@ public class CastToStringOperation extends AbstractCastToOperation {
     private DataOutput dOutInner = abvsInner.getDataOutput();
     int returnTag = ValueTag.XS_STRING_TAG;
     private final char[] hex = { '0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F' };
-    private static final int DOUBLE_MANTISSA_BITS = 52; // size of the mantissa in bits
-    private static final int DOUBLE_MANTISSA_OFFSET = -1075;
-    private static final int DOUBLE_EXPONENT_MAX = 1023;
-    private static final int DOUBLE_EXPONENT_MIN = -1022;
-    private static final int FLOAT_MANTISSA_BITS = 23; // size of the mantissa in bits
-    private static final int FLOAT_MANTISSA_OFFSET = -150;
-    private static final int FLOAT_EXPONENT_MAX = 127;
-    private static final int FLOAT_EXPONENT_MIN = -126;
-    private static final int b = 2; // base of stored value
-    private static final int B = 10; // base of printed value
 
     @Override
     public void convertAnyURI(UTF8StringPointable stringp, DataOutput dOut) throws SystemException, IOException {
@@ -151,14 +141,15 @@ public class CastToStringOperation extends AbstractCastToOperation {
         abvsInner.reset();
         double value = doublep.getDouble();
 
-        if (!Double.isInfinite(value) && !Double.isNaN(value) && Math.abs(value) >= 0.000001
-                && Math.abs(value) <= 1000000) {
-            CastToDecimalOperation castToDecimal = new CastToDecimalOperation();
-            castToDecimal.convertDouble(doublep, dOutInner);
-            XSDecimalPointable decp = (XSDecimalPointable) XSDecimalPointable.FACTORY.createPointable();
-            decp.set(abvsInner.getByteArray(), abvsInner.getStartOffset() + 1,
-                    XSDecimalPointable.TYPE_TRAITS.getFixedLength());
-            convertDecimal(decp, dOut);
+        if (Double.isInfinite(value)) {
+            if (value == Double.NEGATIVE_INFINITY) {
+                FunctionHelper.writeCharSequence("-", dOutInner);
+            }
+            FunctionHelper.writeCharSequence("INF", dOutInner);
+            sendStringDataOutput(dOut);
+        } else if (Double.isNaN(value)) {
+            FunctionHelper.writeCharSequence("NaN", dOutInner);
+            sendStringDataOutput(dOut);
         } else if (value == -0.0 || value == 0.0) {
             long bits = Double.doubleToLongBits(value);
             boolean negative = ((bits >> 63) == 0) ? false : true;
@@ -168,8 +159,23 @@ public class CastToStringOperation extends AbstractCastToOperation {
             }
             FunctionHelper.writeCharSequence("0", dOutInner);
             sendStringDataOutput(dOut);
+        } else if (Math.abs(value) >= 0.000001 && Math.abs(value) <= 10000000) {
+            //the jdk (toString function) does not output number in desired format when 
+            //a number is between one and ten million, so we take care of this 
+            //case separately here.
+            CastToDecimalOperation castToDecimal = new CastToDecimalOperation();
+            castToDecimal.convertDouble(doublep, dOutInner);
+            XSDecimalPointable decp = (XSDecimalPointable) XSDecimalPointable.FACTORY.createPointable();
+            decp.set(abvsInner.getByteArray(), abvsInner.getStartOffset() + 1,
+                    XSDecimalPointable.TYPE_TRAITS.getFixedLength());
+            if (Math.abs(value) <= 1000000) {
+                convertDecimal(decp, dOut);
+            } else {
+                decimalToScientificNotn(decp, dOut);
+            }
         } else {
-            convertDoubleCanonical(doublep, dOut);
+            dOut.write(returnTag);
+            dOut.writeUTF(Double.toString(value));
         }
     }
 
@@ -177,109 +183,9 @@ public class CastToStringOperation extends AbstractCastToOperation {
         abvsInner.reset();
         double value = doublep.getDouble();
 
-        if (Double.isInfinite(value)) {
-            if (value == Double.NEGATIVE_INFINITY) {
-                FunctionHelper.writeCharSequence("-", dOutInner);
-            }
-            FunctionHelper.writeCharSequence("INF", dOutInner);
-        } else if (Double.isNaN(value)) {
-            FunctionHelper.writeCharSequence("NaN", dOutInner);
-        } else {
-            /*
-             * The double to string algorithm is based on a paper by Robert G Burger and 
-             * R Kent Dybvig titled "Print Floating-Point Numbers Quickly and Accurately".
-             */
-            long bits = Double.doubleToLongBits(value);
-            boolean decimalPlaced = false;
-
-            boolean negative = ((bits >> 63) == 0) ? false : true;
-            int e = (int) ((bits >> 52) & 0x7ffL);
-            long f = (e == 0) ? (bits & 0xfffffffffffffL) << 1 : (bits & 0xfffffffffffffL) | 0x10000000000000L;
-            e = e + DOUBLE_MANTISSA_OFFSET;
-
-            if (negative) {
-                FunctionHelper.writeChar('-', dOutInner);
-            }
-            if (value == 0) {
-                FunctionHelper.writeCharSequence("0.0E0", dOutInner);
-            } else {
-                // Initialize variables
-                double r, s, mPlus, mMinus;
-                if (e >= 0) {
-                    if (f == Math.pow(b, DOUBLE_MANTISSA_BITS - 1)) {
-                        r = f * Math.pow(b, e) * 2;
-                        s = 2;
-                        mPlus = Math.pow(b, e);
-                        mMinus = Math.pow(b, e + 1);
-                    } else {
-                        r = f * Math.pow(b, e + 1) * 2;
-                        s = b * 2;
-                        mPlus = Math.pow(b, e);
-                        mMinus = Math.pow(b, e);
-                    }
-                } else {
-                    if (e == DOUBLE_EXPONENT_MIN || f != Math.pow(b, DOUBLE_MANTISSA_BITS - 1)) {
-                        r = f * Math.pow(b, e) * 2;
-                        s = 2;
-                        mPlus = Math.pow(b, e);
-                        mMinus = Math.pow(b, e + 1);
-                    } else {
-                        r = f * Math.pow(b, e + 1) * 2;
-                        s = b * 2;
-                        mPlus = Math.pow(b, e);
-                        mMinus = Math.pow(b, e);
-                    }
-                }
-
-                double k = Math.ceil(Math.log10((r + mPlus) / s));
-                if (k >= 0) {
-                    s = s * Math.pow(B, k);
-                } else {
-                    r = r * Math.pow(B, -k);
-                    mPlus = mPlus * Math.pow(B, -k);
-                    mMinus = mMinus * Math.pow(B, -k);
-                }
-
-                double d;
-                while (!Double.isInfinite(mPlus) && !Double.isNaN(mPlus) && !Double.isInfinite(mMinus)
-                        && !Double.isNaN(mMinus) && !Double.isInfinite(r) && !Double.isNaN(r)) {
-                    if (s == r) {
-                        // Special case where the value is off by a factor of ten.
-                        d = 1;
-                    } else {
-                        d = Math.floor((r * B) / s);
-                    }
-                    r = r * B % s;
-                    mPlus = mPlus * B;
-                    mMinus = mMinus * B;
-
-                    if (r < mMinus && r + mPlus > s) {
-                        if (r * 2 > s) {
-                            d = d + 1;
-                        }
-                        FunctionHelper.writeChar((char) ('0' + d), dOutInner);
-                        break;
-                    } else if (r + mPlus > s) {
-                        d = d + 1;
-                        FunctionHelper.writeChar((char) ('0' + d), dOutInner);
-                        break;
-                    } else if (r < mMinus) {
-                        FunctionHelper.writeChar((char) ('0' + d), dOutInner);
-                        break;
-                    }
-                    FunctionHelper.writeChar((char) ('0' + d), dOutInner);
-                    if (!decimalPlaced) {
-                        decimalPlaced = true;
-                        FunctionHelper.writeChar('.', dOutInner);
-                    }
-                }
-
-                long decimalPlace = FunctionHelper.getPowerOf10(value, DOUBLE_EXPONENT_MAX, DOUBLE_EXPONENT_MIN) - 1;
-                FunctionHelper.writeChar('E', dOutInner);
-                FunctionHelper.writeNumberWithPadding(decimalPlace, 1, dOutInner);
-            }
-        }
-        sendStringDataOutput(dOut);
+        dOut.write(returnTag);
+        dOut.writeUTF(Double.toString(value));
+        return;
     }
 
     @Override
@@ -435,99 +341,9 @@ public class CastToStringOperation extends AbstractCastToOperation {
         } else if (Float.isNaN(value)) {
             FunctionHelper.writeCharSequence("NaN", dOutInner);
         } else {
-            /*
-             * The double to string algorithm is based on a paper by Robert G Burger and 
-             * R Kent Dybvig titled "Print Floating-Point Numbers Quickly and Accurately".
-             */
-            long bits = Float.floatToIntBits(value);
-            boolean decimalPlaced = false;
-
-            boolean negative = ((bits >> 31) == 0) ? false : true;
-            int e = (int) ((bits >> 23) & 0xff);
-            int f = (int) ((e == 0) ? (bits & 0x7fffff) << 1 : (bits & 0x7fffff) | 0x800000);
-            e = e + FLOAT_MANTISSA_OFFSET;
-
-            if (negative) {
-                FunctionHelper.writeChar('-', dOutInner);
-            }
-            if (value == 0) {
-                FunctionHelper.writeCharSequence("0.0E0", dOutInner);
-            } else {
-                // Initialize variables
-                double r, s, mPlus, mMinus;
-                if (e >= 0) {
-                    if (f == Math.pow(b, FLOAT_MANTISSA_BITS - 1)) {
-                        r = f * Math.pow(b, e) * 2;
-                        s = 2;
-                        mPlus = Math.pow(b, e);
-                        mMinus = Math.pow(b, e + 1);
-                    } else {
-                        r = f * Math.pow(b, e + 1) * 2;
-                        s = b * 2;
-                        mPlus = Math.pow(b, e);
-                        mMinus = Math.pow(b, e);
-                    }
-                } else {
-                    if (e == FLOAT_EXPONENT_MIN || f != Math.pow(b, FLOAT_MANTISSA_BITS - 1)) {
-                        r = f * Math.pow(b, e) * 2;
-                        s = 2;
-                        mPlus = Math.pow(b, e);
-                        mMinus = Math.pow(b, e + 1);
-                    } else {
-                        r = f * Math.pow(b, e + 1) * 2;
-                        s = b * 2;
-                        mPlus = Math.pow(b, e);
-                        mMinus = Math.pow(b, e);
-                    }
-                }
-
-                double k = Math.ceil(Math.log10((r + mPlus) / s));
-                if (k >= 0) {
-                    s = s * Math.pow(B, k);
-                } else {
-                    r = r * Math.pow(B, -k);
-                    mPlus = mPlus * Math.pow(B, -k);
-                    mMinus = mMinus * Math.pow(B, -k);
-                }
-
-                double d;
-                while (!Double.isInfinite(mPlus) && !Double.isNaN(mPlus) && !Double.isInfinite(mMinus)
-                        && !Double.isNaN(mMinus)) {
-                    if (s == r) {
-                        // Special case where the value is off by a factor of ten.
-                        d = 1;
-                    } else {
-                        d = Math.floor((r * B) / s);
-                    }
-                    r = r * B % s;
-                    mPlus = mPlus * B;
-                    mMinus = mMinus * B;
-
-                    if (r < mMinus && r + mPlus > s) {
-                        if (r * 2 > s) {
-                            d = d + 1;
-                        }
-                        FunctionHelper.writeChar((char) ('0' + d), dOutInner);
-                        break;
-                    } else if (r + mPlus > s) {
-                        d = d + 1;
-                        FunctionHelper.writeChar((char) ('0' + d), dOutInner);
-                        break;
-                    } else if (r < mMinus) {
-                        FunctionHelper.writeChar((char) ('0' + d), dOutInner);
-                        break;
-                    }
-                    FunctionHelper.writeChar((char) ('0' + d), dOutInner);
-                    if (!decimalPlaced) {
-                        decimalPlaced = true;
-                        FunctionHelper.writeChar('.', dOutInner);
-                    }
-                }
-
-                long decimalPlace = FunctionHelper.getPowerOf10(value, FLOAT_EXPONENT_MAX, FLOAT_EXPONENT_MIN) - 1;
-                FunctionHelper.writeChar('E', dOutInner);
-                FunctionHelper.writeNumberWithPadding(decimalPlace, 1, dOutInner);
-            }
+            dOut.write(returnTag);
+            dOut.writeUTF(Float.toString(value));
+            return;
         }
         sendStringDataOutput(dOut);
     }
@@ -849,4 +665,31 @@ public class CastToStringOperation extends AbstractCastToOperation {
         dOut.write(abvsInner.getByteArray(), abvsInner.getStartOffset(), abvsInner.getLength());
     }
 
+    public void decimalToScientificNotn(XSDecimalPointable decp, DataOutput dOut) throws SystemException, IOException {
+        byte decimalPlace = decp.getDecimalPlace();
+        long value = decp.getDecimalValue();
+        byte nDigits = decp.getDigitCount();
+        abvsInner.reset();
+
+        if (!FunctionHelper.isNumberPostive(value)) {
+            // Negative result, but the rest of the calculations can be based on a positive value.
+            FunctionHelper.writeChar('-', dOutInner);
+            value *= -1;
+        }
+
+        if (value == 0) {
+            FunctionHelper.writeChar('0', dOutInner);
+        } else {
+            long pow10 = (long) Math.pow(10, nDigits - 1);
+            FunctionHelper.writeNumberWithPadding((value / pow10), 0, dOutInner);
+            FunctionHelper.writeChar('.', dOutInner);
+            long mod = value % pow10;
+            FunctionHelper.writeNumberWithPadding(mod, (nDigits - 1), dOutInner);
+            FunctionHelper.writeChar('E', dOutInner);
+            long power = (nDigits - 1) - decimalPlace;
+            FunctionHelper.writeNumberWithPadding(power, 0, dOutInner);
+        }
+        sendStringDataOutput(dOut);
+    }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/c230b902/vxquery-xtest/src/main/java/org/apache/vxquery/xtest/XTest.java
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/main/java/org/apache/vxquery/xtest/XTest.java b/vxquery-xtest/src/main/java/org/apache/vxquery/xtest/XTest.java
index 87f0687..0e2a6e0 100644
--- a/vxquery-xtest/src/main/java/org/apache/vxquery/xtest/XTest.java
+++ b/vxquery-xtest/src/main/java/org/apache/vxquery/xtest/XTest.java
@@ -90,6 +90,9 @@ public class XTest {
             Thread.sleep(opts.keepalive);
         }
         eSvc.shutdown();
+        while (!eSvc.awaitTermination(5L, TimeUnit.SECONDS)) {
+            System.err.println("Failed to close all threads, trying again...");
+        }
         for (ResultReporter r : reporters) {
             r.close();
         }