You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ib...@apache.org on 2021/04/20 14:54:23 UTC

[ignite-3] branch main updated: IGNITE-14372 Fixed configuration update procedure in REST module. (#81)

This is an automated email from the ASF dual-hosted git repository.

ibessonov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 9d852ee  IGNITE-14372 Fixed configuration update procedure in REST module. (#81)
9d852ee is described below

commit 9d852ee01d8a4c2e002376575fa82acdf7707905
Author: ibessonov <be...@gmail.com>
AuthorDate: Tue Apr 20 17:54:16 2021 +0300

    IGNITE-14372 Fixed configuration update procedure in REST module. (#81)
---
 .../ignite/configuration/ConfigurationChanger.java |  46 ++--
 .../configuration/ConfigurationRegistry.java       |   4 +-
 .../ignite/configuration/internal/SuperRoot.java   |  15 +-
 .../java/org/apache/ignite/rest/RestModule.java    |  23 +-
 .../presentation/ConfigurationPresentation.java    |   6 -
 .../ignite/rest/presentation/FormatConverter.java  |  38 ----
 .../rest/presentation/json/JsonConverter.java      | 223 ++++++++++++--------
 .../rest/presentation/json/JsonPresentation.java   |  48 ++---
 .../java/org/apache/ignite/rest/routes/Router.java |   1 +
 .../rest/presentation/json/JsonConverterTest.java  | 234 ++++++++++++++++++---
 .../ignite/configuration/ConfigurationModule.java  |   3 +-
 11 files changed, 422 insertions(+), 219 deletions(-)

diff --git a/modules/configuration/src/main/java/org/apache/ignite/configuration/ConfigurationChanger.java b/modules/configuration/src/main/java/org/apache/ignite/configuration/ConfigurationChanger.java
index 8ee7a8f..8e628f3 100644
--- a/modules/configuration/src/main/java/org/apache/ignite/configuration/ConfigurationChanger.java
+++ b/modules/configuration/src/main/java/org/apache/ignite/configuration/ConfigurationChanger.java
@@ -37,6 +37,7 @@ import org.apache.ignite.configuration.storage.ConfigurationType;
 import org.apache.ignite.configuration.storage.Data;
 import org.apache.ignite.configuration.storage.StorageException;
 import org.apache.ignite.configuration.tree.ConfigurationSource;
+import org.apache.ignite.configuration.tree.ConfigurationVisitor;
 import org.apache.ignite.configuration.tree.InnerNode;
 import org.apache.ignite.configuration.validation.ConfigurationValidationException;
 import org.apache.ignite.configuration.validation.ValidationIssue;
@@ -44,6 +45,7 @@ import org.apache.ignite.configuration.validation.Validator;
 import org.jetbrains.annotations.NotNull;
 
 import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.stream.Collectors.toList;
 import static org.apache.ignite.configuration.internal.util.ConfigurationUtil.addDefaults;
 import static org.apache.ignite.configuration.internal.util.ConfigurationUtil.cleanupMatchingValues;
 import static org.apache.ignite.configuration.internal.util.ConfigurationUtil.fillFromPrefixMap;
@@ -210,19 +212,44 @@ public final class ConfigurationChanger {
         }
     }
 
-    /** Temporary until the IGNITE-14372 */
-    public CompletableFuture<?> changeX(
-        List<String> path,
+    /** */
+    public CompletableFuture<Void> change(
         ConfigurationSource source,
         ConfigurationStorage storage
     ) {
-        assert path.isEmpty() : "Path support is not yet implemented.";
-
         SuperRoot superRoot = new SuperRoot(rootKeys);
 
         source.descend(superRoot);
 
-        return changeInternally(superRoot, storage);
+        Set<ConfigurationType> storagesTypes = new HashSet<>();
+
+        superRoot.traverseChildren(new ConfigurationVisitor<Object>() {
+            @Override public Object visitInnerNode(String key, InnerNode node) {
+                RootKey<?, ?> rootKey = rootKeys.get(key);
+
+                return storagesTypes.add(rootKey.type());
+            }
+        });
+
+        assert !storagesTypes.isEmpty();
+
+        if (storagesTypes.size() != 1) {
+            return CompletableFuture.failedFuture(
+                new ConfigurationChangeException(
+                    "Cannot handle change request with configuration patches belonging to different storages."
+                )
+            );
+        }
+
+        ConfigurationStorage actualStorage = storageInstances.get(storagesTypes.iterator().next());
+
+        if (storage != null && storage != actualStorage) {
+            return CompletableFuture.failedFuture(
+                new ConfigurationChangeException("Mismatched storage passed.")
+            );
+        }
+
+        return changeInternally(superRoot, actualStorage);
     }
 
     /** Stop component. */
@@ -264,12 +291,7 @@ public final class ConfigurationChanger {
 
     /** */
     public SuperRoot mergedSuperRoot() {
-        SuperRoot mergedSuperRoot = new SuperRoot(rootKeys);
-
-        for (StorageRoots storageRoots : storagesRootsMap.values())
-            mergedSuperRoot.append(storageRoots.roots);
-
-        return mergedSuperRoot;
+        return new SuperRoot(rootKeys, storagesRootsMap.values().stream().map(roots -> roots.roots).collect(toList()));
     }
 
     /**
diff --git a/modules/configuration/src/main/java/org/apache/ignite/configuration/ConfigurationRegistry.java b/modules/configuration/src/main/java/org/apache/ignite/configuration/ConfigurationRegistry.java
index 23b058b..e30f396 100644
--- a/modules/configuration/src/main/java/org/apache/ignite/configuration/ConfigurationRegistry.java
+++ b/modules/configuration/src/main/java/org/apache/ignite/configuration/ConfigurationRegistry.java
@@ -123,8 +123,8 @@ public class ConfigurationRegistry {
     }
 
     /** */
-    public CompletableFuture<?> change(List<String> path, ConfigurationSource changesSource, ConfigurationStorage storage) {
-        return changer.changeX(path, changesSource, storage);
+    public CompletableFuture<Void> change(ConfigurationSource changesSource, ConfigurationStorage storage) {
+        return changer.change(changesSource, storage);
     }
 
     /** */
diff --git a/modules/configuration/src/main/java/org/apache/ignite/configuration/internal/SuperRoot.java b/modules/configuration/src/main/java/org/apache/ignite/configuration/internal/SuperRoot.java
index 326c3cd..ed7b231 100644
--- a/modules/configuration/src/main/java/org/apache/ignite/configuration/internal/SuperRoot.java
+++ b/modules/configuration/src/main/java/org/apache/ignite/configuration/internal/SuperRoot.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.configuration.internal;
 
+import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.SortedMap;
@@ -55,6 +56,14 @@ public final class SuperRoot extends InnerNode {
     }
 
     /** */
+    public SuperRoot(Map<String, RootKey<?, ?>> rootKeys, List<SuperRoot> superRoots) {
+        this(rootKeys);
+
+        for (SuperRoot superRoot : superRoots)
+            roots.putAll(superRoot.roots);
+    }
+
+    /** */
     public void addRoot(RootKey<?, ?> rootKey, InnerNode root) {
         assert !roots.containsKey(rootKey.key()) : rootKey.key() + " : " + roots;
         assert allRootKeys.get(rootKey.key()) == rootKey : rootKey.key() + " : " + allRootKeys;
@@ -63,12 +72,6 @@ public final class SuperRoot extends InnerNode {
     }
 
     /** */
-    public void append(SuperRoot otherRoot) {
-        //TODO IGNITE-14372 Revisit API of the super root.
-        roots.putAll(otherRoot.roots);
-    }
-
-    /** */
     public InnerNode getRoot(RootKey<?, ?> rootKey) {
         return roots.get(rootKey.key());
     }
diff --git a/modules/rest/src/main/java/org/apache/ignite/rest/RestModule.java b/modules/rest/src/main/java/org/apache/ignite/rest/RestModule.java
index 4fdf732..867bbac 100644
--- a/modules/rest/src/main/java/org/apache/ignite/rest/RestModule.java
+++ b/modules/rest/src/main/java/org/apache/ignite/rest/RestModule.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.rest;
 
-import com.google.gson.JsonElement;
 import com.google.gson.JsonSyntaxException;
 import io.netty.bootstrap.ServerBootstrap;
 import io.netty.channel.Channel;
@@ -32,16 +31,13 @@ import io.netty.handler.logging.LogLevel;
 import io.netty.handler.logging.LoggingHandler;
 import java.net.BindException;
 import java.nio.charset.StandardCharsets;
-import java.util.List;
 import java.util.Map;
 import org.apache.ignite.configuration.ConfigurationRegistry;
-import org.apache.ignite.configuration.internal.util.ConfigurationUtil;
 import org.apache.ignite.configuration.validation.ConfigurationValidationException;
 import org.apache.ignite.rest.configuration.RestConfiguration;
 import org.apache.ignite.rest.configuration.RestView;
 import org.apache.ignite.rest.netty.RestApiInitializer;
 import org.apache.ignite.rest.presentation.ConfigurationPresentation;
-import org.apache.ignite.rest.presentation.json.JsonConverter;
 import org.apache.ignite.rest.presentation.json.JsonPresentation;
 import org.apache.ignite.rest.routes.Router;
 import org.slf4j.Logger;
@@ -73,21 +69,26 @@ public class RestModule {
     /** */
     private final Logger log;
 
-    /** */
+    /**
+     * @param log Logger.
+     */
     public RestModule(Logger log) {
         this.log = log;
     }
 
-    /** */
+    /**
+     * @param sysCfg Configuration registry.
+     */
     public void prepareStart(ConfigurationRegistry sysCfg) {
         sysConf = sysCfg;
         sysCfg.registerRootKey(RestConfiguration.KEY);
 
-        presentation = new JsonPresentation();
+        presentation = new JsonPresentation(sysCfg);
     }
 
     /**
-     *
+     * @return REST channel future.
+     * @throws InterruptedException If thread has been interupted during the start.
      */
     public ChannelFuture start() throws InterruptedException {
         var router = new Router();
@@ -98,11 +99,7 @@ public class RestModule {
             .get(CONF_URL + ":" + PATH_PARAM, (req, resp) -> {
                 String cfgPath = req.queryParams().get(PATH_PARAM);
                 try {
-                    List<String> path = ConfigurationUtil.split(cfgPath);
-
-                    JsonElement json = sysConf.represent(path, JsonConverter.jsonVisitor());
-
-                    resp.json(json);
+                    resp.json(presentation.representByPath(cfgPath));
                 }
                 catch (IllegalArgumentException pathE) {
                     ErrorResult eRes = new ErrorResult("CONFIG_PATH_UNRECOGNIZED", pathE.getMessage());
diff --git a/modules/rest/src/main/java/org/apache/ignite/rest/presentation/ConfigurationPresentation.java b/modules/rest/src/main/java/org/apache/ignite/rest/presentation/ConfigurationPresentation.java
index 53b3653..821467f 100644
--- a/modules/rest/src/main/java/org/apache/ignite/rest/presentation/ConfigurationPresentation.java
+++ b/modules/rest/src/main/java/org/apache/ignite/rest/presentation/ConfigurationPresentation.java
@@ -40,12 +40,6 @@ public interface ConfigurationPresentation<R> {
     /**
      * Converts and applies configuration update request to system configuration.
      *
-     * Note: for simplisity on this stage of development it is supposed that update request
-     * contains only one root to update despite of multiroot configuration is supported.
-     *
-     * To support updating multiple roots at once more logic should be implemented including
-     * support in {@link FormatConverter} subclasses.
-     *
      * @param configUpdate Configuration update request in representation form.
      */
     void update(R configUpdate);
diff --git a/modules/rest/src/main/java/org/apache/ignite/rest/presentation/FormatConverter.java b/modules/rest/src/main/java/org/apache/ignite/rest/presentation/FormatConverter.java
deleted file mode 100644
index eee1d16..0000000
--- a/modules/rest/src/main/java/org/apache/ignite/rest/presentation/FormatConverter.java
+++ /dev/null
@@ -1,38 +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.rest.presentation;
-
-import java.io.Reader;
-
-/** */
-public interface FormatConverter {
-    /** */
-    String convertTo(Object obj);
-
-    /** */
-    String convertTo(String rootName, Object obj);
-
-    /** */
-    String rootName(String source);
-
-    /** */
-    Object convertFrom(String source, String rootName, Class<?> clazz);
-
-    /** */
-    <T> T convertFrom(Reader source, String rootName, Class<T> clazz);
-}
diff --git a/modules/rest/src/main/java/org/apache/ignite/rest/presentation/json/JsonConverter.java b/modules/rest/src/main/java/org/apache/ignite/rest/presentation/json/JsonConverter.java
index dc2f956..daa584c 100644
--- a/modules/rest/src/main/java/org/apache/ignite/rest/presentation/json/JsonConverter.java
+++ b/modules/rest/src/main/java/org/apache/ignite/rest/presentation/json/JsonConverter.java
@@ -17,41 +17,30 @@
 
 package org.apache.ignite.rest.presentation.json;
 
-import com.google.gson.Gson;
 import com.google.gson.JsonArray;
 import com.google.gson.JsonElement;
 import com.google.gson.JsonNull;
 import com.google.gson.JsonObject;
 import com.google.gson.JsonPrimitive;
-import java.io.Reader;
 import java.io.Serializable;
 import java.lang.reflect.Array;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Deque;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
-import java.util.Optional;
 import org.apache.ignite.configuration.tree.ConfigurationSource;
 import org.apache.ignite.configuration.tree.ConfigurationVisitor;
 import org.apache.ignite.configuration.tree.ConstructableTreeNode;
 import org.apache.ignite.configuration.tree.InnerNode;
 import org.apache.ignite.configuration.tree.NamedListNode;
-import org.apache.ignite.rest.presentation.FormatConverter;
 import org.jetbrains.annotations.NotNull;
 
-/** */
-public class JsonConverter implements FormatConverter {
-    /** */
-    private final Gson gson = new Gson();
-
-    /** {@inheritDoc} */
-    @Override public String convertTo(Object obj) {
-        return gson.toJson(obj);
-    }
+import static org.apache.ignite.configuration.internal.util.ConfigurationUtil.join;
 
+/** */
+public class JsonConverter {
     /** */
     public static ConfigurationVisitor<JsonElement> jsonVisitor() {
         return new ConfigurationVisitor<JsonElement>() {
@@ -108,10 +97,12 @@ public class JsonConverter implements FormatConverter {
                 if (!valClass.isArray())
                     return toJsonPrimitive(val);
 
-                JsonArray jsonArray = new JsonArray();
+                int size = Array.getLength(val);
+
+                JsonArray jsonArray = new JsonArray(size);
 
-                for (int i = 0; i < Array.getLength(val); i++)
-                    jsonArray.add(toJsonPrimitive(Array.get(val, i)));
+                for (int idx = 0; idx < size; idx++)
+                    jsonArray.add(toJsonPrimitive(Array.get(val, idx)));
 
                 return jsonArray;
             }
@@ -127,12 +118,9 @@ public class JsonConverter implements FormatConverter {
                 if (val instanceof String)
                     return new JsonPrimitive((String)val);
 
-                if (val instanceof Number)
-                    return new JsonPrimitive((Number)val);
+                assert val instanceof Number : val.getClass();
 
-                assert false : val;
-
-                throw new IllegalArgumentException(val.getClass().getCanonicalName());
+                return new JsonPrimitive((Number)val);
             }
 
             /**
@@ -148,14 +136,19 @@ public class JsonConverter implements FormatConverter {
         };
     }
 
-    /** */
+    /**
+     * @param jsonElement JSON that has to be converted to the configuration source.
+     * @return JSON-based configuration source.
+     */
     public static ConfigurationSource jsonSource(JsonElement jsonElement) {
-        //TODO IGNITE-14372 Finish this implementation.
+        if (!jsonElement.isJsonObject())
+            throw new IllegalArgumentException("JSON object is expected as a configuration source");
+
         return new JsonObjectConfigurationSource(new ArrayList<>(), jsonElement.getAsJsonObject());
     }
 
     private static class JsonObjectConfigurationSource implements ConfigurationSource {
-        /** Shared. */
+        /** */
         private final List<String> path;
 
         /** */
@@ -166,10 +159,15 @@ public class JsonConverter implements FormatConverter {
             this.jsonObject = jsonObject;
         }
 
+        /** {@inheritDoc} */
         @Override public <T> T unwrap(Class<T> clazz) {
-            throw new IllegalArgumentException(""); //TODO IGNITE-14372 Implement.
+            throw new IllegalArgumentException(
+                "'" + clazz.getSimpleName() + "' is expected as a type for '"
+                    + join(path) + "' configuration value"
+            );
         }
 
+        /** {@inheritDoc} */
         @Override public void descend(ConstructableTreeNode node) {
             for (Map.Entry<String, JsonElement> entry : jsonObject.entrySet()) {
                 String key = entry.getKey();
@@ -177,8 +175,13 @@ public class JsonConverter implements FormatConverter {
                 JsonElement jsonElement = entry.getValue();
 
                 try {
-                    if (jsonElement.isJsonArray() || jsonElement.isJsonPrimitive())
+                    if (jsonElement.isJsonArray() || jsonElement.isJsonPrimitive()) {
+                        List<String> path = new ArrayList<>(this.path.size() + 1);
+                        path.addAll(this.path);
+                        path.add(key);
+
                         node.construct(key, new JsonPrimitiveConfigurationSource(path, jsonElement));
+                    }
                     else if (jsonElement.isJsonNull()) {
                         node.construct(key, null);
                     }
@@ -193,7 +196,16 @@ public class JsonConverter implements FormatConverter {
                     }
                 }
                 catch (NoSuchElementException e) {
-                    throw new IllegalArgumentException(""); //TODO IGNITE-14372 Update comment.
+                    if (path.isEmpty()) {
+                        throw new IllegalArgumentException(
+                            "'" + key + "' configuration root doesn't exist"
+                        );
+                    }
+                    else {
+                        throw new IllegalArgumentException(
+                            "'" + join(path) + "' configuration doesn't have '" + key + "' subconfiguration"
+                        );
+                    }
                 }
             }
         }
@@ -205,37 +217,113 @@ public class JsonConverter implements FormatConverter {
         private final JsonElement jsonLeaf;
 
         private JsonPrimitiveConfigurationSource(List<String> path, JsonElement jsonLeaf) {
+            assert !path.isEmpty();
+
             this.path = path;
             this.jsonLeaf = jsonLeaf;
         }
 
+        /** {@inheritDoc} */
         @Override public <T> T unwrap(Class<T> clazz) {
-            if (clazz.isArray() != jsonLeaf.isJsonArray())
-                throw new IllegalArgumentException(""); //TODO IGNITE-14372 Update comment.
+            if (clazz.isArray()) {
+                if (!jsonLeaf.isJsonArray())
+                    throw wrongTypeException(clazz, -1);
+
+                JsonArray jsonArray = jsonLeaf.getAsJsonArray();
+                int size = jsonArray.size();
+
+                Class<?> componentType = clazz.getComponentType();
+                Class<?> boxedComponentType = box(componentType);
+
+                Object resArray = Array.newInstance(componentType, size);
 
-            if (!clazz.isArray())
-                return unwrap(jsonLeaf.getAsJsonPrimitive(), clazz);
+                for (int idx = 0; idx < size; idx++) {
+                    JsonElement element = jsonArray.get(idx);
 
-            return null;
+                    if (!element.isJsonPrimitive())
+                        throw wrongTypeException(boxedComponentType, idx);
+
+                    Array.set(resArray, idx, unwrap(element.getAsJsonPrimitive(), boxedComponentType, idx));
+                }
+
+                return (T)resArray;
+            }
+            else {
+                if (jsonLeaf.isJsonArray())
+                    throw wrongTypeException(clazz, -1);
+
+                return unwrap(jsonLeaf.getAsJsonPrimitive(), clazz, -1);
+            }
         }
 
+        /** {@inheritDoc} */
         @Override public void descend(ConstructableTreeNode node) {
-            throw new IllegalArgumentException(""); //TODO IGNITE-14372 Update comment.
+            throw new IllegalArgumentException(
+                "'" + join(path) + "' is expected to be a composite configuration node, not a single value"
+            );
         }
 
-        private <T> T unwrap(JsonPrimitive jsonPrimitive, Class<T> clazz) {
+        @NotNull private <T> IllegalArgumentException wrongTypeException(Class<T> clazz, int idx) {
+            return new IllegalArgumentException(
+                "'" + unbox(clazz).getSimpleName() + "' is expected as a type for '"
+                    + join(path) + (idx == -1 ? "" : ("[" + idx + "]")) + "' configuration value"
+            );
+        }
+
+        /**
+         * @param clazz Class object.
+         * @return Same object of it doesn't represent primitive class, boxed version otherwise.
+         */
+        private static Class<?> box(Class<?> clazz) {
+            if (!clazz.isPrimitive())
+                return clazz;
+
+            switch (clazz.getName()) {
+                case "boolean":
+                    return Boolean.class;
+
+                case "int":
+                    return Integer.class;
+
+                case "long":
+                    return Long.class;
+
+                default:
+                    assert clazz == double.class;
+
+                    return Double.class;
+            }
+        }
+
+        private static Class<?> unbox(Class<?> clazz) {
+            assert !clazz.isPrimitive();
+
+            if (clazz == Boolean.class)
+                return boolean.class;
+            else if (clazz == Integer.class)
+                return int.class;
+            else if (clazz == Long.class)
+                return long.class;
+            else if (clazz == Double.class)
+                return double.class;
+            else
+                return clazz;
+        }
+
+        private <T> T unwrap(JsonPrimitive jsonPrimitive, Class<T> clazz, int idx) {
             assert !clazz.isArray();
+            assert !clazz.isPrimitive();
 
             if (clazz == String.class) {
                 if (!jsonPrimitive.isString())
-                    throw new IllegalArgumentException(""); //TODO IGNITE-14372 Update comment.
+                    throw wrongTypeException(clazz, idx);
 
                 return clazz.cast(jsonPrimitive.getAsString());
             }
 
             if (Number.class.isAssignableFrom(clazz)) {
                 if (!jsonPrimitive.isNumber())
-                    throw new IllegalArgumentException(""); //TODO IGNITE-14372 Update comment.
+                    throw wrongTypeException(clazz, idx);
 
                 if (clazz == Double.class)
                     return clazz.cast(jsonPrimitive.getAsDouble());
@@ -243,63 +331,26 @@ public class JsonConverter implements FormatConverter {
                 if (clazz == Long.class)
                     return clazz.cast(jsonPrimitive.getAsLong());
 
-                if (clazz == Integer.class) {
-                    long longValue = jsonPrimitive.getAsLong();
+                assert clazz == Integer.class;
 
-                    if (longValue < Integer.MIN_VALUE || longValue > Integer.MAX_VALUE)
-                        throw new IllegalArgumentException(""); //TODO IGNITE-14372 Update comment.
+                long longValue = jsonPrimitive.getAsLong();
 
-                    return clazz.cast((int)longValue);
+                if (longValue < Integer.MIN_VALUE || longValue > Integer.MAX_VALUE) {
+                    throw new IllegalArgumentException(
+                        "'" + join(path) + "' has integer type"
+                            + " and the value " + longValue + " is out of bounds"
+                    );
                 }
 
-                throw new AssertionError(clazz);
+                return clazz.cast((int)longValue);
             }
 
-            if (clazz == Boolean.class) {
-                if (!jsonPrimitive.isBoolean())
-                    throw new IllegalArgumentException(""); //TODO IGNITE-14372 Update comment.
+            assert clazz == Boolean.class;
 
-                return clazz.cast(jsonPrimitive.getAsBoolean());
-            }
+            if (!jsonPrimitive.isBoolean())
+                throw wrongTypeException(clazz, idx);
 
-            throw new IllegalArgumentException(""); //TODO IGNITE-14372 Update comment.
+            return clazz.cast(jsonPrimitive.getAsBoolean());
         }
     }
-
-    /** {@inheritDoc} */
-    @Override public String convertTo(String rootName, Object src) {
-        Map<String, Object> res = new HashMap<>();
-
-        res.put(rootName, src);
-
-        return gson.toJson(res);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String rootName(String source) {
-        Map<String, Object> map = gson.fromJson(source, Map.class);
-
-        // Peek only first root for simplicity. See comment in ConfigurationPresentation#update for more context.
-        Optional<String> firstOpt = map.keySet().stream().findFirst();
-
-        return firstOpt.isPresent() ? firstOpt.get() : null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object convertFrom(String source, String rootName, Class<?> clazz) {
-        Map map = gson.fromJson(source, Map.class);
-
-        String root = gson.toJson(map.get(rootName));
-
-        return gson.fromJson(root, clazz);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> T convertFrom(Reader source, String rootName, Class<T> clazz) {
-        Map map = gson.fromJson(source, Map.class);
-
-        String root = gson.toJson(map.get(rootName));
-
-        return gson.fromJson(root, clazz);
-    }
 }
diff --git a/modules/rest/src/main/java/org/apache/ignite/rest/presentation/json/JsonPresentation.java b/modules/rest/src/main/java/org/apache/ignite/rest/presentation/json/JsonPresentation.java
index db5020d..ce2c0b1 100644
--- a/modules/rest/src/main/java/org/apache/ignite/rest/presentation/json/JsonPresentation.java
+++ b/modules/rest/src/main/java/org/apache/ignite/rest/presentation/json/JsonPresentation.java
@@ -17,28 +17,29 @@
 
 package org.apache.ignite.rest.presentation.json;
 
+import com.google.gson.JsonElement;
+import com.google.gson.JsonParser;
 import java.util.Collections;
-import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import org.apache.ignite.configuration.ConfigurationRegistry;
+import org.apache.ignite.configuration.internal.util.ConfigurationUtil;
 import org.apache.ignite.rest.presentation.ConfigurationPresentation;
 
 /** */
 public class JsonPresentation implements ConfigurationPresentation<String> {
     /** */
-    private final JsonConverter converter = new JsonConverter();
+    private final ConfigurationRegistry sysCfg;
 
-    /** */
-    public JsonPresentation() {
+    /**
+     * @param sysCfg Configuration registry.
+     */
+    public JsonPresentation(ConfigurationRegistry sysCfg) {
+        this.sysCfg = sysCfg;
     }
 
     /** {@inheritDoc} */
     @Override public String represent() {
-        Map<String, ?> preparedMap = Collections.emptyMap();
-//        configsMap.entrySet().stream().collect(Collectors.toMap(
-//            e -> e.getKey(),
-//            e -> e.getValue().getRoot().value()
-//        ));
-
-        return converter.convertTo(preparedMap);
+        return sysCfg.represent(Collections.emptyList(), JsonConverter.jsonVisitor()).toString();
     }
 
     /** {@inheritDoc} */
@@ -46,26 +47,21 @@ public class JsonPresentation implements ConfigurationPresentation<String> {
         if (path == null || path.isEmpty())
             return represent();
 
-//        String root = path.contains(".") ? path.substring(0, path.indexOf('.')) : path;
-//
-//        Configurator<? extends DynamicConfiguration<?, ?, ?>> configurator = configsMap.get(root);
-//
-//        ConfigurationProperty<Object, Object> prop = configurator.getInternal(BaseSelectors.find(path));
-//
-//        return converter.convertTo(prop.value());
-        return "";
+        return sysCfg.represent(ConfigurationUtil.split(path), JsonConverter.jsonVisitor()).toString();
     }
 
     /** {@inheritDoc} */
     @Override public void update(String configUpdate) {
-        String root = converter.rootName(configUpdate);
+        JsonElement jsonUpdate = JsonParser.parseString(configUpdate);
 
-        if (root == null) {
-            throw new IllegalArgumentException("Invalid request, no root in request: " + configUpdate);
+        try {
+            sysCfg.change(JsonConverter.jsonSource(jsonUpdate), null).get();
+        }
+        catch (InterruptedException e) {
+            throw new RuntimeException(e);
+        }
+        catch (ExecutionException e) {
+            throw new RuntimeException(e.getCause());
         }
-
-//        Object updateObj = converter.convertFrom(configUpdate, root, configurator.getChangeType());
-
-//        configurator.set(BaseSelectors.find(root), updateObj);
     }
 }
diff --git a/modules/rest/src/main/java/org/apache/ignite/rest/routes/Router.java b/modules/rest/src/main/java/org/apache/ignite/rest/routes/Router.java
index 22869b8..e8d10aa 100644
--- a/modules/rest/src/main/java/org/apache/ignite/rest/routes/Router.java
+++ b/modules/rest/src/main/java/org/apache/ignite/rest/routes/Router.java
@@ -93,6 +93,7 @@ public class Router {
      * PUT query helper.
      *
      * @param route Route.
+     * @param acceptType Accept type.
      * @param hnd Actual handler of the request.
      * @return Router
      */
diff --git a/modules/rest/src/test/java/org/apache/ignite/rest/presentation/json/JsonConverterTest.java b/modules/rest/src/test/java/org/apache/ignite/rest/presentation/json/JsonConverterTest.java
index 128d70f..7a60a5a 100644
--- a/modules/rest/src/test/java/org/apache/ignite/rest/presentation/json/JsonConverterTest.java
+++ b/modules/rest/src/test/java/org/apache/ignite/rest/presentation/json/JsonConverterTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.rest.presentation.json;
 
 import com.google.gson.JsonNull;
 import java.util.List;
+import java.util.concurrent.ExecutionException;
 import org.apache.ignite.configuration.ConfigurationRegistry;
 import org.apache.ignite.configuration.annotation.Config;
 import org.apache.ignite.configuration.annotation.ConfigurationRoot;
@@ -28,13 +29,18 @@ import org.apache.ignite.configuration.storage.ConfigurationType;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.function.Executable;
 
 import static com.google.gson.JsonParser.parseString;
 import static java.util.Collections.emptyList;
 import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.ignite.rest.presentation.json.JsonConverter.jsonSource;
 import static org.apache.ignite.rest.presentation.json.JsonConverter.jsonVisitor;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
 import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /** */
 public class JsonConverterTest {
@@ -122,7 +128,7 @@ public class JsonConverterTest {
 
     /** */
     @Test
-    public void toJson() throws Exception {
+    public void toJsonBasic() throws Exception {
         assertEquals(
             parseString("{'root':{'arraysList':{},'primitivesList':{}}}"),
             registry.represent(emptyList(), jsonVisitor())
@@ -138,23 +144,71 @@ public class JsonConverterTest {
             registry.represent(List.of("root", "arraysList"), jsonVisitor())
         );
 
+        IllegalArgumentException e
+            = assertThrows(IllegalArgumentException.class, () -> registry.represent(List.of("doot"), jsonVisitor()));
+        assertEquals("Configuration 'doot' is not found", e.getMessage());
+
+        e = assertThrows(IllegalArgumentException.class, () -> registry.represent(List.of("root", "x"), jsonVisitor()));
+        assertEquals("Configuration 'root.x' is not found", e.getMessage());
+
+        assertEquals(
+            JsonNull.INSTANCE,
+            registry.represent(List.of("root", "primitivesList", "foo"), jsonVisitor())
+        );
+    }
+
+    /** */
+    @Test
+    public void toJsonPrimitives() throws Exception {
         configuration.change(cfg -> cfg
-            .changeArraysList(arraysList -> arraysList
-                .create("name", arrays -> {})
-            )
             .changePrimitivesList(primitivesList -> primitivesList
                 .create("name", primitives -> {})
             )
         ).get(1, SECONDS);
 
         assertEquals(
-            parseString("{'name':{'booleans':[false],'ints':[0],'longs':[0],'doubles':[0.0],'strings':['']}}"),
-            registry.represent(List.of("root", "arraysList"), jsonVisitor())
+            parseString("{'booleanVal':false,'intVal':0,'longVal':0,'doubleVal':0.0,'stringVal':''}"),
+            registry.represent(List.of("root", "primitivesList", "name"), jsonVisitor())
         );
 
         assertEquals(
-            parseString("{'booleanVal':false,'intVal':0,'longVal':0,'doubleVal':0.0,'stringVal':''}"),
-            registry.represent(List.of("root", "primitivesList", "name"), jsonVisitor())
+            parseString("false"),
+            registry.represent(List.of("root", "primitivesList", "name", "booleanVal"), jsonVisitor())
+        );
+
+        assertEquals(
+            parseString("0"),
+            registry.represent(List.of("root", "primitivesList", "name", "intVal"), jsonVisitor())
+        );
+
+        assertEquals(
+            parseString("0"),
+            registry.represent(List.of("root", "primitivesList", "name", "longVal"), jsonVisitor())
+        );
+
+        assertEquals(
+            parseString("0.0"),
+            registry.represent(List.of("root", "primitivesList", "name", "doubleVal"), jsonVisitor())
+        );
+
+        assertEquals(
+            parseString("''"),
+            registry.represent(List.of("root", "primitivesList", "name", "stringVal"), jsonVisitor())
+        );
+    }
+
+    /** */
+    @Test
+    public void toJsonArrays() throws Exception {
+        configuration.change(cfg -> cfg
+            .changeArraysList(arraysList -> arraysList
+                .create("name", arrays -> {})
+            )
+        ).get(1, SECONDS);
+
+        assertEquals(
+            parseString("{'name':{'booleans':[false],'ints':[0],'longs':[0],'doubles':[0.0],'strings':['']}}"),
+            registry.represent(List.of("root", "arraysList"), jsonVisitor())
         );
 
         assertEquals(
@@ -181,39 +235,163 @@ public class JsonConverterTest {
             parseString("['']"),
             registry.represent(List.of("root", "arraysList", "name", "strings"), jsonVisitor())
         );
+    }
 
-        assertEquals(
-            parseString("false"),
-            registry.represent(List.of("root", "primitivesList", "name", "booleanVal"), jsonVisitor())
+    /** */
+    @Test
+    public void fromJsonBasic() throws Exception {
+        // Wrong names:
+        assertThrowsIllegalArgException(
+            () -> change("{'doot' : {}}"),
+            "'doot' configuration root doesn't exist"
         );
 
-        assertEquals(
-            parseString("0"),
-            registry.represent(List.of("root", "primitivesList", "name", "intVal"), jsonVisitor())
+        assertThrowsIllegalArgException(
+            () -> change("{'root':{'foo' : {}}}"),
+            "'root' configuration doesn't have 'foo' subconfiguration"
         );
 
-        assertEquals(
-            parseString("0"),
-            registry.represent(List.of("root", "primitivesList", "name", "longVal"), jsonVisitor())
+        assertThrowsIllegalArgException(
+            () -> change("{'root':{'arraysList':{'name':{'x' : 1}}}}"),
+            "'root.arraysList.name' configuration doesn't have 'x' subconfiguration"
         );
 
-        assertEquals(
-            parseString("0.0"),
-            registry.represent(List.of("root", "primitivesList", "name", "doubleVal"), jsonVisitor())
+        // Wrong node types:
+        assertThrowsIllegalArgException(
+            () -> change("{'root' : 'foo'}"),
+            "'root' is expected to be a composite configuration node, not a single value"
         );
 
-        assertEquals(
-            parseString("''"),
-            registry.represent(List.of("root", "primitivesList", "name", "stringVal"), jsonVisitor())
+        assertThrowsIllegalArgException(
+            () -> change("{'root':{'arraysList' : 'foo'}}"),
+            "'root.arraysList' is expected to be a composite configuration node, not a single value"
+        );
+
+        assertThrowsIllegalArgException(
+            () -> change("{'root':{'arraysList':{'name' : 'foo'}}}"),
+            "'root.arraysList.name' is expected to be a composite configuration node, not a single value"
+        );
+
+        assertThrowsIllegalArgException(
+            () -> change("{'root':{'arraysList':{'name':{'ints' : {}}}}}"),
+            "'int[]' is expected as a type for 'root.arraysList.name.ints' configuration value"
         );
+    }
+
+    /** */
+    @Test
+    public void fromJsonPrimitives() throws Exception {
+        change("{'root':{'primitivesList':{'name' : {}}}}");
 
-        assertThrows(IllegalArgumentException.class, () -> registry.represent(List.of("doot"), jsonVisitor()));
+        JsonPrimitivesConfiguration primitives = configuration.primitivesList().get("name");
+        assertNotNull(primitives);
 
-        assertThrows(IllegalArgumentException.class, () -> registry.represent(List.of("root", "x"), jsonVisitor()));
+        change("{'root':{'primitivesList':{'name':{'booleanVal' : true}}}}");
+        assertTrue(primitives.booleanVal().value());
 
-        assertEquals(
-            JsonNull.INSTANCE,
-            registry.represent(List.of("root", "primitivesList", "foo"), jsonVisitor())
+        change("{'root':{'primitivesList':{'name':{'intVal' : 12345}}}}");
+        assertEquals(12345, primitives.intVal().value());
+
+        change("{'root':{'primitivesList':{'name':{'longVal' : 12345678900}}}}");
+        assertEquals(12345678900L, primitives.longVal().value());
+
+        change("{'root':{'primitivesList':{'name':{'doubleVal' : 2.5}}}}");
+        assertEquals(2.5d, primitives.doubleVal().value());
+
+        change("{'root':{'primitivesList':{'name':{'stringVal' : 'foo'}}}}");
+        assertEquals("foo", primitives.stringVal().value());
+
+        // Wrong value types:
+        assertThrowsIllegalArgException(
+            () -> change("{'root':{'primitivesList':{'name':{'booleanVal' : 'true'}}}}"),
+            "'boolean' is expected as a type for 'root.primitivesList.name.booleanVal' configuration value"
+        );
+
+        assertThrowsIllegalArgException(
+            () -> change("{'root':{'primitivesList':{'name':{'intVal' : 12345678900}}}}"),
+            "'root.primitivesList.name.intVal' has integer type and the value 12345678900 is out of bounds"
+        );
+
+        assertThrowsIllegalArgException(
+            () -> change("{'root':{'primitivesList':{'name':{'intVal' : false}}}}"),
+            "'int' is expected as a type for 'root.primitivesList.name.intVal' configuration value"
+        );
+
+        assertThrowsIllegalArgException(
+            () -> change("{'root':{'primitivesList':{'name':{'stringVal' : 10}}}}"),
+            "'String' is expected as a type for 'root.primitivesList.name.stringVal' configuration value"
+        );
+
+        assertThrowsIllegalArgException(
+            () -> change("{'root':{'primitivesList':{'name':{'doubleVal' : []}}}}"),
+            "'double' is expected as a type for 'root.primitivesList.name.doubleVal' configuration value"
+        );
+    }
+
+    /** */
+    @Test
+    public void fromJsonArrays() throws Exception {
+        change("{'root':{'arraysList':{'name' : {}}}}");
+
+        JsonArraysConfiguration arrays = configuration.arraysList().get("name");
+        assertNotNull(arrays);
+
+        change("{'root':{'arraysList':{'name':{'booleans' : [true]}}}}");
+        assertArrayEquals(new boolean[] {true}, arrays.booleans().value());
+
+        change("{'root':{'arraysList':{'name':{'ints' : [12345]}}}}");
+        assertArrayEquals(new int[] {12345}, arrays.ints().value());
+
+        change("{'root':{'arraysList':{'name':{'longs' : [12345678900]}}}}");
+        assertArrayEquals(new long[] {12345678900L}, arrays.longs().value());
+
+        change("{'root':{'arraysList':{'name':{'doubles' : [2.5]}}}}");
+        assertArrayEquals(new double[] {2.5d}, arrays.doubles().value());
+
+        change("{'root':{'arraysList':{'name':{'strings' : ['foo']}}}}");
+        assertArrayEquals(new String[] {"foo"}, arrays.strings().value());
+
+        // Wrong value types:
+        assertThrowsIllegalArgException(
+            () -> change("{'root':{'arraysList':{'name':{'booleans' : true}}}}"),
+            "'boolean[]' is expected as a type for 'root.arraysList.name.booleans' configuration value"
+        );
+
+        assertThrowsIllegalArgException(
+            () -> change("{'root':{'arraysList':{'name':{'ints' : ['0',0]}}}}"),
+            "'int' is expected as a type for 'root.arraysList.name.ints[0]' configuration value"
+        );
+
+        assertThrowsIllegalArgException(
+            () -> change("{'root':{'arraysList':{'name':{'longs' : [0,'0']}}}}"),
+            "'long' is expected as a type for 'root.arraysList.name.longs[1]' configuration value"
+        );
+
+        assertThrowsIllegalArgException(
+            () -> change("{'root':{'arraysList':{'name':{'booleans' : [{}]}}}}"),
+            "'boolean' is expected as a type for 'root.arraysList.name.booleans[0]' configuration value"
         );
     }
+
+    /** */
+    private void change(String json) throws Exception {
+        try {
+            registry.change(jsonSource(parseString(json)), null).get(1, SECONDS);
+        }
+        catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+
+            if (cause instanceof Exception)
+                throw (Exception)cause;
+
+            throw e;
+        }
+    }
+
+    /** */
+    private static void assertThrowsIllegalArgException(Executable executable, String msg) {
+        IllegalArgumentException e = assertThrows(IllegalArgumentException.class, executable);
+
+        assertEquals(msg, e.getMessage());
+    }
 }
diff --git a/modules/runner/src/main/java/org/apache/ignite/configuration/ConfigurationModule.java b/modules/runner/src/main/java/org/apache/ignite/configuration/ConfigurationModule.java
index ccec3d0..e738974 100644
--- a/modules/runner/src/main/java/org/apache/ignite/configuration/ConfigurationModule.java
+++ b/modules/runner/src/main/java/org/apache/ignite/configuration/ConfigurationModule.java
@@ -19,7 +19,6 @@ package org.apache.ignite.configuration;
 
 import com.google.gson.JsonObject;
 import com.google.gson.JsonParser;
-import java.util.Collections;
 import java.util.concurrent.ExecutionException;
 import org.apache.ignite.configuration.extended.LocalConfiguration;
 import org.apache.ignite.rest.configuration.InMemoryConfigurationStorage;
@@ -52,7 +51,7 @@ public class ConfigurationModule {
         JsonObject jsonCfg = JsonParser.parseString(jsonStr).getAsJsonObject();
 
         try {
-            confRegistry.change(Collections.emptyList(), JsonConverter.jsonSource(jsonCfg), storage).get();
+            confRegistry.change(JsonConverter.jsonSource(jsonCfg), storage).get();
         }
         catch (ExecutionException e) {
             throw new RuntimeException(e.getCause());