You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@vxquery.apache.org by Till Westmann <ti...@westmann.org> on 2013/08/28 07:37:56 UTC

Re: svn commit: r1518074 - in /incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery: compiler/rewriter/ compiler/rewriter/rules/ functions/ runtime/functions/base/ runtime/functions/step/

Hi Preston,

I've got a question on the ChildPathStepUnnestingEvaluatorFactory:
Is there a difference between the setNodeTest method in this class and the setNodeTest method in AbstractPathStepScalarEvaluator that keep us from having just one such method?

Thanks,
Till

On Aug 28, 2013, at 6:39 AM, prestonc@apache.org wrote:

> Author: prestonc
> Date: Wed Aug 28 04:39:56 2013
> New Revision: 1518074
> 
> URL: http://svn.apache.org/r1518074
> Log:
> Added a child path step unnesting evaluator factory to improve the query plan optimization. The commit includes a new rule to use the unnesting function in place of iterate where iterate is operating on a child result.
> 
> Added:
>    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java   (with props)
>    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java   (with props)
> Modified:
>    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
>    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
>    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java
>    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java
> 
> Modified: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
> URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java?rev=1518074&r1=1518073&r2=1518074&view=diff
> ==============================================================================
> --- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java (original)
> +++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java Wed Aug 28 04:39:56 2013
> @@ -22,6 +22,7 @@ import java.util.List;
> import org.apache.vxquery.compiler.rewriter.rules.ConsolidateAssignAggregateRule;
> import org.apache.vxquery.compiler.rewriter.rules.ConvertAssignSortDistinctNodesToOperatorsRule;
> import org.apache.vxquery.compiler.rewriter.rules.ConvertAssignToAggregateRule;
> +import org.apache.vxquery.compiler.rewriter.rules.ConvertAssignToUnnestRule;
> import org.apache.vxquery.compiler.rewriter.rules.EliminateSubplanForSinglePathsRule;
> import org.apache.vxquery.compiler.rewriter.rules.EliminateUnnestAggregateSequencesRule;
> import org.apache.vxquery.compiler.rewriter.rules.EliminateUnnestAggregateSubplanRule;
> @@ -89,6 +90,8 @@ public class RewriteRuleset {
>         // Remove single tuple input subplans and merge unnest aggregate operators.
>         normalization.add(new EliminateSubplanForSinglePathsRule());
>         normalization.add(new EliminateUnnestAggregateSequencesRule());
> +        
> +        normalization.add(new ConvertAssignToUnnestRule());
> 
>         // Convert to a data source scan operator.
>         normalization.add(new SetCollectionDataSourceRule());
> 
> Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java
> URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java?rev=1518074&view=auto
> ==============================================================================
> --- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java (added)
> +++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java Wed Aug 28 04:39:56 2013
> @@ -0,0 +1,111 @@
> +/*
> + * 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.expressions.AbstractFunctionCallExpression;
> +import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
> +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;
> +import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
> +
> +/**
> + * The rule searches for unnest(iterate) operator followed by an assign(child)
> + * operator and merges the assign into the unnest operator.
> + * 
> + * <pre>
> + * Before
> + * 
> + *   plan__parent
> + *   UNNEST( $v2 : iterate( $v1 ) )
> + *   ASSIGN( $v1 : child( $v0 ) )
> + *   plan__child
> + *   
> + *   where plan__parent does not use $v1 and $v0 is defined in plan__child.
> + *   
> + * After
> + * 
> + *   plan__parent
> + *   UNNEST( $v2 : child( $v0 ) )
> + *   plan__child
> + * </pre>
> + * 
> + * @author prestonc
> + */
> +public class ConvertAssignToUnnestRule implements IAlgebraicRewriteRule {
> +    @Override
> +    public boolean rewritePre(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 the iterate operator.
> +        ILogicalExpression logicalExpression = (ILogicalExpression) unnest.getExpressionRef().getValue();
> +        if (logicalExpression.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
> +            return false;
> +        }
> +        AbstractFunctionCallExpression functionCall = (AbstractFunctionCallExpression) logicalExpression;
> +        if (!functionCall.getFunctionIdentifier().equals(BuiltinOperators.ITERATE.getFunctionIdentifier())) {
> +            return false;
> +        }
> +
> +        AbstractLogicalOperator op2 = (AbstractLogicalOperator) unnest.getInputs().get(0).getValue();
> +        if (op2.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
> +            return false;
> +        }
> +        AssignOperator assign = (AssignOperator) op2;
> +
> +        // Check to see if the expression is a child function.
> +        ILogicalExpression logicalExpression2 = (ILogicalExpression) assign.getExpressions().get(0).getValue();
> +        if (logicalExpression2.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
> +            return false;
> +        }
> +        AbstractFunctionCallExpression functionCall2 = (AbstractFunctionCallExpression) logicalExpression2;
> +        if (!functionCall2.getFunctionIdentifier().equals(BuiltinOperators.CHILD.getFunctionIdentifier())) {
> +            return false;
> +        }
> +
> +        // TODO add checks for variables used that have now been removed.
> +
> +        // Update the unnest parameters.
> +        int index = 0;
> +        for (Mutable<ILogicalOperator> input : assign.getInputs()) {
> +            unnest.getInputs().get(index++).setValue(input.getValue());
> +        }
> +
> +        UnnestingFunctionCallExpression child = new UnnestingFunctionCallExpression(BuiltinOperators.CHILD, functionCall2.getArguments());
> +        unnest.getExpressionRef().setValue(child);
> +        
> +        return true;
> +    }
> +
> +    @Override
> +    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
> +        return false;
> +    }
> +}
> 
> Propchange: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java
> ------------------------------------------------------------------------------
>    svn:eol-style = native
> 
> Modified: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
> URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml?rev=1518074&r1=1518073&r2=1518074&view=diff
> ==============================================================================
> --- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml (original)
> +++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml Wed Aug 28 04:39:56 2013
> @@ -869,6 +869,7 @@
>         <param name="arg" type="node()*"/>
>         <return type="node()*"/>
>         <runtime type="scalar" class="org.apache.vxquery.runtime.functions.step.ChildPathStepScalarEvaluatorFactory"/>
> +        <runtime type="unnesting" class="org.apache.vxquery.runtime.functions.step.ChildPathStepUnnestingEvaluatorFactory"/>
>         <property type="DocumentOrder" class="org.apache.vxquery.compiler.rewriter.rules.propagationpolicies.InputPropertyPropagationPolicy">
>             <argument value="0"/>
>         </property>
> 
> Modified: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java
> URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java?rev=1518074&r1=1518073&r2=1518074&view=diff
> ==============================================================================
> --- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java (original)
> +++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java Wed Aug 28 04:39:56 2013
> @@ -17,6 +17,7 @@
> package org.apache.vxquery.runtime.functions.base;
> 
> import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
> +import org.apache.vxquery.exceptions.SystemException;
> 
> import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
> import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
> @@ -41,8 +42,12 @@ public abstract class AbstractTaggedValu
>         for (int i = 0; i < args.length; ++i) {
>             args[i].evaluate(tuple, tvps[i]);
>         }
> -        init(tvps);
> +        try {
> +            init(tvps);
> +        } catch (SystemException e) {
> +            throw new AlgebricksException(e);
> +        }
>     }
> 
> -    protected abstract void init(TaggedValuePointable[] args);
> +    protected abstract void init(TaggedValuePointable[] args) throws SystemException;
> }
> \ No newline at end of file
> 
> Modified: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java
> URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java?rev=1518074&r1=1518073&r2=1518074&view=diff
> ==============================================================================
> --- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java (original)
> +++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java Wed Aug 28 04:39:56 2013
> @@ -16,6 +16,8 @@
>  */
> package org.apache.vxquery.runtime.functions.base;
> 
> +import org.apache.vxquery.context.DynamicContext;
> +
> import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
> import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
> import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
> @@ -25,15 +27,20 @@ import edu.uci.ics.hyracks.api.context.I
> 
> public abstract class AbstractTaggedValueArgumentUnnestingEvaluatorFactory implements IUnnestingEvaluatorFactory {
>     private static final long serialVersionUID = 1L;
> +    
> +    protected DynamicContext dCtx;
> 
>     private final IScalarEvaluatorFactory[] args;
> 
>     public AbstractTaggedValueArgumentUnnestingEvaluatorFactory(IScalarEvaluatorFactory[] args) {
>         this.args = args;
> +        this.dCtx = null;
>     }
> 
>     @Override
>     public final IUnnestingEvaluator createUnnestingEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
> +        dCtx = (DynamicContext) ctx.getJobletContext().getGlobalJobData();
> +
>         IScalarEvaluator[] es = new IScalarEvaluator[args.length];
>         for (int i = 0; i < es.length; ++i) {
>             es[i] = args[i].createScalarEvaluator(ctx);
> 
> Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java
> URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java?rev=1518074&view=auto
> ==============================================================================
> --- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java (added)
> +++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java Wed Aug 28 04:39:56 2013
> @@ -0,0 +1,325 @@
> +/*
> + * 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.runtime.functions.step;
> +
> +import java.io.DataOutput;
> +import java.io.IOException;
> +
> +import org.apache.vxquery.datamodel.accessors.SequencePointable;
> +import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
> +import org.apache.vxquery.datamodel.accessors.atomic.CodedQNamePointable;
> +import org.apache.vxquery.datamodel.accessors.nodes.AttributeNodePointable;
> +import org.apache.vxquery.datamodel.accessors.nodes.DocumentNodePointable;
> +import org.apache.vxquery.datamodel.accessors.nodes.ElementNodePointable;
> +import org.apache.vxquery.datamodel.accessors.nodes.NodeTreePointable;
> +import org.apache.vxquery.datamodel.values.ValueTag;
> +import org.apache.vxquery.datamodel.values.XDMConstants;
> +import org.apache.vxquery.exceptions.ErrorCode;
> +import org.apache.vxquery.exceptions.SystemException;
> +import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentUnnestingEvaluator;
> +import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentUnnestingEvaluatorFactory;
> +import org.apache.vxquery.types.AttributeType;
> +import org.apache.vxquery.types.ElementType;
> +import org.apache.vxquery.types.NameTest;
> +import org.apache.vxquery.types.NodeType;
> +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.algebricks.runtime.base.IScalarEvaluatorFactory;
> +import edu.uci.ics.hyracks.algebricks.runtime.base.IUnnestingEvaluator;
> +import edu.uci.ics.hyracks.data.std.api.IPointable;
> +import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
> +import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
> +import edu.uci.ics.hyracks.data.std.primitive.VoidPointable;
> +import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
> +
> +public class ChildPathStepUnnestingEvaluatorFactory extends AbstractTaggedValueArgumentUnnestingEvaluatorFactory {
> +    private static final long serialVersionUID = 1L;
> +
> +    public ChildPathStepUnnestingEvaluatorFactory(IScalarEvaluatorFactory[] args) {
> +        super(args);
> +    }
> +
> +    @Override
> +    protected IUnnestingEvaluator createEvaluator(IScalarEvaluator[] args) throws AlgebricksException {
> +
> +        final SequencePointable seqp = (SequencePointable) SequencePointable.FACTORY.createPointable();
> +        final TaggedValuePointable rootTVP = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
> +        final DocumentNodePointable dnp = (DocumentNodePointable) DocumentNodePointable.FACTORY.createPointable();
> +        final ElementNodePointable enp = (ElementNodePointable) ElementNodePointable.FACTORY.createPointable();
> +        final IntegerPointable ip = (IntegerPointable) IntegerPointable.FACTORY.createPointable();
> +        final NodeTreePointable ntp = (NodeTreePointable) NodeTreePointable.FACTORY.createPointable();
> +        final TaggedValuePointable itemTvp = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
> +
> +        return new AbstractTaggedValueArgumentUnnestingEvaluator(args) {
> +            private int index;
> +            private int seqLength;
> +
> +            private boolean first;
> +            private ArrayBackedValueStorage nodeAbvs;
> +            private INodeFilter filter;
> +
> +            @Override
> +            public boolean step(IPointable result) throws AlgebricksException {
> +                while (index < seqLength) {
> +                    // Get the next item
> +                    seqp.getEntry(index, itemTvp);
> +                    ++index;
> +                    
> +                    // Test to see if the item fits the path step
> +                    if (matches()) {
> +                        try {
> +                            setNodeToResult(result);
> +                            return true;
> +                        } catch (IOException e) {
> +                            String description = ErrorCode.SYSE0001 + ": " + ErrorCode.SYSE0001.getDescription();
> +                            throw new AlgebricksException(description);
> +                        }
> +                    }
> +                }
> +                return false;
> +            }
> +
> +            @Override
> +            protected void init(TaggedValuePointable[] args) throws SystemException {
> +                first = true;
> +                nodeAbvs = new ArrayBackedValueStorage();
> +
> +                index = 0;
> +                if (first) {
> +                    if (args[1].getTag() != ValueTag.XS_INT_TAG) {
> +                        throw new IllegalArgumentException("Expected int value tag, got: " + args[1].getTag());
> +                    }
> +                    args[1].getValue(ip);
> +                    int typeCode = ip.getInteger();
> +                    SequenceType sType = dCtx.getStaticContext().lookupSequenceType(typeCode);
> +                    setNodeTest(sType);
> +                    first = false;
> +                }
> +                if (args[0].getTag() != ValueTag.NODE_TREE_TAG) {
> +                    throw new SystemException(ErrorCode.SYSE0001);
> +                }
> +                args[0].getValue(ntp);
> +                getSequence(ntp, seqp);
> +                seqLength = seqp.getEntryCount();
> +            }
> +
> +            protected void setNodeTest(SequenceType sType) {
> +                final NodeType nodeType = (NodeType) sType.getItemType();
> +                switch (nodeType.getNodeKind()) {
> +                    case ANY:
> +                        filter = new INodeFilter() {
> +                            @Override
> +                            public boolean accept(NodeTreePointable ntp, TaggedValuePointable tvp) {
> +                                return true;
> +                            }
> +                        };
> +                        break;
> +
> +                    case ATTRIBUTE: {
> +                        AttributeType aType = (AttributeType) nodeType;
> +                        NameTest nameTest = aType.getNameTest();
> +                        byte[] uri = nameTest.getUri();
> +                        byte[] localName = nameTest.getLocalName();
> +                        final UTF8StringPointable urip = (UTF8StringPointable) (uri == null ? null
> +                                : UTF8StringPointable.FACTORY.createPointable());
> +                        final UTF8StringPointable localp = (UTF8StringPointable) (localName == null ? null
> +                                : UTF8StringPointable.FACTORY.createPointable());
> +                        if (uri != null) {
> +                            urip.set(uri, 0, uri.length);
> +                        }
> +                        if (localName != null) {
> +                            localp.set(localName, 0, localName.length);
> +                        }
> +                        final IPointable temp = VoidPointable.FACTORY.createPointable();
> +                        final AttributeNodePointable anp = (AttributeNodePointable) AttributeNodePointable.FACTORY
> +                                .createPointable();
> +                        final CodedQNamePointable cqp = (CodedQNamePointable) CodedQNamePointable.FACTORY
> +                                .createPointable();
> +                        filter = new INodeFilter() {
> +                            @Override
> +                            public boolean accept(NodeTreePointable ntp, TaggedValuePointable tvp) {
> +                                if (tvp.getTag() != ValueTag.ATTRIBUTE_NODE_TAG) {
> +                                    return false;
> +                                }
> +                                tvp.getValue(anp);
> +                                anp.getName(cqp);
> +                                if (urip != null) {
> +                                    ntp.getString(cqp.getNamespaceCode(), temp);
> +                                    if (urip.compareTo(temp) != 0) {
> +                                        return false;
> +                                    }
> +                                }
> +                                if (localp != null) {
> +                                    ntp.getString(cqp.getLocalCode(), temp);
> +                                    if (localp.compareTo(temp) != 0) {
> +                                        return false;
> +                                    }
> +                                }
> +                                return true;
> +                            }
> +                        };
> +                        break;
> +                    }
> +
> +                    case COMMENT:
> +                        filter = new INodeFilter() {
> +                            @Override
> +                            public boolean accept(NodeTreePointable ntp, TaggedValuePointable tvp) {
> +                                return tvp.getTag() == ValueTag.COMMENT_NODE_TAG;
> +                            }
> +                        };
> +                        break;
> +
> +                    case DOCUMENT:
> +                        filter = new INodeFilter() {
> +                            @Override
> +                            public boolean accept(NodeTreePointable ntp, TaggedValuePointable tvp) {
> +                                return tvp.getTag() == ValueTag.DOCUMENT_NODE_TAG;
> +                            }
> +                        };
> +                        break;
> +
> +                    case ELEMENT: {
> +                        ElementType eType = (ElementType) nodeType;
> +                        NameTest nameTest = eType.getNameTest();
> +                        byte[] uri = nameTest.getUri();
> +                        byte[] localName = nameTest.getLocalName();
> +                        final UTF8StringPointable urip = (UTF8StringPointable) (uri == null ? null
> +                                : UTF8StringPointable.FACTORY.createPointable());
> +                        final UTF8StringPointable localp = (UTF8StringPointable) (localName == null ? null
> +                                : UTF8StringPointable.FACTORY.createPointable());
> +                        if (uri != null) {
> +                            urip.set(uri, 0, uri.length);
> +                        }
> +                        if (localName != null) {
> +                            localp.set(localName, 0, localName.length);
> +                        }
> +                        final IPointable temp = VoidPointable.FACTORY.createPointable();
> +                        final ElementNodePointable enp = (ElementNodePointable) ElementNodePointable.FACTORY
> +                                .createPointable();
> +                        final CodedQNamePointable cqp = (CodedQNamePointable) CodedQNamePointable.FACTORY
> +                                .createPointable();
> +                        filter = new INodeFilter() {
> +                            @Override
> +                            public boolean accept(NodeTreePointable ntp, TaggedValuePointable tvp) {
> +                                if (tvp.getTag() != ValueTag.ELEMENT_NODE_TAG) {
> +                                    return false;
> +                                }
> +                                tvp.getValue(enp);
> +                                enp.getName(cqp);
> +                                if (urip != null) {
> +                                    ntp.getString(cqp.getNamespaceCode(), temp);
> +                                    if (urip.compareTo(temp) != 0) {
> +                                        return false;
> +                                    }
> +                                }
> +                                if (localp != null) {
> +                                    ntp.getString(cqp.getLocalCode(), temp);
> +                                    if (localp.compareTo(temp) != 0) {
> +                                        return false;
> +                                    }
> +                                }
> +                                return true;
> +                            }
> +                        };
> +                        break;
> +                    }
> +
> +                    case PI:
> +                        filter = new INodeFilter() {
> +                            @Override
> +                            public boolean accept(NodeTreePointable ntp, TaggedValuePointable tvp) {
> +                                return tvp.getTag() == ValueTag.PI_NODE_TAG;
> +                            }
> +                        };
> +                        break;
> +
> +                    case TEXT:
> +                        filter = new INodeFilter() {
> +                            @Override
> +                            public boolean accept(NodeTreePointable ntp, TaggedValuePointable tvp) {
> +                                return tvp.getTag() == ValueTag.TEXT_NODE_TAG;
> +                            }
> +                        };
> +                        break;
> +                }
> +            }
> +
> +            protected boolean matches() {
> +                return filter.accept(ntp, itemTvp);
> +            }
> +
> +            protected void setNodeToResult(IPointable result) throws IOException {
> +                nodeAbvs.reset();
> +                DataOutput mainOut = nodeAbvs.getDataOutput();
> +                mainOut.write(ValueTag.NODE_TREE_TAG);
> +                boolean hasDictionary = ntp.dictionaryExists() && hasDictionary(itemTvp.getTag());
> +                byte header = (byte) (hasDictionary ? NodeTreePointable.HEADER_DICTIONARY_EXISTS_MASK : 0);
> +                // TODO add all header flags to this setting.
> +                boolean hasNodeIds = ntp.nodeIdExists();
> +                if (hasNodeIds) {
> +                    header |= NodeTreePointable.HEADER_NODEID_EXISTS_MASK;
> +                }
> +                mainOut.write(header);
> +                if (hasNodeIds) {
> +                    mainOut.writeInt(ntp.getRootNodeId());
> +                }
> +                if (hasDictionary) {
> +                    mainOut.write(ntp.getByteArray(), ntp.getDictionaryOffset(), ntp.getDictionarySize());
> +                }
> +                mainOut.write(itemTvp.getByteArray(), itemTvp.getStartOffset(), itemTvp.getLength());
> +                result.set(nodeAbvs.getByteArray(), nodeAbvs.getStartOffset(), nodeAbvs.getLength());
> +            }
> +
> +            private boolean hasDictionary(byte tag) {
> +                switch (tag) {
> +                    case ValueTag.ATTRIBUTE_NODE_TAG:
> +                    case ValueTag.DOCUMENT_NODE_TAG:
> +                    case ValueTag.ELEMENT_NODE_TAG:
> +                        return true;
> +                }
> +                return false;
> +            }
> +
> +            protected void getSequence(NodeTreePointable ntp, SequencePointable seqp) throws SystemException {
> +                ntp.getRootNode(rootTVP);
> +                switch (rootTVP.getTag()) {
> +                    case ValueTag.DOCUMENT_NODE_TAG:
> +                        rootTVP.getValue(dnp);
> +                        dnp.getContent(ntp, seqp);
> +                        return;
> +
> +                    case ValueTag.ELEMENT_NODE_TAG:
> +                        rootTVP.getValue(enp);
> +                        if (enp.childrenChunkExists()) {
> +                            enp.getChildrenSequence(ntp, seqp);
> +                            return;
> +                        }
> +                }
> +                XDMConstants.setEmptySequence(seqp);
> +            }
> +
> +        };
> +    }
> +
> +    private interface INodeFilter {
> +        public boolean accept(NodeTreePointable ntp, TaggedValuePointable tvp);
> +    }
> +
> +}
> \ No newline at end of file
> 
> Propchange: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java
> ------------------------------------------------------------------------------
>    svn:eol-style = native
> 
> 


Re: svn commit: r1518074 - in /incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery: compiler/rewriter/ compiler/rewriter/rules/ functions/ runtime/functions/base/ runtime/functions/step/

Posted by Till Westmann <ti...@westmann.org>.
The method setNodeTest basically does 2 things:
1) convert a SequenceType to an INodeFilter
2) assign the INodeFilter to a member.
If we pull 1) into a static method all the complexity should be in one place. We could just put a class into org.apache.vxquery.runtime.functions.step that contains such a method.

For the node tree building we could indeed have a NodeTreeBuilder class that encapsulates the members that are currently used in setNodeToResult and give each of the evaluators exactly one such builder.

Does this make sense?

Cheers,
Till

On Aug 28, 2013, at 9:15 PM, Eldon Carman <ec...@ucr.edu> wrote:

> Yes, Till.
> 
> The path steps could benefit from sharing code between unnest and scalar
> evaluators. I am a little confused on how to best share the code between
> the two. Each one is build on an abstract class with different interfaces.
> The interface defines the evaluate or step function, yet all the code in
> between is shared for the most part. Maybe we create two new classes for
> node testing and building node trees: NodeTest and NodeTreeBuilder.
> 
> What do you think?
> Preston
> 
> 
> 
> 
> On Tue, Aug 27, 2013 at 10:37 PM, Till Westmann <ti...@westmann.org> wrote:
> 
>> Hi Preston,
>> 
>> I've got a question on the ChildPathStepUnnestingEvaluatorFactory:
>> Is there a difference between the setNodeTest method in this class and the
>> setNodeTest method in AbstractPathStepScalarEvaluator that keep us from
>> having just one such method?
>> 
>> Thanks,
>> Till
>> 
>> On Aug 28, 2013, at 6:39 AM, prestonc@apache.org wrote:
>> 
>>> Author: prestonc
>>> Date: Wed Aug 28 04:39:56 2013
>>> New Revision: 1518074
>>> 
>>> URL: http://svn.apache.org/r1518074
>>> Log:
>>> Added a child path step unnesting evaluator factory to improve the query
>> plan optimization. The commit includes a new rule to use the unnesting
>> function in place of iterate where iterate is operating on a child result.
>>> 
>>> Added:
>>> 
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java
>>  (with props)
>>> 
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java
>>  (with props)
>>> Modified:
>>> 
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
>>> 
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
>>> 
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java
>>> 
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java
>>> 
>>> Modified:
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
>>> URL:
>> http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java?rev=1518074&r1=1518073&r2=1518074&view=diff
>>> 
>> ==============================================================================
>>> ---
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
>> (original)
>>> +++
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
>> Wed Aug 28 04:39:56 2013
>>> @@ -22,6 +22,7 @@ import java.util.List;
>>> import
>> org.apache.vxquery.compiler.rewriter.rules.ConsolidateAssignAggregateRule;
>>> import
>> org.apache.vxquery.compiler.rewriter.rules.ConvertAssignSortDistinctNodesToOperatorsRule;
>>> import
>> org.apache.vxquery.compiler.rewriter.rules.ConvertAssignToAggregateRule;
>>> +import
>> org.apache.vxquery.compiler.rewriter.rules.ConvertAssignToUnnestRule;
>>> import
>> org.apache.vxquery.compiler.rewriter.rules.EliminateSubplanForSinglePathsRule;
>>> import
>> org.apache.vxquery.compiler.rewriter.rules.EliminateUnnestAggregateSequencesRule;
>>> import
>> org.apache.vxquery.compiler.rewriter.rules.EliminateUnnestAggregateSubplanRule;
>>> @@ -89,6 +90,8 @@ public class RewriteRuleset {
>>>        // Remove single tuple input subplans and merge unnest aggregate
>> operators.
>>>        normalization.add(new EliminateSubplanForSinglePathsRule());
>>>        normalization.add(new EliminateUnnestAggregateSequencesRule());
>>> +
>>> +        normalization.add(new ConvertAssignToUnnestRule());
>>> 
>>>        // Convert to a data source scan operator.
>>>        normalization.add(new SetCollectionDataSourceRule());
>>> 
>>> Added:
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java
>>> URL:
>> http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java?rev=1518074&view=auto
>>> 
>> ==============================================================================
>>> ---
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java
>> (added)
>>> +++
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java
>> Wed Aug 28 04:39:56 2013
>>> @@ -0,0 +1,111 @@
>>> +/*
>>> + * 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.expressions.AbstractFunctionCallExpression;
>>> +import
>> edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
>>> +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;
>>> +import
>> edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
>>> +
>>> +/**
>>> + * The rule searches for unnest(iterate) operator followed by an
>> assign(child)
>>> + * operator and merges the assign into the unnest operator.
>>> + *
>>> + * <pre>
>>> + * Before
>>> + *
>>> + *   plan__parent
>>> + *   UNNEST( $v2 : iterate( $v1 ) )
>>> + *   ASSIGN( $v1 : child( $v0 ) )
>>> + *   plan__child
>>> + *
>>> + *   where plan__parent does not use $v1 and $v0 is defined in
>> plan__child.
>>> + *
>>> + * After
>>> + *
>>> + *   plan__parent
>>> + *   UNNEST( $v2 : child( $v0 ) )
>>> + *   plan__child
>>> + * </pre>
>>> + *
>>> + * @author prestonc
>>> + */
>>> +public class ConvertAssignToUnnestRule implements IAlgebraicRewriteRule
>> {
>>> +    @Override
>>> +    public boolean rewritePre(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 the iterate operator.
>>> +        ILogicalExpression logicalExpression = (ILogicalExpression)
>> unnest.getExpressionRef().getValue();
>>> +        if (logicalExpression.getExpressionTag() !=
>> LogicalExpressionTag.FUNCTION_CALL) {
>>> +            return false;
>>> +        }
>>> +        AbstractFunctionCallExpression functionCall =
>> (AbstractFunctionCallExpression) logicalExpression;
>>> +        if
>> (!functionCall.getFunctionIdentifier().equals(BuiltinOperators.ITERATE.getFunctionIdentifier()))
>> {
>>> +            return false;
>>> +        }
>>> +
>>> +        AbstractLogicalOperator op2 = (AbstractLogicalOperator)
>> unnest.getInputs().get(0).getValue();
>>> +        if (op2.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
>>> +            return false;
>>> +        }
>>> +        AssignOperator assign = (AssignOperator) op2;
>>> +
>>> +        // Check to see if the expression is a child function.
>>> +        ILogicalExpression logicalExpression2 = (ILogicalExpression)
>> assign.getExpressions().get(0).getValue();
>>> +        if (logicalExpression2.getExpressionTag() !=
>> LogicalExpressionTag.FUNCTION_CALL) {
>>> +            return false;
>>> +        }
>>> +        AbstractFunctionCallExpression functionCall2 =
>> (AbstractFunctionCallExpression) logicalExpression2;
>>> +        if
>> (!functionCall2.getFunctionIdentifier().equals(BuiltinOperators.CHILD.getFunctionIdentifier()))
>> {
>>> +            return false;
>>> +        }
>>> +
>>> +        // TODO add checks for variables used that have now been
>> removed.
>>> +
>>> +        // Update the unnest parameters.
>>> +        int index = 0;
>>> +        for (Mutable<ILogicalOperator> input : assign.getInputs()) {
>>> +            unnest.getInputs().get(index++).setValue(input.getValue());
>>> +        }
>>> +
>>> +        UnnestingFunctionCallExpression child = new
>> UnnestingFunctionCallExpression(BuiltinOperators.CHILD,
>> functionCall2.getArguments());
>>> +        unnest.getExpressionRef().setValue(child);
>>> +
>>> +        return true;
>>> +    }
>>> +
>>> +    @Override
>>> +    public boolean rewritePost(Mutable<ILogicalOperator> opRef,
>> IOptimizationContext context) {
>>> +        return false;
>>> +    }
>>> +}
>>> 
>>> Propchange:
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java
>>> 
>> ------------------------------------------------------------------------------
>>>   svn:eol-style = native
>>> 
>>> Modified:
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
>>> URL:
>> http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml?rev=1518074&r1=1518073&r2=1518074&view=diff
>>> 
>> ==============================================================================
>>> ---
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
>> (original)
>>> +++
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
>> Wed Aug 28 04:39:56 2013
>>> @@ -869,6 +869,7 @@
>>>        <param name="arg" type="node()*"/>
>>>        <return type="node()*"/>
>>>        <runtime type="scalar"
>> class="org.apache.vxquery.runtime.functions.step.ChildPathStepScalarEvaluatorFactory"/>
>>> +        <runtime type="unnesting"
>> class="org.apache.vxquery.runtime.functions.step.ChildPathStepUnnestingEvaluatorFactory"/>
>>>        <property type="DocumentOrder"
>> class="org.apache.vxquery.compiler.rewriter.rules.propagationpolicies.InputPropertyPropagationPolicy">
>>>            <argument value="0"/>
>>>        </property>
>>> 
>>> Modified:
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java
>>> URL:
>> http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java?rev=1518074&r1=1518073&r2=1518074&view=diff
>>> 
>> ==============================================================================
>>> ---
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java
>> (original)
>>> +++
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java
>> Wed Aug 28 04:39:56 2013
>>> @@ -17,6 +17,7 @@
>>> package org.apache.vxquery.runtime.functions.base;
>>> 
>>> import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
>>> +import org.apache.vxquery.exceptions.SystemException;
>>> 
>>> import
>> edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
>>> import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
>>> @@ -41,8 +42,12 @@ public abstract class AbstractTaggedValu
>>>        for (int i = 0; i < args.length; ++i) {
>>>            args[i].evaluate(tuple, tvps[i]);
>>>        }
>>> -        init(tvps);
>>> +        try {
>>> +            init(tvps);
>>> +        } catch (SystemException e) {
>>> +            throw new AlgebricksException(e);
>>> +        }
>>>    }
>>> 
>>> -    protected abstract void init(TaggedValuePointable[] args);
>>> +    protected abstract void init(TaggedValuePointable[] args) throws
>> SystemException;
>>> }
>>> \ No newline at end of file
>>> 
>>> Modified:
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java
>>> URL:
>> http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java?rev=1518074&r1=1518073&r2=1518074&view=diff
>>> 
>> ==============================================================================
>>> ---
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java
>> (original)
>>> +++
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java
>> Wed Aug 28 04:39:56 2013
>>> @@ -16,6 +16,8 @@
>>> */
>>> package org.apache.vxquery.runtime.functions.base;
>>> 
>>> +import org.apache.vxquery.context.DynamicContext;
>>> +
>>> import
>> edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
>>> import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
>>> import
>> edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
>>> @@ -25,15 +27,20 @@ import edu.uci.ics.hyracks.api.context.I
>>> 
>>> public abstract class
>> AbstractTaggedValueArgumentUnnestingEvaluatorFactory implements
>> IUnnestingEvaluatorFactory {
>>>    private static final long serialVersionUID = 1L;
>>> +
>>> +    protected DynamicContext dCtx;
>>> 
>>>    private final IScalarEvaluatorFactory[] args;
>>> 
>>>    public
>> AbstractTaggedValueArgumentUnnestingEvaluatorFactory(IScalarEvaluatorFactory[]
>> args) {
>>>        this.args = args;
>>> +        this.dCtx = null;
>>>    }
>>> 
>>>    @Override
>>>    public final IUnnestingEvaluator
>> createUnnestingEvaluator(IHyracksTaskContext ctx) throws
>> AlgebricksException {
>>> +        dCtx = (DynamicContext)
>> ctx.getJobletContext().getGlobalJobData();
>>> +
>>>        IScalarEvaluator[] es = new IScalarEvaluator[args.length];
>>>        for (int i = 0; i < es.length; ++i) {
>>>            es[i] = args[i].createScalarEvaluator(ctx);
>>> 
>>> Added:
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java
>>> URL:
>> http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java?rev=1518074&view=auto
>>> 
>> ==============================================================================
>>> ---
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java
>> (added)
>>> +++
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java
>> Wed Aug 28 04:39:56 2013
>>> @@ -0,0 +1,325 @@
>>> +/*
>>> + * 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.runtime.functions.step;
>>> +
>>> +import java.io.DataOutput;
>>> +import java.io.IOException;
>>> +
>>> +import org.apache.vxquery.datamodel.accessors.SequencePointable;
>>> +import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
>>> +import
>> org.apache.vxquery.datamodel.accessors.atomic.CodedQNamePointable;
>>> +import
>> org.apache.vxquery.datamodel.accessors.nodes.AttributeNodePointable;
>>> +import
>> org.apache.vxquery.datamodel.accessors.nodes.DocumentNodePointable;
>>> +import
>> org.apache.vxquery.datamodel.accessors.nodes.ElementNodePointable;
>>> +import org.apache.vxquery.datamodel.accessors.nodes.NodeTreePointable;
>>> +import org.apache.vxquery.datamodel.values.ValueTag;
>>> +import org.apache.vxquery.datamodel.values.XDMConstants;
>>> +import org.apache.vxquery.exceptions.ErrorCode;
>>> +import org.apache.vxquery.exceptions.SystemException;
>>> +import
>> org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentUnnestingEvaluator;
>>> +import
>> org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentUnnestingEvaluatorFactory;
>>> +import org.apache.vxquery.types.AttributeType;
>>> +import org.apache.vxquery.types.ElementType;
>>> +import org.apache.vxquery.types.NameTest;
>>> +import org.apache.vxquery.types.NodeType;
>>> +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.algebricks.runtime.base.IScalarEvaluatorFactory;
>>> +import edu.uci.ics.hyracks.algebricks.runtime.base.IUnnestingEvaluator;
>>> +import edu.uci.ics.hyracks.data.std.api.IPointable;
>>> +import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
>>> +import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
>>> +import edu.uci.ics.hyracks.data.std.primitive.VoidPointable;
>>> +import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
>>> +
>>> +public class ChildPathStepUnnestingEvaluatorFactory extends
>> AbstractTaggedValueArgumentUnnestingEvaluatorFactory {
>>> +    private static final long serialVersionUID = 1L;
>>> +
>>> +    public
>> ChildPathStepUnnestingEvaluatorFactory(IScalarEvaluatorFactory[] args) {
>>> +        super(args);
>>> +    }
>>> +
>>> +    @Override
>>> +    protected IUnnestingEvaluator createEvaluator(IScalarEvaluator[]
>> args) throws AlgebricksException {
>>> +
>>> +        final SequencePointable seqp = (SequencePointable)
>> SequencePointable.FACTORY.createPointable();
>>> +        final TaggedValuePointable rootTVP = (TaggedValuePointable)
>> TaggedValuePointable.FACTORY.createPointable();
>>> +        final DocumentNodePointable dnp = (DocumentNodePointable)
>> DocumentNodePointable.FACTORY.createPointable();
>>> +        final ElementNodePointable enp = (ElementNodePointable)
>> ElementNodePointable.FACTORY.createPointable();
>>> +        final IntegerPointable ip = (IntegerPointable)
>> IntegerPointable.FACTORY.createPointable();
>>> +        final NodeTreePointable ntp = (NodeTreePointable)
>> NodeTreePointable.FACTORY.createPointable();
>>> +        final TaggedValuePointable itemTvp = (TaggedValuePointable)
>> TaggedValuePointable.FACTORY.createPointable();
>>> +
>>> +        return new AbstractTaggedValueArgumentUnnestingEvaluator(args) {
>>> +            private int index;
>>> +            private int seqLength;
>>> +
>>> +            private boolean first;
>>> +            private ArrayBackedValueStorage nodeAbvs;
>>> +            private INodeFilter filter;
>>> +
>>> +            @Override
>>> +            public boolean step(IPointable result) throws
>> AlgebricksException {
>>> +                while (index < seqLength) {
>>> +                    // Get the next item
>>> +                    seqp.getEntry(index, itemTvp);
>>> +                    ++index;
>>> +
>>> +                    // Test to see if the item fits the path step
>>> +                    if (matches()) {
>>> +                        try {
>>> +                            setNodeToResult(result);
>>> +                            return true;
>>> +                        } catch (IOException e) {
>>> +                            String description = ErrorCode.SYSE0001 +
>> ": " + ErrorCode.SYSE0001.getDescription();
>>> +                            throw new AlgebricksException(description);
>>> +                        }
>>> +                    }
>>> +                }
>>> +                return false;
>>> +            }
>>> +
>>> +            @Override
>>> +            protected void init(TaggedValuePointable[] args) throws
>> SystemException {
>>> +                first = true;
>>> +                nodeAbvs = new ArrayBackedValueStorage();
>>> +
>>> +                index = 0;
>>> +                if (first) {
>>> +                    if (args[1].getTag() != ValueTag.XS_INT_TAG) {
>>> +                        throw new IllegalArgumentException("Expected
>> int value tag, got: " + args[1].getTag());
>>> +                    }
>>> +                    args[1].getValue(ip);
>>> +                    int typeCode = ip.getInteger();
>>> +                    SequenceType sType =
>> dCtx.getStaticContext().lookupSequenceType(typeCode);
>>> +                    setNodeTest(sType);
>>> +                    first = false;
>>> +                }
>>> +                if (args[0].getTag() != ValueTag.NODE_TREE_TAG) {
>>> +                    throw new SystemException(ErrorCode.SYSE0001);
>>> +                }
>>> +                args[0].getValue(ntp);
>>> +                getSequence(ntp, seqp);
>>> +                seqLength = seqp.getEntryCount();
>>> +            }
>>> +
>>> +            protected void setNodeTest(SequenceType sType) {
>>> +                final NodeType nodeType = (NodeType)
>> sType.getItemType();
>>> +                switch (nodeType.getNodeKind()) {
>>> +                    case ANY:
>>> +                        filter = new INodeFilter() {
>>> +                            @Override
>>> +                            public boolean accept(NodeTreePointable
>> ntp, TaggedValuePointable tvp) {
>>> +                                return true;
>>> +                            }
>>> +                        };
>>> +                        break;
>>> +
>>> +                    case ATTRIBUTE: {
>>> +                        AttributeType aType = (AttributeType) nodeType;
>>> +                        NameTest nameTest = aType.getNameTest();
>>> +                        byte[] uri = nameTest.getUri();
>>> +                        byte[] localName = nameTest.getLocalName();
>>> +                        final UTF8StringPointable urip =
>> (UTF8StringPointable) (uri == null ? null
>>> +                                :
>> UTF8StringPointable.FACTORY.createPointable());
>>> +                        final UTF8StringPointable localp =
>> (UTF8StringPointable) (localName == null ? null
>>> +                                :
>> UTF8StringPointable.FACTORY.createPointable());
>>> +                        if (uri != null) {
>>> +                            urip.set(uri, 0, uri.length);
>>> +                        }
>>> +                        if (localName != null) {
>>> +                            localp.set(localName, 0, localName.length);
>>> +                        }
>>> +                        final IPointable temp =
>> VoidPointable.FACTORY.createPointable();
>>> +                        final AttributeNodePointable anp =
>> (AttributeNodePointable) AttributeNodePointable.FACTORY
>>> +                                .createPointable();
>>> +                        final CodedQNamePointable cqp =
>> (CodedQNamePointable) CodedQNamePointable.FACTORY
>>> +                                .createPointable();
>>> +                        filter = new INodeFilter() {
>>> +                            @Override
>>> +                            public boolean accept(NodeTreePointable
>> ntp, TaggedValuePointable tvp) {
>>> +                                if (tvp.getTag() !=
>> ValueTag.ATTRIBUTE_NODE_TAG) {
>>> +                                    return false;
>>> +                                }
>>> +                                tvp.getValue(anp);
>>> +                                anp.getName(cqp);
>>> +                                if (urip != null) {
>>> +
>> ntp.getString(cqp.getNamespaceCode(), temp);
>>> +                                    if (urip.compareTo(temp) != 0) {
>>> +                                        return false;
>>> +                                    }
>>> +                                }
>>> +                                if (localp != null) {
>>> +                                    ntp.getString(cqp.getLocalCode(),
>> temp);
>>> +                                    if (localp.compareTo(temp) != 0) {
>>> +                                        return false;
>>> +                                    }
>>> +                                }
>>> +                                return true;
>>> +                            }
>>> +                        };
>>> +                        break;
>>> +                    }
>>> +
>>> +                    case COMMENT:
>>> +                        filter = new INodeFilter() {
>>> +                            @Override
>>> +                            public boolean accept(NodeTreePointable
>> ntp, TaggedValuePointable tvp) {
>>> +                                return tvp.getTag() ==
>> ValueTag.COMMENT_NODE_TAG;
>>> +                            }
>>> +                        };
>>> +                        break;
>>> +
>>> +                    case DOCUMENT:
>>> +                        filter = new INodeFilter() {
>>> +                            @Override
>>> +                            public boolean accept(NodeTreePointable
>> ntp, TaggedValuePointable tvp) {
>>> +                                return tvp.getTag() ==
>> ValueTag.DOCUMENT_NODE_TAG;
>>> +                            }
>>> +                        };
>>> +                        break;
>>> +
>>> +                    case ELEMENT: {
>>> +                        ElementType eType = (ElementType) nodeType;
>>> +                        NameTest nameTest = eType.getNameTest();
>>> +                        byte[] uri = nameTest.getUri();
>>> +                        byte[] localName = nameTest.getLocalName();
>>> +                        final UTF8StringPointable urip =
>> (UTF8StringPointable) (uri == null ? null
>>> +                                :
>> UTF8StringPointable.FACTORY.createPointable());
>>> +                        final UTF8StringPointable localp =
>> (UTF8StringPointable) (localName == null ? null
>>> +                                :
>> UTF8StringPointable.FACTORY.createPointable());
>>> +                        if (uri != null) {
>>> +                            urip.set(uri, 0, uri.length);
>>> +                        }
>>> +                        if (localName != null) {
>>> +                            localp.set(localName, 0, localName.length);
>>> +                        }
>>> +                        final IPointable temp =
>> VoidPointable.FACTORY.createPointable();
>>> +                        final ElementNodePointable enp =
>> (ElementNodePointable) ElementNodePointable.FACTORY
>>> +                                .createPointable();
>>> +                        final CodedQNamePointable cqp =
>> (CodedQNamePointable) CodedQNamePointable.FACTORY
>>> +                                .createPointable();
>>> +                        filter = new INodeFilter() {
>>> +                            @Override
>>> +                            public boolean accept(NodeTreePointable
>> ntp, TaggedValuePointable tvp) {
>>> +                                if (tvp.getTag() !=
>> ValueTag.ELEMENT_NODE_TAG) {
>>> +                                    return false;
>>> +                                }
>>> +                                tvp.getValue(enp);
>>> +                                enp.getName(cqp);
>>> +                                if (urip != null) {
>>> +
>> ntp.getString(cqp.getNamespaceCode(), temp);
>>> +                                    if (urip.compareTo(temp) != 0) {
>>> +                                        return false;
>>> +                                    }
>>> +                                }
>>> +                                if (localp != null) {
>>> +                                    ntp.getString(cqp.getLocalCode(),
>> temp);
>>> +                                    if (localp.compareTo(temp) != 0) {
>>> +                                        return false;
>>> +                                    }
>>> +                                }
>>> +                                return true;
>>> +                            }
>>> +                        };
>>> +                        break;
>>> +                    }
>>> +
>>> +                    case PI:
>>> +                        filter = new INodeFilter() {
>>> +                            @Override
>>> +                            public boolean accept(NodeTreePointable
>> ntp, TaggedValuePointable tvp) {
>>> +                                return tvp.getTag() ==
>> ValueTag.PI_NODE_TAG;
>>> +                            }
>>> +                        };
>>> +                        break;
>>> +
>>> +                    case TEXT:
>>> +                        filter = new INodeFilter() {
>>> +                            @Override
>>> +                            public boolean accept(NodeTreePointable
>> ntp, TaggedValuePointable tvp) {
>>> +                                return tvp.getTag() ==
>> ValueTag.TEXT_NODE_TAG;
>>> +                            }
>>> +                        };
>>> +                        break;
>>> +                }
>>> +            }
>>> +
>>> +            protected boolean matches() {
>>> +                return filter.accept(ntp, itemTvp);
>>> +            }
>>> +
>>> +            protected void setNodeToResult(IPointable result) throws
>> IOException {
>>> +                nodeAbvs.reset();
>>> +                DataOutput mainOut = nodeAbvs.getDataOutput();
>>> +                mainOut.write(ValueTag.NODE_TREE_TAG);
>>> +                boolean hasDictionary = ntp.dictionaryExists() &&
>> hasDictionary(itemTvp.getTag());
>>> +                byte header = (byte) (hasDictionary ?
>> NodeTreePointable.HEADER_DICTIONARY_EXISTS_MASK : 0);
>>> +                // TODO add all header flags to this setting.
>>> +                boolean hasNodeIds = ntp.nodeIdExists();
>>> +                if (hasNodeIds) {
>>> +                    header |=
>> NodeTreePointable.HEADER_NODEID_EXISTS_MASK;
>>> +                }
>>> +                mainOut.write(header);
>>> +                if (hasNodeIds) {
>>> +                    mainOut.writeInt(ntp.getRootNodeId());
>>> +                }
>>> +                if (hasDictionary) {
>>> +                    mainOut.write(ntp.getByteArray(),
>> ntp.getDictionaryOffset(), ntp.getDictionarySize());
>>> +                }
>>> +                mainOut.write(itemTvp.getByteArray(),
>> itemTvp.getStartOffset(), itemTvp.getLength());
>>> +                result.set(nodeAbvs.getByteArray(),
>> nodeAbvs.getStartOffset(), nodeAbvs.getLength());
>>> +            }
>>> +
>>> +            private boolean hasDictionary(byte tag) {
>>> +                switch (tag) {
>>> +                    case ValueTag.ATTRIBUTE_NODE_TAG:
>>> +                    case ValueTag.DOCUMENT_NODE_TAG:
>>> +                    case ValueTag.ELEMENT_NODE_TAG:
>>> +                        return true;
>>> +                }
>>> +                return false;
>>> +            }
>>> +
>>> +            protected void getSequence(NodeTreePointable ntp,
>> SequencePointable seqp) throws SystemException {
>>> +                ntp.getRootNode(rootTVP);
>>> +                switch (rootTVP.getTag()) {
>>> +                    case ValueTag.DOCUMENT_NODE_TAG:
>>> +                        rootTVP.getValue(dnp);
>>> +                        dnp.getContent(ntp, seqp);
>>> +                        return;
>>> +
>>> +                    case ValueTag.ELEMENT_NODE_TAG:
>>> +                        rootTVP.getValue(enp);
>>> +                        if (enp.childrenChunkExists()) {
>>> +                            enp.getChildrenSequence(ntp, seqp);
>>> +                            return;
>>> +                        }
>>> +                }
>>> +                XDMConstants.setEmptySequence(seqp);
>>> +            }
>>> +
>>> +        };
>>> +    }
>>> +
>>> +    private interface INodeFilter {
>>> +        public boolean accept(NodeTreePointable ntp,
>> TaggedValuePointable tvp);
>>> +    }
>>> +
>>> +}
>>> \ No newline at end of file
>>> 
>>> Propchange:
>> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java
>>> 
>> ------------------------------------------------------------------------------
>>>   svn:eol-style = native
>>> 
>>> 
>> 
>> 


Re: svn commit: r1518074 - in /incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery: compiler/rewriter/ compiler/rewriter/rules/ functions/ runtime/functions/base/ runtime/functions/step/

Posted by Eldon Carman <ec...@ucr.edu>.
Yes, Till.

The path steps could benefit from sharing code between unnest and scalar
evaluators. I am a little confused on how to best share the code between
the two. Each one is build on an abstract class with different interfaces.
The interface defines the evaluate or step function, yet all the code in
between is shared for the most part. Maybe we create two new classes for
node testing and building node trees: NodeTest and NodeTreeBuilder.

What do you think?
Preston




On Tue, Aug 27, 2013 at 10:37 PM, Till Westmann <ti...@westmann.org> wrote:

> Hi Preston,
>
> I've got a question on the ChildPathStepUnnestingEvaluatorFactory:
> Is there a difference between the setNodeTest method in this class and the
> setNodeTest method in AbstractPathStepScalarEvaluator that keep us from
> having just one such method?
>
> Thanks,
> Till
>
> On Aug 28, 2013, at 6:39 AM, prestonc@apache.org wrote:
>
> > Author: prestonc
> > Date: Wed Aug 28 04:39:56 2013
> > New Revision: 1518074
> >
> > URL: http://svn.apache.org/r1518074
> > Log:
> > Added a child path step unnesting evaluator factory to improve the query
> plan optimization. The commit includes a new rule to use the unnesting
> function in place of iterate where iterate is operating on a child result.
> >
> > Added:
> >
>  incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java
>   (with props)
> >
>  incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java
>   (with props)
> > Modified:
> >
>  incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
> >
>  incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
> >
>  incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java
> >
>  incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java
> >
> > Modified:
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
> > URL:
> http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java?rev=1518074&r1=1518073&r2=1518074&view=diff
> >
> ==============================================================================
> > ---
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
> (original)
> > +++
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
> Wed Aug 28 04:39:56 2013
> > @@ -22,6 +22,7 @@ import java.util.List;
> > import
> org.apache.vxquery.compiler.rewriter.rules.ConsolidateAssignAggregateRule;
> > import
> org.apache.vxquery.compiler.rewriter.rules.ConvertAssignSortDistinctNodesToOperatorsRule;
> > import
> org.apache.vxquery.compiler.rewriter.rules.ConvertAssignToAggregateRule;
> > +import
> org.apache.vxquery.compiler.rewriter.rules.ConvertAssignToUnnestRule;
> > import
> org.apache.vxquery.compiler.rewriter.rules.EliminateSubplanForSinglePathsRule;
> > import
> org.apache.vxquery.compiler.rewriter.rules.EliminateUnnestAggregateSequencesRule;
> > import
> org.apache.vxquery.compiler.rewriter.rules.EliminateUnnestAggregateSubplanRule;
> > @@ -89,6 +90,8 @@ public class RewriteRuleset {
> >         // Remove single tuple input subplans and merge unnest aggregate
> operators.
> >         normalization.add(new EliminateSubplanForSinglePathsRule());
> >         normalization.add(new EliminateUnnestAggregateSequencesRule());
> > +
> > +        normalization.add(new ConvertAssignToUnnestRule());
> >
> >         // Convert to a data source scan operator.
> >         normalization.add(new SetCollectionDataSourceRule());
> >
> > Added:
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java
> > URL:
> http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java?rev=1518074&view=auto
> >
> ==============================================================================
> > ---
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java
> (added)
> > +++
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java
> Wed Aug 28 04:39:56 2013
> > @@ -0,0 +1,111 @@
> > +/*
> > + * 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.expressions.AbstractFunctionCallExpression;
> > +import
> edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
> > +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;
> > +import
> edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
> > +
> > +/**
> > + * The rule searches for unnest(iterate) operator followed by an
> assign(child)
> > + * operator and merges the assign into the unnest operator.
> > + *
> > + * <pre>
> > + * Before
> > + *
> > + *   plan__parent
> > + *   UNNEST( $v2 : iterate( $v1 ) )
> > + *   ASSIGN( $v1 : child( $v0 ) )
> > + *   plan__child
> > + *
> > + *   where plan__parent does not use $v1 and $v0 is defined in
> plan__child.
> > + *
> > + * After
> > + *
> > + *   plan__parent
> > + *   UNNEST( $v2 : child( $v0 ) )
> > + *   plan__child
> > + * </pre>
> > + *
> > + * @author prestonc
> > + */
> > +public class ConvertAssignToUnnestRule implements IAlgebraicRewriteRule
> {
> > +    @Override
> > +    public boolean rewritePre(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 the iterate operator.
> > +        ILogicalExpression logicalExpression = (ILogicalExpression)
> unnest.getExpressionRef().getValue();
> > +        if (logicalExpression.getExpressionTag() !=
> LogicalExpressionTag.FUNCTION_CALL) {
> > +            return false;
> > +        }
> > +        AbstractFunctionCallExpression functionCall =
> (AbstractFunctionCallExpression) logicalExpression;
> > +        if
> (!functionCall.getFunctionIdentifier().equals(BuiltinOperators.ITERATE.getFunctionIdentifier()))
> {
> > +            return false;
> > +        }
> > +
> > +        AbstractLogicalOperator op2 = (AbstractLogicalOperator)
> unnest.getInputs().get(0).getValue();
> > +        if (op2.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
> > +            return false;
> > +        }
> > +        AssignOperator assign = (AssignOperator) op2;
> > +
> > +        // Check to see if the expression is a child function.
> > +        ILogicalExpression logicalExpression2 = (ILogicalExpression)
> assign.getExpressions().get(0).getValue();
> > +        if (logicalExpression2.getExpressionTag() !=
> LogicalExpressionTag.FUNCTION_CALL) {
> > +            return false;
> > +        }
> > +        AbstractFunctionCallExpression functionCall2 =
> (AbstractFunctionCallExpression) logicalExpression2;
> > +        if
> (!functionCall2.getFunctionIdentifier().equals(BuiltinOperators.CHILD.getFunctionIdentifier()))
> {
> > +            return false;
> > +        }
> > +
> > +        // TODO add checks for variables used that have now been
> removed.
> > +
> > +        // Update the unnest parameters.
> > +        int index = 0;
> > +        for (Mutable<ILogicalOperator> input : assign.getInputs()) {
> > +            unnest.getInputs().get(index++).setValue(input.getValue());
> > +        }
> > +
> > +        UnnestingFunctionCallExpression child = new
> UnnestingFunctionCallExpression(BuiltinOperators.CHILD,
> functionCall2.getArguments());
> > +        unnest.getExpressionRef().setValue(child);
> > +
> > +        return true;
> > +    }
> > +
> > +    @Override
> > +    public boolean rewritePost(Mutable<ILogicalOperator> opRef,
> IOptimizationContext context) {
> > +        return false;
> > +    }
> > +}
> >
> > Propchange:
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java
> >
> ------------------------------------------------------------------------------
> >    svn:eol-style = native
> >
> > Modified:
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
> > URL:
> http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml?rev=1518074&r1=1518073&r2=1518074&view=diff
> >
> ==============================================================================
> > ---
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
> (original)
> > +++
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
> Wed Aug 28 04:39:56 2013
> > @@ -869,6 +869,7 @@
> >         <param name="arg" type="node()*"/>
> >         <return type="node()*"/>
> >         <runtime type="scalar"
> class="org.apache.vxquery.runtime.functions.step.ChildPathStepScalarEvaluatorFactory"/>
> > +        <runtime type="unnesting"
> class="org.apache.vxquery.runtime.functions.step.ChildPathStepUnnestingEvaluatorFactory"/>
> >         <property type="DocumentOrder"
> class="org.apache.vxquery.compiler.rewriter.rules.propagationpolicies.InputPropertyPropagationPolicy">
> >             <argument value="0"/>
> >         </property>
> >
> > Modified:
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java
> > URL:
> http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java?rev=1518074&r1=1518073&r2=1518074&view=diff
> >
> ==============================================================================
> > ---
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java
> (original)
> > +++
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluator.java
> Wed Aug 28 04:39:56 2013
> > @@ -17,6 +17,7 @@
> > package org.apache.vxquery.runtime.functions.base;
> >
> > import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
> > +import org.apache.vxquery.exceptions.SystemException;
> >
> > import
> edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
> > import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
> > @@ -41,8 +42,12 @@ public abstract class AbstractTaggedValu
> >         for (int i = 0; i < args.length; ++i) {
> >             args[i].evaluate(tuple, tvps[i]);
> >         }
> > -        init(tvps);
> > +        try {
> > +            init(tvps);
> > +        } catch (SystemException e) {
> > +            throw new AlgebricksException(e);
> > +        }
> >     }
> >
> > -    protected abstract void init(TaggedValuePointable[] args);
> > +    protected abstract void init(TaggedValuePointable[] args) throws
> SystemException;
> > }
> > \ No newline at end of file
> >
> > Modified:
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java
> > URL:
> http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java?rev=1518074&r1=1518073&r2=1518074&view=diff
> >
> ==============================================================================
> > ---
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java
> (original)
> > +++
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/base/AbstractTaggedValueArgumentUnnestingEvaluatorFactory.java
> Wed Aug 28 04:39:56 2013
> > @@ -16,6 +16,8 @@
> >  */
> > package org.apache.vxquery.runtime.functions.base;
> >
> > +import org.apache.vxquery.context.DynamicContext;
> > +
> > import
> edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
> > import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
> > import
> edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
> > @@ -25,15 +27,20 @@ import edu.uci.ics.hyracks.api.context.I
> >
> > public abstract class
> AbstractTaggedValueArgumentUnnestingEvaluatorFactory implements
> IUnnestingEvaluatorFactory {
> >     private static final long serialVersionUID = 1L;
> > +
> > +    protected DynamicContext dCtx;
> >
> >     private final IScalarEvaluatorFactory[] args;
> >
> >     public
> AbstractTaggedValueArgumentUnnestingEvaluatorFactory(IScalarEvaluatorFactory[]
> args) {
> >         this.args = args;
> > +        this.dCtx = null;
> >     }
> >
> >     @Override
> >     public final IUnnestingEvaluator
> createUnnestingEvaluator(IHyracksTaskContext ctx) throws
> AlgebricksException {
> > +        dCtx = (DynamicContext)
> ctx.getJobletContext().getGlobalJobData();
> > +
> >         IScalarEvaluator[] es = new IScalarEvaluator[args.length];
> >         for (int i = 0; i < es.length; ++i) {
> >             es[i] = args[i].createScalarEvaluator(ctx);
> >
> > Added:
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java
> > URL:
> http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java?rev=1518074&view=auto
> >
> ==============================================================================
> > ---
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java
> (added)
> > +++
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java
> Wed Aug 28 04:39:56 2013
> > @@ -0,0 +1,325 @@
> > +/*
> > + * 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.runtime.functions.step;
> > +
> > +import java.io.DataOutput;
> > +import java.io.IOException;
> > +
> > +import org.apache.vxquery.datamodel.accessors.SequencePointable;
> > +import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
> > +import
> org.apache.vxquery.datamodel.accessors.atomic.CodedQNamePointable;
> > +import
> org.apache.vxquery.datamodel.accessors.nodes.AttributeNodePointable;
> > +import
> org.apache.vxquery.datamodel.accessors.nodes.DocumentNodePointable;
> > +import
> org.apache.vxquery.datamodel.accessors.nodes.ElementNodePointable;
> > +import org.apache.vxquery.datamodel.accessors.nodes.NodeTreePointable;
> > +import org.apache.vxquery.datamodel.values.ValueTag;
> > +import org.apache.vxquery.datamodel.values.XDMConstants;
> > +import org.apache.vxquery.exceptions.ErrorCode;
> > +import org.apache.vxquery.exceptions.SystemException;
> > +import
> org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentUnnestingEvaluator;
> > +import
> org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentUnnestingEvaluatorFactory;
> > +import org.apache.vxquery.types.AttributeType;
> > +import org.apache.vxquery.types.ElementType;
> > +import org.apache.vxquery.types.NameTest;
> > +import org.apache.vxquery.types.NodeType;
> > +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.algebricks.runtime.base.IScalarEvaluatorFactory;
> > +import edu.uci.ics.hyracks.algebricks.runtime.base.IUnnestingEvaluator;
> > +import edu.uci.ics.hyracks.data.std.api.IPointable;
> > +import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
> > +import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
> > +import edu.uci.ics.hyracks.data.std.primitive.VoidPointable;
> > +import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
> > +
> > +public class ChildPathStepUnnestingEvaluatorFactory extends
> AbstractTaggedValueArgumentUnnestingEvaluatorFactory {
> > +    private static final long serialVersionUID = 1L;
> > +
> > +    public
> ChildPathStepUnnestingEvaluatorFactory(IScalarEvaluatorFactory[] args) {
> > +        super(args);
> > +    }
> > +
> > +    @Override
> > +    protected IUnnestingEvaluator createEvaluator(IScalarEvaluator[]
> args) throws AlgebricksException {
> > +
> > +        final SequencePointable seqp = (SequencePointable)
> SequencePointable.FACTORY.createPointable();
> > +        final TaggedValuePointable rootTVP = (TaggedValuePointable)
> TaggedValuePointable.FACTORY.createPointable();
> > +        final DocumentNodePointable dnp = (DocumentNodePointable)
> DocumentNodePointable.FACTORY.createPointable();
> > +        final ElementNodePointable enp = (ElementNodePointable)
> ElementNodePointable.FACTORY.createPointable();
> > +        final IntegerPointable ip = (IntegerPointable)
> IntegerPointable.FACTORY.createPointable();
> > +        final NodeTreePointable ntp = (NodeTreePointable)
> NodeTreePointable.FACTORY.createPointable();
> > +        final TaggedValuePointable itemTvp = (TaggedValuePointable)
> TaggedValuePointable.FACTORY.createPointable();
> > +
> > +        return new AbstractTaggedValueArgumentUnnestingEvaluator(args) {
> > +            private int index;
> > +            private int seqLength;
> > +
> > +            private boolean first;
> > +            private ArrayBackedValueStorage nodeAbvs;
> > +            private INodeFilter filter;
> > +
> > +            @Override
> > +            public boolean step(IPointable result) throws
> AlgebricksException {
> > +                while (index < seqLength) {
> > +                    // Get the next item
> > +                    seqp.getEntry(index, itemTvp);
> > +                    ++index;
> > +
> > +                    // Test to see if the item fits the path step
> > +                    if (matches()) {
> > +                        try {
> > +                            setNodeToResult(result);
> > +                            return true;
> > +                        } catch (IOException e) {
> > +                            String description = ErrorCode.SYSE0001 +
> ": " + ErrorCode.SYSE0001.getDescription();
> > +                            throw new AlgebricksException(description);
> > +                        }
> > +                    }
> > +                }
> > +                return false;
> > +            }
> > +
> > +            @Override
> > +            protected void init(TaggedValuePointable[] args) throws
> SystemException {
> > +                first = true;
> > +                nodeAbvs = new ArrayBackedValueStorage();
> > +
> > +                index = 0;
> > +                if (first) {
> > +                    if (args[1].getTag() != ValueTag.XS_INT_TAG) {
> > +                        throw new IllegalArgumentException("Expected
> int value tag, got: " + args[1].getTag());
> > +                    }
> > +                    args[1].getValue(ip);
> > +                    int typeCode = ip.getInteger();
> > +                    SequenceType sType =
> dCtx.getStaticContext().lookupSequenceType(typeCode);
> > +                    setNodeTest(sType);
> > +                    first = false;
> > +                }
> > +                if (args[0].getTag() != ValueTag.NODE_TREE_TAG) {
> > +                    throw new SystemException(ErrorCode.SYSE0001);
> > +                }
> > +                args[0].getValue(ntp);
> > +                getSequence(ntp, seqp);
> > +                seqLength = seqp.getEntryCount();
> > +            }
> > +
> > +            protected void setNodeTest(SequenceType sType) {
> > +                final NodeType nodeType = (NodeType)
> sType.getItemType();
> > +                switch (nodeType.getNodeKind()) {
> > +                    case ANY:
> > +                        filter = new INodeFilter() {
> > +                            @Override
> > +                            public boolean accept(NodeTreePointable
> ntp, TaggedValuePointable tvp) {
> > +                                return true;
> > +                            }
> > +                        };
> > +                        break;
> > +
> > +                    case ATTRIBUTE: {
> > +                        AttributeType aType = (AttributeType) nodeType;
> > +                        NameTest nameTest = aType.getNameTest();
> > +                        byte[] uri = nameTest.getUri();
> > +                        byte[] localName = nameTest.getLocalName();
> > +                        final UTF8StringPointable urip =
> (UTF8StringPointable) (uri == null ? null
> > +                                :
> UTF8StringPointable.FACTORY.createPointable());
> > +                        final UTF8StringPointable localp =
> (UTF8StringPointable) (localName == null ? null
> > +                                :
> UTF8StringPointable.FACTORY.createPointable());
> > +                        if (uri != null) {
> > +                            urip.set(uri, 0, uri.length);
> > +                        }
> > +                        if (localName != null) {
> > +                            localp.set(localName, 0, localName.length);
> > +                        }
> > +                        final IPointable temp =
> VoidPointable.FACTORY.createPointable();
> > +                        final AttributeNodePointable anp =
> (AttributeNodePointable) AttributeNodePointable.FACTORY
> > +                                .createPointable();
> > +                        final CodedQNamePointable cqp =
> (CodedQNamePointable) CodedQNamePointable.FACTORY
> > +                                .createPointable();
> > +                        filter = new INodeFilter() {
> > +                            @Override
> > +                            public boolean accept(NodeTreePointable
> ntp, TaggedValuePointable tvp) {
> > +                                if (tvp.getTag() !=
> ValueTag.ATTRIBUTE_NODE_TAG) {
> > +                                    return false;
> > +                                }
> > +                                tvp.getValue(anp);
> > +                                anp.getName(cqp);
> > +                                if (urip != null) {
> > +
>  ntp.getString(cqp.getNamespaceCode(), temp);
> > +                                    if (urip.compareTo(temp) != 0) {
> > +                                        return false;
> > +                                    }
> > +                                }
> > +                                if (localp != null) {
> > +                                    ntp.getString(cqp.getLocalCode(),
> temp);
> > +                                    if (localp.compareTo(temp) != 0) {
> > +                                        return false;
> > +                                    }
> > +                                }
> > +                                return true;
> > +                            }
> > +                        };
> > +                        break;
> > +                    }
> > +
> > +                    case COMMENT:
> > +                        filter = new INodeFilter() {
> > +                            @Override
> > +                            public boolean accept(NodeTreePointable
> ntp, TaggedValuePointable tvp) {
> > +                                return tvp.getTag() ==
> ValueTag.COMMENT_NODE_TAG;
> > +                            }
> > +                        };
> > +                        break;
> > +
> > +                    case DOCUMENT:
> > +                        filter = new INodeFilter() {
> > +                            @Override
> > +                            public boolean accept(NodeTreePointable
> ntp, TaggedValuePointable tvp) {
> > +                                return tvp.getTag() ==
> ValueTag.DOCUMENT_NODE_TAG;
> > +                            }
> > +                        };
> > +                        break;
> > +
> > +                    case ELEMENT: {
> > +                        ElementType eType = (ElementType) nodeType;
> > +                        NameTest nameTest = eType.getNameTest();
> > +                        byte[] uri = nameTest.getUri();
> > +                        byte[] localName = nameTest.getLocalName();
> > +                        final UTF8StringPointable urip =
> (UTF8StringPointable) (uri == null ? null
> > +                                :
> UTF8StringPointable.FACTORY.createPointable());
> > +                        final UTF8StringPointable localp =
> (UTF8StringPointable) (localName == null ? null
> > +                                :
> UTF8StringPointable.FACTORY.createPointable());
> > +                        if (uri != null) {
> > +                            urip.set(uri, 0, uri.length);
> > +                        }
> > +                        if (localName != null) {
> > +                            localp.set(localName, 0, localName.length);
> > +                        }
> > +                        final IPointable temp =
> VoidPointable.FACTORY.createPointable();
> > +                        final ElementNodePointable enp =
> (ElementNodePointable) ElementNodePointable.FACTORY
> > +                                .createPointable();
> > +                        final CodedQNamePointable cqp =
> (CodedQNamePointable) CodedQNamePointable.FACTORY
> > +                                .createPointable();
> > +                        filter = new INodeFilter() {
> > +                            @Override
> > +                            public boolean accept(NodeTreePointable
> ntp, TaggedValuePointable tvp) {
> > +                                if (tvp.getTag() !=
> ValueTag.ELEMENT_NODE_TAG) {
> > +                                    return false;
> > +                                }
> > +                                tvp.getValue(enp);
> > +                                enp.getName(cqp);
> > +                                if (urip != null) {
> > +
>  ntp.getString(cqp.getNamespaceCode(), temp);
> > +                                    if (urip.compareTo(temp) != 0) {
> > +                                        return false;
> > +                                    }
> > +                                }
> > +                                if (localp != null) {
> > +                                    ntp.getString(cqp.getLocalCode(),
> temp);
> > +                                    if (localp.compareTo(temp) != 0) {
> > +                                        return false;
> > +                                    }
> > +                                }
> > +                                return true;
> > +                            }
> > +                        };
> > +                        break;
> > +                    }
> > +
> > +                    case PI:
> > +                        filter = new INodeFilter() {
> > +                            @Override
> > +                            public boolean accept(NodeTreePointable
> ntp, TaggedValuePointable tvp) {
> > +                                return tvp.getTag() ==
> ValueTag.PI_NODE_TAG;
> > +                            }
> > +                        };
> > +                        break;
> > +
> > +                    case TEXT:
> > +                        filter = new INodeFilter() {
> > +                            @Override
> > +                            public boolean accept(NodeTreePointable
> ntp, TaggedValuePointable tvp) {
> > +                                return tvp.getTag() ==
> ValueTag.TEXT_NODE_TAG;
> > +                            }
> > +                        };
> > +                        break;
> > +                }
> > +            }
> > +
> > +            protected boolean matches() {
> > +                return filter.accept(ntp, itemTvp);
> > +            }
> > +
> > +            protected void setNodeToResult(IPointable result) throws
> IOException {
> > +                nodeAbvs.reset();
> > +                DataOutput mainOut = nodeAbvs.getDataOutput();
> > +                mainOut.write(ValueTag.NODE_TREE_TAG);
> > +                boolean hasDictionary = ntp.dictionaryExists() &&
> hasDictionary(itemTvp.getTag());
> > +                byte header = (byte) (hasDictionary ?
> NodeTreePointable.HEADER_DICTIONARY_EXISTS_MASK : 0);
> > +                // TODO add all header flags to this setting.
> > +                boolean hasNodeIds = ntp.nodeIdExists();
> > +                if (hasNodeIds) {
> > +                    header |=
> NodeTreePointable.HEADER_NODEID_EXISTS_MASK;
> > +                }
> > +                mainOut.write(header);
> > +                if (hasNodeIds) {
> > +                    mainOut.writeInt(ntp.getRootNodeId());
> > +                }
> > +                if (hasDictionary) {
> > +                    mainOut.write(ntp.getByteArray(),
> ntp.getDictionaryOffset(), ntp.getDictionarySize());
> > +                }
> > +                mainOut.write(itemTvp.getByteArray(),
> itemTvp.getStartOffset(), itemTvp.getLength());
> > +                result.set(nodeAbvs.getByteArray(),
> nodeAbvs.getStartOffset(), nodeAbvs.getLength());
> > +            }
> > +
> > +            private boolean hasDictionary(byte tag) {
> > +                switch (tag) {
> > +                    case ValueTag.ATTRIBUTE_NODE_TAG:
> > +                    case ValueTag.DOCUMENT_NODE_TAG:
> > +                    case ValueTag.ELEMENT_NODE_TAG:
> > +                        return true;
> > +                }
> > +                return false;
> > +            }
> > +
> > +            protected void getSequence(NodeTreePointable ntp,
> SequencePointable seqp) throws SystemException {
> > +                ntp.getRootNode(rootTVP);
> > +                switch (rootTVP.getTag()) {
> > +                    case ValueTag.DOCUMENT_NODE_TAG:
> > +                        rootTVP.getValue(dnp);
> > +                        dnp.getContent(ntp, seqp);
> > +                        return;
> > +
> > +                    case ValueTag.ELEMENT_NODE_TAG:
> > +                        rootTVP.getValue(enp);
> > +                        if (enp.childrenChunkExists()) {
> > +                            enp.getChildrenSequence(ntp, seqp);
> > +                            return;
> > +                        }
> > +                }
> > +                XDMConstants.setEmptySequence(seqp);
> > +            }
> > +
> > +        };
> > +    }
> > +
> > +    private interface INodeFilter {
> > +        public boolean accept(NodeTreePointable ntp,
> TaggedValuePointable tvp);
> > +    }
> > +
> > +}
> > \ No newline at end of file
> >
> > Propchange:
> incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepUnnestingEvaluatorFactory.java
> >
> ------------------------------------------------------------------------------
> >    svn:eol-style = native
> >
> >
>
>