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/07/12 20:55:17 UTC

vxquery git commit: Implementing jn:parse-json and testing

Repository: vxquery
Updated Branches:
  refs/heads/master 1acfe1fec -> b96caaa0f


Implementing jn:parse-json and testing


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

Branch: refs/heads/master
Commit: b96caaa0f01bcfde59d9e85c7d2e8548031ea268
Parents: 1acfe1f
Author: Christina Pavlopoulou <cp...@ucr.edu>
Authored: Sat Jul 9 11:34:08 2016 -0700
Committer: Christina Pavlopoulou <cp...@ucr.edu>
Committed: Tue Jul 12 13:25:43 2016 -0700

----------------------------------------------------------------------
 .../vxquery/functions/builtin-functions.xml     |  15 +++
 .../apache/vxquery/jsonparser/JSONParser.java   |  40 +++---
 .../json/JnParseJsonScalarEvaluatorFactory.java | 126 +++++++++++++++++++
 .../runtime/functions/util/FunctionHelper.java  |  21 +++-
 .../org/apache/vxquery/xmlparser/IParser.java   |   6 +-
 .../org/apache/vxquery/xmlparser/XMLParser.java |  37 +-----
 .../Json/Parser/q02_parser.txt                  |   1 -
 .../Json/Parser/q03_parser.txt                  |   1 -
 .../Json/Parser/q04_parser.txt                  |   1 -
 .../Json/Parser/q10_parser.txt                  |   3 +
 .../Json/Parser/q11_parser.txt                  |   3 +
 .../Json/Parser/q12_parser.txt                  |   1 +
 .../Queries/XQuery/Json/Parser/q02_parser.xq    |  20 ---
 .../Queries/XQuery/Json/Parser/q03_parser.xq    |  20 ---
 .../Queries/XQuery/Json/Parser/q04_parser.xq    |  20 ---
 .../Queries/XQuery/Json/Parser/q10_parser.xq    |  21 ++++
 .../Queries/XQuery/Json/Parser/q11_parser.xq    |  22 ++++
 .../Queries/XQuery/Json/Parser/q12_parser.xq    |  22 ++++
 .../Queries/XQuery/Json/Parser/q13_parser.xq    |  22 ++++
 .../test/resources/cat/JsonParserQueries.xml    |  35 +++---
 20 files changed, 300 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/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 60d57dd..0e64d46 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
@@ -330,6 +330,21 @@
         <return type="json-item()?"/>
         <runtime type="scalar" class="org.apache.vxquery.runtime.functions.json.JnDocScalarEvaluatorFactory"/>
     </function>
+    
+    <!-- jn:parse-json($arg as  xs:string?) as json-item()*  -->
+    <function name="jn:parse-json">
+        <param name="arg" type="xs:string?"/>
+        <return type="json-item()*"/>
+        <runtime type="scalar" class="org.apache.vxquery.runtime.functions.json.JnParseJsonScalarEvaluatorFactory"/>
+    </function>
+    
+    <!-- jn:parse-json($arg as  xs:string?, $options as object()) as json-item()*  -->
+    <function name="jn:parse-json">
+        <param name="arg" type="xs:string?"/>
+        <param name="options" type="item()"/>
+        <return type="json-item()*"/>
+        <runtime type="scalar" class="org.apache.vxquery.runtime.functions.json.JnParseJsonScalarEvaluatorFactory"/>
+    </function>
 
     <!-- fn:doc-available($uri  as xs:string?) as  xs:boolean -->
     <function name="fn:doc-available">

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-core/src/main/java/org/apache/vxquery/jsonparser/JSONParser.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/jsonparser/JSONParser.java b/vxquery-core/src/main/java/org/apache/vxquery/jsonparser/JSONParser.java
index 9e8bffe..71220cf 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/jsonparser/JSONParser.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/jsonparser/JSONParser.java
@@ -15,9 +15,8 @@
 package org.apache.vxquery.jsonparser;
 
 import java.io.DataOutput;
-import java.io.File;
 import java.io.IOException;
