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/18 22:22:33 UTC

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

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/master
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) {