You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@vxquery.apache.org by ti...@apache.org on 2010/06/16 23:23:48 UTC

svn commit: r955397 - in /incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery: functions/ runtime/functions/ xmlquery/query/

Author: tillw
Date: Wed Jun 16 21:23:47 2010
New Revision: 955397

URL: http://svn.apache.org/viewvc?rev=955397&view=rev
Log:
- added node comparison iterators

Added:
    incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/AbstractNodeComparisonIterator.java
    incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/OpIsSameNodeIterator.java
    incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/OpNodeAfterIterator.java
    incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/OpNodeBeforeIterator.java
Modified:
    incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
    incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryTranslator.java

Modified: incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/functions/builtin-operators.xml?rev=955397&r1=955396&r2=955397&view=diff
==============================================================================
--- incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/functions/builtin-operators.xml (original)
+++ incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/functions/builtin-operators.xml Wed Jun 16 21:23:47 2010
@@ -233,14 +233,14 @@
         <return type="xs:boolean"/>
     </operator>
 
-    <!-- op:hexBinary-equal($value1  as xs:hexBinary, $value2 as xs:hexBinary)  as xs:boolean -->
+    <!-- op:hexBinary-equal($value1 as xs:hexBinary, $value2 as xs:hexBinary) as xs:boolean -->
     <operator name="op:hexBinary-equal">
         <param name="arg1" type="xs:hexBinary"/>
         <param name="arg2" type="xs:hexBinary"/>
         <return type="xs:boolean"/>
     </operator>
 
-    <!-- op:intersect($parameter1  as node()*, $parameter2 as node()*)  as node()* -->
+    <!-- op:intersect($parameter1 as node()*, $parameter2 as node()*) as node()* -->
     <operator name="op:intersect">
         <param name="parameter1" type="node()*"/>
         <param name="parameter2" type="node()*"/>
@@ -249,11 +249,12 @@
         <runtime class="org.apache.vxquery.runtime.functions.OpIntersectIterator"/>
     </operator>
 
-    <!-- op:is-same-node($parameter1  as node(), $parameter2 as node())  as xs:boolean -->
+    <!-- op:is-same-node($parameter1 as node(), $parameter2 as node()) as xs:boolean -->
     <operator name="op:is-same-node">
-        <param name="parameter1" type="node()"/>
-        <param name="parameter2" type="node()"/>
+        <param name="parameter1" type="node()?"/>
+        <param name="parameter2" type="node()?"/>
         <return type="xs:boolean"/>
+        <runtime class="org.apache.vxquery.runtime.functions.OpIsSameNodeIterator"/>
     </operator>
 
     <!-- op:multiply-dayTimeDuration( $arg1 as xs:dayTimeDuration, $arg2 as xs:double)  as xs:dayTimeDuration -->
@@ -272,16 +273,18 @@
 
     <!-- op:node-after($parameter1  as node(), $parameter2 as node())  as xs:boolean -->
     <operator name="op:node-after">
-        <param name="parameter1" type="node()"/>
-        <param name="parameter2" type="node()"/>
+        <param name="parameter1" type="node()?"/>
+        <param name="parameter2" type="node()?"/>
         <return type="xs:boolean"/>
+        <runtime class="org.apache.vxquery.runtime.functions.OpNodeAfterIterator"/>
     </operator>
 
     <!-- op:node-before($parameter1  as node(), $parameter2 as node())  as xs:boolean -->
     <operator name="op:node-before">
-        <param name="parameter1" type="node()"/>
-        <param name="parameter2" type="node()"/>
+        <param name="parameter1" type="node()?"/>
+        <param name="parameter2" type="node()?"/>
         <return type="xs:boolean"/>
+        <runtime class="org.apache.vxquery.runtime.functions.OpNodeBeforeIterator"/>
     </operator>
 
     <!-- op:NOTATION-equal($arg1  as xs:NOTATION, $arg2 as xs:NOTATION)  as xs:boolean -->
@@ -669,20 +672,6 @@
         <runtime class="org.apache.vxquery.runtime.functions.BooleanOrIterator"/>
     </operator>
 
-    <!-- opext:precedes($arg1 as node()?, $arg2 as node()?) as xs:boolean? -->
-    <operator name="opext:precedes">
-        <param name="arg1" type="node()?"/>
-        <param name="arg2" type="node()?"/>
-        <return type="xs:boolean?"/>
-    </operator>
-
-    <!-- opext:follows($arg1 as node()?, $arg2 as node()?) as xs:boolean? -->
-    <operator name="opext:follows">
-        <param name="arg1" type="node()?"/>
-        <param name="arg2" type="node()?"/>
-        <return type="xs:boolean?"/>
-    </operator>
-
     <!-- opext:sort-nodes-asc($arg as node()*) as node()* -->
     <operator name="opext:sort-nodes-asc">
         <param name="arg" type="node()*"/>

