You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@vxquery.apache.org by pr...@apache.org on 2013/01/16 02:53:59 UTC

svn commit: r1433797 - in /incubator/vxquery/trunk/vxquery: src/site/ src/site/apt/ vxquery-core/src/main/java/org/apache/vxquery/functions/ vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ vxquery-core/src/main/java/org/apache/vxq...

Author: prestonc
Date: Wed Jan 16 01:53:59 2013
New Revision: 1433797

URL: http://svn.apache.org/viewvc?rev=1433797&view=rev
Log:
Current work on self, descendant-or-self and a working copy of descendant. Includes and addition to the site documentation about the xml file node pointable structure.

Added:
    incubator/vxquery/trunk/vxquery/src/site/apt/development_xml_node_details.apt
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractDescendantPathStepScalarEvaluator.java
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractSinglePathStepScalarEvaluator.java
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepScalarEvaluator.java
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepScalarEvaluatorFactory.java
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantPathStepScalarEvaluator.java
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantPathStepScalarEvaluatorFactory.java
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/SelfPathStepScalarEvaluator.java
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/SelfPathStepScalarEvaluatorFactory.java
Modified:
    incubator/vxquery/trunk/vxquery/src/site/site.xml
    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/step/AbstractPathStepScalarEvaluator.java
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AttributePathStepScalarEvaluator.java
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepScalarEvaluator.java
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/provider/VXQueryBinaryHashFunctionFactoryProvider.java