-import java.io.InputStream;
+import java.io.Reader;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -31,6 +30,7 @@ import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.vxquery.datamodel.builders.atomic.StringValueBuilder;
 import org.apache.vxquery.datamodel.builders.jsonitem.ArrayBuilder;
 import org.apache.vxquery.datamodel.builders.jsonitem.ObjectBuilder;
+import org.apache.vxquery.datamodel.builders.sequence.SequenceBuilder;
 import org.apache.vxquery.datamodel.values.ValueTag;
 import org.apache.vxquery.xmlparser.IParser;
 
@@ -44,8 +44,9 @@ public class JSONParser implements IParser {
     protected final List<ArrayBackedValueStorage> keyStack;
     protected final List<UTF8StringPointable> spStack;
     protected final StringValueBuilder svb;
+    protected final SequenceBuilder sb;
     protected final DataOutput out;
-    protected itemType checkItem, startItem;
+    protected itemType checkItem;
     protected int levelArray, levelObject;
 
     enum itemType {
@@ -65,20 +66,23 @@ public class JSONParser implements IParser {
         spStack = new ArrayList<UTF8StringPointable>();
         itemStack = new ArrayList<itemType>();
         svb = new StringValueBuilder();
+        sb = new SequenceBuilder();
         abvsStack.add(atomic);
         out = abvsStack.get(abvsStack.size() - 1).getDataOutput();
 
     }
 
-    public void parseDocument(File file, ArrayBackedValueStorage result) throws HyracksDataException {
+    public int parse(Reader input, ArrayBackedValueStorage result) throws HyracksDataException {
+        int items = 0;
         try {
             DataOutput outResult = result.getDataOutput();
-            JsonParser parser = factory.createParser(file);
+            JsonParser parser = factory.createParser(input);
             JsonToken token = parser.nextToken();
             checkItem = null;
-            startItem = null;
+
             levelArray = 0;
             levelObject = 0;
+            sb.reset(result);
             while (token != null) {
                 if (itemStack.size() > 1) {
                     checkItem = itemStack.get(itemStack.size() - 2);
@@ -138,8 +142,11 @@ public class JSONParser implements IParser {
                             }
                         }
                         itemStack.remove(itemStack.size() - 1);
-                        startItem = itemType.ARRAY;
                         levelArray--;
+                        if (levelArray + levelObject == 0) {
+                            sb.addItem(abvsStack.get(1));
+                            items++;
+                        }
                         break;
                     case END_OBJECT:
                         obStack.get(levelObject - 1).finish();
@@ -152,23 +159,23 @@ public class JSONParser implements IParser {
                             }
                         }
                         itemStack.remove(itemStack.size() - 1);
-                        startItem = itemType.OBJECT;
                         levelObject--;
+                        if (levelObject + levelArray == 0) {
+                            sb.addItem(abvsStack.get(1));
+                            items++;
+                        }
                         break;
                     default:
                         break;
                 }
                 token = parser.nextToken();
             }
-            if (startItem == itemType.ARRAY || startItem == itemType.OBJECT) {
-                outResult.write(abvsStack.get(1).getByteArray());
-            } else {
-                //the atomic value is always set to be at the bottom of the arraybackedvaluestorage stack.
-                outResult.write(abvsStack.get(0).getByteArray());
-            }
+            sb.finish();
+            outResult.write(result.getByteArray());
         } catch (Exception e) {
             throw new HyracksDataException(e.toString());
         }
+        return items;
     }
 
     public void atomicValues(int tag, JsonParser parser, DataOutput out, StringValueBuilder svb, int levelArray,
@@ -190,9 +197,4 @@ public class JSONParser implements IParser {
             }
         }
     }
-
-    @Override
-    public void parseHDFSDocument(InputStream in, ArrayBackedValueStorage abvs) throws HyracksDataException {
-
-    }
 }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnParseJsonScalarEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnParseJsonScalarEvaluatorFactory.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnParseJsonScalarEvaluatorFactory.java
