You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2021/02/24 14:40:00 UTC

[GitHub] [ignite-3] ibessonov opened a new pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

ibessonov opened a new pull request #56:
URL: https://github.com/apache/ignite-3/pull/56


   Signed-off-by: ibessonov <be...@gmail.com>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] sergey-chugunov-1985 commented on pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
sergey-chugunov-1985 commented on pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#issuecomment-787841075


   Now patch looks better to me. There are still things we can improve but they are minor and could be done later in other tasks.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] ibessonov commented on a change in pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r582917148



##########
File path: modules/configuration/src/main/java/org/apache/ignite/configuration/ConfigurationChanger.java
##########
@@ -74,63 +78,65 @@ private StorageRoots(Map<RootKey<?>, InnerNode> roots, long version) {
     private final Map<Class<? extends ConfigurationStorage>, ConfigurationStorage> storageInstances = new HashMap<>();
 
     /** Constructor. */
-    public ConfigurationChanger(ConfigurationStorage... configurationStorages) {
-        for (ConfigurationStorage storage : configurationStorages)
-            storageInstances.put(storage.getClass(), storage);
+    public ConfigurationChanger(RootKey<?>... rootKeys) {
+        this.rootKeys.addAll(Arrays.asList(rootKeys));
+    }
+
+    /** */
+    public void addRootKey(RootKey<?> rootKey) {
+        assert !storageInstances.containsKey(rootKey.getStorageType());
+
+        rootKeys.add(rootKey);
     }
 
     /**
      * Initialize changer.
      */
     // ConfigurationChangeException, really?
-    public void init(RootKey<?>... rootKeys) throws ConfigurationChangeException {
-        Map<Class<? extends ConfigurationStorage>, Set<RootKey<?>>> rootsByStorage = new HashMap<>();
-
-        for (RootKey<?> rootKey : rootKeys) {
-            Class<? extends ConfigurationStorage> storageType = rootKey.getStorageType();
+    public void init(ConfigurationStorage configurationStorage) throws ConfigurationChangeException {
+        storageInstances.put(configurationStorage.getClass(), configurationStorage);
 
-            rootsByStorage.computeIfAbsent(storageType, c -> new HashSet<>()).add(rootKey);
-        }
+        Set<RootKey<?>> storageRootKeys = rootKeys.stream().filter(
+            rootKey -> configurationStorage.getClass() == rootKey.getStorageType()
+        ).collect(Collectors.toSet());
 
-        for (ConfigurationStorage configurationStorage : storageInstances.values()) {
-            Data data;
+        Data data;
 
-            try {
-                data = configurationStorage.readAll();
-            }
-            catch (StorageException e) {
-                throw new ConfigurationChangeException("Failed to initialize configuration: " + e.getMessage(), e);
-            }
+        try {
+            data = configurationStorage.readAll();
+        }
+        catch (StorageException e) {
+            throw new ConfigurationChangeException("Failed to initialize configuration: " + e.getMessage(), e);
+        }
 
-            Map<RootKey<?>, InnerNode> storageRootsMap = new HashMap<>();
+        Map<RootKey<?>, InnerNode> storageRootsMap = new HashMap<>();
 
-            Map<String, ?> dataValuesPrefixMap = ConfigurationUtil.toPrefixMap(data.values());
+        Map<String, ?> dataValuesPrefixMap = ConfigurationUtil.toPrefixMap(data.values());
 
-            for (RootKey<?> rootKey : rootsByStorage.get(configurationStorage.getClass())) {
-                Map<String, ?> rootPrefixMap = (Map<String, ?>)dataValuesPrefixMap.get(rootKey.key());
+        for (RootKey<?> rootKey : storageRootKeys) {
+            Map<String, ?> rootPrefixMap = (Map<String, ?>)dataValuesPrefixMap.get(rootKey.key());
 
-                if (rootPrefixMap == null) {
-                    //TODO IGNITE-14193 Init with defaults.
-                    storageRootsMap.put(rootKey, rootKey.createRootNode());
-                }
-                else {
-                    InnerNode rootNode = rootKey.createRootNode();
+            if (rootPrefixMap == null) {
+                //TODO IGNITE-14193 Init with defaults.
+                storageRootsMap.put(rootKey, rootKey.createRootNode());
+            }
+            else {
+                InnerNode rootNode = rootKey.createRootNode();
 
-                    ConfigurationUtil.fillFromPrefixMap(rootNode, rootPrefixMap);
+                ConfigurationUtil.fillFromPrefixMap(rootNode, rootPrefixMap);
 
-                    storageRootsMap.put(rootKey, rootNode);
-                }
+                storageRootsMap.put(rootKey, rootNode);
             }
+        }
 
-            storagesRootsMap.put(configurationStorage.getClass(), new StorageRoots(storageRootsMap, data.version()));
+        storagesRootsMap.put(configurationStorage.getClass(), new StorageRoots(storageRootsMap, data.version()));
 
-            configurationStorage.addListener(changedEntries -> updateFromListener(
-                configurationStorage.getClass(),
-                changedEntries
-            ));
+        configurationStorage.addListener(changedEntries -> updateFromListener(
+            configurationStorage.getClass(),
+            changedEntries
+        ));
 
-            // TODO: IGNITE-14118 iterate over data and fill Configurators
-        }
+        // TODO: IGNITE-14118 iterate over data and fill Configurators

Review comment:
       Removed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] sergey-chugunov-1985 commented on a change in pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
sergey-chugunov-1985 commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r582609046



##########
File path: modules/configuration/src/main/java/org/apache/ignite/configuration/ConfigurationChanger.java
##########
@@ -74,63 +78,65 @@ private StorageRoots(Map<RootKey<?>, InnerNode> roots, long version) {
     private final Map<Class<? extends ConfigurationStorage>, ConfigurationStorage> storageInstances = new HashMap<>();
 
     /** Constructor. */
-    public ConfigurationChanger(ConfigurationStorage... configurationStorages) {
-        for (ConfigurationStorage storage : configurationStorages)
-            storageInstances.put(storage.getClass(), storage);
+    public ConfigurationChanger(RootKey<?>... rootKeys) {
+        this.rootKeys.addAll(Arrays.asList(rootKeys));
+    }
+
+    /** */
+    public void addRootKey(RootKey<?> rootKey) {
+        assert !storageInstances.containsKey(rootKey.getStorageType());
+
+        rootKeys.add(rootKey);
     }
 
     /**
      * Initialize changer.
      */
     // ConfigurationChangeException, really?
-    public void init(RootKey<?>... rootKeys) throws ConfigurationChangeException {
-        Map<Class<? extends ConfigurationStorage>, Set<RootKey<?>>> rootsByStorage = new HashMap<>();
-
-        for (RootKey<?> rootKey : rootKeys) {
-            Class<? extends ConfigurationStorage> storageType = rootKey.getStorageType();
+    public void init(ConfigurationStorage configurationStorage) throws ConfigurationChangeException {
+        storageInstances.put(configurationStorage.getClass(), configurationStorage);
 
-            rootsByStorage.computeIfAbsent(storageType, c -> new HashSet<>()).add(rootKey);
-        }
+        Set<RootKey<?>> storageRootKeys = rootKeys.stream().filter(
+            rootKey -> configurationStorage.getClass() == rootKey.getStorageType()
+        ).collect(Collectors.toSet());
 
-        for (ConfigurationStorage configurationStorage : storageInstances.values()) {
-            Data data;
+        Data data;
 
-            try {
-                data = configurationStorage.readAll();
-            }
-            catch (StorageException e) {
-                throw new ConfigurationChangeException("Failed to initialize configuration: " + e.getMessage(), e);
-            }
+        try {
+            data = configurationStorage.readAll();
+        }
+        catch (StorageException e) {
+            throw new ConfigurationChangeException("Failed to initialize configuration: " + e.getMessage(), e);
+        }
 
-            Map<RootKey<?>, InnerNode> storageRootsMap = new HashMap<>();
+        Map<RootKey<?>, InnerNode> storageRootsMap = new HashMap<>();
 
-            Map<String, ?> dataValuesPrefixMap = ConfigurationUtil.toPrefixMap(data.values());
+        Map<String, ?> dataValuesPrefixMap = ConfigurationUtil.toPrefixMap(data.values());
 
-            for (RootKey<?> rootKey : rootsByStorage.get(configurationStorage.getClass())) {
-                Map<String, ?> rootPrefixMap = (Map<String, ?>)dataValuesPrefixMap.get(rootKey.key());
+        for (RootKey<?> rootKey : storageRootKeys) {
+            Map<String, ?> rootPrefixMap = (Map<String, ?>)dataValuesPrefixMap.get(rootKey.key());
 
-                if (rootPrefixMap == null) {
-                    //TODO IGNITE-14193 Init with defaults.
-                    storageRootsMap.put(rootKey, rootKey.createRootNode());
-                }
-                else {
-                    InnerNode rootNode = rootKey.createRootNode();
+            if (rootPrefixMap == null) {
+                //TODO IGNITE-14193 Init with defaults.
+                storageRootsMap.put(rootKey, rootKey.createRootNode());
+            }
+            else {
+                InnerNode rootNode = rootKey.createRootNode();
 
-                    ConfigurationUtil.fillFromPrefixMap(rootNode, rootPrefixMap);
+                ConfigurationUtil.fillFromPrefixMap(rootNode, rootPrefixMap);
 
-                    storageRootsMap.put(rootKey, rootNode);
-                }
+                storageRootsMap.put(rootKey, rootNode);
             }
+        }
 
-            storagesRootsMap.put(configurationStorage.getClass(), new StorageRoots(storageRootsMap, data.version()));
+        storagesRootsMap.put(configurationStorage.getClass(), new StorageRoots(storageRootsMap, data.version()));
 
-            configurationStorage.addListener(changedEntries -> updateFromListener(
-                configurationStorage.getClass(),
-                changedEntries
-            ));
+        configurationStorage.addListener(changedEntries -> updateFromListener(
+            configurationStorage.getClass(),
+            changedEntries
+        ));
 
-            // TODO: IGNITE-14118 iterate over data and fill Configurators
-        }
+        // TODO: IGNITE-14118 iterate over data and fill Configurators

Review comment:
       obsolete




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] ibessonov commented on a change in pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r582901496



##########
File path: modules/configuration/src/main/java/org/apache/ignite/configuration/internal/DynamicConfiguration.java
##########
@@ -108,43 +85,50 @@ protected DynamicConfiguration(
     ) {
         members.put(member.key(), member);
 
-        configurator.addValidations((Class<? extends ConfigurationTree<?, ?>>) getClass(), member.key(), validators);
+//        configurator.addValidations((Class<? extends ConfigurationTree<?, ?>>) getClass(), member.key(), validators);
     }
 
     /** {@inheritDoc} */
-    @Override public void change(CHANGE change) throws ConfigurationValidationException {
-        configurator.set(BaseSelectors.find(qualifiedName), change);
+    @Override public Future<Void> change(Consumer<CHANGE> change) throws ConfigurationValidationException {
+        Objects.requireNonNull(change, "Configuration consumer cannot be null.");
+
+        InnerNode rootNodeChange = ((RootKeyImpl)rootKey).createRootNode();
+
+        if (keys.size() == 1)
+            change.accept((CHANGE)rootNodeChange);
+        else {
+            // TODO Not optimal, can be improved. Do it when tests are ready.
+            fillFromPrefixMap(rootNodeChange, toPrefixMap(Collections.singletonMap(join(keys), null)));

Review comment:
       Done, I hope it's better now




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] sergey-chugunov-1985 commented on a change in pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
sergey-chugunov-1985 commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r582622447



##########
File path: modules/configuration/src/main/java/org/apache/ignite/configuration/internal/DynamicConfiguration.java
##########
@@ -108,43 +85,50 @@ protected DynamicConfiguration(
     ) {
         members.put(member.key(), member);
 
-        configurator.addValidations((Class<? extends ConfigurationTree<?, ?>>) getClass(), member.key(), validators);
+//        configurator.addValidations((Class<? extends ConfigurationTree<?, ?>>) getClass(), member.key(), validators);
     }
 
     /** {@inheritDoc} */
-    @Override public void change(CHANGE change) throws ConfigurationValidationException {
-        configurator.set(BaseSelectors.find(qualifiedName), change);
+    @Override public Future<Void> change(Consumer<CHANGE> change) throws ConfigurationValidationException {
+        Objects.requireNonNull(change, "Configuration consumer cannot be null.");
+
+        InnerNode rootNodeChange = ((RootKeyImpl)rootKey).createRootNode();
+
+        if (keys.size() == 1)
+            change.accept((CHANGE)rootNodeChange);
+        else {
+            // TODO Not optimal, can be improved. Do it when tests are ready.
+            fillFromPrefixMap(rootNodeChange, toPrefixMap(Collections.singletonMap(join(keys), null)));

Review comment:
       this code needs to be refactored for more clarity




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] ibessonov commented on a change in pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r582908711



##########
File path: modules/configuration/src/main/java/org/apache/ignite/configuration/internal/ConfigurationNode.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.configuration.internal;
+
+import java.util.List;
+import java.util.NoSuchElementException;
+import org.apache.ignite.configuration.ConfigurationChanger;
+import org.apache.ignite.configuration.RootKey;
+import org.apache.ignite.configuration.internal.util.ConfigurationUtil;
+import org.apache.ignite.configuration.internal.util.KeyNotFoundException;
+import org.apache.ignite.configuration.tree.TraversableTreeNode;
+
+/**
+ * Super class for dynamic configuration tree nodes. Has all common data and value retrieving algorithm in it.
+ */
+public abstract class ConfigurationNode<VIEW> {
+    /** Full path to the current node. */
+    protected final List<String> keys;
+
+    /** Name of the current node. Same as last element of {@link #keys}. */
+    protected final String key;
+
+    /** Root key instance for the current trees root. */
+    protected final RootKey<?> rootKey;
+
+    /** Configuration changer instance to get latest value of the root. */
+    protected final ConfigurationChanger changer;
+
+    /**
+     * Cached value of current trees root. Useful to determine whether you have the latest configuration value or not.
+     */
+    private volatile TraversableTreeNode cachedRootNode;
+
+    /** Cached configuration value. Immutable. */
+    private VIEW val;
+
+    /**
+     * Validity flag. Configuration is declared invalid if it's a part of named list configuration and corresponding
+     * entry is already removed.
+     */
+    private boolean invalid;
+
+    /**
+     * Constructor.
+     *
+     * @param prefix Configuration prefix.
+     * @param key Configuration key.
+     * @param rootKey Root key.
+     * @param changer Configuration changer.
+     */
+    protected ConfigurationNode(List<String> prefix, String key, RootKey<?> rootKey, ConfigurationChanger changer) {
+        this.keys = ConfigurationUtil.appendKey(prefix, key);
+        this.key = key;
+        this.rootKey = rootKey;
+        this.changer = changer;
+    }
+
+    /**
+     * Returns latest value of the configuration or throws exception.
+     *
+     * @return Latest configuration value.
+     * @throws NoSuchElementException If configuration is a part of already deleted named list configuration entry.
+     */
+    protected final VIEW refreshValue() throws NoSuchElementException {
+        if (invalid)

Review comment:
       I am still not sure that second synchronized is required. Gotta think about it. Anyway, we don't need sync access for reading this field, "cachedRootNode" volatility guarantees everything.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] ibessonov commented on a change in pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r583674964



##########
File path: modules/configuration-annotation-processor/src/test/java/org/apache/ignite/configuration/sample/UsageTest.java
##########
@@ -35,74 +33,88 @@
      * Test creation of configuration and calling configuration API methods.
      */
     @Test
-    public void test() {
-        InitLocal initLocal = new InitLocal().withBaseline(
-            new InitBaseline()
-                .withNodes(
-                    new NamedList<>(
-                        Collections.singletonMap("node1", new InitNode().withConsistentId("test").withPort(1000))
+    public void test() throws Exception {
+        var registry = new ConfigurationRegistry();
+
+        registry.registerRootKey(LocalConfiguration.KEY);
+
+        registry.registerStorage(new TestConfigurationStorage());
+
+        LocalConfiguration root = registry.getConfiguration(LocalConfiguration.KEY);
+
+        root.change(local ->
+            local.changeBaseline(baseline ->
+                baseline.changeNodes(nodes ->
+                    nodes.create("node1", node ->
+                        node.initConsistentId("test").initPort(1000)
                     )
+                ).changeAutoAdjust(autoAdjust ->
+                    autoAdjust.changeEnabled(true).changeTimeout(100_000L)
                 )
-                .withAutoAdjust(new InitAutoAdjust().withEnabled(true).withTimeout(100000L))
-        );
+            )
+        ).get();

Review comment:
       Ok, fair point




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] ibessonov commented on a change in pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r582904285



##########
File path: modules/configuration/src/main/java/org/apache/ignite/configuration/ConfigurationValue.java
##########
@@ -17,9 +17,20 @@
 
 package org.apache.ignite.configuration;
 
+import java.util.concurrent.Future;
+import org.apache.ignite.configuration.validation.ConfigurationValidationException;
+
 /**
  * Configuration value.
  * @param <VALUE> Type of the value.
  */
 public interface ConfigurationValue<VALUE> extends ConfigurationProperty<VALUE, VALUE> {
+
+    /**
+     * Change this configuration node value.
+     * @param change CHANGE object.
+     * @throws ConfigurationValidationException If validation failed.
+     */
+    //TODO Rename.
+    Future<Void> change(VALUE change) throws ConfigurationValidationException;

Review comment:
       Future can potentially throw that exception, not the method itself




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] ibessonov commented on a change in pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r582900177



##########
File path: modules/configuration-annotation-processor/src/test/java/org/apache/ignite/configuration/sample/UsageTest.java
##########
@@ -35,36 +31,42 @@
      * Test creation of configuration and calling configuration API methods.
      */
     @Test
-    public void test() {
-        InitLocal initLocal = new InitLocal().withBaseline(
-            new InitBaseline()
-                .withNodes(
-                    new NamedList<>(
-                        Collections.singletonMap("node1", new InitNode().withConsistentId("test").withPort(1000))
+    public void test() throws Exception {
+        var registry = new ConfigurationRegistry();
+
+        registry.registerRootKey(LocalConfiguration.KEY);
+
+        registry.registerStorage(new TestConfigurationStorage());
+
+        LocalConfiguration root = registry.getConfiguration(LocalConfiguration.KEY);
+
+        root.change(local ->
+            local.changeBaseline(baseline ->
+                baseline.changeNodes(nodes ->
+                    nodes.create("node1", node ->
+                        node.initConsistentId("test").initPort(1000)
                     )
+                ).changeAutoAdjust(autoAdjust ->
+                    autoAdjust.changeEnabled(true).changeTimeout(100_000L)
                 )
-                .withAutoAdjust(new InitAutoAdjust().withEnabled(true).withTimeout(100000L))
-        );
-
-        final Configurator<LocalConfigurationImpl> configurator = Configurator.create(
-            LocalConfigurationImpl::new,
-            initLocal
-        );
-
-        final LocalConfiguration root = configurator.getRoot();
-        root.baseline().autoAdjust().enabled().value();
-
-        Assertions.assertThrows(ConfigurationValidationException.class, () -> {
-            configurator.set(Selectors.LOCAL_BASELINE_AUTO_ADJUST_ENABLED, false);
-        });
-        configurator.set(Selectors.LOCAL_BASELINE_AUTO_ADJUST, new ChangeAutoAdjust().withEnabled(false).withTimeout(0L));
-        configurator.getRoot().baseline().nodes().get("node1").autoAdjustEnabled(false);
-        configurator.getRoot().baseline().autoAdjust().enabled(true);
-        configurator.getRoot().baseline().nodes().get("node1").autoAdjustEnabled(true);
-
-        Assertions.assertThrows(ConfigurationValidationException.class, () -> {
-            configurator.getRoot().baseline().autoAdjust().enabled(false);
-        });
+            )
+        ).get();
+
+        assertTrue(root.baseline().autoAdjust().enabled().value());
+
+//        assertThrows(ConfigurationValidationException.class, () -> {

Review comment:
       Removed this code completely




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] ibessonov commented on a change in pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r582901102



##########
File path: modules/configuration/src/main/java/org/apache/ignite/configuration/ConfigurationChanger.java
##########
@@ -279,8 +286,11 @@ private ValidationResult validate(
 
             final Configurator<?> configurator = configurators.get(rootKey);
 
-            List<ValidationIssue> list = configurator.validateChanges(changesForRoot);
-            issues.addAll(list);
+            // TODO This will be fixed later.

Review comment:
       Done

##########
File path: modules/configuration/src/main/java/org/apache/ignite/configuration/internal/ConfigurationNode.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.configuration.internal;
+
+import java.util.List;
+import java.util.NoSuchElementException;
+import org.apache.ignite.configuration.ConfigurationChanger;
+import org.apache.ignite.configuration.RootKey;
+import org.apache.ignite.configuration.internal.util.ConfigurationUtil;
+import org.apache.ignite.configuration.internal.util.KeyNotFoundException;
+import org.apache.ignite.configuration.tree.TraversableTreeNode;
+
+/** */
+public abstract class ConfigurationNode<VIEW> {
+    protected final List<String> keys;
+
+    protected final String key;
+
+    protected final ConfigurationChanger changer;
+
+    protected final RootKey<?> rootKey;
+
+    private volatile TraversableTreeNode cachedRootNode;
+
+    protected VIEW val;
+
+    private boolean invalid;
+
+    protected ConfigurationNode(List<String> prefix, String key, RootKey<?> rootKey, ConfigurationChanger changer) {
+        this.keys = ConfigurationUtil.appendKey(prefix, key);
+        this.key = key;
+        this.rootKey = rootKey;
+        this.changer = changer;
+    }
+
+    /** */
+    protected VIEW refresh() {
+        checkValueValidity();
+
+        TraversableTreeNode newRootNode = changer.getRootNode(rootKey);
+        TraversableTreeNode oldRootNode = cachedRootNode;
+
+        if (oldRootNode == newRootNode)
+            return val;
+
+        try {
+            VIEW newVal = (VIEW)ConfigurationUtil.find(keys.subList(1, keys.size()), newRootNode);
+
+            synchronized (this) {
+                if (cachedRootNode == oldRootNode) {
+                    cachedRootNode = newRootNode;
+
+                    refresh0(newVal);
+
+                    return val = newVal;
+                }
+                else {
+                    checkValueValidity();
+
+                    return val;
+                }
+            }
+        }
+        catch (KeyNotFoundException e) {
+            invalid = true;

Review comment:
       Fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] sergey-chugunov-1985 commented on a change in pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
sergey-chugunov-1985 commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r582616308



##########
File path: modules/configuration/src/main/java/org/apache/ignite/configuration/internal/ConfigurationNode.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.configuration.internal;
+
+import java.util.List;
+import java.util.NoSuchElementException;
+import org.apache.ignite.configuration.ConfigurationChanger;
+import org.apache.ignite.configuration.RootKey;
+import org.apache.ignite.configuration.internal.util.ConfigurationUtil;
+import org.apache.ignite.configuration.internal.util.KeyNotFoundException;
+import org.apache.ignite.configuration.tree.TraversableTreeNode;
+
+/** */
+public abstract class ConfigurationNode<VIEW> {
+    protected final List<String> keys;
+
+    protected final String key;
+
+    protected final ConfigurationChanger changer;
+
+    protected final RootKey<?> rootKey;
+
+    private volatile TraversableTreeNode cachedRootNode;
+
+    protected VIEW val;
+
+    private boolean invalid;
+
+    protected ConfigurationNode(List<String> prefix, String key, RootKey<?> rootKey, ConfigurationChanger changer) {
+        this.keys = ConfigurationUtil.appendKey(prefix, key);
+        this.key = key;
+        this.rootKey = rootKey;
+        this.changer = changer;
+    }
+
+    /** */
+    protected VIEW refresh() {
+        checkValueValidity();
+
+        TraversableTreeNode newRootNode = changer.getRootNode(rootKey);
+        TraversableTreeNode oldRootNode = cachedRootNode;
+
+        if (oldRootNode == newRootNode)
+            return val;
+
+        try {
+            VIEW newVal = (VIEW)ConfigurationUtil.find(keys.subList(1, keys.size()), newRootNode);
+
+            synchronized (this) {
+                if (cachedRootNode == oldRootNode) {
+                    cachedRootNode = newRootNode;
+
+                    refresh0(newVal);
+
+                    return val = newVal;
+                }
+                else {
+                    checkValueValidity();
+
+                    return val;
+                }
+            }
+        }
+        catch (KeyNotFoundException e) {
+            invalid = true;

Review comment:
       possible visibility issue




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] SammyVimes commented on a change in pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r582752035



##########
File path: modules/configuration-annotation-processor/src/test/java/org/apache/ignite/configuration/sample/UsageTest.java
##########
@@ -35,74 +33,88 @@
      * Test creation of configuration and calling configuration API methods.
      */
     @Test
-    public void test() {
-        InitLocal initLocal = new InitLocal().withBaseline(
-            new InitBaseline()
-                .withNodes(
-                    new NamedList<>(
-                        Collections.singletonMap("node1", new InitNode().withConsistentId("test").withPort(1000))
+    public void test() throws Exception {
+        var registry = new ConfigurationRegistry();
+
+        registry.registerRootKey(LocalConfiguration.KEY);
+
+        registry.registerStorage(new TestConfigurationStorage());
+
+        LocalConfiguration root = registry.getConfiguration(LocalConfiguration.KEY);
+
+        root.change(local ->
+            local.changeBaseline(baseline ->
+                baseline.changeNodes(nodes ->
+                    nodes.create("node1", node ->
+                        node.initConsistentId("test").initPort(1000)
                     )
+                ).changeAutoAdjust(autoAdjust ->
+                    autoAdjust.changeEnabled(true).changeTimeout(100_000L)
                 )
-                .withAutoAdjust(new InitAutoAdjust().withEnabled(true).withTimeout(100000L))
-        );
+            )
+        ).get();

Review comment:
       Should we use Future#get without some sensible timeout?

##########
File path: modules/configuration/src/main/java/org/apache/ignite/configuration/internal/NamedListConfiguration.java
##########
@@ -38,89 +44,41 @@
      * Constructor.
      * @param prefix Configuration prefix.
      * @param key Configuration key.
-     * @param configurator Configurator that this object is attached to.
-     * @param root Root configuration.
+     * @param rootKey Root key.
+     * @param changer Configuration changer.
      * @param creator Underlying configuration creator function.
      */
     public NamedListConfiguration(
-        String prefix,
+        List<String> prefix,
         String key,
-        Configurator<? extends DynamicConfiguration<?, ?, ?>> configurator,
-        DynamicConfiguration<?, ?, ?> root,
-        BiFunction<String, String, T> creator
-    ) {
-        super(prefix, key, false, configurator, root);
+        RootKey<?> rootKey,
+        ConfigurationChanger changer,
+        BiFunction<List<String>, String, T> creator) {
+        super(prefix, key, rootKey, changer);
         this.creator = creator;
     }
 
-    /**
-     * Copy constructor.
-     * @param base Base to copy from.
-     * @param configurator Configurator to attach to.
-     * @param root Root of the configuration.
-     */
-    private NamedListConfiguration(
-        NamedListConfiguration<VIEW, T, INIT, CHANGE> base,
-        Configurator<? extends DynamicConfiguration<?, ?, ?>> configurator,
-        DynamicConfiguration<?, ?, ?> root
-    ) {
-        super(base.prefix, base.key, false, configurator, root);
-
-        this.creator = base.creator;
-
-        for (Map.Entry<String, T> entry : base.values.entrySet()) {
-            String k = entry.getKey();
-            T value = entry.getValue();
-
-            final T copy = (T) ((DynamicConfiguration<VIEW, INIT, CHANGE>) value).copy(root);
-            add(copy);
-
-            this.values.put(k, copy);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void init(NamedList<INIT> list) {
-        list.getValues().forEach((key, init) -> {
-            if (!values.containsKey(key)) {
-                final T created = creator.apply(qualifiedName, key);
-                add(created);
-                values.put(key, created);
-            }
-
-            values.get(key).init(init);
-        });
-    }
-
     /**
      * Get named configuration by name.
      * @param name Name.
      * @return Configuration.
      */
     public T get(String name) {
-        return values.get(name);
-    }
+        refreshValue();
 
-    /** {@inheritDoc} */
-    @Override public NamedList<VIEW> value() {
-        return new NamedList<>(values.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, it -> it.getValue().value())));
+        return values.get(name); //TODO Exceptions.
     }
 
     /** {@inheritDoc} */
-    @Override public void changeWithoutValidation(NamedList<CHANGE> list) {
-        list.getValues().forEach((key, change) -> {
-            if (!values.containsKey(key)) {
-                final T created = creator.apply(qualifiedName, key);
-                add(created);
-                values.put(key, created);
-            }
+    @Override protected synchronized void refreshValue0(NamedListView<VIEW> newValue) {
+        //TODO Just swap it, we don't need actual concurrent access.
+        Set<String> newKeys = newValue.namedListKeys();
 
-            values.get(key).changeWithoutValidation(change);
-        });
-    }
+        values.keySet().removeIf(key -> !newKeys.contains(key));
 
-    /** {@inheritDoc} */
-    @Override public NamedListConfiguration<VIEW, T, INIT, CHANGE> copy(DynamicConfiguration<?, ?, ?> root) {
-        return new NamedListConfiguration<>(this, configurator, root);
+        for (String newKey : newKeys) {
+            if (!values.containsKey(newKey))
+                values.put(newKey, creator.apply(keys, newKey));

Review comment:
       Doesn't this create default values?

##########
File path: modules/configuration/src/main/java/org/apache/ignite/configuration/ConfigurationValue.java
##########
@@ -17,9 +17,20 @@
 
 package org.apache.ignite.configuration;
 
+import java.util.concurrent.Future;
+import org.apache.ignite.configuration.validation.ConfigurationValidationException;
+
 /**
  * Configuration value.
  * @param <VALUE> Type of the value.
  */
 public interface ConfigurationValue<VALUE> extends ConfigurationProperty<VALUE, VALUE> {
+
+    /**
+     * Change this configuration node value.
+     * @param change CHANGE object.
+     * @throws ConfigurationValidationException If validation failed.
+     */
+    //TODO Rename.
+    Future<Void> change(VALUE change) throws ConfigurationValidationException;

Review comment:
       I think this method also throws ConfigurationChangeException

##########
File path: modules/configuration/src/main/java/org/apache/ignite/configuration/internal/ConfigurationNode.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.configuration.internal;
+
+import java.util.List;
+import java.util.NoSuchElementException;
+import org.apache.ignite.configuration.ConfigurationChanger;
+import org.apache.ignite.configuration.RootKey;
+import org.apache.ignite.configuration.internal.util.ConfigurationUtil;
+import org.apache.ignite.configuration.internal.util.KeyNotFoundException;
+import org.apache.ignite.configuration.tree.TraversableTreeNode;
+
+/**
+ * Super class for dynamic configuration tree nodes. Has all common data and value retrieving algorithm in it.
+ */
+public abstract class ConfigurationNode<VIEW> {
+    /** Full path to the current node. */
+    protected final List<String> keys;
+
+    /** Name of the current node. Same as last element of {@link #keys}. */
+    protected final String key;
+
+    /** Root key instance for the current trees root. */
+    protected final RootKey<?> rootKey;
+
+    /** Configuration changer instance to get latest value of the root. */
+    protected final ConfigurationChanger changer;
+
+    /**
+     * Cached value of current trees root. Useful to determine whether you have the latest configuration value or not.
+     */
+    private volatile TraversableTreeNode cachedRootNode;
+
+    /** Cached configuration value. Immutable. */
+    private VIEW val;
+
+    /**
+     * Validity flag. Configuration is declared invalid if it's a part of named list configuration and corresponding
+     * entry is already removed.
+     */
+    private boolean invalid;
+
+    /**
+     * Constructor.
+     *
+     * @param prefix Configuration prefix.
+     * @param key Configuration key.
+     * @param rootKey Root key.
+     * @param changer Configuration changer.
+     */
+    protected ConfigurationNode(List<String> prefix, String key, RootKey<?> rootKey, ConfigurationChanger changer) {
+        this.keys = ConfigurationUtil.appendKey(prefix, key);
+        this.key = key;
+        this.rootKey = rootKey;
+        this.changer = changer;
+    }
+
+    /**
+     * Returns latest value of the configuration or throws exception.
+     *
+     * @return Latest configuration value.
+     * @throws NoSuchElementException If configuration is a part of already deleted named list configuration entry.
+     */
+    protected final VIEW refreshValue() throws NoSuchElementException {
+        if (invalid)

Review comment:
       all subsequent accesses to this variable are synchronized, don't we need synchronization here too?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] ibessonov commented on a change in pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r582908711



##########
File path: modules/configuration/src/main/java/org/apache/ignite/configuration/internal/ConfigurationNode.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.configuration.internal;
+
+import java.util.List;
+import java.util.NoSuchElementException;
+import org.apache.ignite.configuration.ConfigurationChanger;
+import org.apache.ignite.configuration.RootKey;
+import org.apache.ignite.configuration.internal.util.ConfigurationUtil;
+import org.apache.ignite.configuration.internal.util.KeyNotFoundException;
+import org.apache.ignite.configuration.tree.TraversableTreeNode;
+
+/**
+ * Super class for dynamic configuration tree nodes. Has all common data and value retrieving algorithm in it.
+ */
+public abstract class ConfigurationNode<VIEW> {
+    /** Full path to the current node. */
+    protected final List<String> keys;
+
+    /** Name of the current node. Same as last element of {@link #keys}. */
+    protected final String key;
+
+    /** Root key instance for the current trees root. */
+    protected final RootKey<?> rootKey;
+
+    /** Configuration changer instance to get latest value of the root. */
+    protected final ConfigurationChanger changer;
+
+    /**
+     * Cached value of current trees root. Useful to determine whether you have the latest configuration value or not.
+     */
+    private volatile TraversableTreeNode cachedRootNode;
+
+    /** Cached configuration value. Immutable. */
+    private VIEW val;
+
+    /**
+     * Validity flag. Configuration is declared invalid if it's a part of named list configuration and corresponding
+     * entry is already removed.
+     */
+    private boolean invalid;
+
+    /**
+     * Constructor.
+     *
+     * @param prefix Configuration prefix.
+     * @param key Configuration key.
+     * @param rootKey Root key.
+     * @param changer Configuration changer.
+     */
+    protected ConfigurationNode(List<String> prefix, String key, RootKey<?> rootKey, ConfigurationChanger changer) {
+        this.keys = ConfigurationUtil.appendKey(prefix, key);
+        this.key = key;
+        this.rootKey = rootKey;
+        this.changer = changer;
+    }
+
+    /**
+     * Returns latest value of the configuration or throws exception.
+     *
+     * @return Latest configuration value.
+     * @throws NoSuchElementException If configuration is a part of already deleted named list configuration entry.
+     */
+    protected final VIEW refreshValue() throws NoSuchElementException {
+        if (invalid)

Review comment:
       I am still not sure that second synchronized is required. Gotta think about it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] ibessonov commented on a change in pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r582902884



##########
File path: modules/configuration-annotation-processor/src/test/java/org/apache/ignite/configuration/sample/UsageTest.java
##########
@@ -35,74 +33,88 @@
      * Test creation of configuration and calling configuration API methods.
      */
     @Test
-    public void test() {
-        InitLocal initLocal = new InitLocal().withBaseline(
-            new InitBaseline()
-                .withNodes(
-                    new NamedList<>(
-                        Collections.singletonMap("node1", new InitNode().withConsistentId("test").withPort(1000))
+    public void test() throws Exception {
+        var registry = new ConfigurationRegistry();
+
+        registry.registerRootKey(LocalConfiguration.KEY);
+
+        registry.registerStorage(new TestConfigurationStorage());
+
+        LocalConfiguration root = registry.getConfiguration(LocalConfiguration.KEY);
+
+        root.change(local ->
+            local.changeBaseline(baseline ->
+                baseline.changeNodes(nodes ->
+                    nodes.create("node1", node ->
+                        node.initConsistentId("test").initPort(1000)
                     )
+                ).changeAutoAdjust(autoAdjust ->
+                    autoAdjust.changeEnabled(true).changeTimeout(100_000L)
                 )
-                .withAutoAdjust(new InitAutoAdjust().withEnabled(true).withTimeout(100000L))
-        );
+            )
+        ).get();

Review comment:
       I don't know) In production code - probably not, in tests - sure we can




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] ibessonov commented on a change in pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r582907528



##########
File path: modules/configuration/src/main/java/org/apache/ignite/configuration/internal/NamedListConfiguration.java
##########
@@ -38,89 +44,41 @@
      * Constructor.
      * @param prefix Configuration prefix.
      * @param key Configuration key.
-     * @param configurator Configurator that this object is attached to.
-     * @param root Root configuration.
+     * @param rootKey Root key.
+     * @param changer Configuration changer.
      * @param creator Underlying configuration creator function.
      */
     public NamedListConfiguration(
-        String prefix,
+        List<String> prefix,
         String key,
-        Configurator<? extends DynamicConfiguration<?, ?, ?>> configurator,
-        DynamicConfiguration<?, ?, ?> root,
-        BiFunction<String, String, T> creator
-    ) {
-        super(prefix, key, false, configurator, root);
+        RootKey<?> rootKey,
+        ConfigurationChanger changer,
+        BiFunction<List<String>, String, T> creator) {
+        super(prefix, key, rootKey, changer);
         this.creator = creator;
     }
 
-    /**
-     * Copy constructor.
-     * @param base Base to copy from.
-     * @param configurator Configurator to attach to.
-     * @param root Root of the configuration.
-     */
-    private NamedListConfiguration(
-        NamedListConfiguration<VIEW, T, INIT, CHANGE> base,
-        Configurator<? extends DynamicConfiguration<?, ?, ?>> configurator,
-        DynamicConfiguration<?, ?, ?> root
-    ) {
-        super(base.prefix, base.key, false, configurator, root);
-
-        this.creator = base.creator;
-
-        for (Map.Entry<String, T> entry : base.values.entrySet()) {
-            String k = entry.getKey();
-            T value = entry.getValue();
-
-            final T copy = (T) ((DynamicConfiguration<VIEW, INIT, CHANGE>) value).copy(root);
-            add(copy);
-
-            this.values.put(k, copy);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void init(NamedList<INIT> list) {
-        list.getValues().forEach((key, init) -> {
-            if (!values.containsKey(key)) {
-                final T created = creator.apply(qualifiedName, key);
-                add(created);
-                values.put(key, created);
-            }
-
-            values.get(key).init(init);
-        });
-    }
-
     /**
      * Get named configuration by name.
      * @param name Name.
      * @return Configuration.
      */
     public T get(String name) {
-        return values.get(name);
-    }
+        refreshValue();
 
-    /** {@inheritDoc} */
-    @Override public NamedList<VIEW> value() {
-        return new NamedList<>(values.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, it -> it.getValue().value())));
+        return values.get(name); //TODO Exceptions.
     }
 
     /** {@inheritDoc} */
-    @Override public void changeWithoutValidation(NamedList<CHANGE> list) {
-        list.getValues().forEach((key, change) -> {
-            if (!values.containsKey(key)) {
-                final T created = creator.apply(qualifiedName, key);
-                add(created);
-                values.put(key, created);
-            }
+    @Override protected synchronized void refreshValue0(NamedListView<VIEW> newValue) {
+        //TODO Just swap it, we don't need actual concurrent access.
+        Set<String> newKeys = newValue.namedListKeys();
 
-            values.get(key).changeWithoutValidation(change);
-        });
-    }
+        values.keySet().removeIf(key -> !newKeys.contains(key));
 
-    /** {@inheritDoc} */
-    @Override public NamedListConfiguration<VIEW, T, INIT, CHANGE> copy(DynamicConfiguration<?, ?, ?> root) {
-        return new NamedListConfiguration<>(this, configurator, root);
+        for (String newKey : newKeys) {
+            if (!values.containsKey(newKey))
+                values.put(newKey, creator.apply(keys, newKey));

Review comment:
       See no problems with that ;) These defaults couldn't be read because of "refresh" method guarantees.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] SammyVimes commented on a change in pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r583671260



##########
File path: modules/configuration-annotation-processor/src/test/java/org/apache/ignite/configuration/sample/UsageTest.java
##########
@@ -35,74 +33,88 @@
      * Test creation of configuration and calling configuration API methods.
      */
     @Test
-    public void test() {
-        InitLocal initLocal = new InitLocal().withBaseline(
-            new InitBaseline()
-                .withNodes(
-                    new NamedList<>(
-                        Collections.singletonMap("node1", new InitNode().withConsistentId("test").withPort(1000))
+    public void test() throws Exception {
+        var registry = new ConfigurationRegistry();
+
+        registry.registerRootKey(LocalConfiguration.KEY);
+
+        registry.registerStorage(new TestConfigurationStorage());
+
+        LocalConfiguration root = registry.getConfiguration(LocalConfiguration.KEY);
+
+        root.change(local ->
+            local.changeBaseline(baseline ->
+                baseline.changeNodes(nodes ->
+                    nodes.create("node1", node ->
+                        node.initConsistentId("test").initPort(1000)
                     )
+                ).changeAutoAdjust(autoAdjust ->
+                    autoAdjust.changeEnabled(true).changeTimeout(100_000L)
                 )
-                .withAutoAdjust(new InitAutoAdjust().withEnabled(true).withTimeout(100000L))
-        );
+            )
+        ).get();

Review comment:
       I can recall some test being run for 125 hours on teamcity :D 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] sergey-chugunov-1985 commented on a change in pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
sergey-chugunov-1985 commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r582610282



##########
File path: modules/configuration-annotation-processor/src/test/java/org/apache/ignite/configuration/sample/UsageTest.java
##########
@@ -35,36 +31,42 @@
      * Test creation of configuration and calling configuration API methods.
      */
     @Test
-    public void test() {
-        InitLocal initLocal = new InitLocal().withBaseline(
-            new InitBaseline()
-                .withNodes(
-                    new NamedList<>(
-                        Collections.singletonMap("node1", new InitNode().withConsistentId("test").withPort(1000))
+    public void test() throws Exception {
+        var registry = new ConfigurationRegistry();
+
+        registry.registerRootKey(LocalConfiguration.KEY);
+
+        registry.registerStorage(new TestConfigurationStorage());
+
+        LocalConfiguration root = registry.getConfiguration(LocalConfiguration.KEY);
+
+        root.change(local ->
+            local.changeBaseline(baseline ->
+                baseline.changeNodes(nodes ->
+                    nodes.create("node1", node ->
+                        node.initConsistentId("test").initPort(1000)
                     )
+                ).changeAutoAdjust(autoAdjust ->
+                    autoAdjust.changeEnabled(true).changeTimeout(100_000L)
                 )
-                .withAutoAdjust(new InitAutoAdjust().withEnabled(true).withTimeout(100000L))
-        );
-
-        final Configurator<LocalConfigurationImpl> configurator = Configurator.create(
-            LocalConfigurationImpl::new,
-            initLocal
-        );
-
-        final LocalConfiguration root = configurator.getRoot();
-        root.baseline().autoAdjust().enabled().value();
-
-        Assertions.assertThrows(ConfigurationValidationException.class, () -> {
-            configurator.set(Selectors.LOCAL_BASELINE_AUTO_ADJUST_ENABLED, false);
-        });
-        configurator.set(Selectors.LOCAL_BASELINE_AUTO_ADJUST, new ChangeAutoAdjust().withEnabled(false).withTimeout(0L));
-        configurator.getRoot().baseline().nodes().get("node1").autoAdjustEnabled(false);
-        configurator.getRoot().baseline().autoAdjust().enabled(true);
-        configurator.getRoot().baseline().nodes().get("node1").autoAdjustEnabled(true);
-
-        Assertions.assertThrows(ConfigurationValidationException.class, () -> {
-            configurator.getRoot().baseline().autoAdjust().enabled(false);
-        });
+            )
+        ).get();
+
+        assertTrue(root.baseline().autoAdjust().enabled().value());
+
+//        assertThrows(ConfigurationValidationException.class, () -> {

Review comment:
       add comment why it is turned off




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] sergey-chugunov-1985 commented on a change in pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
sergey-chugunov-1985 commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r582613104



##########
File path: modules/configuration/src/main/java/org/apache/ignite/configuration/ConfigurationChanger.java
##########
@@ -279,8 +286,11 @@ private ValidationResult validate(
 
             final Configurator<?> configurator = configurators.get(rootKey);
 
-            List<ValidationIssue> list = configurator.validateChanges(changesForRoot);
-            issues.addAll(list);
+            // TODO This will be fixed later.

Review comment:
       need a reference to ticket




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] ibessonov merged pull request #56: IGNITE-14230 Port DynamicConfiguration to new underlying configuration framework.

Posted by GitBox <gi...@apache.org>.
ibessonov merged pull request #56:
URL: https://github.com/apache/ignite-3/pull/56


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org