Added: incubator/vxquery/trunk/vxquery/src/site/apt/development_xml_node_details.apt
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/src/site/apt/development_xml_node_details.apt?rev=1433797&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/src/site/apt/development_xml_node_details.apt (added)
+++ incubator/vxquery/trunk/vxquery/src/site/apt/development_xml_node_details.apt Wed Jan 16 01:53:59 2013
@@ -0,0 +1,162 @@
+~~ 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.
+
+Data XML and Node Types
+
+  XML is used as the data source for XQuery and must be parsed into Hyracks data. Each
+  node type defined in XPath and XQuery can be mapped into pointable defined in Apache 
+  VXQuery.
+
+* XPath Node Types
+
+*--------------------------------+----------------------+---------------+
+| <<Data Type>>                  | <<Pointable Name>>   | <<Data Size>> |
+*--------------------------------+----------------------+---------------:
+| Attribute Nodes                | {{{https://svn.apache.org/repos/asf/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/nodes/AttributeNodePointable.java}AttributeNodePointable}}      |    1 + length |
+*--------------------------------+----------------------+---------------:
+| Document Nodes                 | {{{https://svn.apache.org/repos/asf/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/nodes/DocumentNodePointable.java}DocumentNodePointable}}      |    1 + length |
+*--------------------------------+----------------------+---------------:
+| Element Nodes                  | {{{https://svn.apache.org/repos/asf/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/nodes/ElementNodePointable.java}ElementNodePointable}}      |    1 + length |
+*--------------------------------+----------------------+---------------:
+| Node Tree Nodes                | {{{https://svn.apache.org/repos/asf/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/nodes/NodeTreePointable.java}NodeTreePointable}}      |    1 + length |
+*--------------------------------+----------------------+---------------:
+| Processing Instruction Nodes   | {{{https://svn.apache.org/repos/asf/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/nodes/PINodePointable.java}PINodePointable}}      |    1 + length |
+*--------------------------------+----------------------+---------------:
+| Comment Nodes                  | {{{https://svn.apache.org/repos/asf/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/nodes/TextOrCommentNodePointable.java}TextOrCommentNodePointable}}      |    1 + length |
+*--------------------------------+----------------------+---------------:
+| Text Nodes                     | {{{https://svn.apache.org/repos/asf/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/datamodel/accessors/nodes/TextOrCommentNodePointable.java}TextOrCommentNodePointable}}      |    1 + length |
+*--------------------------------+----------------------+---------------:
+
+
+* XML Mapping
+
+  The XML mapping to Hyracks pointables is fairly straight forward. The following example
+  shows how each node is mapped and saved into a byte array used by Hyracks.
+
+** Example XML File
+
+  The example XML file comes from W3School XQuery tutorial.
+  
+----------------------------------------
+<?xml version="1.0" encoding="ISO-8859-1"?>
+<!-- Edited by XMLSpyÆ -->
+<bookstore>
+
+    <book category="COOKING">
+        <title lang="en">Everyday Italian</title>
+        <author>Giada De Laurentiis</author>
+        <year>2005</year>
+        <price>30.00</price>
+    </book>
+    
+    <book category="CHILDREN">
+        <title lang="en">Harry Potter</title>
+        <author>J K. Rowling</author>
+        <year>2005</year>
+        <price>29.99</price>
+    </book>
+    
+    <book category="WEB">
+        <title lang="en">XQuery Kick Start</title>
+        <author>James McGovern</author>
+        <author>Per Bothner</author>
+        <author>Kurt Cagle</author>
+        <author>James Linn</author>
+        <author>Vaidyanathan Nagarajan</author>
+        <year>2003</year>
+        <price>49.99</price>
+    </book>
+    
+    <book category="WEB">
+        <title lang="en">Learning XML</title>
+        <author>Erik T. Ray</author>
+        <year>2003</year>
+        <price>39.95</price>
+    </book>
+
+</bookstore>
+----------------------------------------
+  
+** Example Hyracks Mapping
+
+  The mapping is explained through using some short hand for the above example XML file.
+  Realize the direct bytes will not be explained although the pointable names are used for
+  each piece of information.
+  
+----------------------------------------
+NodeTree {
+    DocumentNode {bookstore}
+        sequence (children) {
+            ElementNode {book}
+                sequence (attributes) {
+                    AttributeNode {category}
+                }
+                sequence (children) {
+                    ElementNode {title:Everyday Italian}
+                        sequence (attributes) {
+                            AttributeNode {lang}
+                        }
+                    ElementNode {author}
+                    ElementNode {year}
+                    ElementNode {price}
+                }
+            ElementNode {book}
+                sequence (attributes) {
+                    AttributeNode {category}
+                }
+                sequence (children) {
+                    ElementNode {title:Harry Potter}
+                        sequence (attributes) {
+                           AttributeNode {lang}
+                        }
+                    ElementNode {author}
+                    ElementNode {year}
+                    ElementNode {price}
+                }
+            ElementNode {book}
+                sequence (attributes) {
+                    AttributeNode {category}
+                }
+                sequence (children) {
+                    ElementNode {title:XQuery Kick Start}
+                        sequence (attributes) {
+                            AttributeNode {lang}
+                        }
+                    ElementNode {author}
+                    ElementNode {author}
+                    ElementNode {author}
+                    ElementNode {author}
+                    ElementNode {author}
+                    ElementNode {year}
+                    ElementNode {price}
+                }
+            ElementNode {book}
+                sequence (attributes) {
+                    AttributeNode {category}
+                }
+                sequence (children) {
+                    ElementNode {title:Learning XML}
+                        sequence (attributes) {
+                            AttributeNode {lang}
+                        }
+                    ElementNode {author}
+                    ElementNode {year}
+                    ElementNode {price}
+                }
+        }
+}
+----------------------------------------
+  
+  
\ No newline at end of file

Modified: incubator/vxquery/trunk/vxquery/src/site/site.xml
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/src/site/site.xml?rev=1433797&r1=1433796&r2=1433797&view=diff
==============================================================================
--- incubator/vxquery/trunk/vxquery/src/site/site.xml (original)
+++ incubator/vxquery/trunk/vxquery/src/site/site.xml Wed Jan 16 01:53:59 2013
@@ -32,7 +32,8 @@ limitations under the License.
         <menu ref="modules"/>
       
         <menu name="For Developers">
-            <item name="Development Tips" href="development_tips.html"/>
+            <item name="Data Basic Types" href="development_tips.html"/>
+            <item name="Data XML and Node Types" href="development_xml_node_details.html"/>
             <item name="Eclipse Setup" href="development_eclipse_setup.html"/>
         </menu>
 

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=1433797&r1=1433796&r2=1433797&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 Jan 16 01:53:59 2013
@@ -794,12 +794,14 @@
     <operator name="opext:descendant">
         <param name="arg" type="node()*"/>
         <return type="node()*"/>
+        <runtime type="scalar" class="org.apache.vxquery.runtime.functions.step.DescendantPathStepScalarEvaluatorFactory"/>
     </operator>
 
     <!-- opext:descendant-or-self($arg as node()*) as node()* -->
     <operator name="opext:descendant-or-self">
         <param name="arg" type="node()*"/>
         <return type="node()*"/>
+        <runtime type="scalar" class="org.apache.vxquery.runtime.functions.step.DescendantOrSelfPathStepScalarEvaluatorFactory"/>
     </operator>
 
     <!-- opext:following($arg as node()*) as node()* -->
@@ -836,6 +838,7 @@
     <operator name="opext:self">
         <param name="arg" type="node()*"/>
         <return type="node()*"/>
+        <runtime type="scalar" class="org.apache.vxquery.runtime.functions.step.SelfPathStepScalarEvaluatorFactory"/>
     </operator>
 
     <!-- opext:sequence($arg as item()*) as item()* -->

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractDescendantPathStepScalarEvaluator.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractDescendantPathStepScalarEvaluator.java?rev=1433797&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractDescendantPathStepScalarEvaluator.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractDescendantPathStepScalarEvaluator.java Wed Jan 16 01:53:59 2013
@@ -0,0 +1,97 @@
+/*
+ * 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.IOException;
+
+import org.apache.vxquery.datamodel.accessors.SequencePointable;
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+import org.apache.vxquery.datamodel.accessors.nodes.DocumentNodePointable;
+import org.apache.vxquery.datamodel.accessors.nodes.ElementNodePointable;
+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 edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+
+public abstract class AbstractDescendantPathStepScalarEvaluator extends AbstractPathStepScalarEvaluator {
+    private final DocumentNodePointable dnp;
+
+    private final ElementNodePointable enp;
+
+    public AbstractDescendantPathStepScalarEvaluator(IScalarEvaluator[] args, IHyracksTaskContext ctx) {
+        super(args, ctx);
+        dnp = (DocumentNodePointable) DocumentNodePointable.FACTORY.createPointable();
+        enp = (ElementNodePointable) ElementNodePointable.FACTORY.createPointable();
+    }
+
+    @Override
+    protected abstract void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException;
+
+    /**
+     * Search through all tree children and children's children.
+     * 
+     * @param nodePointable
+     * @throws SystemException
+     */
+    protected void searchSubtree(TaggedValuePointable nodePointable) throws SystemException {
+        try {
+            SequencePointable seqp = (SequencePointable) SequencePointable.FACTORY.createPointable();
+
+            // Find all child element to search.
+            switch (nodePointable.getTag()) {
+                case ValueTag.DOCUMENT_NODE_TAG:
+                    nodePointable.getValue(dnp);
+                    dnp.getContent(ntp, seqp);
+                    break;
+
+                case ValueTag.ELEMENT_NODE_TAG:
+                    nodePointable.getValue(enp);
+                    if (enp.childrenChunkExists()) {
+                        enp.getChildrenSequence(ntp, seqp);
+                    }
+                    break;
+
+                default:
+                    XDMConstants.setEmptySequence(seqp);
+                    break;
+            }
+
+            int seqSize = seqp.getEntryCount();
+            for (int i = 0; i < seqSize; ++i) {
+                seqp.getEntry(i, itemTvp);
+                // Only search element nodes.
+                if (itemTvp.getTag() == ValueTag.ELEMENT_NODE_TAG) {
+                    if (matches()) {
+                        appendNodeToResult();
+                    }
+                    // Now check this elements children.
+                    TaggedValuePointable tvpTemp = (TaggedValuePointable) TaggedValuePointable.FACTORY
+                            .createPointable();
+                    tvpTemp.set(itemTvp);
+                    searchSubtree(tvpTemp);
+                }
+            }
+        } catch (IOException e) {
+            throw new SystemException(ErrorCode.SYSE0001, e);
+        }
+    }
+
+}
\ No newline at end of file

Modified: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractPathStepScalarEvaluator.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractPathStepScalarEvaluator.java?rev=1433797&r1=1433796&r2=1433797&view=diff
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractPathStepScalarEvaluator.java (original)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractPathStepScalarEvaluator.java Wed Jan 16 01:53:59 2013
@@ -20,7 +20,6 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.vxquery.context.DynamicContext;
-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;
@@ -28,7 +27,6 @@ import org.apache.vxquery.datamodel.acce
 import org.apache.vxquery.datamodel.accessors.nodes.NodeTreePointable;
 import org.apache.vxquery.datamodel.builders.sequence.SequenceBuilder;
 import org.apache.vxquery.datamodel.values.ValueTag;
-import org.apache.vxquery.exceptions.ErrorCode;
 import org.apache.vxquery.exceptions.SystemException;
 import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluator;
 import org.apache.vxquery.types.AttributeType;
@@ -40,7 +38,6 @@ import org.apache.vxquery.types.Sequence
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.data.std.api.IPointable;
-import edu.uci.ics.hyracks.data.std.primitive.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;
@@ -48,38 +45,26 @@ import edu.uci.ics.hyracks.data.std.util
 public abstract class AbstractPathStepScalarEvaluator extends AbstractTaggedValueArgumentScalarEvaluator {
     protected final DynamicContext dCtx;
 
-    private final IntegerPointable ip;
+    protected final SequenceBuilder seqb;
 
-    private final NodeTreePointable ntp;
-
-    private final SequencePointable seqp;
-
-    private final ArrayBackedValueStorage seqAbvs;
-
-    private final SequenceBuilder seqb;
+    protected final NodeTreePointable ntp;
 
     private final ArrayBackedValueStorage nodeAbvs;
 
-    private final TaggedValuePointable itemTvp;
+    protected final TaggedValuePointable itemTvp;
 
     private INodeFilter filter;
 
-    private boolean first;
-
     public AbstractPathStepScalarEvaluator(IScalarEvaluator[] args, IHyracksTaskContext ctx) {
         super(args);
         dCtx = (DynamicContext) ctx.getJobletContext().getGlobalJobData();
-        ip = (IntegerPointable) IntegerPointable.FACTORY.createPointable();
         ntp = (NodeTreePointable) NodeTreePointable.FACTORY.createPointable();
-        seqp = (SequencePointable) SequencePointable.FACTORY.createPointable();
-        seqAbvs = new ArrayBackedValueStorage();
         seqb = new SequenceBuilder();
         nodeAbvs = new ArrayBackedValueStorage();
         itemTvp = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
-        first = true;
     }
 
-    private void setNodeTest(SequenceType sType) {
+    protected void setNodeTest(SequenceType sType) {
         final NodeType nodeType = (NodeType) sType.getItemType();
         switch (nodeType.getNodeKind()) {
             case ANY:
@@ -218,47 +203,14 @@ public abstract class AbstractPathStepSc
         }
     }
 
-    protected abstract void getSequence(NodeTreePointable ntp, SequencePointable seqp) throws SystemException;
-
     @Override
-    protected final void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException {
-        try {
-            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);
-            seqAbvs.reset();
-            seqb.reset(seqAbvs);
-            int seqSize = seqp.getEntryCount();
-            for (int i = 0; i < seqSize; ++i) {
-                seqp.getEntry(i, itemTvp);
-                if (matches()) {
-                    appendNodeToResult();
-                }
-            }
-            seqb.finish();
-            result.set(seqAbvs);
-        } catch (IOException e) {
-            throw new SystemException(ErrorCode.SYSE0001, e);
-        }
-    }
+    protected abstract void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException;
 
-    private boolean matches() {
+    protected boolean matches() {
         return filter.accept(ntp, itemTvp);
     }
 
-    private void appendNodeToResult() throws IOException {
+    protected void appendNodeToResult() throws IOException {
         nodeAbvs.reset();
         DataOutput mainOut = nodeAbvs.getDataOutput();
         mainOut.write(ValueTag.NODE_TREE_TAG);

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractSinglePathStepScalarEvaluator.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractSinglePathStepScalarEvaluator.java?rev=1433797&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractSinglePathStepScalarEvaluator.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AbstractSinglePathStepScalarEvaluator.java Wed Jan 16 01:53:59 2013
@@ -0,0 +1,88 @@
+/*
+ * 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.IOException;
+
+import org.apache.vxquery.datamodel.accessors.SequencePointable;
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+import org.apache.vxquery.datamodel.accessors.nodes.NodeTreePointable;
+import org.apache.vxquery.datamodel.values.ValueTag;
+import org.apache.vxquery.exceptions.ErrorCode;
+import org.apache.vxquery.exceptions.SystemException;
+import org.apache.vxquery.types.SequenceType;
+
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+
+public abstract class AbstractSinglePathStepScalarEvaluator extends AbstractPathStepScalarEvaluator {
+
+    private final IntegerPointable ip;
+
+    private final SequencePointable seqp;
+
+    private final ArrayBackedValueStorage seqAbvs;
+
+    private boolean first;
+
+    public AbstractSinglePathStepScalarEvaluator(IScalarEvaluator[] args, IHyracksTaskContext ctx) {
+        super(args, ctx);
+        ip = (IntegerPointable) IntegerPointable.FACTORY.createPointable();
+        seqp = (SequencePointable) SequencePointable.FACTORY.createPointable();
+        seqAbvs = new ArrayBackedValueStorage();
+        first = true;
+    }
+
+    protected abstract void getSequence(NodeTreePointable ntp, SequencePointable seqp) throws SystemException;
+
+    @Override
+    protected final void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException {
+        try {
+            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);
+            seqAbvs.reset();
+            seqb.reset(seqAbvs);
+            int seqSize = seqp.getEntryCount();
+            for (int i = 0; i < seqSize; ++i) {
+                seqp.getEntry(i, itemTvp);
+                if (matches()) {
+                    appendNodeToResult();
+                }
+            }
+            seqb.finish();
+            result.set(seqAbvs);
+        } catch (IOException e) {
+            throw new SystemException(ErrorCode.SYSE0001, e);
+        }
+    }
+}
\ No newline at end of file

Modified: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AttributePathStepScalarEvaluator.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AttributePathStepScalarEvaluator.java?rev=1433797&r1=1433796&r2=1433797&view=diff
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AttributePathStepScalarEvaluator.java (original)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/AttributePathStepScalarEvaluator.java Wed Jan 16 01:53:59 2013
@@ -27,7 +27,7 @@ import org.apache.vxquery.exceptions.Sys
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
-public class AttributePathStepScalarEvaluator extends AbstractPathStepScalarEvaluator {
+public class AttributePathStepScalarEvaluator extends AbstractSinglePathStepScalarEvaluator {
     private final TaggedValuePointable rootTVP;
 
     private final ElementNodePointable enp;

Modified: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepScalarEvaluator.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepScalarEvaluator.java?rev=1433797&r1=1433796&r2=1433797&view=diff
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepScalarEvaluator.java (original)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/ChildPathStepScalarEvaluator.java Wed Jan 16 01:53:59 2013
@@ -28,7 +28,7 @@ import org.apache.vxquery.exceptions.Sys
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
-public class ChildPathStepScalarEvaluator extends AbstractPathStepScalarEvaluator {
+public class ChildPathStepScalarEvaluator extends AbstractSinglePathStepScalarEvaluator {
     private final TaggedValuePointable rootTVP;
 
     private final DocumentNodePointable dnp;

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepScalarEvaluator.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepScalarEvaluator.java?rev=1433797&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepScalarEvaluator.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepScalarEvaluator.java Wed Jan 16 01:53:59 2013
@@ -0,0 +1,86 @@
+/*
+ * 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.IOException;
+
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+import org.apache.vxquery.datamodel.values.ValueTag;
+import org.apache.vxquery.exceptions.ErrorCode;
+import org.apache.vxquery.exceptions.SystemException;
+import org.apache.vxquery.types.ElementType;
+import org.apache.vxquery.types.Quantifier;
+import org.apache.vxquery.types.SequenceType;
+
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+
+public class DescendantOrSelfPathStepScalarEvaluator extends AbstractDescendantPathStepScalarEvaluator {
+    private final TaggedValuePointable rootTVP;
+
+    private final IntegerPointable ip;
+
+    private final ArrayBackedValueStorage seqAbvs;
+
+    public DescendantOrSelfPathStepScalarEvaluator(IScalarEvaluator[] args, IHyracksTaskContext ctx) {
+        super(args, ctx);
+        rootTVP = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+        ip = (IntegerPointable) IntegerPointable.FACTORY.createPointable();
+        seqAbvs = new ArrayBackedValueStorage();
+    }
+
+    @Override
+    protected final void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException {
+        try {
+            if (args[0].getTag() != ValueTag.NODE_TREE_TAG) {
+                throw new SystemException(ErrorCode.SYSE0001);
+            }
+            args[0].getValue(ntp);
+
+            // Set up the result sequence and get the root node.
+            seqAbvs.reset();
+            seqb.reset(seqAbvs);
+            ntp.getRootNode(rootTVP);
+
+            // Solve for self.
+            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);
+            rootTVP.set(itemTvp);
+            if (matches()) {
+                appendNodeToResult();
+            }
+
+            // Solve for descendants.
+            setNodeTest(SequenceType.create(ElementType.ANYELEMENT, Quantifier.QUANT_ONE));
+            searchSubtree(rootTVP);
+
+            seqb.finish();
+            result.set(seqAbvs);
+        } catch (IOException e) {
+            throw new SystemException(ErrorCode.SYSE0001, e);
+        }
+    }
+
+}
\ No newline at end of file

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepScalarEvaluatorFactory.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepScalarEvaluatorFactory.java?rev=1433797&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepScalarEvaluatorFactory.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantOrSelfPathStepScalarEvaluatorFactory.java Wed Jan 16 01:53:59 2013
@@ -0,0 +1,38 @@
+/*
+ * 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 org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluatorFactory;
+
+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.api.context.IHyracksTaskContext;
+
+public class DescendantOrSelfPathStepScalarEvaluatorFactory extends AbstractTaggedValueArgumentScalarEvaluatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    public DescendantOrSelfPathStepScalarEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        super(args);
+    }
+
+    @Override
+    protected IScalarEvaluator createEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args)
+            throws AlgebricksException {
+        return new DescendantOrSelfPathStepScalarEvaluator(args, ctx);
+    }
+}
\ No newline at end of file

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantPathStepScalarEvaluator.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantPathStepScalarEvaluator.java?rev=1433797&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantPathStepScalarEvaluator.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantPathStepScalarEvaluator.java Wed Jan 16 01:53:59 2013
@@ -0,0 +1,69 @@
+/*
+ * 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.IOException;
+
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+import org.apache.vxquery.datamodel.values.ValueTag;
+import org.apache.vxquery.exceptions.ErrorCode;
+import org.apache.vxquery.exceptions.SystemException;
+import org.apache.vxquery.types.ElementType;
+import org.apache.vxquery.types.Quantifier;
+import org.apache.vxquery.types.SequenceType;
+
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+
+public class DescendantPathStepScalarEvaluator extends AbstractDescendantPathStepScalarEvaluator {
+    private final TaggedValuePointable rootTVP;
+
+    private final ArrayBackedValueStorage seqAbvs;
+
+    public DescendantPathStepScalarEvaluator(IScalarEvaluator[] args, IHyracksTaskContext ctx) {
+        super(args, ctx);
+        rootTVP = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+        seqAbvs = new ArrayBackedValueStorage();
+    }
+
+    @Override
+    protected final void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException {
+        try {
+            if (args[0].getTag() != ValueTag.NODE_TREE_TAG) {
+                throw new SystemException(ErrorCode.SYSE0001);
+            }
+            args[0].getValue(ntp);
+
+            // Set up the result sequence and get the root node.
+            seqAbvs.reset();
+            seqb.reset(seqAbvs);
+            ntp.getRootNode(rootTVP);
+
+            // Solve for descendants.
+            setNodeTest(SequenceType.create(ElementType.ANYELEMENT, Quantifier.QUANT_ONE));
+            searchSubtree(rootTVP);
+
+            seqb.finish();
+            result.set(seqAbvs);
+        } catch (IOException e) {
+            throw new SystemException(ErrorCode.SYSE0001, e);
+        }
+    }
+
+}
\ No newline at end of file

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantPathStepScalarEvaluatorFactory.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantPathStepScalarEvaluatorFactory.java?rev=1433797&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantPathStepScalarEvaluatorFactory.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/DescendantPathStepScalarEvaluatorFactory.java Wed Jan 16 01:53:59 2013
@@ -0,0 +1,38 @@
+/*
+ * 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 org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluatorFactory;
+
+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.api.context.IHyracksTaskContext;
+
+public class DescendantPathStepScalarEvaluatorFactory extends AbstractTaggedValueArgumentScalarEvaluatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    public DescendantPathStepScalarEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        super(args);
+    }
+
+    @Override
+    protected IScalarEvaluator createEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args)
+            throws AlgebricksException {
+        return new DescendantOrSelfPathStepScalarEvaluator(args, ctx);
+    }
+}
\ No newline at end of file

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/SelfPathStepScalarEvaluator.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/SelfPathStepScalarEvaluator.java?rev=1433797&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/SelfPathStepScalarEvaluator.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/SelfPathStepScalarEvaluator.java Wed Jan 16 01:53:59 2013
@@ -0,0 +1,59 @@
+/*
+ * 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.IOException;
+
+import org.apache.vxquery.datamodel.accessors.SequencePointable;
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+import org.apache.vxquery.datamodel.accessors.nodes.NodeTreePointable;
+import org.apache.vxquery.datamodel.builders.sequence.SequenceBuilder;
+import org.apache.vxquery.exceptions.ErrorCode;
+import org.apache.vxquery.exceptions.SystemException;
+
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+
+public class SelfPathStepScalarEvaluator extends AbstractSinglePathStepScalarEvaluator {
+    private final TaggedValuePointable rootTVP;
+
+    final SequenceBuilder sb = new SequenceBuilder();
+
+    private ArrayBackedValueStorage abvs = new ArrayBackedValueStorage();
+
+    public SelfPathStepScalarEvaluator(IScalarEvaluator[] args, IHyracksTaskContext ctx) {
+        super(args, ctx);
+        rootTVP = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+    }
+
+    @Override
+    protected void getSequence(NodeTreePointable ntp, SequencePointable seqp) throws SystemException {
+        ntp.getRootNode(rootTVP);
+
+        // Create sequence with node.
+        try {
+            abvs.reset();
+            sb.reset(abvs);
+            sb.addItem(rootTVP);
+            sb.finish();
+            seqp.set(abvs);
+        } catch (IOException e) {
+            throw new SystemException(ErrorCode.SYSE0001);
+        }
+    }
+}
\ No newline at end of file

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/SelfPathStepScalarEvaluatorFactory.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/SelfPathStepScalarEvaluatorFactory.java?rev=1433797&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/SelfPathStepScalarEvaluatorFactory.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/step/SelfPathStepScalarEvaluatorFactory.java Wed Jan 16 01:53:59 2013
@@ -0,0 +1,38 @@
+/*
+ * 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 org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluatorFactory;
+
+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.api.context.IHyracksTaskContext;
+
+public class SelfPathStepScalarEvaluatorFactory extends AbstractTaggedValueArgumentScalarEvaluatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    public SelfPathStepScalarEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        super(args);
+    }
+
+    @Override
+    protected IScalarEvaluator createEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args)
+            throws AlgebricksException {
+        return new SelfPathStepScalarEvaluator(args, ctx);
+    }
+}
\ No newline at end of file

Modified: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/provider/VXQueryBinaryHashFunctionFactoryProvider.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/provider/VXQueryBinaryHashFunctionFactoryProvider.java?rev=1433797&r1=1433796&r2=1433797&view=diff
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/provider/VXQueryBinaryHashFunctionFactoryProvider.java (original)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/provider/VXQueryBinaryHashFunctionFactoryProvider.java Wed Jan 16 01:53:59 2013
@@ -31,10 +31,10 @@ public class VXQueryBinaryHashFunctionFa
 
     @Override
     public IBinaryHashFunctionFactory getBinaryHashFunctionFactory(Object type) throws AlgebricksException {
-        
+
         // Treat all data as a raw array of bytes.
         return VXQueryRawBinaryHashFunctionFactory.INSTANCE;
-        
+
     }
 
 }