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 2016/06/23 22:44:15 UTC

vxquery git commit: JSONiq size function, array navigation and tests

Repository: vxquery
Updated Branches:
  refs/heads/master 020a16a68 -> 95c7cd2b9


JSONiq size function, array navigation and tests

Conflicts:
	vxquery-xtest/src/test/resources/VXQueryCatalog.xml


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

Branch: refs/heads/master
Commit: 95c7cd2b9db79cdc95d31fc6540a7dd02a547c95
Parents: 020a16a
Author: Christina Pavlopoulou <cp...@ucr.edu>
Authored: Fri Jun 17 17:14:07 2016 -0700
Committer: Christina Pavlopoulou <cp...@ucr.edu>
Committed: Thu Jun 23 14:53:02 2016 -0700

----------------------------------------------------------------------
 .../vxquery/functions/builtin-functions.xml     |  7 ++
 .../vxquery/functions/builtin-operators.xml     |  8 ++
 .../functions/json/ValueScalarEvaluator.java    | 86 ++++++++++++++++++++
 .../json/ValueScalarEvaluatorFactory.java       | 39 +++++++++
 .../misc/FnSizeScalarEvaluatorFactory.java      | 77 ++++++++++++++++++
 .../org/apache/vxquery/xmlquery/ast/ASTTag.java |  3 +-
 .../vxquery/xmlquery/ast/ArgumentListNode.java  | 42 ++++++++++
 .../vxquery/xmlquery/ast/FilterExprNode.java    | 51 ------------
 .../vxquery/xmlquery/ast/PostfixExprNode.java   | 51 ++++++++++++
 .../xmlquery/translator/XMLQueryTranslator.java | 38 +++++++--
 vxquery-core/src/main/javacc/xquery-grammar.jj  | 55 +++++++++++--
 .../Array/Navigation/q01_array_navigation.txt   |  1 +
 .../Array/Navigation/q02_array_navigation.txt   |  1 +
 .../Array/Navigation/q03_array_navigation.txt   |  1 +
 .../Array/Navigation/q04_array_navigation.txt   |  0
 .../Array/Navigation/q05_array_navigation.txt   |  0
 .../Array/Navigation/q01_array_navigation.xq    | 21 +++++
 .../Array/Navigation/q02_array_navigation.xq    | 22 +++++
 .../Array/Navigation/q03_array_navigation.xq    | 22 +++++
 .../Array/Navigation/q04_array_navigation.xq    | 21 +++++
 .../Array/Navigation/q05_array_navigation.xq    | 21 +++++
 .../cat/JsonArrayNavigationQueries.xml          | 48 +++++++++++
 22 files changed, 549 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-functions.xml
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-functions.xml b/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-functions.xml
index 8379ccf..adeef38 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-functions.xml
+++ b/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-functions.xml
@@ -237,6 +237,13 @@
             <argument value="0"/>
         </property>
     </function>
+    
+    <!-- fn:size($expr as  item()*) as xs:anyAtomicType* -->
+    <function name="fn:size">
+        <param name="expr" type="item()*"/>
+        <return type="xs:integer"/>
+        <runtime type="scalar" class="org.apache.vxquery.runtime.functions.misc.FnSizeScalarEvaluatorFactory"/>
+    </function>
 
     <!-- fn:dateTime($arg1  as xs:date?, $arg2 as xs:time?)  as xs:dateTime?  -->
     <function name="fn:dateTime">

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml b/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
index e28d405..877208c 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
+++ b/vxquery-core/src/main/java/org/apache/vxquery/functions/builtin-operators.xml
@@ -646,6 +646,14 @@
         <return type="xs:anyAtomicType?"/>
         <runtime type="scalar" class="org.apache.vxquery.runtime.functions.arithmetic.AddScalarEvaluatorFactory"/>
     </operator>