new file mode 100644
index 0000000..8c7951c
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnParseJsonScalarEvaluatorFactory.java
@@ -0,0 +1,126 @@
+/*
+ * 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.BufferedReader;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+
+import org.apache.commons.io.IOUtils;
+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.primitive.BooleanPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+import org.apache.vxquery.datamodel.accessors.jsonitem.ObjectPointable;
+import org.apache.vxquery.datamodel.values.ValueTag;
+import org.apache.vxquery.exceptions.ErrorCode;
+import org.apache.vxquery.exceptions.SystemException;
+import org.apache.vxquery.jsonparser.JSONParser;
+import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluator;
+import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluatorFactory;
+import org.apache.vxquery.runtime.functions.util.FunctionHelper;
+import org.apache.vxquery.xmlparser.IParser;
+
+public class JnParseJsonScalarEvaluatorFactory extends AbstractTaggedValueArgumentScalarEvaluatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public JnParseJsonScalarEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        super(args);
+    }
+
+    @Override
+    protected IScalarEvaluator createEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args)
+            throws AlgebricksException {
+        final UTF8StringPointable stringp = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable();
+        final UTF8StringPointable stringp2 = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable();
+        final ObjectPointable op = (ObjectPointable) ObjectPointable.FACTORY.createPointable();
+        final BooleanPointable bp = (BooleanPointable) BooleanPointable.FACTORY.createPointable();
+        final ByteBufferInputStream bbis = new ByteBufferInputStream();
+        final DataInputStream di = new DataInputStream(bbis);
+        final ArrayBackedValueStorage abvs = new ArrayBackedValueStorage();
+
+        return new AbstractTaggedValueArgumentScalarEvaluator(args) {
+
+            @Override
+            protected void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException {
+                TaggedValuePointable tvp = args[0];
+                if (tvp.getTag() != ValueTag.XS_STRING_TAG) {
+                    throw new SystemException(ErrorCode.FORG0006);
+                }
+                tvp.getValue(stringp);
+                if (args.length == 1) {
+                    callParser();
+                    result.set(abvs);
+                }
+                if (args.length > 1) {
+                    TaggedValuePointable tvp1 = args[1];
+                    if (tvp1.getTag() != ValueTag.OBJECT_TAG) {
+                        throw new SystemException(ErrorCode.FORG0006);
+                    }
+                    tvp1.getValue(op);
+                    TaggedValuePointable tempTvp = ppool.takeOne(TaggedValuePointable.class);
+                    try {
+                        op.getKeys(tvp1);
+                        tvp1.getValue(stringp2);
+                        op.getValue(stringp2, tempTvp);
+                    } catch (IOException e1) {
+                        throw new SystemException(ErrorCode.SYSE0001, e1);
+                    }
+                    if (tempTvp.getTag() != ValueTag.XS_BOOLEAN_TAG) {
+                        throw new SystemException(ErrorCode.JNTY0020);
+                    }
+                    tempTvp.getValue(bp);
+                    if (bp.getBoolean() == true) {
+                        callParser();
+                        result.set(abvs);
+                    } else {
+                        int items = callParser();
+                        if (items > 1) {
+                            throw new SystemException(ErrorCode.JNDY0021);
+                        } else {
+                            result.set(abvs);
+                        }
+                    }
+                    ppool.giveBack(tempTvp);
+                }
+            }
+
+            public int callParser() throws SystemException {
+                int items = 0;
+                try {
+                    IParser parser = new JSONParser();
+                    String input = FunctionHelper.getStringFromPointable(stringp, bbis, di);
+                    InputStreamReader isr = new InputStreamReader(IOUtils.toInputStream(input));
+                    items = parser.parse(new BufferedReader(isr), abvs);
+                } catch (IOException e) {
+                    throw new SystemException(ErrorCode.FODC0002, e);
+                }
+                return items;
+            }
+
+        };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/util/FunctionHelper.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/util/FunctionHelper.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/util/FunctionHelper.java
index 8aa7696..6d280f2 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/util/FunctionHelper.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/util/FunctionHelper.java
@@ -16,11 +16,15 @@
  */
 package org.apache.vxquery.runtime.functions.util;
 