Added: incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/AbstractNodeComparisonIterator.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/AbstractNodeComparisonIterator.java?rev=955397&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/AbstractNodeComparisonIterator.java (added)
+++ incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/AbstractNodeComparisonIterator.java Wed Jun 16 21:23:47 2010
@@ -0,0 +1,47 @@
+/*
+ * 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;
+
+import org.apache.vxquery.context.StaticContext;
+import org.apache.vxquery.datamodel.XDMNode;
+import org.apache.vxquery.exceptions.SystemException;
+import org.apache.vxquery.functions.Function;
+import org.apache.vxquery.runtime.CallStackFrame;
+import org.apache.vxquery.runtime.RegisterAllocator;
+import org.apache.vxquery.runtime.base.AbstractEagerlyEvaluatedFunctionIterator;
+import org.apache.vxquery.runtime.base.RuntimeIterator;
+
+public abstract class AbstractNodeComparisonIterator extends AbstractEagerlyEvaluatedFunctionIterator {
+    public AbstractNodeComparisonIterator(RegisterAllocator rAllocator, Function fn, RuntimeIterator[] arguments,
+            StaticContext ctx) {
+        super(rAllocator, fn, arguments, ctx);
+    }
+
+    @Override
+    public Object evaluateEagerly(CallStackFrame frame) throws SystemException {
+        final XDMNode n1 = (XDMNode) arguments[0].evaluateEagerly(frame);
+        if (n1 == null) {
+            return null;
+        }
+        final XDMNode n2 = (XDMNode) arguments[1].evaluateEagerly(frame);
+        if (n2 == null) {
+            return null;
+        }
+        final boolean res = compare(n1, n2);
+        return frame.getRuntimeControlBlock().getAtomicValueFactory().createBoolean(res);
+    }
+
+    protected abstract boolean compare(XDMNode n1, XDMNode n2);
+}
\ No newline at end of file

Added: incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/OpIsSameNodeIterator.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/OpIsSameNodeIterator.java?rev=955397&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/OpIsSameNodeIterator.java (added)
+++ incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/OpIsSameNodeIterator.java Wed Jun 16 21:23:47 2010
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+import org.apache.vxquery.context.StaticContext;
+import org.apache.vxquery.datamodel.XDMNode;
+import org.apache.vxquery.functions.Function;
+import org.apache.vxquery.runtime.RegisterAllocator;
+import org.apache.vxquery.runtime.base.RuntimeIterator;
+
+public class OpIsSameNodeIterator extends AbstractNodeComparisonIterator {
+    public OpIsSameNodeIterator(RegisterAllocator rAllocator, Function fn, RuntimeIterator[] arguments,
+            StaticContext ctx) {
+        super(rAllocator, fn, arguments, ctx);
+    }
+
+    @Override
+    protected boolean compare(XDMNode n1, XDMNode n2) {
+        return n1.compareDocumentOrder(n2) == 0;
+    }
+}
\ No newline at end of file

Added: incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/OpNodeAfterIterator.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/OpNodeAfterIterator.java?rev=955397&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/OpNodeAfterIterator.java (added)
+++ incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/OpNodeAfterIterator.java Wed Jun 16 21:23:47 2010
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+import org.apache.vxquery.context.StaticContext;
+import org.apache.vxquery.datamodel.XDMNode;
+import org.apache.vxquery.functions.Function;
+import org.apache.vxquery.runtime.RegisterAllocator;
+import org.apache.vxquery.runtime.base.RuntimeIterator;
+
+public class OpNodeAfterIterator extends AbstractNodeComparisonIterator {
+    public OpNodeAfterIterator(RegisterAllocator rAllocator, Function fn, RuntimeIterator[] arguments, StaticContext ctx) {
+        super(rAllocator, fn, arguments, ctx);
+    }
+
+    @Override
+    protected boolean compare(XDMNode n1, XDMNode n2) {
+        return n1.compareDocumentOrder(n2) > 0;
+    }
+}
\ No newline at end of file

Added: incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/OpNodeBeforeIterator.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/OpNodeBeforeIterator.java?rev=955397&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/OpNodeBeforeIterator.java (added)
+++ incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/runtime/functions/OpNodeBeforeIterator.java Wed Jun 16 21:23:47 2010
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+import org.apache.vxquery.context.StaticContext;
+import org.apache.vxquery.datamodel.XDMNode;
+import org.apache.vxquery.functions.Function;
+import org.apache.vxquery.runtime.RegisterAllocator;
+import org.apache.vxquery.runtime.base.RuntimeIterator;
+
+public class OpNodeBeforeIterator extends AbstractNodeComparisonIterator {
+    public OpNodeBeforeIterator(RegisterAllocator rAllocator, Function fn, RuntimeIterator[] arguments,
+            StaticContext ctx) {
+        super(rAllocator, fn, arguments, ctx);
+    }
+
+    @Override
+    protected boolean compare(XDMNode n1, XDMNode n2) {
+        return n1.compareDocumentOrder(n2) < 0;
+    }
+}
\ No newline at end of file

Modified: incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryTranslator.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryTranslator.java?rev=955397&r1=955396&r2=955397&view=diff
==============================================================================
--- incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryTranslator.java (original)
+++ incubator/vxquery/trunk/vxquery/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryTranslator.java Wed Jun 16 21:23:47 2010
@@ -1565,7 +1565,7 @@ final class XMLQueryTranslator {
                 return BuiltinOperators.EXCEPT;
 
             case FOLLOWS:
-                return BuiltinOperators.FOLLOWS;
+                return BuiltinOperators.NODE_AFTER;
 
             case GENERAL_EQ:
                 return BuiltinOperators.GENERAL_EQ;
@@ -1610,7 +1610,7 @@ final class XMLQueryTranslator {
                 return BuiltinOperators.ADD;
 
             case PRECEDES:
-                return BuiltinOperators.PRECEDES;
+                return BuiltinOperators.NODE_BEFORE;
 
             case RANGE:
                 return BuiltinOperators.TO;