You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by se...@apache.org on 2015/07/20 13:39:17 UTC

[07/21] incubator-ignite git commit: # ignite-962 simple json object implementation

# ignite-962 simple json object implementation


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

Branch: refs/heads/ignite-1121
Commit: e6ba441dc4c16464c4fa6f8af26edcdd4f72607f
Parents: c7e7c46
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jul 17 17:10:52 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jul 17 17:10:52 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/GridKernalContext.java      |   6 +
 .../ignite/internal/GridKernalContextImpl.java  |  14 +-
 .../ignite/internal/IgniteComponentType.java    |   8 +
 .../apache/ignite/internal/IgniteKernal.java    |   1 +
 .../IgniteCacheObjectProcessorImpl.java         |  10 +
 .../json/IgniteJsonNoopProcessor.java           |  67 +++
 .../processors/json/IgniteJsonProcessor.java    |  71 +++
 .../processors/query/GridQueryProcessor.java    | 235 ++++++++-
 .../org/apache/ignite/JsonArrayBuilderImpl.java | 130 -----
 .../java/org/apache/ignite/JsonArrayImpl.java   | 131 -----
 .../org/apache/ignite/JsonGeneratorImpl.java    | 500 -------------------
 .../org/apache/ignite/JsonLocationImpl.java     |  60 ---
 .../java/org/apache/ignite/JsonNumberImpl.java  | 116 -----
 .../apache/ignite/JsonObjectBuilderImpl.java    | 141 ------
 .../java/org/apache/ignite/JsonObjectImpl.java  | 116 -----
 .../org/apache/ignite/JsonProviderImpl.java     | 104 ----
 .../java/org/apache/ignite/JsonStringImpl.java  |  71 ---
 .../processors/json/IgniteJsonArray.java        | 132 +++++
 .../processors/json/IgniteJsonArrayBuilder.java | 130 +++++
 .../processors/json/IgniteJsonGenerator.java    | 500 +++++++++++++++++++
 .../processors/json/IgniteJsonLocation.java     |  60 +++
 .../json/IgniteJsonMessageFactory.java          |  36 ++
 .../processors/json/IgniteJsonNumber.java       | 116 +++++
 .../processors/json/IgniteJsonObject.java       | 117 +++++
 .../json/IgniteJsonObjectBuilder.java           | 141 ++++++
 .../json/IgniteJsonProcessorImpl.java           | 115 +++++
 .../processors/json/IgniteJsonProvider.java     | 104 ++++
 .../processors/json/IgniteJsonString.java       |  77 +++
 .../processors/json/JsonCacheObject.java        | 103 ++++
 .../java/org/apache/ignite/json/IgniteJson.java |  36 ++
 .../processors/json/IgniteJsonCacheTest.java    | 194 +++++++
 31 files changed, 2249 insertions(+), 1393 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index addc808..bf0ba65 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal;
 
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.processors.json.*;
 import org.apache.ignite.internal.managers.checkpoint.*;
 import org.apache.ignite.internal.managers.collision.*;
 import org.apache.ignite.internal.managers.communication.*;