+import java.io.BufferedReader;
 import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 
@@ -1232,10 +1236,18 @@ public class FunctionHelper {
     public static void readInDocFromString(String fName, ByteBufferInputStream bbis, DataInputStream di,
             ArrayBackedValueStorage abvs, IParser parser)
                     throws NumberFormatException, JsonParseException, IOException {
+        int bufferSize = Integer.parseInt(System.getProperty("vxquery.buffer_size", "-1"));
+        Reader input;
         if (!fName.contains("hdfs:/")) {
             File file = new File(fName);
+
             if (file.exists()) {
-                parser.parseDocument(file, abvs);
+                if (bufferSize > 0) {
+                    input = new BufferedReader(new InputStreamReader(new FileInputStream(file)), bufferSize);
+                } else {
+                    input = new InputStreamReader(new FileInputStream(file));
+                }
+                parser.parse(input, abvs);
             }
         }
         //else check in HDFS file system
@@ -1248,7 +1260,12 @@ public class FunctionHelper {
                     Path xmlDocument = new Path(fHdfsName);
                     if (fs.exists(xmlDocument)) {
                         InputStream in = fs.open(xmlDocument).getWrappedStream();
-                        parser.parseHDFSDocument(in, abvs);
+                        if (bufferSize > 0) {
+                            input = new BufferedReader(new InputStreamReader(in), bufferSize);
+                        } else {
+                            input = new InputStreamReader(in);
+                        }
+                        parser.parse(input, abvs);
                         in.close();
                     }
                     fs.close();

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/IParser.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/IParser.java b/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/IParser.java
index 7397f03..bb62339 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/IParser.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/IParser.java
@@ -16,14 +16,12 @@
  */
 package org.apache.vxquery.xmlparser;
 
-import java.io.File;
-import java.io.InputStream;
+import java.io.Reader;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 
 public interface IParser {
-    public void parseDocument(File file, ArrayBackedValueStorage abvs) throws HyracksDataException;
+    public int parse(Reader input, ArrayBackedValueStorage abvs) throws HyracksDataException;
 
-    public void parseHDFSDocument(InputStream in, ArrayBackedValueStorage abvs) throws HyracksDataException;
 }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/XMLParser.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/XMLParser.java b/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/XMLParser.java
index d11e1c1..2f9f8c5 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/XMLParser.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/xmlparser/XMLParser.java
@@ -73,31 +73,18 @@ public class XMLParser implements IParser {
         }
     }
 
-    public void parseDocument(File file, ArrayBackedValueStorage abvs) throws HyracksDataException {
+    public int parse(Reader input, ArrayBackedValueStorage abvs) throws HyracksDataException {
         try {
-            Reader input;
-            if (bufferSize > 0) {
-                input = new BufferedReader(new InputStreamReader(new FileInputStream(file)), bufferSize);
-            } else {
-                input = new InputStreamReader(new FileInputStream(file));
-            }
             in.setCharacterStream(input);
             parser.parse(in);
             handler.writeDocument(abvs);
             input.close();
-        } catch (FileNotFoundException e) {
-            HyracksDataException hde = new VXQueryFileNotFoundException(e, file);
-            hde.setNodeId(nodeId);
-            throw hde;
-        } catch (SAXException e) {
-            HyracksDataException hde = new VXQueryParseException(e, file);
-            hde.setNodeId(nodeId);
-            throw hde;
-        } catch (IOException e) {
+        } catch (Exception e) {
             HyracksDataException hde = new HyracksDataException(e);
             hde.setNodeId(nodeId);
             throw hde;
         }
+        return 0;
     }
 
     public void parseElements(File file, IFrameWriter writer, int tupleIndex) throws HyracksDataException {
@@ -147,22 +134,4 @@ public class XMLParser implements IParser {
         }
     }
 
-    public void parseHDFSDocument(InputStream inputStream, ArrayBackedValueStorage abvs) throws HyracksDataException {
-        try {
-            Reader input;
-            if (bufferSize > 0) {
-                input = new BufferedReader(new InputStreamReader(inputStream), bufferSize);
-            } else {
-                input = new InputStreamReader(inputStream);
-            }
-            in.setCharacterStream(input);
-            parser.parse(in);
-            handler.writeDocument(abvs);
-            input.close();
-        } catch (Exception e) {
-            HyracksDataException hde = new HyracksDataException(e);
-            hde.setNodeId(nodeId);
-            throw hde;
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q02_parser.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q02_parser.txt b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q02_parser.txt
deleted file mode 100644
index d00491f..0000000
--- a/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q02_parser.txt
+++ /dev/null
@@ -1 +0,0 @@
-1

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q03_parser.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q03_parser.txt b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q03_parser.txt
deleted file mode 100644
index 5625e59..0000000
--- a/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q03_parser.txt
+++ /dev/null
@@ -1 +0,0 @@
-1.2

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q04_parser.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q04_parser.txt b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q04_parser.txt
deleted file mode 100644
index ec186f1..0000000
--- a/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q04_parser.txt
+++ /dev/null
@@ -1 +0,0 @@
-string
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q10_parser.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q10_parser.txt b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q10_parser.txt
new file mode 100644
index 0000000..c87f444
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q10_parser.txt
@@ -0,0 +1,3 @@
+{"foo":"bar"}
+{"bar":"foo"}
+[1,2,3]

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q11_parser.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q11_parser.txt b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q11_parser.txt
new file mode 100644
index 0000000..c87f444
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q11_parser.txt
@@ -0,0 +1,3 @@
+{"foo":"bar"}
+{"bar":"foo"}
+[1,2,3]

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q12_parser.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q12_parser.txt b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q12_parser.txt
new file mode 100644
index 0000000..2393cd0
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Parser/q12_parser.txt
@@ -0,0 +1 @@
+{"foo":"bar"}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q02_parser.xq
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q02_parser.xq b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q02_parser.xq
deleted file mode 100644
index 9d73500..0000000
--- a/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q02_parser.xq
+++ /dev/null
@@ -1,20 +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. :)
-
-(: Json Parser Query :)
-(: parse a file containing an atomic int :)
-    jn:json-doc("int_json_file")

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q03_parser.xq
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q03_parser.xq b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q03_parser.xq
deleted file mode 100644
index 2ba7384..0000000
--- a/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q03_parser.xq
+++ /dev/null
@@ -1,20 +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. :)
-
-(: Json Parser Query :)
-(: parse a file containing a double :)
-    jn:json-doc("double_json_file")

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q04_parser.xq
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q04_parser.xq b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q04_parser.xq
deleted file mode 100644
index 6f15917..0000000
--- a/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q04_parser.xq
+++ /dev/null
@@ -1,20 +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. :)
-
-(: Json Parser Query :)
-(: parse a file containing a string :)
-    jn:json-doc("string_json_file")

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q10_parser.xq
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q10_parser.xq b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q10_parser.xq
new file mode 100644
index 0000000..cf58429
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q10_parser.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 Parser Query :)
+(: parse a string with arrays :)
+let $x:="{&quot;foo&quot;:&quot;bar&quot;} {&quot;bar&quot;:&quot;foo&quot;} [1,2,3]"
+return jn:parse-json($x)

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q11_parser.xq
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q11_parser.xq b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q11_parser.xq
new file mode 100644
index 0000000..75a3079
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q11_parser.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 Parser Query :)
+(: parse a string with arrays :)
+let $x:="{&quot;foo&quot;:&quot;bar&quot;} {&quot;bar&quot;:&quot;foo&quot;} [1,2,3]"
+let $y:={"jsoniq-multiple-top-level-items":xs:boolean("true")}
+return jn:parse-json($x,$y)

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q12_parser.xq
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q12_parser.xq b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q12_parser.xq
new file mode 100644
index 0000000..2d94a1d
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q12_parser.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 Parser Query :)
+(: parse a string with arrays :)
+let $x:="{&quot;foo&quot;:&quot;bar&quot;}"
+let $y:={"jsoniq-multiple-top-level-items":xs:boolean("false")}
+return jn:parse-json($x,$y)

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q13_parser.xq
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q13_parser.xq b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q13_parser.xq
new file mode 100644
index 0000000..f4a8d31
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Parser/q13_parser.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 Parser Query :)
+(: parse a string with arrays :)
+let $x:="{&quot;foo&quot;:&quot;bar&quot;} [1]"
+let $y:={"jsoniq-multiple-top-level-items":xs:boolean("false")}
+return jn:parse-json($x,$y)

