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/08/16 15:13:13 UTC

vxquery git commit: Implement libjn:accumulate and libjn:intersect

Repository: vxquery
Updated Branches:
  refs/heads/master 782ab8842 -> 3641b1435


Implement libjn:accumulate and libjn:intersect


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

Branch: refs/heads/master
Commit: 3641b1435862a8d9c6f5b6e65bf5691f1bb97c1f
Parents: 782ab88
Author: riyafa <ri...@cse.mrt.ac.lk>
Authored: Wed Jul 27 14:12:51 2016 +0530
Committer: riyafa <ri...@cse.mrt.ac.lk>
Committed: Sat Aug 13 19:02:26 2016 +0530

----------------------------------------------------------------------
 .../vxquery/functions/builtin-functions.xml     |  14 ++
 .../AbstractLibjnAccumulateScalarEvaluator.java | 166 +++++++++++++++++++
 .../functions/json/JnKeysScalarEvaluator.java   |  16 +-
 .../functions/json/JnNullScalarEvaluator.java   |   2 -
 .../json/LibjnAccumulateScalarEvaluator.java    |  38 +++++
 .../LibjnAccumulateScalarEvaluatorFactory.java  |  39 +++++
 .../json/LibjnIntersectScalarEvaluator.java     |  37 +++++
 .../LibjnIntersectScalarEvaluatorFactory.java   |  39 +++++
 .../runtime/functions/util/FunctionHelper.java  |  15 ++
 .../Json/Libraries/accumulate.txt               |   5 +
 .../Json/Libraries/intersect.txt                |   1 +
 .../Queries/XQuery/Json/Libraries/accumulate.xq |  41 +++++
 .../Queries/XQuery/Json/Libraries/intersect.xq  |  28 ++++
 .../test/resources/cat/LibrariesInJSONiq.xml    |  11 +-
 14 files changed, 434 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/vxquery/blob/3641b143/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 43839f8..9156331 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
@@ -1257,4 +1257,18 @@
         <return type="item()*"/>
         <runtime type="scalar" class="org.apache.vxquery.runtime.functions.json.LibjnRemoveKeysScalarEvaluatorFactory"/>
     </function>
+
+    <!-- libjn:accumulate($sequence as item()*) as object() -->
+    <function name="libjn:accumulate">
+        <param name="sequence" type="item()*"/>
+        <return type="object()"/>
+        <runtime type="scalar" class="org.apache.vxquery.runtime.functions.json.LibjnAccumulateScalarEvaluatorFactory"/>
+    </function>
+
+    <!-- libjn:intersect($sequence as item()*) as object() -->
+    <function name="libjn:intersect">
+        <param name="sequence" type="item()*"/>
+        <return type="object()"/>
+        <runtime type="scalar" class="org.apache.vxquery.runtime.functions.json.LibjnIntersectScalarEvaluatorFactory"/>
+    </function>
 </functions>