@@ -555,6 +556,11 @@ public interface GridKernalContext extends Iterable<GridComponent> {
     public ClusterProcessor cluster();
 
     /**
+     * @return Json processor.
+     */
+    public IgniteJsonProcessor json();
+
+    /**
      * Gets marshaller context.
      *
      * @return Marshaller context.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 5c3da6a..36b263c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.processors.json.*;
 import org.apache.ignite.internal.managers.checkpoint.*;
 import org.apache.ignite.internal.managers.collision.*;
 import org.apache.ignite.internal.managers.communication.*;
@@ -248,6 +249,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
 
     /** */
     @GridToStringExclude
+    private IgniteJsonProcessor json;
+
+    /** */
+    @GridToStringExclude
     private DataStructuresProcessor dataStructuresProc;
 
     /** */
@@ -491,7 +496,9 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         else if (comp instanceof ClusterProcessor)
             cluster = (ClusterProcessor)comp;
         else if (comp instanceof IgniteScriptingProcessor)
-            scriptProc = (IgniteScriptingProcessor) comp;
+            scriptProc = (IgniteScriptingProcessor)comp;
+        else if (comp instanceof IgniteJsonProcessor)
+            json = (IgniteJsonProcessor)comp;
         else if (!(comp instanceof DiscoveryNodeValidationProcessor))
             assert (comp instanceof GridPluginComponent) : "Unknown manager class: " + comp.getClass();
 
@@ -919,6 +926,11 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
+    @Override public IgniteJsonProcessor json() {
+        return json;
+    }
+
+    /** {@inheritDoc} */
     @Override public MarshallerContextImpl marshallerContext() {
         return marshCtx;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java
index 4b937b1..1e91c88 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java
@@ -83,6 +83,14 @@ public enum IgniteComponentType {
         "org.apache.ignite.internal.processors.schedule.IgniteNoopScheduleProcessor",
         "org.apache.ignite.internal.processors.schedule.IgniteScheduleProcessor",
         "ignite-schedule"
+    ),
+
+    /** JSON API. */
+    JSON(
+        "org.apache.ignite.internal.processors.json.IgniteJsonNoopProcessor",
+        "org.apache.ignite.internal.processors.json.IgniteJsonProcessorImpl",
+        "ignite-json",
+        "org.apache.ignite.internal.processors.json.IgniteJsonMessageFactory"
     );
 
     /** No-op class name. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 081ccf0..3118242 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -786,6 +786,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 IgniteComponentType.HADOOP.createIfInClassPath(ctx, cfg.getHadoopConfiguration() != null)));
             startProcessor(new GridServiceProcessor(ctx));
             startProcessor(new DataStructuresProcessor(ctx));
+            startProcessor((GridProcessor)IgniteComponentType.JSON.createIfInClassPath(ctx, false));
 
             // Start plugins.
             for (PluginProvider provider : ctx.plugins().allProviders()) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index 3e59b10..c6ab6f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -94,6 +94,11 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
         if (obj instanceof KeyCacheObject)
             return (KeyCacheObject)obj;
 
+        KeyCacheObject key = this.ctx.json().toCacheKeyObject(ctx, obj, userObj);
+
+        if (key != null)
+            return key;
+
         return toCacheKeyObject0(obj, userObj);
     }
 
@@ -158,6 +163,11 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
         if (obj == null || obj instanceof CacheObject)
             return (CacheObject)obj;
 
+        CacheObject cacheObj = this.ctx.json().toCacheKeyObject(ctx, obj, userObj);
+
+        if (cacheObj != null)
+            return cacheObj;
+
         return toCacheObject0(obj, userObj);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/core/src/main/java/org/apache/ignite/internal/processors/json/IgniteJsonNoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/json/IgniteJsonNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/json/IgniteJsonNoopProcessor.java
new file mode 100644
index 0000000..ff1762d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/json/IgniteJsonNoopProcessor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.ignite.internal.processors.json;
+
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.jetbrains.annotations.*;
+
+/**
+ *
+ */
+public class IgniteJsonNoopProcessor extends GridProcessorAdapter implements IgniteJsonProcessor {
+    /**
+     * @param ctx Context.
+     */
+    public IgniteJsonNoopProcessor(GridKernalContext ctx) {
+        super(ctx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public KeyCacheObject toCacheKeyObject(CacheObjectContext ctx, Object obj, boolean userObj) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public CacheObject toCacheObject(CacheObjectContext ctx,
+        @Nullable Object obj,
+        boolean userObj) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean jsonType(Class<?> cls) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean jsonObject(Object obj) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasField(Object obj, String fieldName) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object field(Object obj, String fieldName) {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/core/src/main/java/org/apache/ignite/internal/processors/json/IgniteJsonProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/json/IgniteJsonProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/json/IgniteJsonProcessor.java
new file mode 100644
index 0000000..92db316
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/json/IgniteJsonProcessor.java
@@ -0,0 +1,71 @@
+/*
+ * 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.ignite.internal.processors.json;
+
+import org.apache.ignite.internal.processors.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.jetbrains.annotations.*;
+
+/**
+ *
+ */
+public interface IgniteJsonProcessor extends GridProcessor {
+    /**
+     * @param ctx Cache context.
+     * @param obj Key value.
+     * @param userObj If {@code true} then given object is object provided by user and should be copied
+     *        before stored in cache.
+     * @return Cache key object.
+     */
+    public KeyCacheObject toCacheKeyObject(CacheObjectContext ctx, Object obj, boolean userObj);
+
+    /**
+     * @param ctx Cache context.
+     * @param obj Object.
+     * @param userObj If {@code true} then given object is object provided by user and should be copied
+     *        before stored in cache.
+     * @return Cache object.
+     */
+    @Nullable public CacheObject toCacheObject(CacheObjectContext ctx, @Nullable Object obj, boolean userObj);
+
+    /**
+     * @param cls Class.
+     * @return {@code True} if given type is json object type.
+     */
+    public boolean jsonType(Class<?> cls);
+
+    /**
+     * @param obj Object.
+     * @return {@code True} if given object is JSON object.
+     */
+    public boolean jsonObject(Object obj);
+
+    /**
+     * @param obj Object.
+     * @param fieldName Field name.
+     * @return {@code True} if field is set.
+     */
+    public boolean hasField(Object obj, String fieldName);
+
+    /**
+     * @param obj Object.
+     * @param fieldName Field name.
+     * @return Field value.
+     */
+    public Object field(Object obj, String fieldName);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index f3ad4b2..ea57a12 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -70,6 +70,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /** */
     private final GridQueryIndexing idx;
 
+    /** */
+    private TypeId jsonTypeId;
+
     /**
      * @param ctx Kernal context.
      */
@@ -145,6 +148,13 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                         typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType()));
                     }
+                    else if (ctx.json().jsonType(desc.keyClass()) || ctx.json().jsonType(desc.valueClass())) {
+                        processJsonMeta(meta, desc);
+
+                        typeId = new TypeId(ccfg.getName(), valCls);
+
+                        jsonTypeId = typeId;
+                    }
                     else {
                         processClassMeta(meta, desc);
 
@@ -455,39 +465,48 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             if (coctx == null)
                 coctx = cacheObjectContext(space);
 
-            Class<?> valCls = null;
+            TypeDescriptor desc;
 
-            TypeId id;
+            if (ctx.json().jsonObject(val)) {
+                desc = types.get(jsonTypeId);
 
-            boolean portableVal = ctx.cacheObjects().isPortableObject(val);
-
-            if (portableVal) {
-                int typeId = ctx.cacheObjects().typeId(val);
-
-                id = new TypeId(space, typeId);
+                assert desc != null && desc.registered() : desc;
             }
             else {
-                valCls = val.value(coctx, false).getClass();
+                Class<?> valCls = null;
 
-                id = new TypeId(space, valCls);
-            }
+                TypeId id;
 
-            TypeDescriptor desc = types.get(id);
+                boolean portableVal = ctx.cacheObjects().isPortableObject(val);
 
-            if (desc == null || !desc.registered())
-                return;
+                if (portableVal) {
+                    int typeId = ctx.cacheObjects().typeId(val);
+
+                    id = new TypeId(space, typeId);
+                }
+                else  {
+                    valCls = val.value(coctx, false).getClass();
+
+                    id = new TypeId(space, valCls);
+                }
 
-            if (!portableVal && !desc.valueClass().isAssignableFrom(valCls))
-                throw new IgniteCheckedException("Failed to update index due to class name conflict" +
-                    "(multiple classes with same simple name are stored in the same cache) " +
-                    "[expCls=" + desc.valueClass().getName() + ", actualCls=" + valCls.getName() + ']');
+                desc = types.get(id);
 
-            if (!ctx.cacheObjects().isPortableObject(key)) {
-                Class<?> keyCls = key.value(coctx, false).getClass();
+                if (desc == null || !desc.registered())
+                    return;
 
-                if (!desc.keyClass().isAssignableFrom(keyCls))
-                    throw new IgniteCheckedException("Failed to update index, incorrect key class [expCls=" +
-                        desc.keyClass().getName() + ", actualCls=" + keyCls.getName() + "]");
+                if (!portableVal && !desc.valueClass().isAssignableFrom(valCls))
+                    throw new IgniteCheckedException("Failed to update index due to class name conflict" +
+                        "(multiple classes with same simple name are stored in the same cache) " +
+                        "[expCls=" + desc.valueClass().getName() + ", actualCls=" + valCls.getName() + ']');
+
+                if (!ctx.cacheObjects().isPortableObject(key)) {
+                    Class<?> keyCls = key.value(coctx, false).getClass();
+
+                    if (!desc.keyClass().isAssignableFrom(keyCls))
+                        throw new IgniteCheckedException("Failed to update index, incorrect key class [expCls=" +
+                            desc.keyClass().getName() + ", actualCls=" + keyCls.getName() + "]");
+                }
             }
 
             idx.store(space, desc, key, val, ver, expirationTime);
@@ -1236,6 +1255,98 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Processes declarative metadata for json object.
+     *
+     * @param meta Declared metadata.
+     * @param d Type descriptor.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void processJsonMeta(CacheTypeMetadata meta, TypeDescriptor d)
+        throws IgniteCheckedException {
+        for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet()) {
+            JsonProperty prop = buildJsonProperty(entry.getKey(), entry.getValue());
+
+            d.addProperty(prop, false);
+
+            String idxName = prop.name() + "_idx";
+
+            d.addIndex(idxName, idx.isGeometryClass(prop.type()) ? GEO_SPATIAL : SORTED);
+
+            d.addFieldToIndex(idxName, prop.name(), 0, false);
+        }
+
+        for (Map.Entry<String, Class<?>> entry : meta.getDescendingFields().entrySet()) {
+            JsonProperty prop = buildJsonProperty(entry.getKey(), entry.getValue());
+
+            d.addProperty(prop, false);
+
+            String idxName = prop.name() + "_idx";
+
+            d.addIndex(idxName, idx.isGeometryClass(prop.type()) ? GEO_SPATIAL : SORTED);
+
+            d.addFieldToIndex(idxName, prop.name(), 0, true);
+        }
+
+        for (String txtIdx : meta.getTextFields()) {
+            JsonProperty prop = buildJsonProperty(txtIdx, String.class);
+
+            d.addProperty(prop, false);
+
+            d.addFieldToTextIndex(prop.name());
+        }
+
+        Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> grps = meta.getGroups();
+
+        if (grps != null) {
+            for (Map.Entry<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> entry : grps.entrySet()) {
+                String idxName = entry.getKey();
+
+                LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>> idxFields = entry.getValue();
+
+                int order = 0;
+
+                for (Map.Entry<String, IgniteBiTuple<Class<?>, Boolean>> idxField : idxFields.entrySet()) {
+                    JsonProperty prop = buildJsonProperty(idxField.getKey(), idxField.getValue().get1());
+
+                    d.addProperty(prop, false);
+
+                    Boolean descending = idxField.getValue().get2();
+
+                    d.addFieldToIndex(idxName, prop.name(), order, descending != null && descending);
+
+                    order++;
+                }
+            }
+        }
+
+        for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
+            JsonProperty prop = buildJsonProperty(entry.getKey(), entry.getValue());
+
+            if (!d.props.containsKey(prop.name()))
+                d.addProperty(prop, false);
+        }
+    }
+
+    /**
+     * Builds portable object property.
+     *
+     * @param pathStr String representing path to the property. May contains dots '.' to identify
+     *      nested fields.
+     * @param resType Result type.
+     * @return Portable property.
+     */
+    private JsonProperty buildJsonProperty(String pathStr, Class<?> resType) {
+        String[] path = pathStr.split("\\.");
+
+        JsonProperty res = null;
+
+        for (String prop : path)
+            res = new JsonProperty(prop, res, resType);
+
+        return res;
+    }
+
+    /**
      * Processes declarative metadata for portable object.
      *
      * @param meta Declared metadata.
@@ -1626,6 +1737,84 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /**
      *
      */
+    private class JsonProperty extends Property {
+        /** Property name. */
+        private String propName;
+
+        /** Parent property. */
+        private JsonProperty parent;
+
+        /** Result class. */
+        private Class<?> type;
+
+        /** */
+        private volatile int isKeyProp;
+
+        /**
+         * Constructor.
+         *
+         * @param propName Property name.
+         * @param parent Parent property.
+         * @param type Result type.
+         */
+        private JsonProperty(String propName, JsonProperty parent, Class<?> type) {
+            this.propName = propName;
+            this.parent = parent;
+            this.type = type;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object value(Object key, Object val) throws IgniteCheckedException {
+            Object obj;
+
+            if (parent != null) {
+                obj = parent.value(key, val);
+
+                if (obj == null)
+                    return null;
+
+                if (!ctx.json().jsonObject(obj))
+                    throw new IgniteCheckedException("Non-json object received as a result of property extraction " +
+                        "[parent=" + parent + ", propName=" + propName + ", obj=" + obj + ']');
+            }
+            else {
+                int isKeyProp0 = isKeyProp;
+
+                if (isKeyProp0 == 0) {
+                    // Key is allowed to be a non-portable object here.
+                    // We check key before value consistently with ClassProperty.
+                    if (ctx.json().jsonObject(key) && ctx.json().hasField(key, propName))
+                        isKeyProp = isKeyProp0 = 1;
+                    else if (ctx.json().hasField(val, propName))
+                        isKeyProp = isKeyProp0 = -1;
+                    else {
+                        U.warn(log, "Neither key nor value have property " +
+                            "[propName=" + propName + ", key=" + key + ", val=" + val + "]");
+
+                        return null;
+                    }
+                }
+
+                obj = isKeyProp0 == 1 ? key : val;
+            }
+
+            return ctx.json().field(obj, propName);
+        }
+
+        /** {@inheritDoc} */
+        @Override public String name() {
+            return propName;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Class<?> type() {
+            return type;
+        }
+    }
+
+    /**
+     *
+     */
     private class PortableProperty extends Property {
         /** Property name. */
         private String propName;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/json/src/main/java/org/apache/ignite/JsonArrayBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/json/src/main/java/org/apache/ignite/JsonArrayBuilderImpl.java b/modules/json/src/main/java/org/apache/ignite/JsonArrayBuilderImpl.java
deleted file mode 100644
index 97b3e9a..0000000
--- a/modules/json/src/main/java/org/apache/ignite/JsonArrayBuilderImpl.java
+++ /dev/null
@@ -1,130 +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.ignite;
-
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import javax.json.*;
-import java.math.*;
-import java.util.*;
-
-/**
- * Json array builder.
- */
-public class JsonArrayBuilderImpl implements JsonArrayBuilder {
-    /** Json array list. */
-    private List<JsonValue> jsonList = new ArrayList<>();
-
-    /** {@inheritDoc} */
-    @Override public JsonArrayBuilder add(JsonValue val) {
-        A.notNull(val, "value");
-
-        jsonList.add(val);
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonArrayBuilder add(String val) {
-        A.notNull(val, "value");
-
-        jsonList.add(new JsonStringImpl(val));
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonArrayBuilder add(BigDecimal val) {
-        A.notNull(val, "value");
-
-        jsonList.add(new JsonNumberImpl(val));
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonArrayBuilder add(BigInteger val) {
-        A.notNull(val, "value");
-
-        //TODO: optimize for value
-        jsonList.add(new JsonNumberImpl(new BigDecimal(val)));
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonArrayBuilder add(int val) {
-        //TODO: optimize for value
-        jsonList.add(new JsonNumberImpl(new BigDecimal(val)));
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonArrayBuilder add(long val) {
-        //TODO: optimize for value
-        jsonList.add(new JsonNumberImpl(new BigDecimal(val)));
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonArrayBuilder add(double val) {
-        //TODO: optimize for value
-        jsonList.add(new JsonNumberImpl(new BigDecimal(val)));
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonArrayBuilder add(boolean val) {
-        jsonList.add(val ? JsonValue.TRUE : JsonValue.FALSE);
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonArrayBuilder addNull() {
-        jsonList.add(JsonValue.NULL);
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonArrayBuilder add(JsonObjectBuilder bld) {
-        A.notNull(bld, "value");
-
-        jsonList.add(bld.build());
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonArrayBuilder add(JsonArrayBuilder bld) {
-        A.notNull(bld, "value");
-
-        jsonList.add(bld.build());
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonArray build() {
-        return new JsonArrayImpl(jsonList);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/json/src/main/java/org/apache/ignite/JsonArrayImpl.java
----------------------------------------------------------------------
diff --git a/modules/json/src/main/java/org/apache/ignite/JsonArrayImpl.java b/modules/json/src/main/java/org/apache/ignite/JsonArrayImpl.java
deleted file mode 100644
index 9005c22..0000000
--- a/modules/json/src/main/java/org/apache/ignite/JsonArrayImpl.java
+++ /dev/null
@@ -1,131 +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.ignite;
-
-import javax.json.*;
-import java.util.*;
-
-/**
- * Implementation of JsonArray
- */
-public class JsonArrayImpl extends ArrayList<JsonValue> implements JsonArray {
-    /** Values for getValueAs. */
-    private List<JsonValue> val;
-
-    /**
-     * @param val List json values.
-     */
-    public JsonArrayImpl(List<JsonValue> val) {
-        super(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonObject getJsonObject(int idx) {
-        return (JsonObject)get(idx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonArray getJsonArray(int idx) {
-        return (JsonArray)get(idx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonNumber getJsonNumber(int idx) {
-        return (JsonNumber)get(idx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonString getJsonString(int idx) {
-        return (JsonString)get(idx);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public <T extends JsonValue> List<T> getValuesAs(Class<T> clazz) {
-        if (val == null) {
-            val = new ArrayList(this.size());
-
-            for (int i = 0; i < size(); ++i)
-                val.add(get(i));
-
-            val = Collections.unmodifiableList(val);
-        }
-        return (List<T>) val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getString(int idx) {
-        return getJsonString(idx).getString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getString(int idx, String dfltVal) {
-        try {
-            return getString(idx);
-        }
-        catch (Exception e) {
-            return dfltVal;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getInt(int idx) {
-        return getJsonNumber(idx).intValue();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getInt(int idx, int dfltVal) {
-        try {
-            return getInt(idx);
-        } catch (Exception e) {
-            return dfltVal;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean getBoolean(int idx) {
-        JsonValue val = get(idx);
-
-        if (val.equals(JsonValue.TRUE))
-            return true;
-
-        if (val.equals(JsonValue.FALSE))
-            return false;
-
-        throw new ClassCastException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean getBoolean(int idx, boolean dfltVal) {
-        try {
-            return getBoolean(idx);
-        } catch (Exception e) {
-            return dfltVal;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isNull(int idx) {
-        return get(idx).equals(JsonValue.NULL);
-    }
-
-    /** {@inheritDoc} */
-    @Override public ValueType getValueType() {
-        return ValueType.ARRAY;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/json/src/main/java/org/apache/ignite/JsonGeneratorImpl.java
----------------------------------------------------------------------
diff --git a/modules/json/src/main/java/org/apache/ignite/JsonGeneratorImpl.java b/modules/json/src/main/java/org/apache/ignite/JsonGeneratorImpl.java
deleted file mode 100644
index 95b336f..0000000
--- a/modules/json/src/main/java/org/apache/ignite/JsonGeneratorImpl.java
+++ /dev/null
@@ -1,500 +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.ignite;
-
-import javax.json.*;
-import javax.json.stream.*;
-import java.io.*;
-import java.math.*;
-import java.util.*;
-
-/**
- * Json generator implementation.
- */
-public class JsonGeneratorImpl implements JsonGenerator {
-    /** Writer. */
-    private final BufferedWriter writer;
-
-    private LinkedList<Element> ctx = new LinkedList();
-
-    /**
-     * @param writer Writer.
-     */
-    public JsonGeneratorImpl(Writer writer) {
-        this.writer = new BufferedWriter(writer);
-
-        ctx.push(new Element(Context.NONE, true));
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator writeStartObject() {
-        try {
-            if (ctx.getLast().context() == Context.OBJECT ||
-                (ctx.getLast().context() == Context.NONE && !ctx.getLast().isFirst()))
-                throw new JsonGenerationException("No name for object field.");
-
-            writeComma();
-            writer.write('{');
-
-            ctx.addLast(new Element(Context.OBJECT, true));
-
-            return this;
-        }
-        catch (IOException e) {
-            throw new JsonException("Writer fails.", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator writeStartObject(String name) {
-        try {
-            if (ctx.getLast().context() != Context.OBJECT)
-                throw new JsonGenerationException("Object with name in not object scope.");
-
-            writeComma();
-            writeString(name);
-            writer.write(":");
-            writer.write('{');
-
-            ctx.addLast(new Element(Context.OBJECT, true));
-
-            return this;
-        }
-        catch (IOException e) {
-            throw new JsonException("Writer fails.", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator writeStartArray() {
-        try {
-            if (ctx.getLast().context() == Context.OBJECT ||
-                (ctx.getLast().context() == Context.NONE && !ctx.getLast().isFirst()))
-                throw new JsonGenerationException("Array in object scope.");
-
-            writeComma();
-            writer.write("[");
-
-            ctx.addLast(new Element(Context.ARRAY, true));
-
-            return this;
-        }
-        catch (IOException e) {
-            throw new JsonException("Writer fails.", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator writeStartArray(String name) {
-        try {
-            if (ctx.getLast().context() != Context.OBJECT)
-                throw new JsonGenerationException("Array with name in not object scope.");
-
-            writeComma();
-            writeString(name);
-            writer.write(":");
-            writer.write('[');
-
-            ctx.addLast(new Element(Context.ARRAY, true));
-
-            return this;
-        }
-        catch (IOException e) {
-            throw new JsonException("Writer fails.", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator write(String name, JsonValue val) {
-        if (ctx.getLast().context() != Context.OBJECT)
-            throw new JsonGenerationException("Json value with name in not object scope.");
-
-        try {
-            switch (val.getValueType()) {
-                case ARRAY: {
-                    JsonArray arr = (JsonArray) val;
-
-                    writeStartArray(name);
-
-                    for (JsonValue el : arr)
-                        write(el);
-
-                    writeEnd();
-
-                    break;
-                }
-
-                case OBJECT: {
-                    JsonObject o = (JsonObject) val;
-
-                    writeStartObject(name);
-
-                    for (Map.Entry<String, JsonValue> member : o.entrySet())
-                        write(member.getKey(), member.getValue());
-
-                    writeEnd();
-
-                    break;
-                }
-
-                case STRING: {
-                    JsonString str = (JsonString) val;
-
-                    write(name, str.getString());
-
-                    break;
-                }
-
-                case NUMBER: {
-                    JsonNumber n = (JsonNumber) val;
-
-                    writeComma();
-                    writeString(name);
-                    writer.write(":");
-                    writeString(n.toString());
-
-                    break;
-                }
-                case TRUE: {
-                    write(name, true);
-
-                    break;
-                }
-
-                case FALSE: {
-                    write(name, false);
-
-                    break;
-                }
-
-                case NULL: {
-                    writeNull(name);
-
-                    break;
-                }
-            }
-            return this;
-        }
-        catch (IOException e) {
-            throw new JsonException("Writer fails.", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator write(String name, String val) {
-        return writeSimpleField(name, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator write(String name, BigInteger val) {
-        return writeSimpleField(name, String.valueOf(val));
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator write(String name, BigDecimal val) {
-        return writeSimpleField(name, String.valueOf(val));
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator write(String name, int val) {
-        return writeSimpleField(name, String.valueOf(val));
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator write(String name, long val) {
-        return writeSimpleField(name, String.valueOf(val));
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator write(String name, double val) {
-        return writeSimpleField(name, String.valueOf(val));
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator write(String name, boolean val) {
-        return writeSimpleField(name, String.valueOf(val));
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator writeNull(String name) {
-        return writeSimpleField(name, "null");
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator writeEnd() {
-        if (ctx.getLast().context() == Context.NONE)
-            throw new JsonGenerationException("Cannot call writeEnd in none context.");
-
-        try {
-            if (ctx.getLast().context() == Context.ARRAY)
-                writer.write("]");
-
-            if (ctx.getLast().context() == Context.OBJECT)
-                writer.write("}");
-
-            ctx.removeLast();
-
-            return this;
-        }
-        catch(IOException e) {
-            throw new JsonException("Writer fails.", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator write(JsonValue val) {
-        if (ctx.getLast().context() != Context.ARRAY)
-            throw new JsonGenerationException("Json value without name in not array scope.");
-
-        try {
-            switch (val.getValueType()) {
-                case ARRAY: {
-                    JsonArray arr = (JsonArray) val;
-
-                    writeStartArray();
-
-                    for (JsonValue el : arr)
-                        write(el);
-
-                    writeEnd();
-
-                    break;
-                }
-
-                case OBJECT: {
-                    JsonObject o = (JsonObject) val;
-
-                    writeStartObject();
-
-                    for (Map.Entry<String, JsonValue> member : o.entrySet())
-                        write(member.getKey(), member.getValue());
-
-                    writeEnd();
-
-                    break;
-                }
-
-                case STRING: {
-                    JsonString str = (JsonString) val;
-
-                    write(str.getString());
-
-                    break;
-                }
-
-                case NUMBER: {
-                    JsonNumber n = (JsonNumber) val;
-
-                    writeComma();
-                    writeString(n.toString());
-
-                    break;
-                }
-                case TRUE: {
-                    write(true);
-
-                    break;
-                }
-
-                case FALSE: {
-                    write(false);
-
-                    break;
-                }
-
-                case NULL: {
-                    writeNull();
-
-                    break;
-                }
-            }
-            return this;
-        }
-        catch (IOException e) {
-            throw new JsonException("Writer fails.", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator write(String val) {
-        return writeSimpleArrayElement(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator write(BigDecimal val) {
-        return writeSimpleArrayElement(String.valueOf(val));
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator write(BigInteger val) {
-        return writeSimpleArrayElement(String.valueOf(val));
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator write(int val) {
-        return writeSimpleArrayElement(String.valueOf(val));
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator write(long val) {
-        return writeSimpleArrayElement(String.valueOf(val));
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator write(double val) {
-        return writeSimpleArrayElement(String.valueOf(val));
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator write(boolean val) {
-        return writeSimpleArrayElement(String.valueOf(val));
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator writeNull() {
-        return writeSimpleArrayElement("null");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() {
-        try {
-            writer.close();
-        }
-        catch (IOException e) {
-            throw new JsonException("Could not close writer.", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void flush() {
-        try {
-            writer.flush();
-        }
-        catch (IOException e) {
-            throw new JsonException("Could not flush buffer to writer.", e);
-        }
-    }
-
-    /**
-     * Write comma if object is not first.
-     *
-     * @throws IOException If failed.
-     */
-    private void writeComma() throws IOException{
-        if (!ctx.getLast().isFirst())
-            writer.write(",");
-
-        ctx.getLast().isFirst = false;
-    }
-
-    /**
-     * @param name Field name.
-     * @param val Field value.
-     */
-    private JsonGenerator writeSimpleField(String name, String val) {
-        if (ctx.getLast().context() != Context.OBJECT)
-            throw new JsonGenerationException("String with name in not object scope.");
-
-        try {
-            writeComma();
-            writeString(name);
-            writer.write(":");
-            writeString(val);
-
-            return this;
-        }
-        catch (IOException e) {
-            throw new JsonException("Writer fails.", e);
-        }
-    }
-
-
-    /**
-     * @param val Field value.
-     */
-    private JsonGenerator writeSimpleArrayElement(String val) {
-        if (ctx.getLast().context() != Context.ARRAY)
-            throw new JsonGenerationException("String without name in not array scope.");
-
-        try {
-            writeComma();
-            writeString(val);
-
-            return this;
-        }
-        catch (IOException e) {
-            throw new JsonException("Writer fails.", e);
-        }
-    }
-
-    /**
-     * @param str String to write.
-     * @throws IOException If failed.
-     * //TODO: escape string.
-     */
-    private void writeString(String str) throws IOException {
-        writer.write(str);
-    }
-
-    /**
-     * Generator element.
-     */
-    private static class Element {
-        /** Context. */
-        private Context ctx;
-
-        /** First element flag. */
-        private boolean isFirst;
-
-        /**
-         * @param ctx Context.
-         * @param isFirst First element flag.
-         */
-        public Element(Context ctx, boolean isFirst) {
-            this.ctx = ctx;
-            this.isFirst = isFirst;
-        }
-
-        /**
-         * @return First element flag.
-         */
-        public boolean isFirst() {
-            return isFirst;
-        }
-
-        /**
-         * @return Context.
-         */
-        public Context context() {
-            return ctx;
-        }
-    }
-    /**
-     * Context for writer.
-     */
-    private enum Context {
-        /** Writing object. */
-        OBJECT,
-
-        /** Writing array. */
-        ARRAY,
-
-        /** Not in object or in array. */
-        NONE
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/json/src/main/java/org/apache/ignite/JsonLocationImpl.java
----------------------------------------------------------------------
diff --git a/modules/json/src/main/java/org/apache/ignite/JsonLocationImpl.java b/modules/json/src/main/java/org/apache/ignite/JsonLocationImpl.java
deleted file mode 100644
index 078c16f..0000000
--- a/modules/json/src/main/java/org/apache/ignite/JsonLocationImpl.java
+++ /dev/null
@@ -1,60 +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.ignite;
-
-import javax.json.stream.*;
-
-/**
- * Json location implementation.
- */
-public class JsonLocationImpl implements JsonLocation {
-    /** Column number. */
-    private final long col;
-
-    /** Line number. */
-    private final long line;
-
-    /** Stream offset. */
-    private final long off;
-
-    /**
-     * @param line Line number.
-     * @param col Column number.
-     * @param streamOff Stream offset.
-     */
-    JsonLocationImpl(long line, long col, long streamOff) {
-        this.line = line;
-        this.col = col;
-        this.off = streamOff;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getLineNumber() {
-        return line;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getColumnNumber() {
-        return col;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getStreamOffset() {
-        return off;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/json/src/main/java/org/apache/ignite/JsonNumberImpl.java
----------------------------------------------------------------------
diff --git a/modules/json/src/main/java/org/apache/ignite/JsonNumberImpl.java b/modules/json/src/main/java/org/apache/ignite/JsonNumberImpl.java
deleted file mode 100644
index 7cd0a68..0000000
--- a/modules/json/src/main/java/org/apache/ignite/JsonNumberImpl.java
+++ /dev/null
@@ -1,116 +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.ignite;
-
-import javax.json.*;
-import java.math.*;
-
-/**
- * Json number implementation.
- * //TODO: optimize for int, long, double...
- */
-public class JsonNumberImpl implements JsonNumber {
-    /** Value. */
-    private final BigDecimal val;
-
-    /**
-     * @param val Value.
-     */
-    public JsonNumberImpl(BigDecimal val){
-        this.val = val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isIntegral() {
-        if (val == null)
-            return false;
-
-        return val.scale() == 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int intValue() {
-        return val.intValue();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int intValueExact() {
-        return val.intValueExact();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long longValue() {
-        return val.longValue();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long longValueExact() {
-        return val.longValueExact();
-    }
-
-    /** {@inheritDoc} */
-    @Override public BigInteger bigIntegerValue() {
-        return val.toBigInteger();
-    }
-
-    /** {@inheritDoc} */
-    @Override public BigInteger bigIntegerValueExact() {
-        return val.toBigIntegerExact();
-    }
-
-    /** {@inheritDoc} */
-    @Override public double doubleValue() {
-        return val.doubleValue();
-    }
-
-    /** {@inheritDoc} */
-    @Override public BigDecimal bigDecimalValue() {
-        return val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public ValueType getValueType() {
-        return ValueType.NUMBER;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return val.toString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        if (val == null)
-            return 0;
-
-        return val.hashCode();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object obj) {
-        if (obj == null || !(obj instanceof JsonNumberImpl))
-            return false;
-
-        BigDecimal val0 = ((JsonNumberImpl)obj).val;
-
-        if (val == null)
-            return val0 == null;
-
-        return val.equals(val0);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/json/src/main/java/org/apache/ignite/JsonObjectBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/json/src/main/java/org/apache/ignite/JsonObjectBuilderImpl.java b/modules/json/src/main/java/org/apache/ignite/JsonObjectBuilderImpl.java
deleted file mode 100644
index 6246794..0000000
--- a/modules/json/src/main/java/org/apache/ignite/JsonObjectBuilderImpl.java
+++ /dev/null
@@ -1,141 +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.ignite;
-
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import javax.json.*;
-import java.math.*;
-import java.util.*;
-
-/**
- * Json object builder implementation.
- */
-public class JsonObjectBuilderImpl implements JsonObjectBuilder {
-    /** Json object map. */
-    private Map<String, JsonValue> jsonMap = new HashMap<>();
-
-    /** {@inheritDoc} */
-    @Override public JsonObjectBuilder add(String name, JsonValue val) {
-        A.notNull(name, "key", val, "value");
-
-        jsonMap.put(name, val);
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonObjectBuilder add(String name, String val) {
-        A.notNull(name, "key", val, "value");
-
-        jsonMap.put(name, new JsonStringImpl(val));
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonObjectBuilder add(String name, BigInteger val) {
-        A.notNull(name, "key", val, "value");
-
-        //TODO: optimize for value
-        jsonMap.put(name, new JsonNumberImpl(new BigDecimal(val)));
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonObjectBuilder add(String name, BigDecimal val) {
-        A.notNull(name, "key", val, "value");
-
-        //TODO: optimize for value
-        jsonMap.put(name, new JsonNumberImpl(val));
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonObjectBuilder add(String name, int val) {
-        A.notNull(name, "key");
-
-        //TODO: optimize for value
-        jsonMap.put(name, new JsonNumberImpl(new BigDecimal(val)));
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonObjectBuilder add(String name, long val) {
-        A.notNull(name, "key");
-
-            //TODO: optimize for value
-            jsonMap.put(name, new JsonNumberImpl(new BigDecimal(val)));
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonObjectBuilder add(String name, double val) {
-        A.notNull(name, "key");
-
-        //TODO: optimize for value
-        jsonMap.put(name, new JsonNumberImpl(new BigDecimal(val)));
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonObjectBuilder add(String name, boolean val) {
-        A.notNull(name, "key");
-
-        jsonMap.put(name, val ? JsonValue.TRUE : JsonValue.FALSE);
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonObjectBuilder addNull(String name) {
-        A.notNull(name, "key");
-
-        jsonMap.put(name, JsonValue.NULL);
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonObjectBuilder add(String name, JsonObjectBuilder bld) {
-        A.notNull(name, "key", bld, "value");
-
-        jsonMap.put(name, bld.build());
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonObjectBuilder add(String name, JsonArrayBuilder bld) {
-        A.notNull(name, "key", bld, "value");
-
-        jsonMap.put(name, bld.build());
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonObject build() {
-        return new JsonObjectImpl(jsonMap);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/json/src/main/java/org/apache/ignite/JsonObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/json/src/main/java/org/apache/ignite/JsonObjectImpl.java b/modules/json/src/main/java/org/apache/ignite/JsonObjectImpl.java
deleted file mode 100644
index 8cb7637..0000000
--- a/modules/json/src/main/java/org/apache/ignite/JsonObjectImpl.java
+++ /dev/null
@@ -1,116 +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.ignite;
-
-import javax.json.*;
-import java.util.*;
-
-/**
- * JsonObject implementation.
- */
-public class JsonObjectImpl extends HashMap<String, JsonValue> implements JsonObject {
-    /**
-     * @param val Map to store.
-     */
-    public JsonObjectImpl(Map<String, JsonValue> val) {
-        super(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonArray getJsonArray(String name) {
-        return (JsonArray)get(name);
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonObject getJsonObject(String name) {
-        return (JsonObject)get(name);
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonNumber getJsonNumber(String name) {
-        return (JsonNumber)get(name);
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonString getJsonString(String name) {
-        return (JsonString)get(name);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getString(String name) {
-        return getJsonString(name).getString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getString(String name, String dfltVal) {
-        try {
-            return getString(name);
-        }
-        catch (Exception e) {
-            return dfltVal;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getInt(String name) {
-        return getJsonNumber(name).intValue();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getInt(String name, int dfltVal) {
-        try {
-            return getInt(name);
-        }
-        catch (Exception e) {
-            return dfltVal;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean getBoolean(String name) {
-        JsonValue val = get(name);
-
-        if (val.equals(JsonValue.TRUE))
-            return true;
-
-        if (val.equals(JsonValue.FALSE))
-            return false;
-
-        throw new ClassCastException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean getBoolean(String name, boolean dfltVal) {
-        try {
-            return getBoolean(name);
-        }
-        catch (Exception e) {
-            return dfltVal;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isNull(String name) {
-        return get(name).equals(JsonValue.NULL);
-    }
-
-    /** {@inheritDoc} */
-    @Override public ValueType getValueType() {
-        return ValueType.OBJECT;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/json/src/main/java/org/apache/ignite/JsonProviderImpl.java
----------------------------------------------------------------------
diff --git a/modules/json/src/main/java/org/apache/ignite/JsonProviderImpl.java b/modules/json/src/main/java/org/apache/ignite/JsonProviderImpl.java
deleted file mode 100644
index 9bb6305..0000000
--- a/modules/json/src/main/java/org/apache/ignite/JsonProviderImpl.java
+++ /dev/null
@@ -1,104 +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.ignite;
-
-import javax.json.*;
-import javax.json.spi.*;
-import javax.json.stream.*;
-import java.io.*;
-import java.util.*;
-
-/**
- * Json provider implementation.
- */
-public class JsonProviderImpl extends JsonProvider {
-    /** {@inheritDoc} */
-    @Override public JsonParser createParser(Reader reader) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonParser createParser(InputStream in) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonParserFactory createParserFactory(Map<String, ?> config) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator createGenerator(Writer writer) {
-        return new JsonGeneratorImpl(writer);
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGenerator createGenerator(OutputStream out) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonGeneratorFactory createGeneratorFactory(Map<String, ?> config) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonReader createReader(Reader reader) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonReader createReader(InputStream in) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonWriter createWriter(Writer writer) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonWriter createWriter(OutputStream out) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonWriterFactory createWriterFactory(Map<String, ?> config) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonReaderFactory createReaderFactory(Map<String, ?> config) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonObjectBuilder createObjectBuilder() {
-        return new JsonObjectBuilderImpl();
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonArrayBuilder createArrayBuilder() {
-        return new JsonArrayBuilderImpl();
-    }
-
-    /** {@inheritDoc} */
-    @Override public JsonBuilderFactory createBuilderFactory(Map<String, ?> config) {
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/json/src/main/java/org/apache/ignite/JsonStringImpl.java
----------------------------------------------------------------------
diff --git a/modules/json/src/main/java/org/apache/ignite/JsonStringImpl.java b/modules/json/src/main/java/org/apache/ignite/JsonStringImpl.java
deleted file mode 100644
index 6314826..0000000
--- a/modules/json/src/main/java/org/apache/ignite/JsonStringImpl.java
+++ /dev/null
@@ -1,71 +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.ignite;
-
-import javax.json.*;
-
-/**
- * Json string implementation.
- */
-public class JsonStringImpl implements JsonString {
-    /** Value. */
-    private final String val;
-
-    /**
-     * @param val Value.
-     */
-    public JsonStringImpl(String val) {
-        this.val = val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getString() {
-        return val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public CharSequence getChars() {
-        return val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public ValueType getValueType() {
-        return ValueType.STRING;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        if (val == null)
-            return 0;
-
-        return val.hashCode();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object obj) {
-        if (obj == null || !(obj instanceof JsonString))
-            return false;
-
-        JsonString other = (JsonString)obj;
-
-        if (val == null)
-            return other.getString() == null;
-
-        return val.equals(other.getString());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/json/src/main/java/org/apache/ignite/internal/processors/json/IgniteJsonArray.java
----------------------------------------------------------------------
diff --git a/modules/json/src/main/java/org/apache/ignite/internal/processors/json/IgniteJsonArray.java b/modules/json/src/main/java/org/apache/ignite/internal/processors/json/IgniteJsonArray.java
new file mode 100644
index 0000000..11267ba
--- /dev/null
+++ b/modules/json/src/main/java/org/apache/ignite/internal/processors/json/IgniteJsonArray.java
@@ -0,0 +1,132 @@
+/*
+ * 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.ignite.internal.processors.json;
+
+import javax.json.*;
+import java.io.*;
+import java.util.*;
+
+/**
+ * Implementation of JsonArray
+ */
+public class IgniteJsonArray extends ArrayList<JsonValue> implements JsonArray, Serializable {
+    /** Values for getValueAs. */
+    private List<JsonValue> val;
+
+    /**
+     * @param val List json values.
+     */
+    public IgniteJsonArray(List<JsonValue> val) {
+        super(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteJsonObject getJsonObject(int idx) {
+        return (IgniteJsonObject)get(idx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public JsonArray getJsonArray(int idx) {
+        return (JsonArray)get(idx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public JsonNumber getJsonNumber(int idx) {
+        return (JsonNumber)get(idx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public JsonString getJsonString(int idx) {
+        return (JsonString)get(idx);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <T extends JsonValue> List<T> getValuesAs(Class<T> clazz) {
+        if (val == null) {
+            val = new ArrayList(this.size());
+
+            for (int i = 0; i < size(); ++i)
+                val.add(get(i));
+
+            val = Collections.unmodifiableList(val);
+        }
+        return (List<T>) val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getString(int idx) {
+        return getJsonString(idx).getString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getString(int idx, String dfltVal) {
+        try {
+            return getString(idx);
+        }
+        catch (Exception e) {
+            return dfltVal;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getInt(int idx) {
+        return getJsonNumber(idx).intValue();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getInt(int idx, int dfltVal) {
+        try {
+            return getInt(idx);
+        } catch (Exception e) {
+            return dfltVal;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean getBoolean(int idx) {
+        JsonValue val = get(idx);
+
+        if (val.equals(JsonValue.TRUE))
+            return true;
+
+        if (val.equals(JsonValue.FALSE))
+            return false;
+
+        throw new ClassCastException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean getBoolean(int idx, boolean dfltVal) {
+        try {
+            return getBoolean(idx);
+        } catch (Exception e) {
+            return dfltVal;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isNull(int idx) {
+        return get(idx).equals(JsonValue.NULL);
+    }
+
+    /** {@inheritDoc} */
+    @Override public ValueType getValueType() {
+        return ValueType.ARRAY;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6ba441d/modules/json/src/main/java/org/apache/ignite/internal/processors/json/IgniteJsonArrayBuilder.java
----------------------------------------------------------------------
diff --git a/modules/json/src/main/java/org/apache/ignite/internal/processors/json/IgniteJsonArrayBuilder.java b/modules/json/src/main/java/org/apache/ignite/internal/processors/json/IgniteJsonArrayBuilder.java
new file mode 100644
index 0000000..b65fa6c
--- /dev/null
+++ b/modules/json/src/main/java/org/apache/ignite/internal/processors/json/IgniteJsonArrayBuilder.java
@@ -0,0 +1,130 @@
+/*
+ * 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.ignite.internal.processors.json;
+
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import javax.json.*;
+import java.math.*;
+import java.util.*;
+
+/**
+ * Json array builder.
+ */
+public class IgniteJsonArrayBuilder implements JsonArrayBuilder {
+    /** Json array list. */
+    private List<JsonValue> jsonList = new ArrayList<>();
+
+    /** {@inheritDoc} */
+    @Override public JsonArrayBuilder add(JsonValue val) {
+        A.notNull(val, "value");
+
+        jsonList.add(val);
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public JsonArrayBuilder add(String val) {
+        A.notNull(val, "value");
+
+        jsonList.add(new IgniteJsonString(val));
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public JsonArrayBuilder add(BigDecimal val) {
+        A.notNull(val, "value");
+
+        jsonList.add(new IgniteJsonNumber(val));
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public JsonArrayBuilder add(BigInteger val) {
+        A.notNull(val, "value");
+
+        //TODO: optimize for value
+        jsonList.add(new IgniteJsonNumber(new BigDecimal(val)));
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public JsonArrayBuilder add(int val) {
+        //TODO: optimize for value
+        jsonList.add(new IgniteJsonNumber(new BigDecimal(val)));
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public JsonArrayBuilder add(long val) {
+        //TODO: optimize for value
+        jsonList.add(new IgniteJsonNumber(new BigDecimal(val)));
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public JsonArrayBuilder add(double val) {
+        //TODO: optimize for value
+        jsonList.add(new IgniteJsonNumber(new BigDecimal(val)));
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public JsonArrayBuilder add(boolean val) {
+        jsonList.add(val ? JsonValue.TRUE : JsonValue.FALSE);
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public JsonArrayBuilder addNull() {
+        jsonList.add(JsonValue.NULL);
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public JsonArrayBuilder add(JsonObjectBuilder bld) {
+        A.notNull(bld, "value");
+
+        jsonList.add(bld.build());
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public JsonArrayBuilder add(JsonArrayBuilder bld) {
+        A.notNull(bld, "value");
+
+        jsonList.add(bld.build());
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public JsonArray build() {
+        return new IgniteJsonArray(jsonList);
+    }
+}