http://git-wip-us.apache.org/repos/asf/vxquery/blob/b96caaa0/vxquery-xtest/src/test/resources/cat/JsonParserQueries.xml
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/cat/JsonParserQueries.xml b/vxquery-xtest/src/test/resources/cat/JsonParserQueries.xml
index d61a1e8..74f7b00 100644
--- a/vxquery-xtest/src/test/resources/cat/JsonParserQueries.xml
+++ b/vxquery-xtest/src/test/resources/cat/JsonParserQueries.xml
@@ -25,21 +25,6 @@
       <query name="q01_parser" date="2016-06-28"/>
       <output-file compare="Text">q01_parser.txt</output-file>
    </test-case>
-   <test-case name="json-parser-q02" FilePath="Json/Parser/" Creator="Christina Pavlopoulou">
-      <description>Parsing atomic int from json file.</description>
-      <query name="q02_parser" date="2016-06-28"/>
-      <output-file compare="Text">q02_parser.txt</output-file>
-   </test-case>
-   <test-case name="json-parser-q03" FilePath="Json/Parser/" Creator="Christina Pavlopoulou">
-      <description>Parsing atomic double from a json file.</description>
-      <query name="q03_parser" date="2016-06-28"/>
-      <output-file compare="Text">q03_parser.txt</output-file>
-   </test-case>
-   <test-case name="json-parser-q04" FilePath="Json/Parser/" Creator="Christina Pavlopoulou">
-      <description>Parsing atomic string from a json file.</description>
-      <query name="q04_parser" date="2016-06-28"/>
-      <output-file compare="Text">q04_parser.txt</output-file>
-   </test-case>
    <test-case name="json-parser-q05" FilePath="Json/Parser/" Creator="Christina Pavlopoulou">
       <description>Parsing nested arrays from a json file.</description>
       <query name="q05_parser" date="2016-06-29"/>