+    
+    <!-- opext:value($expr as xs:item(), $arg as xs:anyAtomicType?) as item() -->
+    <operator name="opext:value">
+        <param name="expr" type="item()"/>
+        <param name="argExpr" type="xs:anyAtomicType"/>
+        <return type="item()"/>
+        <runtime type="scalar" class="org.apache.vxquery.runtime.functions.json.ValueScalarEvaluatorFactory"/>
+    </operator>
 
     <!-- opext:subtract($arg1 as xs:anyAtomicType?, $arg2 as xs:anyAtomicType?) as xs:anyAtomicType? -->
     <operator name="opext:subtract">

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/ValueScalarEvaluator.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/ValueScalarEvaluator.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/ValueScalarEvaluator.java
new file mode 100644
index 0000000..d7e3e51
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/ValueScalarEvaluator.java
@@ -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.json;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+import org.apache.vxquery.datamodel.accessors.jsonitem.ArrayPointable;
+import org.apache.vxquery.datamodel.builders.sequence.SequenceBuilder;
+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.AbstractTaggedValueArgumentScalarEvaluator;
+
+public class ValueScalarEvaluator extends AbstractTaggedValueArgumentScalarEvaluator {
+    protected final IHyracksTaskContext ctx;
+    private final ArrayBackedValueStorage mvs;
+    private final ArrayPointable ap;
+    private final LongPointable lp;
+    private final SequenceBuilder ab;
+    protected DataOutput out;
+
+    public ValueScalarEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args) {
+        super(args);
+        this.ctx = ctx;
+        ap = (ArrayPointable) ArrayPointable.FACTORY.createPointable();
+        lp = (LongPointable) LongPointable.FACTORY.createPointable();
+        mvs = new ArrayBackedValueStorage();
+        ab = new SequenceBuilder();
+    }
+
+    @Override
+    protected void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException {
+        TaggedValuePointable tvp1 = args[0];
+        TaggedValuePointable tvp2 = args[1];
+        if (!(tvp1.getTag() == ValueTag.ARRAY_TAG || tvp1.getTag() == ValueTag.OBJECT_TAG)) {
+            throw new SystemException(ErrorCode.FORG0006);
+        }
+        if (tvp1.getTag() == ValueTag.ARRAY_TAG) {
+            if (tvp2.getTag() != ValueTag.XS_INTEGER_TAG) {
+                throw new SystemException(ErrorCode.FORG0006);
+            }
+            TaggedValuePointable tempTvp = ppool.takeOne(TaggedValuePointable.class);
+            mvs.reset();
+            try {
+                ab.reset(mvs);
+                tvp1.getValue(ap);
+                tvp2.getValue(lp);
+                if ((int) lp.getLong() > ap.getEntryCount()) {
+                    XDMConstants.setEmptySequence(result);
+                    return;
+                }
+                ap.getEntry((int) lp.getLong() - 1, tempTvp);
+                ab.addItem(ap.getEntryCount() != 0 ? tempTvp : tvp1);
+                ab.finish();
+                result.set(mvs);
+            } catch (IOException e) {
+                throw new SystemException(ErrorCode.SYSE0001, e);
+            } finally {
+                ppool.giveBack(tempTvp);
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/ValueScalarEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/ValueScalarEvaluatorFactory.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/ValueScalarEvaluatorFactory.java
new file mode 100644
index 0000000..7e754c2
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/ValueScalarEvaluatorFactory.java
@@ -0,0 +1,39 @@
+/*
+* 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.json;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluatorFactory;
+
+public class ValueScalarEvaluatorFactory extends AbstractTaggedValueArgumentScalarEvaluatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public ValueScalarEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        super(args);
+    }
+
+    @Override
+    protected IScalarEvaluator createEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args)
+            throws AlgebricksException {
+        return new ValueScalarEvaluator(ctx, args);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/misc/FnSizeScalarEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/misc/FnSizeScalarEvaluatorFactory.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/misc/FnSizeScalarEvaluatorFactory.java
new file mode 100644
index 0000000..3ff935f
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/misc/FnSizeScalarEvaluatorFactory.java
@@ -0,0 +1,77 @@
+/*
+ * 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.misc;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+import org.apache.vxquery.datamodel.accessors.jsonitem.ArrayPointable;
+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.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluatorFactory;
+
+public class FnSizeScalarEvaluatorFactory extends AbstractTaggedValueArgumentScalarEvaluatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public FnSizeScalarEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        super(args);
+    }
+
+    @Override
+    protected IScalarEvaluator createEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args)
+            throws AlgebricksException {
+        return new FnSizeScalarEvaluator(args);
+    }
+
+    private static class FnSizeScalarEvaluator extends AbstractTaggedValueArgumentScalarEvaluator {
+        final ArrayBackedValueStorage abvs = new ArrayBackedValueStorage();
+        final ArrayPointable ap = (ArrayPointable) ArrayPointable.FACTORY.createPointable();
+
+        public FnSizeScalarEvaluator(IScalarEvaluator[] args) {
+            super(args);
+        }
+
+        @Override
+        protected void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException {
+            TaggedValuePointable tvp = args[0];
+            if (!(tvp.getTag() != ValueTag.ARRAY_TAG || tvp.getTag() != ValueTag.OBJECT_TAG)) {
+                throw new SystemException(ErrorCode.FORG0006);
+            }
+            abvs.reset();
+            tvp.getValue(ap);
+            DataOutput out = abvs.getDataOutput();
+            ap.getEntryCount();
+            try {
+                out.write(ValueTag.XS_INTEGER_TAG);
+                out.writeLong(ap.getEntryCount());
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+            result.set(abvs);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/ast/ASTTag.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/ast/ASTTag.java b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/ast/ASTTag.java
index 3a91e80..a68dbd7 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/ast/ASTTag.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/ast/ASTTag.java
@@ -17,6 +17,7 @@
 package org.apache.vxquery.xmlquery.ast;
 
 public enum ASTTag {
+    ARGUMENT_LIST,
     VERSION_DECL,
     PROLOG,
     LIBRARY_MODULE,
@@ -79,7 +80,7 @@ public enum ASTTag {
     PATH_EXPRESSION,
     AXIS_STEP,
     NAME_TEST,
-    FILTER_EXPRESSION,
+    POSTFIX_EXPRESSION,
     LITERAL,
     VARIABLE_REFERENCE,
     PARENTHESIZED_EXPRESSION,

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/ast/ArgumentListNode.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/ast/ArgumentListNode.java b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/ast/ArgumentListNode.java
new file mode 100644
index 0000000..ad6f0f5
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/ast/ArgumentListNode.java
@@ -0,0 +1,42 @@
+/*
+* 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.xmlquery.ast;
+
+import java.util.List;
+
+import org.apache.vxquery.util.SourceLocation;
+
+public class ArgumentListNode extends ASTNode {
+    private List<ASTNode> arg;
+
+    public ArgumentListNode(SourceLocation loc) {
+        super(loc);
+    }
+
+    @Override
+    public ASTTag getTag() {
+        return ASTTag.ARGUMENT_LIST;
+    }
+
+    public List<ASTNode> getArg() {
+        return arg;
+    }
+
+    public void setArg(List<ASTNode> arg) {
+        this.arg = arg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/ast/FilterExprNode.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/ast/FilterExprNode.java b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/ast/FilterExprNode.java
deleted file mode 100644
index a19749b..0000000
--- a/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/ast/FilterExprNode.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one or more
-* contributor license agreements.  See the NOTICE file distributed with
-* this work for additional information regarding copyright ownership.
-* The ASF licenses this file to You under the Apache License, Version 2.0
-* (the "License"); you may not use this file except in compliance with
-* the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-package org.apache.vxquery.xmlquery.ast;
-
-import java.util.List;
-
-import org.apache.vxquery.util.SourceLocation;
-
-public class FilterExprNode extends ASTNode {
-    private ASTNode expr;
-    private List<ASTNode> predicates;
-
-    public FilterExprNode(SourceLocation loc) {
-        super(loc);
-    }
-
-    @Override
-    public ASTTag getTag() {
-        return ASTTag.FILTER_EXPRESSION;
-    }
-
-    public ASTNode getExpr() {
-        return expr;
-    }
-
-    public void setExpr(ASTNode expr) {
-        this.expr = expr;
-    }
-
-    public List<ASTNode> getPredicates() {
-        return predicates;
-    }
-
-    public void setPredicates(List<ASTNode> predicates) {
-        this.predicates = predicates;
-    }
-}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/ast/PostfixExprNode.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/ast/PostfixExprNode.java b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/ast/PostfixExprNode.java
new file mode 100644
index 0000000..a1ce4fa
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/ast/PostfixExprNode.java
@@ -0,0 +1,51 @@
+/*
+* 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.xmlquery.ast;
+
+import java.util.List;
+
+import org.apache.vxquery.util.SourceLocation;
+
+public class PostfixExprNode extends ASTNode {
+    private ASTNode expr;
+    private List<ASTNode> args;
+
+    public PostfixExprNode(SourceLocation loc) {
+        super(loc);
+    }
+
+    @Override
+    public ASTTag getTag() {
+        return ASTTag.POSTFIX_EXPRESSION;
+    }
+
+    public ASTNode getExpr() {
+        return expr;
+    }
+
+    public void setExpr(ASTNode expr) {
+        this.expr = expr;
+    }
+
+    public List<ASTNode> getArgs() {
+        return args;
+    }
+
+    public void setArgs(List<ASTNode> args) {
+        this.args = args;
+    }
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java
index 5f44fea..8ae5828 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java
@@ -98,6 +98,7 @@ import org.apache.vxquery.types.TextType;
 import org.apache.vxquery.types.TypeUtils;
 import org.apache.vxquery.xmlquery.ast.ASTNode;
 import org.apache.vxquery.xmlquery.ast.ASTTag;
+import org.apache.vxquery.xmlquery.ast.ArgumentListNode;
 import org.apache.vxquery.xmlquery.ast.ArrayConstructor;
 import org.apache.vxquery.xmlquery.ast.AtomicTypeNode;
 import org.apache.vxquery.xmlquery.ast.AttributeTestNode;
@@ -129,7 +130,6 @@ import org.apache.vxquery.xmlquery.ast.ExprNode;
 import org.apache.vxquery.xmlquery.ast.ExtensionExprNode;
 import org.apache.vxquery.xmlquery.ast.FLWORClauseNode;
 import org.apache.vxquery.xmlquery.ast.FLWORExprNode;
-import org.apache.vxquery.xmlquery.ast.FilterExprNode;
 import org.apache.vxquery.xmlquery.ast.ForClauseNode;
 import org.apache.vxquery.xmlquery.ast.ForVarDeclNode;
 import org.apache.vxquery.xmlquery.ast.FunctionDeclNode;
@@ -158,6 +158,7 @@ import org.apache.vxquery.xmlquery.ast.PairConstructor;
 import org.apache.vxquery.xmlquery.ast.ParamNode;
 import org.apache.vxquery.xmlquery.ast.ParenthesizedExprNode;
 import org.apache.vxquery.xmlquery.ast.PathExprNode;
+import org.apache.vxquery.xmlquery.ast.PostfixExprNode;
 import org.apache.vxquery.xmlquery.ast.PrologNode;
 import org.apache.vxquery.xmlquery.ast.QNameNode;
 import org.apache.vxquery.xmlquery.ast.QuantifiedExprNode;
@@ -1536,10 +1537,37 @@ public class XMLQueryTranslator {
                             treat(ctxExpr, SequenceType.create(AnyNodeType.INSTANCE, Quantifier.QUANT_STAR)),
                             ce(SequenceType.create(BuiltinTypeRegistry.XS_INT, Quantifier.QUANT_ONE), ntCode));
                     asc = isForwardAxis(axis);
-                } else if (ASTTag.FILTER_EXPRESSION.equals(pathNode.getTag())) {
-                    FilterExprNode filterNode = (FilterExprNode) pathNode;
-                    predicates = filterNode.getPredicates();
-                    ctxExpr = vre(translateExpression(filterNode.getExpr(), tCtx));
+                } else if (ASTTag.POSTFIX_EXPRESSION.equals(pathNode.getTag())) {
+                    PostfixExprNode postfixNode = (PostfixExprNode) pathNode;
+                    List<ASTNode> args = postfixNode.getArgs();
+                    ILogicalExpression expr = vre(translateExpression(postfixNode.getExpr(), tCtx));
+                    List<ILogicalExpression> arguments = new ArrayList<ILogicalExpression>();
+                    if (args != null && !args.isEmpty()) {
+                        for (ASTNode an : args) {
+                            if (an.getTag() == ASTTag.ARGUMENT_LIST) {
+                                ArgumentListNode argNode = (ArgumentListNode) an;
+                                for (ASTNode en : argNode.getArg()) {
+                                    ILogicalExpression argument = vre(translateExpression(en, tCtx));
+                                    arguments.add(argument);
+                                    //if this is the first argument, then the first parameter in the value
+                                    //is the whole expression,
+                                    //otherwise it is the result of the value function
+                                    if (an == args.get(0)) {
+                                        ctxExpr = sfce(BuiltinOperators.VALUE, expr, argument);
+                                    } else {
+                                        ctxExpr = sfce(BuiltinOperators.VALUE, ctxExpr, argument);
+                                    }
+                                }
+                                if (arguments.size() == 0)
+                                    ctxExpr = expr;
+                            } else {
+                                predicates = postfixNode.getArgs();
+                                ctxExpr = expr;
+                            }
+                        }
+                    } else {
+                        ctxExpr = expr;
+                    }
                 } else {
                     throw new IllegalStateException("Unknown path node: " + pathNode.getTag());
                 }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-core/src/main/javacc/xquery-grammar.jj
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/javacc/xquery-grammar.jj b/vxquery-core/src/main/javacc/xquery-grammar.jj
index 32e3b5b..df92e33 100644
--- a/vxquery-core/src/main/javacc/xquery-grammar.jj
+++ b/vxquery-core/src/main/javacc/xquery-grammar.jj
@@ -1541,7 +1541,7 @@ ASTNode StepExpr():
     ASTNode expr;
 }
 {
-    LOOKAHEAD(3) expr = FilterExpr() {
+    LOOKAHEAD(3) expr = PostfixExpr() {
         return expr;
     }
     | expr = AxisStep() {
@@ -1743,17 +1743,54 @@ NameTestNode Wildcard():
     }
 }
 
-ASTNode FilterExpr()   :
+ASTNode PostfixExpr() :
 {
     ASTNode expr;
-    List<ASTNode> pList;
+    ASTNode arguments;
+    List<ASTNode> argList=new ArrayList<ASTNode>();
 }
 {
-    expr = PrimaryExpr() pList = PredicateList() {
-        FilterExprNode fe = new FilterExprNode(expr.getSourceLocation());
-        fe.setExpr(expr);
-        fe.setPredicates(pList);
-        return fe;
+    expr = PrimaryExpr() (arguments = Predicate() {
+        argList.add(arguments);
+    } | arguments = Arguments() {
+        argList.add(arguments);
+    } )* {
+        PostfixExprNode pe = new PostfixExprNode(expr.getSourceLocation());
+        pe.setExpr(expr);
+        pe.setArgs(argList);
+        return pe;
+    }
+}
+
+ASTNode Arguments() :
+{
+    List<ASTNode> arg;
+    Token start;
+}
+{
+    start= "(" arg = ArgumentList() {
+        ArgumentListNode an = new ArgumentListNode(createSourceLocation(start));
+        an.setArg(arg);
+        return an;
+    }
+}
+
+List<ASTNode> ArgumentList() :
+{
+    ASTNode expr;
+    List<ASTNode> args = new ArrayList<ASTNode>();
+}
+{
+    [
+        expr = ExprSingle() {
+            args.add(expr);
+        } (
+            "," expr = ExprSingle() {
+                args.add(expr);
+            }
+        )*
+    ] ")" {
+        return args;
     }
 }
 
@@ -4084,4 +4121,4 @@ TOKEN :
  < NotNumber : ( (
 "." <Digits>) |  (
 <Digits> ("." (["0" - "9"])*)?)) (["e", "E"] (["+", "-"])? <Digits>)? ["a" - "z", "A" - "Z"] (["0" - "9", "a" - "z", "A" - "Z"])* >
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q01_array_navigation.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q01_array_navigation.txt b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q01_array_navigation.txt
new file mode 100644
index 0000000..d8263ee
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q01_array_navigation.txt
@@ -0,0 +1 @@
+2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q02_array_navigation.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q02_array_navigation.txt b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q02_array_navigation.txt
new file mode 100644
index 0000000..d8263ee
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q02_array_navigation.txt
@@ -0,0 +1 @@
+2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q03_array_navigation.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q03_array_navigation.txt b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q03_array_navigation.txt
new file mode 100644
index 0000000..d8263ee
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q03_array_navigation.txt
@@ -0,0 +1 @@
+2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q04_array_navigation.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q04_array_navigation.txt b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q04_array_navigation.txt
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q05_array_navigation.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q05_array_navigation.txt b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Array/Navigation/q05_array_navigation.txt
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q01_array_navigation.xq
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q01_array_navigation.xq b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q01_array_navigation.xq
new file mode 100644
index 0000000..a45c196
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q01_array_navigation.xq
@@ -0,0 +1,21 @@
+(: 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. :)
+
+(: Json Array Navigation Query :)
+(: Just navigate through an array :)
+    let $x:=[2,5]
+    return $x(1)

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q02_array_navigation.xq
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q02_array_navigation.xq b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q02_array_navigation.xq
new file mode 100644
index 0000000..585bf5e
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q02_array_navigation.xq
@@ -0,0 +1,22 @@
+(: 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. :)
+
+(: Json Array Navigation Query :)
+(: Just navigate through an array :)
+    let $x:=[1,[2]]
+    return $x(2)(1)
+    
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q03_array_navigation.xq
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q03_array_navigation.xq b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q03_array_navigation.xq
new file mode 100644
index 0000000..00afac3
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q03_array_navigation.xq
@@ -0,0 +1,22 @@
+(: 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. :)
+
+(: Json Array Size Query :)
+(: Size of an array :)
+    let $x:=[2,[2,1,3]]
+    return size($x)
+    
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q04_array_navigation.xq
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q04_array_navigation.xq b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q04_array_navigation.xq
new file mode 100644
index 0000000..222c41a
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q04_array_navigation.xq
@@ -0,0 +1,21 @@
+(: 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. :)
+   
+(: Json Array Navigation Query :)
+(: Navigate through an empty array :)
+let $x:=[]
+return $x(1)

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q05_array_navigation.xq
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q05_array_navigation.xq b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q05_array_navigation.xq
new file mode 100644
index 0000000..fcee2b9
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Array/Navigation/q05_array_navigation.xq
@@ -0,0 +1,21 @@
+(: 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. :)
+   
+(: Json Array Navigation Query :)
+(: Ask for an index out of the array size:)
+let $x:=[1]
+return $x(2)

http://git-wip-us.apache.org/repos/asf/vxquery/blob/95c7cd2b/vxquery-xtest/src/test/resources/cat/JsonArrayNavigationQueries.xml
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/cat/JsonArrayNavigationQueries.xml b/vxquery-xtest/src/test/resources/cat/JsonArrayNavigationQueries.xml
new file mode 100644
index 0000000..f40db91
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/cat/JsonArrayNavigationQueries.xml
@@ -0,0 +1,48 @@
+<!--
+  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.
+-->
+
+<test-group xmlns="http://www.w3.org/2005/02/query-test-XQTSCatalog" name="JsonArrayNavigationQueries" featureOwner="VXQuery">
+    <GroupInfo>
+        <title>Json Array Navigation</title>
+        <description/>
+    </GroupInfo>
+    <test-case name="json-array-navigation-q01" FilePath="Json/Array/Navigation" Creator="Christina Pavlopoulou">
+        <description>Navigate through an array.</description>
+        <query name="q01_array_navigation" date="2016-06-16"/>
+        <output-file compare="Text">q01_array_navigation.txt</output-file>
+    </test-case>
+    <test-case name="json-array-navigation-q02" FilePath="Json/Array/Navigation" Creator="Christina Pavlopoulou">
+        <description>Navigate through an array.</description>
+        <query name="q02_array_navigation" date="2016-06-16"/>
+        <output-file compare="Text">q02_array_navigation.txt</output-file>
+    </test-case>
+    <test-case name="json-array-navigation-q03" FilePath="Json/Array/Navigation" Creator="Christina Pavlopoulou">
+        <description>Size of an array.</description>
+        <query name="q03_array_navigation" date="2016-06-16"/>
+        <output-file compare="Text">q03_array_navigation.txt</output-file>
+    </test-case>
+    <test-case name="json-array-navigation-q04" FilePath="Json/Array/Navigation" Creator="Christina Pavlopoulou">
+        <description>Navigate through an empty array.</description>
+        <query name="q04_array_navigation" date="2016-06-18"/>
+        <output-file compare="Text">q04_array_navigation.txt</output-file>
+    </test-case>
+    <test-case name="json-array-navigation-q05" FilePath="Json/Array/Navigation" Creator="Christina Pavlopoulou">
+        <description>Ask for an index out of the array size.</description>
+        <query name="q05_array_navigation" date="2016-06-18"/>
+        <output-file compare="Text">q05_array_navigation.txt</output-file>
+    </test-case>
+</test-group>