http://git-wip-us.apache.org/repos/asf/vxquery/blob/3641b143/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/AbstractLibjnAccumulateScalarEvaluator.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/AbstractLibjnAccumulateScalarEvaluator.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/AbstractLibjnAccumulateScalarEvaluator.java
new file mode 100644
index 0000000..90dc034
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/AbstractLibjnAccumulateScalarEvaluator.java
@@ -0,0 +1,166 @@
+/*
+* 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.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.UTF8StringPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.vxquery.datamodel.accessors.SequencePointable;
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+import org.apache.vxquery.datamodel.accessors.jsonitem.ObjectPointable;
+import org.apache.vxquery.datamodel.builders.jsonitem.ArrayBuilder;
+import org.apache.vxquery.datamodel.builders.jsonitem.ObjectBuilder;
+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.util.FunctionHelper;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public abstract class AbstractLibjnAccumulateScalarEvaluator extends AbstractTaggedValueArgumentScalarEvaluator {
+    protected final IHyracksTaskContext ctx;
+    private final SequencePointable sp, sp1;
+    protected final ObjectBuilder ob;
+    private final ArrayBuilder ab;
+    protected TaggedValuePointable key, value;
+    private final UTF8StringPointable stringKey;
+    private final ArrayBackedValueStorage abvs, abvs1;
+    private final Map<TaggedValuePointable, List<TaggedValuePointable>> tvps;
+    protected List<TaggedValuePointable> values;
+
+    public AbstractLibjnAccumulateScalarEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args) {
+        super(args);
+        this.ctx = ctx;
+        sp = (SequencePointable) SequencePointable.FACTORY.createPointable();
+        sp1 = (SequencePointable) SequencePointable.FACTORY.createPointable();
+        stringKey = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable();
+        abvs = new ArrayBackedValueStorage();
+        abvs1 = new ArrayBackedValueStorage();
+        ob = new ObjectBuilder();
+        ab = new ArrayBuilder();
+        tvps = new LinkedHashMap<>();
+
+    }
+
+    @Override
+    protected void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException {
+        TaggedValuePointable arg = args[0];
+        if (!(arg.getTag() == ValueTag.SEQUENCE_TAG || arg.getTag() == ValueTag.OBJECT_TAG)) {
+            throw new SystemException(ErrorCode.FORG0006);
+        }
+        TaggedValuePointable tempTvp = ppool.takeOne(TaggedValuePointable.class);
+        try {
+            abvs.reset();
+            ob.reset(abvs);
+            tvps.clear();
+            if (arg.getTag() == ValueTag.SEQUENCE_TAG) {
+                arg.getValue(sp);
+                for (int i = 0; i < sp.getEntryCount(); ++i) {
+                    sp.getEntry(i, tempTvp);
+                    if (tempTvp.getTag() == ValueTag.OBJECT_TAG) {
+                        addPairs(tempTvp);
+                    }
+                }
+            } else if (arg.getTag() == ValueTag.OBJECT_TAG) {
+                addPairs(arg);
+            }
+            for (TaggedValuePointable key1 : tvps.keySet()) {
+                key1.getValue(stringKey);
+                values = tvps.get(key1);
+                if (values.size() > 1) {
+                    FunctionHelper.removeDuplicates(values);
+                    if (values.size() > 1) {
+                        abvs1.reset();
+                        ab.reset(abvs1);
+                        for (TaggedValuePointable pointable : values) {
+                            ab.addItem(pointable);
+                        }
+                        ab.finish();
+                        ob.addItem(stringKey, abvs1);
+                    } else {
+                        ob.addItem(stringKey, values.get(0));
+                    }
+                } else {
+                    addKeyValue(stringKey, values.get(0));
+                }
+            }
+            ob.finish();
+            result.set(abvs);
+        } catch (IOException e) {
+            throw new SystemException(ErrorCode.SYSE0001, e);
+        } finally {
+            ppool.giveBack(tempTvp);
+            for (TaggedValuePointable key1 : tvps.keySet()) {
+                for (TaggedValuePointable value1 : tvps.get(key1)) {
+                    ppool.giveBack(value1);
+                }
+                ppool.giveBack(key1);
+            }
+        }
+    }
+
+    protected abstract void addKeyValue(UTF8StringPointable stringKey, TaggedValuePointable taggedValuePointable)
+            throws IOException;
+
+    private void addPair(TaggedValuePointable tvp1, ObjectPointable op) throws IOException, SystemException {
+        TaggedValuePointable tvp = isDuplicateKeys(tvp1, tvps.keySet());
+        value = ppool.takeOne(TaggedValuePointable.class);
+        tvp1.getValue(stringKey);
+        op.getValue(stringKey, value);
+        if (tvp == null) {
+            key = ppool.takeOne(TaggedValuePointable.class);
+            key.set(tvp1);
+            values = new ArrayList<>();
+            values.add(value);
+            tvps.put(key, values);
+        } else {
+            tvps.get(tvp).add(value);
+        }
+    }
+
+    private void addPairs(TaggedValuePointable tvp1) throws IOException, SystemException {
+        ObjectPointable op = (ObjectPointable) ObjectPointable.FACTORY.createPointable();
+        tvp1.getValue(op);
+        op.getKeys(tvp1);
+        if (tvp1.getTag() == ValueTag.XS_STRING_TAG) {
+            addPair(tvp1, op);
+        } else if (tvp1.getTag() == ValueTag.SEQUENCE_TAG) {
+            tvp1.getValue(sp1);
+            for (int j = 0; j < sp1.getEntryCount(); ++j) {
+                sp1.getEntry(j, tvp1);
+                addPair(tvp1, op);
+            }
+        }
+    }
+
+    private TaggedValuePointable isDuplicateKeys(TaggedValuePointable key, Set<TaggedValuePointable> pointables) {
+        for (TaggedValuePointable tvp : pointables) {
+            if (tvp != null && FunctionHelper.arraysEqual(tvp, key)) {
+                return tvp;
+            }
+        }
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/3641b143/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnKeysScalarEvaluator.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnKeysScalarEvaluator.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnKeysScalarEvaluator.java
index 4b093eb..c90472e 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnKeysScalarEvaluator.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnKeysScalarEvaluator.java
@@ -79,7 +79,7 @@ public class JnKeysScalarEvaluator extends AbstractTaggedValueArgumentScalarEval
                         }
                     }
                 }
-                removeDuplicates(pointables);
+                FunctionHelper.removeDuplicates(pointables);
                 abvs.reset();
                 sb.reset(abvs);
                 for (TaggedValuePointable tvp : pointables) {
@@ -104,18 +104,4 @@ public class JnKeysScalarEvaluator extends AbstractTaggedValueArgumentScalarEval
             XDMConstants.setEmptySequence(result);
         }
     }
-
-    private void removeDuplicates(List<TaggedValuePointable> pointables) {
-        int size = pointables.size();
-        for (int i = 0; i < size - 1; i++) {
-            for (int j = i + 1; j < size; j++) {
-                if (!FunctionHelper.arraysEqual(pointables.get(j), pointables.get(i))) {
-                    continue;
-                }
-                pointables.remove(j);
-                j--;
-                size--;
-            }
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/3641b143/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnNullScalarEvaluator.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnNullScalarEvaluator.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnNullScalarEvaluator.java
index 772f30a..3137b67 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnNullScalarEvaluator.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/JnNullScalarEvaluator.java
@@ -27,12 +27,10 @@ import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScal
 
 public class JnNullScalarEvaluator extends AbstractTaggedValueArgumentScalarEvaluator {
     protected final IHyracksTaskContext ctx;
-    private final ArrayBackedValueStorage abvs;
 
     public JnNullScalarEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args) {
         super(args);
         this.ctx = ctx;
-        abvs = new ArrayBackedValueStorage();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/vxquery/blob/3641b143/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/LibjnAccumulateScalarEvaluator.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/LibjnAccumulateScalarEvaluator.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/LibjnAccumulateScalarEvaluator.java
new file mode 100644
index 0000000..378c53d
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/LibjnAccumulateScalarEvaluator.java
@@ -0,0 +1,38 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.vxquery.runtime.functions.json;
+
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+
+import java.io.IOException;
+
+public class LibjnAccumulateScalarEvaluator extends AbstractLibjnAccumulateScalarEvaluator {
+
+    public LibjnAccumulateScalarEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args) {
+        super(ctx, args);
+    }
+
+    @Override
+    protected void addKeyValue(UTF8StringPointable stringKey, TaggedValuePointable taggedValuePointable)
+            throws IOException {
+        ob.addItem(stringKey, values.get(0));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/3641b143/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/LibjnAccumulateScalarEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/LibjnAccumulateScalarEvaluatorFactory.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/LibjnAccumulateScalarEvaluatorFactory.java
new file mode 100644
index 0000000..fbbb997
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/LibjnAccumulateScalarEvaluatorFactory.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 LibjnAccumulateScalarEvaluatorFactory extends AbstractTaggedValueArgumentScalarEvaluatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public LibjnAccumulateScalarEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        super(args);
+    }
+
+    @Override
+    protected IScalarEvaluator createEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args)
+            throws AlgebricksException {
+        return new LibjnAccumulateScalarEvaluator(ctx, args);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/3641b143/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/LibjnIntersectScalarEvaluator.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/LibjnIntersectScalarEvaluator.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/LibjnIntersectScalarEvaluator.java
new file mode 100644
index 0000000..b8fcaa3
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/LibjnIntersectScalarEvaluator.java
@@ -0,0 +1,37 @@
+/*
+* 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.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+
+import java.io.IOException;
+
+public class LibjnIntersectScalarEvaluator extends AbstractLibjnAccumulateScalarEvaluator {
+
+    public LibjnIntersectScalarEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args) {
+        super(ctx, args);
+    }
+
+    @Override
+    protected void addKeyValue(UTF8StringPointable stringKey, TaggedValuePointable taggedValuePointable)
+            throws IOException {
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/3641b143/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/LibjnIntersectScalarEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/LibjnIntersectScalarEvaluatorFactory.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/LibjnIntersectScalarEvaluatorFactory.java
new file mode 100644
index 0000000..a7d7b91
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/json/LibjnIntersectScalarEvaluatorFactory.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 LibjnIntersectScalarEvaluatorFactory extends AbstractTaggedValueArgumentScalarEvaluatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public LibjnIntersectScalarEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        super(args);
+    }
+
+    @Override
+    protected IScalarEvaluator createEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args)
+            throws AlgebricksException {
+        return new LibjnIntersectScalarEvaluator(ctx, args);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/3641b143/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 0b3ea28..4176b20 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
@@ -28,6 +28,7 @@ import java.io.InputStreamReader;
 import java.io.Reader;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
+import java.util.List;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -1425,4 +1426,18 @@ public class FunctionHelper {
         }
     }
 
+    public static void removeDuplicates(List<TaggedValuePointable> pointables) {
+        int size = pointables.size();
+        for (int i = 0; i < size - 1; i++) {
+            for (int j = i + 1; j < size; j++) {
+                if (!FunctionHelper.arraysEqual(pointables.get(j), pointables.get(i))) {
+                    continue;
+                }
+                pointables.remove(j);
+                j--;
+                size--;
+            }
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/3641b143/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Libraries/accumulate.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Libraries/accumulate.txt b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Libraries/accumulate.txt
new file mode 100644
index 0000000..2ac05a3
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Libraries/accumulate.txt
@@ -0,0 +1,5 @@
+{"Sunday":1,"Monday":2,"Tuesday":3,"Wednesday":4,"Thursday":5,"Friday":6,"Saturday":7}
+{"a":"b","c":"d"}
+{"a":["b","d"]}
+{"foo":[{"bar":[1,2]},"bar"],"bar":"foo","coa":123,"456":[789,"number"]}
+{"a":[{"c":"d"},{"e":"f"}]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/3641b143/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Libraries/intersect.txt
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Libraries/intersect.txt b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Libraries/intersect.txt
new file mode 100644
index 0000000..ce52d9a
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/ExpectedTestResults/Json/Libraries/intersect.txt
@@ -0,0 +1 @@
+{"foo":[{"bar":[1,2]},"bar"],"coa":123,"456":[789,"number"]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/vxquery/blob/3641b143/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Libraries/accumulate.xq
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Libraries/accumulate.xq b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Libraries/accumulate.xq
new file mode 100644
index 0000000..dfaaa8f
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Libraries/accumulate.xq
@@ -0,0 +1,41 @@
+(: 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. :)
+
+(: JSONiq libjn:accumulate :)
+libjn:accumulate(
+        for $d at $i in ("Sunday",
+        "Monday",
+        "Tuesday",
+        "Wednesday",
+        "Thursday",
+        "Friday",
+        "Saturday" )
+        return { $d : $i }
+),
+libjn:accumulate(({ "a": "b" }, { "c": "d" })),
+libjn:accumulate(({ "a": "b" }, { "a": "d" })),
+libjn:accumulate(
+        (
+                { "foo" : { "bar" : [ 1, 2 ] } },
+                [ 1, 2, 3, 4 ],
+                { "foo" : "bar", "bar" : "foo", "coa":123 , 456: 789},
+                {"foo" : "bar", "coa" :123, 456 :"number"},
+                1,
+                jn:null()
+        )
+),
+libjn:accumulate(({ "a": { "c": "d" } }, { "a": { "e": "f" } }))

http://git-wip-us.apache.org/repos/asf/vxquery/blob/3641b143/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Libraries/intersect.xq
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Libraries/intersect.xq b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Libraries/intersect.xq
new file mode 100644
index 0000000..2b48b61
--- /dev/null
+++ b/vxquery-xtest/src/test/resources/Queries/XQuery/Json/Libraries/intersect.xq
@@ -0,0 +1,28 @@
+(: 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. :)
+
+(: JSONiq libjn:intersect :)
+libjn:intersect(
+    (
+        { "foo" : { "bar" : [ 1, 2 ] } },
+        [ 1, 2, 3, 4 ],
+        { "foo" : "bar", "bar" : "foo", "coa":123 , 456: 789},
+        {"foo" : "bar", "coa" :123, 456 :"number"},
+        1,
+        jn:null()
+    )
+)

http://git-wip-us.apache.org/repos/asf/vxquery/blob/3641b143/vxquery-xtest/src/test/resources/cat/LibrariesInJSONiq.xml
----------------------------------------------------------------------
diff --git a/vxquery-xtest/src/test/resources/cat/LibrariesInJSONiq.xml b/vxquery-xtest/src/test/resources/cat/LibrariesInJSONiq.xml
index 66cab65..a656aba 100644
--- a/vxquery-xtest/src/test/resources/cat/LibrariesInJSONiq.xml
+++ b/vxquery-xtest/src/test/resources/cat/LibrariesInJSONiq.xml
@@ -40,7 +40,6 @@
       <query name="flatten" date="2016-07-20"/>
       <output-file compare="Text">flatten.txt</output-file>
    </test-case>
-
     <test-case name="values" FilePath="Json/Libraries/" Creator="Riyafa Abdul Hameed">
         <description>Json Libraries.</description>
         <query name="values" date="2016-07-31"/>
@@ -66,4 +65,14 @@
       <query name="remove_keys" date="2016-07-31"/>
       <output-file compare="Text">remove_keys.txt</output-file>
    </test-case>
+   <test-case name="accumulate" FilePath="Json/Libraries/" Creator="Riyafa Abdul Hameed">
+      <description>Json Libraries.</description>
+      <query name="accumulate" date="2016-07-27"/>
+      <output-file compare="Text">accumulate.txt</output-file>
+   </test-case>
+   <test-case name="intersect" FilePath="Json/Libraries/" Creator="Riyafa Abdul Hameed">
+      <description>Json Libraries.</description>
+      <query name="intersect" date="2016-07-30"/>
+      <output-file compare="Text">intersect.txt</output-file>
+   </test-case>
 </test-group>