@@ -65,4 +50,24 @@
       <query name="q09_parser" date="2016-07-05"/>
       <output-file compare="Text">q09_parser.txt</output-file>
    </test-case>
+   <test-case name="json-parser-q10" FilePath="Json/Parser/" Creator="Christina Pavlopoulou">
+      <description>Parsing arrays and objects from a json string.</description>
+      <query name="q10_parser" date="2016-07-07"/>
+      <output-file compare="Text">q10_parser.txt</output-file>
+   </test-case>
+   <test-case name="json-parser-q11" FilePath="Json/Parser/" Creator="Christina Pavlopoulou">
+      <description>Parsing arrays and objects from a json string.</description>
+      <query name="q11_parser" date="2016-07-09"/>
+      <output-file compare="Text">q11_parser.txt</output-file>
+   </test-case>
+   <test-case name="json-parser-q12" FilePath="Json/Parser/" Creator="Christina Pavlopoulou">
+      <description>Parsing arrays and objects from a json string.</description>
+      <query name="q12_parser" date="2016-07-10"/>
+      <output-file compare="Text">q12_parser.txt</output-file>
+   </test-case>
+    <test-case name="json-parser-q13" FilePath="Json/Parser/" Creator="Christina Pavlopoulou">
+      <description>Parsing arrays and objects from a json string.</description>
+      <query name="q13_parser" date="2016-07-12"/>
+      <expected-error>JNDY0021</expected-error>
+   </test-case>
 </test-group>