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/09/15 10:13:39 UTC

[GitHub] [ignite-3] tkalkirill opened a new pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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


   https://issues.apache.org/jira/browse/IGNITE-15409


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] tkalkirill commented on a change in pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/CollectionUtils.java
##########
@@ -142,6 +143,41 @@ else if (ts == null || ts.length == 0)
         }
     }
 
+    /**
+     * Create a view for an iterable with conversion of elements.
+     *
+     * @param iterable Iterable.
+     * @param mapper Conversion function.
+     * @param <T1> Type of the elements.
+     * @param <T2> Converted type of the elements.
+     * @return View of iterable.
+     */
+    public static <T1, T2> Iterable<T2> view(

Review comment:
       In order not to create collections.
   WDYT?




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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



##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationNode.java
##########
@@ -17,24 +17,27 @@
 
 package org.apache.ignite.internal.configuration;
 
-import java.util.Collections;
+import java.util.Collection;
 import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.Objects;
-import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.configuration.ConfigurationListenOnlyException;
 import org.apache.ignite.configuration.ConfigurationProperty;
 import org.apache.ignite.configuration.RootKey;
 import org.apache.ignite.configuration.notifications.ConfigurationListener;
 import org.apache.ignite.internal.configuration.tree.TraversableTreeNode;
 import org.apache.ignite.internal.configuration.util.ConfigurationUtil;
 import org.apache.ignite.internal.configuration.util.KeyNotFoundException;
 
+import static java.util.Collections.unmodifiableCollection;
+import static java.util.concurrent.ConcurrentHashMap.newKeySet;
+
 /**
  * Super class for dynamic configuration tree nodes. Has all common data and value retrieving algorithm in it.
  */
 public abstract class ConfigurationNode<VIEW, CHANGE> implements ConfigurationProperty<VIEW, CHANGE> {
     /** Listeners of property update. */
-    protected final List<ConfigurationListener<VIEW>> updateListeners = new CopyOnWriteArrayList<>();
+    protected final Collection<ConfigurationListener<VIEW>> updateListeners = newKeySet();

Review comment:
       ```suggestion
       protected final Collection<ConfigurationListener<VIEW>> updateListeners = ConcurrentHashMap.newKeySet();
   ```




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] tkalkirill commented on a change in pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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



##########
File path: modules/configuration/src/test/java/org/apache/ignite/internal/configuration/notifications/ConfigurationListenerTest.java
##########
@@ -644,4 +667,366 @@ public void dataRace() throws Exception {
 
         assertEquals(List.of("deleted"), log);
     }
+
+    /** */
+    @Test
+    void testNoGetOrUpdateConfigValueForAny() throws Exception {
+        ChildConfiguration any0 = configuration.elements().any();
+
+        assertThrows(ConfigurationException.class, () -> any0.value());
+        assertThrows(ConfigurationException.class, () -> any0.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any0.str().value());
+        assertThrows(ConfigurationException.class, () -> any0.str().update(""));
+
+        assertThrows(ConfigurationException.class, () -> any0.child2().value());
+        assertThrows(ConfigurationException.class, () -> any0.child2().change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any0.child2().i().value());
+        assertThrows(ConfigurationException.class, () -> any0.child2().i().update(100));
+
+        assertThrows(ConfigurationException.class, () -> any0.elements2().value());
+        assertThrows(ConfigurationException.class, () -> any0.elements2().change(doNothingConsumer()));
+        assertThrows(ConfigurationException.class, () -> any0.elements2().get("test"));
+
+        Child2Configuration any1 = any0.elements2().any();
+
+        assertThrows(ConfigurationException.class, () -> any1.value());
+        assertThrows(ConfigurationException.class, () -> any1.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any1.i().value());
+        assertThrows(ConfigurationException.class, () -> any1.i().update(200));
+
+        configuration.elements().change(c0 -> c0.create("test", c1 -> c1.changeStr("foo"))).get(1, SECONDS);
+
+        Child2Configuration any2 = configuration.elements().get("test").elements2().any();
+
+        assertThrows(ConfigurationException.class, () -> any2.value());
+        assertThrows(ConfigurationException.class, () -> any2.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any2.i().value());
+        assertThrows(ConfigurationException.class, () -> any2.i().update(300));
+    }
+
+    /** */
+    @Test
+    void testAnyListeners() throws Exception {
+        List<String> events = new ArrayList<>();
+
+        // Add "regular" listeners.
+        configuration.listen(configListener(ctx -> events.add("root")));
+
+        configuration.child().listen(configListener(ctx -> events.add("root.child")));
+        configuration.child().str().listen(configListener(ctx -> events.add("root.child.str")));
+        configuration.child().child2().listen(configListener(ctx -> events.add("root.child.child2")));
+        configuration.child().child2().i().listen(configListener(ctx -> events.add("root.child.child2.i")));
+
+        configuration.elements().listen(configListener(ctx -> events.add("root.elements")));
+        configuration.elements().listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.onCrt")));
+        configuration.elements().listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.onUpd")));
+        configuration.elements().listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.onRen")));
+        configuration.elements().listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.onDel")));
+
+        configuration.elements().change(c -> c.create("0", doNothingConsumer())).get(1, SECONDS);
+
+        ChildConfiguration childCfg = this.configuration.elements().get("0");
+
+        childCfg.listen(configListener(ctx -> events.add("root.elements.0")));
+        childCfg.str().listen(configListener(ctx -> events.add("root.elements.0.str")));
+        childCfg.child2().listen(configListener(ctx -> events.add("root.elements.0.child2")));
+        childCfg.child2().i().listen(configListener(ctx -> events.add("root.elements.0.child2.i")));
+
+        NamedConfigurationTree<Child2Configuration, Child2View, Child2Change> elements2 = childCfg.elements2();
+
+        elements2.listen(configListener(ctx -> events.add("root.elements.0.elements2")));
+        elements2.listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.0.elements2.onCrt")));
+        elements2.listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.0.elements2.onUpd")));
+        elements2.listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.0.elements2.onRen")));
+        elements2.listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.0.elements2.onDel")));
+
+        elements2.change(c -> c.create("0", doNothingConsumer())).get(1, SECONDS);
+
+        Child2Configuration child2 = elements2.get("0");
+
+        child2.listen(configListener(ctx -> events.add("root.elements.0.elements2.0")));
+        child2.i().listen(configListener(ctx -> events.add("root.elements.0.elements2.0.i")));
+
+        // Adding "any" listeners.
+        ChildConfiguration anyChild = configuration.elements().any();
+
+        anyChild.listen(configListener(ctx -> events.add("root.elements.any")));
+        anyChild.str().listen(configListener(ctx -> events.add("root.elements.any.str")));
+        anyChild.child2().listen(configListener(ctx -> events.add("root.elements.any.child2")));
+        anyChild.child2().i().listen(configListener(ctx -> events.add("root.elements.any.child2.i")));
+
+        NamedConfigurationTree<Child2Configuration, Child2View, Child2Change> anyEl2 = anyChild.elements2();
+
+        anyEl2.listen(configListener(ctx -> events.add("root.elements.any.elements2")));
+        anyEl2.listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.any.elements2.onCrt")));
+        anyEl2.listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.any.elements2.onUpd")));
+        anyEl2.listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.any.elements2.onRen")));
+        anyEl2.listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.any.elements2.onDel")));
+
+        Child2Configuration anyChild2 = anyEl2.any();
+
+        anyChild2.listen(configListener(ctx -> events.add("root.elements.any.elements2.any")));
+        anyChild2.i().listen(configListener(ctx -> events.add("root.elements.any.elements2.any.i")));
+
+        childCfg.elements2().any().listen(configListener(ctx -> events.add("root.elements.0.elements2.any")));
+        childCfg.elements2().any().i().listen(configListener(ctx -> events.add("root.elements.0.elements2.any.i")));
+
+        // Tests.
+        checkListeners(
+            () -> configuration.child().change(c -> c.changeStr("x").changeChild2(c0 -> c0.changeI(100))),
+            List.of("root", "root.child", "root.child.str", "root.child.child2", "root.child.child2.i"),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").str().update("x"),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.str",
+                "root.elements.0.str"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").elements2().get("0").i().update(200),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.elements2",
+                "root.elements.0.elements2",
+                "root.elements.any.elements2.onUpd",
+                "root.elements.0.elements2.onUpd",
+                //
+                "root.elements.any.elements2.any",
+                "root.elements.0.elements2.any",
+                "root.elements.0.elements2.0",
+                //
+                "root.elements.any.elements2.any.i",
+                "root.elements.0.elements2.any.i",
+                "root.elements.0.elements2.0.i"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").elements2().change(c -> c.create("1", doNothingConsumer())),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.elements2",
+                "root.elements.0.elements2",
+                "root.elements.any.elements2.onCrt",
+                "root.elements.0.elements2.onCrt",
+                //
+                "root.elements.any.elements2.onUpd",
+                "root.elements.0.elements2.onUpd"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements()

Review comment:
       I'll try to implement 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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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



##########
File path: modules/configuration-api/src/main/java/org/apache/ignite/configuration/ConfigurationListenOnlyException.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+/**
+ * This exception is used if an attempt was made to get/update a configuration value in listen-only mode.
+ */
+public class ConfigurationListenOnlyException extends RuntimeException {

Review comment:
       ```suggestion
   public class ConfigurationListenOnlyException extends IgniteException {
   ```




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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



##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/DynamicConfiguration.java
##########
@@ -133,4 +140,9 @@ public DynamicConfiguration(
     public Map<String, ConfigurationProperty<?>> touchMembers() {
         return members();
     }
+
+    /**
+     * @return Configuration interface, for example {@code RootConfiguration}.

Review comment:
       What should it return for named lists? I'd like that to be documented here

##########
File path: modules/configuration-api/src/main/java/org/apache/ignite/configuration/NamedConfigurationTree.java
##########
@@ -43,4 +43,14 @@
      * @param listener Listener.
      */
     void listenElements(ConfigurationNamedListListener<VIEW> listener);
+
+    /**
+     * Get a placeholder that allows you to add listeners for changing configuration value

Review comment:
       Usually method descriptions formulated like "Returns blah-blah" rather than "Do this and that", can you please run through your javadocs and correct that?

##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/util/ConfigurationNotificationsUtil.java
##########
@@ -30,66 +32,115 @@
 import org.apache.ignite.configuration.notifications.ConfigurationListener;
 import org.apache.ignite.configuration.notifications.ConfigurationNamedListListener;
 import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
-import org.apache.ignite.internal.configuration.ConfigurationNode;
 import org.apache.ignite.internal.configuration.DynamicConfiguration;
 import org.apache.ignite.internal.configuration.DynamicProperty;
 import org.apache.ignite.internal.configuration.NamedListConfiguration;
 import org.apache.ignite.internal.configuration.tree.ConfigurationVisitor;
 import org.apache.ignite.internal.configuration.tree.InnerNode;
 import org.apache.ignite.internal.configuration.tree.NamedListNode;
+import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.innerNodeVisitor;
 import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.leafNodeVisitor;
 import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.namedListNodeVisitor;
+import static org.apache.ignite.internal.util.CollectionUtils.viewReadOnly;
 
 /** */
 public class ConfigurationNotificationsUtil {
     /**
      * Recursively notifies all public configuration listeners, accumulating resulting futures in {@code futures} list.
+     *
      * @param oldInnerNode Old configuration values root.
      * @param newInnerNode New configuration values root.
      * @param cfgNode Public configuration tree node corresponding to the current inner nodes.
      * @param storageRevision Storage revision.
      * @param futures Write-only list of futures to accumulate results.
      */
     public static void notifyListeners(
-        InnerNode oldInnerNode,
+        @Nullable InnerNode oldInnerNode,
         InnerNode newInnerNode,
         DynamicConfiguration<InnerNode, ?> cfgNode,
         long storageRevision,
         List<CompletableFuture<?>> futures
+    ) {
+        notifyListeners(oldInnerNode, newInnerNode, cfgNode, storageRevision, futures, List.of(), new HashMap<>());
+    }
+
+    /**
+     * Recursively notifies all public configuration listeners, accumulating resulting futures in {@code futures} list.
+     *
+     * @param oldInnerNode Old configuration values root.
+     * @param newInnerNode New configuration values root.
+     * @param cfgNode Public configuration tree node corresponding to the current inner nodes.
+     * @param storageRevision Storage revision.
+     * @param futures Write-only list of futures to accumulate results.
+     * @param anyConfigs Current {@link NamedListConfiguration#any "any"} configurations.
+     * @param eventConfigs Configuration containers for {@link ConfigurationNotificationEvent}.
+     */
+    private static void notifyListeners(

Review comment:
       you know, this method became way too big, can you extract some parts so that it's easier to navigate the code?

##########
File path: modules/configuration-api/src/main/java/org/apache/ignite/configuration/ConfigurationListenOnlyException.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+/**
+ * This exception is used if an attempt was made to get/update a configuration value in listen-only mode.
+ */
+public class ConfigurationListenOnlyException extends RuntimeException {

Review comment:
       I think it is appropriate to have RuntimeException here. Maybe we should use IgniteException as a base class, I'm not sure yet

##########
File path: modules/configuration-api/src/main/java/org/apache/ignite/configuration/NamedConfigurationTree.java
##########
@@ -43,4 +43,14 @@
      * @param listener Listener.
      */
     void listenElements(ConfigurationNamedListListener<VIEW> listener);

Review comment:
       I see no "stop" method here. Is this what's intended?

##########
File path: modules/configuration-api/src/main/java/org/apache/ignite/configuration/notifications/ConfigurationNotificationEvent.java
##########
@@ -50,4 +50,28 @@
      * @return Counter value.
      */
     long storageRevision();
+
+    /**
+     * Get the parent (any from the root) or current configuration.
+     * <p>
+     * For example, if we changed the child configuration, then we can get both the parent
+     * and the current child configuration.
+     *
+     * @param configClass Configuration interface, for example {@code RootConfiguration}.
+     * @param <T> Configuration type.
+     * @return Configuration instance.
+     */
+    @Nullable <T extends ConfigurationProperty> T config(Class<T> configClass);
+
+    /**
+     * Get the key of a named list item for the parent (any from the root) or current configuration.
+     * <p>
+     * For example, if a column of a table has changed, then we can get the name of the table and columns
+     * for which the changes have occurred.
+     *
+     * @param configClass Configuration interface, for example {@code TableConfiguration}.
+     * @param <T> Configuration type.
+     * @return Configuration instance.
+     */
+    @Nullable <T extends ConfigurationProperty> String keyNamedConfig(Class<T> configClass);

Review comment:
       Type T is excessive here, you should use "Class<? extends ConfigurationProperty>"

##########
File path: modules/configuration-api/src/main/java/org/apache/ignite/configuration/notifications/ConfigurationNotificationEvent.java
##########
@@ -50,4 +50,28 @@
      * @return Counter value.
      */
     long storageRevision();
+
+    /**
+     * Get the parent (any from the root) or current configuration.
+     * <p>
+     * For example, if we changed the child configuration, then we can get both the parent
+     * and the current child configuration.
+     *
+     * @param configClass Configuration interface, for example {@code RootConfiguration}.
+     * @param <T> Configuration type.
+     * @return Configuration instance.
+     */
+    @Nullable <T extends ConfigurationProperty> T config(Class<T> configClass);
+
+    /**
+     * Get the key of a named list item for the parent (any from the root) or current configuration.
+     * <p>
+     * For example, if a column of a table has changed, then we can get the name of the table and columns
+     * for which the changes have occurred.
+     *
+     * @param configClass Configuration interface, for example {@code TableConfiguration}.
+     * @param <T> Configuration type.
+     * @return Configuration instance.
+     */
+    @Nullable <T extends ConfigurationProperty> String keyNamedConfig(Class<T> configClass);

Review comment:
       Name implies that you return config, but instead you return name. I think you should rename this method

##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationNode.java
##########
@@ -17,24 +17,27 @@
 
 package org.apache.ignite.internal.configuration;
 
-import java.util.Collections;
+import java.util.Collection;
 import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.Objects;
-import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.configuration.ConfigurationListenOnlyException;
 import org.apache.ignite.configuration.ConfigurationProperty;
 import org.apache.ignite.configuration.RootKey;
 import org.apache.ignite.configuration.notifications.ConfigurationListener;
 import org.apache.ignite.internal.configuration.tree.TraversableTreeNode;
 import org.apache.ignite.internal.configuration.util.ConfigurationUtil;
 import org.apache.ignite.internal.configuration.util.KeyNotFoundException;
 
+import static java.util.Collections.unmodifiableCollection;
+
 /**
  * Super class for dynamic configuration tree nodes. Has all common data and value retrieving algorithm in it.
  */
 public abstract class ConfigurationNode<VIEW> implements ConfigurationProperty<VIEW> {
     /** Listeners of property update. */
-    private final List<ConfigurationListener<VIEW>> updateListeners = new CopyOnWriteArrayList<>();
+    private final Collection<ConfigurationListener<VIEW>> updateListeners = ConcurrentHashMap.newKeySet();

Review comment:
       This one does not preserve order of the elements, right? I think that guys who implement tables flow might be upset, we should find better collection




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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



##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/util/ConfigurationNotificationsUtil.java
##########
@@ -30,66 +32,115 @@
 import org.apache.ignite.configuration.notifications.ConfigurationListener;
 import org.apache.ignite.configuration.notifications.ConfigurationNamedListListener;
 import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
-import org.apache.ignite.internal.configuration.ConfigurationNode;
 import org.apache.ignite.internal.configuration.DynamicConfiguration;
 import org.apache.ignite.internal.configuration.DynamicProperty;
 import org.apache.ignite.internal.configuration.NamedListConfiguration;
 import org.apache.ignite.internal.configuration.tree.ConfigurationVisitor;
 import org.apache.ignite.internal.configuration.tree.InnerNode;
 import org.apache.ignite.internal.configuration.tree.NamedListNode;
+import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.innerNodeVisitor;
 import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.leafNodeVisitor;
 import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.namedListNodeVisitor;
+import static org.apache.ignite.internal.util.CollectionUtils.viewReadOnly;
 
 /** */
 public class ConfigurationNotificationsUtil {
     /**
      * Recursively notifies all public configuration listeners, accumulating resulting futures in {@code futures} list.
+     *
      * @param oldInnerNode Old configuration values root.
      * @param newInnerNode New configuration values root.
      * @param cfgNode Public configuration tree node corresponding to the current inner nodes.
      * @param storageRevision Storage revision.
      * @param futures Write-only list of futures to accumulate results.
      */
     public static void notifyListeners(
-        InnerNode oldInnerNode,
+        @Nullable InnerNode oldInnerNode,
         InnerNode newInnerNode,
         DynamicConfiguration<InnerNode, ?> cfgNode,
         long storageRevision,
         List<CompletableFuture<?>> futures
+    ) {
+        notifyListeners(oldInnerNode, newInnerNode, cfgNode, storageRevision, futures, List.of(), new HashMap<>());
+    }
+
+    /**
+     * Recursively notifies all public configuration listeners, accumulating resulting futures in {@code futures} list.
+     *
+     * @param oldInnerNode Old configuration values root.
+     * @param newInnerNode New configuration values root.
+     * @param cfgNode Public configuration tree node corresponding to the current inner nodes.
+     * @param storageRevision Storage revision.
+     * @param futures Write-only list of futures to accumulate results.
+     * @param anyConfigs Current {@link NamedListConfiguration#any "any"} configurations.
+     * @param eventConfigs Configuration containers for {@link ConfigurationNotificationEvent}.
+     */
+    private static void notifyListeners(

Review comment:
       Ok, we can refactor it later




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] tkalkirill commented on a change in pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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



##########
File path: modules/configuration-api/src/main/java/org/apache/ignite/configuration/ConfigurationException.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+/**
+ * This exception is used to indicate any error condition in the configuration.

Review comment:
       I propose to discuss this in a separate ticket.

##########
File path: modules/configuration/src/test/java/org/apache/ignite/internal/configuration/notifications/ConfigurationListenerTest.java
##########
@@ -644,4 +667,366 @@ public void dataRace() throws Exception {
 
         assertEquals(List.of("deleted"), log);
     }
+
+    /** */
+    @Test
+    void testNoGetOrUpdateConfigValueForAny() throws Exception {
+        ChildConfiguration any0 = configuration.elements().any();
+
+        assertThrows(ConfigurationException.class, () -> any0.value());
+        assertThrows(ConfigurationException.class, () -> any0.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any0.str().value());
+        assertThrows(ConfigurationException.class, () -> any0.str().update(""));
+
+        assertThrows(ConfigurationException.class, () -> any0.child2().value());
+        assertThrows(ConfigurationException.class, () -> any0.child2().change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any0.child2().i().value());
+        assertThrows(ConfigurationException.class, () -> any0.child2().i().update(100));
+
+        assertThrows(ConfigurationException.class, () -> any0.elements2().value());
+        assertThrows(ConfigurationException.class, () -> any0.elements2().change(doNothingConsumer()));
+        assertThrows(ConfigurationException.class, () -> any0.elements2().get("test"));
+
+        Child2Configuration any1 = any0.elements2().any();
+
+        assertThrows(ConfigurationException.class, () -> any1.value());
+        assertThrows(ConfigurationException.class, () -> any1.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any1.i().value());
+        assertThrows(ConfigurationException.class, () -> any1.i().update(200));
+
+        configuration.elements().change(c0 -> c0.create("test", c1 -> c1.changeStr("foo"))).get(1, SECONDS);
+
+        Child2Configuration any2 = configuration.elements().get("test").elements2().any();
+
+        assertThrows(ConfigurationException.class, () -> any2.value());
+        assertThrows(ConfigurationException.class, () -> any2.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any2.i().value());
+        assertThrows(ConfigurationException.class, () -> any2.i().update(300));
+    }
+
+    /** */
+    @Test
+    void testAnyListeners() throws Exception {
+        List<String> events = new ArrayList<>();
+
+        // Add "regular" listeners.
+        configuration.listen(configListener(ctx -> events.add("root")));
+
+        configuration.child().listen(configListener(ctx -> events.add("root.child")));
+        configuration.child().str().listen(configListener(ctx -> events.add("root.child.str")));
+        configuration.child().child2().listen(configListener(ctx -> events.add("root.child.child2")));
+        configuration.child().child2().i().listen(configListener(ctx -> events.add("root.child.child2.i")));
+
+        configuration.elements().listen(configListener(ctx -> events.add("root.elements")));
+        configuration.elements().listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.onCrt")));
+        configuration.elements().listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.onUpd")));
+        configuration.elements().listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.onRen")));
+        configuration.elements().listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.onDel")));
+
+        configuration.elements().change(c -> c.create("0", doNothingConsumer())).get(1, SECONDS);
+
+        ChildConfiguration childCfg = this.configuration.elements().get("0");
+
+        childCfg.listen(configListener(ctx -> events.add("root.elements.0")));
+        childCfg.str().listen(configListener(ctx -> events.add("root.elements.0.str")));
+        childCfg.child2().listen(configListener(ctx -> events.add("root.elements.0.child2")));
+        childCfg.child2().i().listen(configListener(ctx -> events.add("root.elements.0.child2.i")));
+
+        NamedConfigurationTree<Child2Configuration, Child2View, Child2Change> elements2 = childCfg.elements2();
+
+        elements2.listen(configListener(ctx -> events.add("root.elements.0.elements2")));
+        elements2.listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.0.elements2.onCrt")));
+        elements2.listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.0.elements2.onUpd")));
+        elements2.listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.0.elements2.onRen")));
+        elements2.listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.0.elements2.onDel")));
+
+        elements2.change(c -> c.create("0", doNothingConsumer())).get(1, SECONDS);
+
+        Child2Configuration child2 = elements2.get("0");
+
+        child2.listen(configListener(ctx -> events.add("root.elements.0.elements2.0")));
+        child2.i().listen(configListener(ctx -> events.add("root.elements.0.elements2.0.i")));
+
+        // Adding "any" listeners.
+        ChildConfiguration anyChild = configuration.elements().any();
+
+        anyChild.listen(configListener(ctx -> events.add("root.elements.any")));
+        anyChild.str().listen(configListener(ctx -> events.add("root.elements.any.str")));
+        anyChild.child2().listen(configListener(ctx -> events.add("root.elements.any.child2")));
+        anyChild.child2().i().listen(configListener(ctx -> events.add("root.elements.any.child2.i")));
+
+        NamedConfigurationTree<Child2Configuration, Child2View, Child2Change> anyEl2 = anyChild.elements2();
+
+        anyEl2.listen(configListener(ctx -> events.add("root.elements.any.elements2")));
+        anyEl2.listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.any.elements2.onCrt")));
+        anyEl2.listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.any.elements2.onUpd")));
+        anyEl2.listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.any.elements2.onRen")));
+        anyEl2.listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.any.elements2.onDel")));
+
+        Child2Configuration anyChild2 = anyEl2.any();
+
+        anyChild2.listen(configListener(ctx -> events.add("root.elements.any.elements2.any")));
+        anyChild2.i().listen(configListener(ctx -> events.add("root.elements.any.elements2.any.i")));
+
+        childCfg.elements2().any().listen(configListener(ctx -> events.add("root.elements.0.elements2.any")));
+        childCfg.elements2().any().i().listen(configListener(ctx -> events.add("root.elements.0.elements2.any.i")));
+
+        // Tests.
+        checkListeners(
+            () -> configuration.child().change(c -> c.changeStr("x").changeChild2(c0 -> c0.changeI(100))),
+            List.of("root", "root.child", "root.child.str", "root.child.child2", "root.child.child2.i"),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").str().update("x"),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.str",
+                "root.elements.0.str"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").elements2().get("0").i().update(200),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.elements2",
+                "root.elements.0.elements2",
+                "root.elements.any.elements2.onUpd",
+                "root.elements.0.elements2.onUpd",
+                //
+                "root.elements.any.elements2.any",
+                "root.elements.0.elements2.any",
+                "root.elements.0.elements2.0",
+                //
+                "root.elements.any.elements2.any.i",
+                "root.elements.0.elements2.any.i",
+                "root.elements.0.elements2.0.i"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").elements2().change(c -> c.create("1", doNothingConsumer())),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.elements2",
+                "root.elements.0.elements2",
+                "root.elements.any.elements2.onCrt",
+                "root.elements.0.elements2.onCrt",
+                //
+                "root.elements.any.elements2.onUpd",
+                "root.elements.0.elements2.onUpd"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements()

Review comment:
       I propose to discuss this in a separate 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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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



##########
File path: modules/configuration-api/src/main/java/org/apache/ignite/configuration/ConfigurationException.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+/**
+ * This exception is used to indicate any error condition in the configuration.

Review comment:
       Please put any in quotes or rephrase, it's hard to understand what's going on. Maybe class name should be changed as well

##########
File path: modules/configuration-api/src/main/java/org/apache/ignite/configuration/NamedConfigurationTree.java
##########
@@ -43,4 +43,12 @@
      * @param listener Listener.
      */
     void listenElements(ConfigurationNamedListListener<VIEW> listener);
+
+    /**
+     * Get a placeholder that only allows you to add listeners to
+     * track configuration changes, as well as the entire sub-configuration.

Review comment:
       I think that this comment should be expanded

##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/asm/ConfigurationAsmGenerator.java
##########
@@ -994,31 +997,40 @@ private void addConfigurationImplConstructor(
             arg("prefix", List.class),
             arg("key", String.class),
             arg("rootKey", RootKey.class),
-            arg("changer", DynamicConfigurationChanger.class)
+            arg("changer", DynamicConfigurationChanger.class),
+            arg("listenOnly", boolean.class)
         );
 
+        Variable rootKeyVar = ctor.getScope().getVariable("rootKey");
+        Variable changerVar = ctor.getScope().getVariable("changer");
+        Variable listenOnlyVar = ctor.getScope().getVariable("listenOnly");
+
         BytecodeBlock ctorBody = ctor.getBody()
             .append(ctor.getThis())
             .append(ctor.getScope().getVariable("prefix"))
             .append(ctor.getScope().getVariable("key"))
-            .append(ctor.getScope().getVariable("rootKey"))
-            .append(ctor.getScope().getVariable("changer"))
+            .append(rootKeyVar)
+            .append(changerVar)
+            .append(listenOnlyVar)
             .invokeConstructor(DYNAMIC_CONFIGURATION_CTOR);
 
+        BytecodeExpression thisKeysVar = ctor.getThis().getField("keys", List.class);
+
         int newIdx = 0;
         for (Field schemaField : concat(asList(schemaFields), extensionsFields)) {
             FieldDefinition fieldDef = fieldDefs.get(schemaField.getName());
 
             BytecodeExpression newValue;
 
             if (isValue(schemaField)) {
-                // newValue = new DynamicProperty(super.keys, fieldName, rootKey, changer);
+                // newValue = new DynamicProperty(super.keys, fieldName, rootKey, changer, boolean);

Review comment:
       ```suggestion
                   // newValue = new DynamicProperty(super.keys, fieldName, rootKey, changer, listenOnly);
   ```

##########
File path: modules/configuration-api/src/main/java/org/apache/ignite/configuration/NamedConfigurationTree.java
##########
@@ -43,4 +43,12 @@
      * @param listener Listener.
      */
     void listenElements(ConfigurationNamedListListener<VIEW> listener);
+
+    /**
+     * Get a placeholder that only allows you to add listeners to

Review comment:
       ```suggestion
        * Returns a placeholder that only allows you to add listeners to
   ```

##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/util/ConfigurationNotificationsUtil.java
##########
@@ -233,7 +364,7 @@ public static void notifyListeners(
      * Invoke {@link ConfigurationListener#onUpdate(ConfigurationNotificationEvent)} on all passed listeners and put
      * results in {@code futures}. Not recursively.
      *
-     * @param listeners List o flisteners.
+     * @param listeners List o listeners.

Review comment:
       ```suggestion
        * @param listeners List of listeners.
   ```

##########
File path: modules/configuration/src/test/java/org/apache/ignite/internal/configuration/notifications/ConfigurationListenerTest.java
##########
@@ -644,4 +667,366 @@ public void dataRace() throws Exception {
 
         assertEquals(List.of("deleted"), log);
     }
+
+    /** */
+    @Test
+    void testNoGetOrUpdateConfigValueForAny() throws Exception {
+        ChildConfiguration any0 = configuration.elements().any();
+
+        assertThrows(ConfigurationException.class, () -> any0.value());
+        assertThrows(ConfigurationException.class, () -> any0.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any0.str().value());
+        assertThrows(ConfigurationException.class, () -> any0.str().update(""));
+
+        assertThrows(ConfigurationException.class, () -> any0.child2().value());
+        assertThrows(ConfigurationException.class, () -> any0.child2().change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any0.child2().i().value());
+        assertThrows(ConfigurationException.class, () -> any0.child2().i().update(100));
+
+        assertThrows(ConfigurationException.class, () -> any0.elements2().value());
+        assertThrows(ConfigurationException.class, () -> any0.elements2().change(doNothingConsumer()));
+        assertThrows(ConfigurationException.class, () -> any0.elements2().get("test"));
+
+        Child2Configuration any1 = any0.elements2().any();
+
+        assertThrows(ConfigurationException.class, () -> any1.value());
+        assertThrows(ConfigurationException.class, () -> any1.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any1.i().value());
+        assertThrows(ConfigurationException.class, () -> any1.i().update(200));
+
+        configuration.elements().change(c0 -> c0.create("test", c1 -> c1.changeStr("foo"))).get(1, SECONDS);
+
+        Child2Configuration any2 = configuration.elements().get("test").elements2().any();
+
+        assertThrows(ConfigurationException.class, () -> any2.value());
+        assertThrows(ConfigurationException.class, () -> any2.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any2.i().value());
+        assertThrows(ConfigurationException.class, () -> any2.i().update(300));
+    }
+
+    /** */
+    @Test
+    void testAnyListeners() throws Exception {
+        List<String> events = new ArrayList<>();
+
+        // Add "regular" listeners.
+        configuration.listen(configListener(ctx -> events.add("root")));
+
+        configuration.child().listen(configListener(ctx -> events.add("root.child")));
+        configuration.child().str().listen(configListener(ctx -> events.add("root.child.str")));
+        configuration.child().child2().listen(configListener(ctx -> events.add("root.child.child2")));
+        configuration.child().child2().i().listen(configListener(ctx -> events.add("root.child.child2.i")));
+
+        configuration.elements().listen(configListener(ctx -> events.add("root.elements")));
+        configuration.elements().listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.onCrt")));
+        configuration.elements().listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.onUpd")));
+        configuration.elements().listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.onRen")));
+        configuration.elements().listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.onDel")));
+
+        configuration.elements().change(c -> c.create("0", doNothingConsumer())).get(1, SECONDS);
+
+        ChildConfiguration childCfg = this.configuration.elements().get("0");
+
+        childCfg.listen(configListener(ctx -> events.add("root.elements.0")));
+        childCfg.str().listen(configListener(ctx -> events.add("root.elements.0.str")));
+        childCfg.child2().listen(configListener(ctx -> events.add("root.elements.0.child2")));
+        childCfg.child2().i().listen(configListener(ctx -> events.add("root.elements.0.child2.i")));
+
+        NamedConfigurationTree<Child2Configuration, Child2View, Child2Change> elements2 = childCfg.elements2();
+
+        elements2.listen(configListener(ctx -> events.add("root.elements.0.elements2")));
+        elements2.listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.0.elements2.onCrt")));
+        elements2.listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.0.elements2.onUpd")));
+        elements2.listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.0.elements2.onRen")));
+        elements2.listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.0.elements2.onDel")));
+
+        elements2.change(c -> c.create("0", doNothingConsumer())).get(1, SECONDS);
+
+        Child2Configuration child2 = elements2.get("0");
+
+        child2.listen(configListener(ctx -> events.add("root.elements.0.elements2.0")));
+        child2.i().listen(configListener(ctx -> events.add("root.elements.0.elements2.0.i")));
+
+        // Adding "any" listeners.
+        ChildConfiguration anyChild = configuration.elements().any();
+
+        anyChild.listen(configListener(ctx -> events.add("root.elements.any")));
+        anyChild.str().listen(configListener(ctx -> events.add("root.elements.any.str")));
+        anyChild.child2().listen(configListener(ctx -> events.add("root.elements.any.child2")));
+        anyChild.child2().i().listen(configListener(ctx -> events.add("root.elements.any.child2.i")));
+
+        NamedConfigurationTree<Child2Configuration, Child2View, Child2Change> anyEl2 = anyChild.elements2();
+
+        anyEl2.listen(configListener(ctx -> events.add("root.elements.any.elements2")));
+        anyEl2.listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.any.elements2.onCrt")));
+        anyEl2.listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.any.elements2.onUpd")));
+        anyEl2.listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.any.elements2.onRen")));
+        anyEl2.listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.any.elements2.onDel")));
+
+        Child2Configuration anyChild2 = anyEl2.any();
+
+        anyChild2.listen(configListener(ctx -> events.add("root.elements.any.elements2.any")));
+        anyChild2.i().listen(configListener(ctx -> events.add("root.elements.any.elements2.any.i")));
+
+        childCfg.elements2().any().listen(configListener(ctx -> events.add("root.elements.0.elements2.any")));
+        childCfg.elements2().any().i().listen(configListener(ctx -> events.add("root.elements.0.elements2.any.i")));
+
+        // Tests.
+        checkListeners(
+            () -> configuration.child().change(c -> c.changeStr("x").changeChild2(c0 -> c0.changeI(100))),
+            List.of("root", "root.child", "root.child.str", "root.child.child2", "root.child.child2.i"),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").str().update("x"),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.str",
+                "root.elements.0.str"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").elements2().get("0").i().update(200),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.elements2",
+                "root.elements.0.elements2",
+                "root.elements.any.elements2.onUpd",
+                "root.elements.0.elements2.onUpd",
+                //
+                "root.elements.any.elements2.any",
+                "root.elements.0.elements2.any",
+                "root.elements.0.elements2.0",
+                //
+                "root.elements.any.elements2.any.i",
+                "root.elements.0.elements2.any.i",
+                "root.elements.0.elements2.0.i"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").elements2().change(c -> c.create("1", doNothingConsumer())),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.elements2",
+                "root.elements.0.elements2",
+                "root.elements.any.elements2.onCrt",
+                "root.elements.0.elements2.onCrt",
+                //
+                "root.elements.any.elements2.onUpd",
+                "root.elements.0.elements2.onUpd"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements()

Review comment:
       Should you call "any" listener in case of create operation? Seems logical, previous value is zero in this case. What do you think?

##########
File path: modules/configuration/src/test/java/org/apache/ignite/internal/configuration/notifications/ConfigurationListenerTest.java
##########
@@ -644,4 +667,366 @@ public void dataRace() throws Exception {
 
         assertEquals(List.of("deleted"), log);
     }
+
+    /** */
+    @Test
+    void testNoGetOrUpdateConfigValueForAny() throws Exception {
+        ChildConfiguration any0 = configuration.elements().any();
+
+        assertThrows(ConfigurationException.class, () -> any0.value());
+        assertThrows(ConfigurationException.class, () -> any0.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any0.str().value());
+        assertThrows(ConfigurationException.class, () -> any0.str().update(""));
+
+        assertThrows(ConfigurationException.class, () -> any0.child2().value());
+        assertThrows(ConfigurationException.class, () -> any0.child2().change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any0.child2().i().value());
+        assertThrows(ConfigurationException.class, () -> any0.child2().i().update(100));
+
+        assertThrows(ConfigurationException.class, () -> any0.elements2().value());
+        assertThrows(ConfigurationException.class, () -> any0.elements2().change(doNothingConsumer()));
+        assertThrows(ConfigurationException.class, () -> any0.elements2().get("test"));
+
+        Child2Configuration any1 = any0.elements2().any();
+
+        assertThrows(ConfigurationException.class, () -> any1.value());
+        assertThrows(ConfigurationException.class, () -> any1.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any1.i().value());
+        assertThrows(ConfigurationException.class, () -> any1.i().update(200));
+
+        configuration.elements().change(c0 -> c0.create("test", c1 -> c1.changeStr("foo"))).get(1, SECONDS);
+
+        Child2Configuration any2 = configuration.elements().get("test").elements2().any();
+
+        assertThrows(ConfigurationException.class, () -> any2.value());
+        assertThrows(ConfigurationException.class, () -> any2.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any2.i().value());
+        assertThrows(ConfigurationException.class, () -> any2.i().update(300));
+    }
+
+    /** */
+    @Test
+    void testAnyListeners() throws Exception {

Review comment:
       Test is huge, maybe we should split it. Like "create" should be tested separately from "delete" and so on

##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/util/ConfigurationNotificationsUtil.java
##########
@@ -284,4 +418,70 @@ public static void touch(DynamicConfiguration<?, ?> cfg) {
                 touch((DynamicConfiguration<?, ?>)value);
         }
     }
+
+    /**
+     * Get the dynamic property of the leaf.
+     *
+     * @param dynamicConfig Dynamic configuration.
+     * @param nodeName Name of the child node.
+     * @return Dynamic property of a leaf.
+     */
+    private static DynamicProperty<Serializable> dynamicProperty(
+        DynamicConfiguration<InnerNode, ?> dynamicConfig,
+        String nodeName
+    ) {
+        return (DynamicProperty<Serializable>)dynamicConfig.members().get(nodeName);
+    }
+
+    /**
+     * Get the dynamic configuration of the child node.
+     *
+     * @param dynamicConfig Dynamic configuration.
+     * @param nodeName Name of the child node.
+     * @return Dynamic configuration of the child node.
+     */
+    private static DynamicConfiguration<InnerNode, ?> dynamicConfig(
+        DynamicConfiguration<InnerNode, ?> dynamicConfig,
+        String nodeName
+    ) {
+        return (DynamicConfiguration<InnerNode, ?>)dynamicConfig.members().get(nodeName);
+    }
+
+    /**
+     * Get the named dynamic configuration of the child node.
+     *
+     * @param dynamicConfig Dynamic configuration.
+     * @param nodeName Name of the child node.
+     * @return Named dynamic configuration of the child node.
+     */
+    private static NamedListConfiguration<?, InnerNode, ?> namedDynamicConfig(
+        DynamicConfiguration<InnerNode, ?> dynamicConfig,
+        String nodeName
+    ) {
+        return (NamedListConfiguration<?, InnerNode, ?>)dynamicConfig.members().get(nodeName);
+    }
+
+    /**
+     * Get the dynamic configuration of the {@link NamedListConfiguration#any any} node.
+     *
+     * @param namedConfig Dynamic configuration.
+     * @return Dynamic configuration of the "any" node.
+     */
+    private static DynamicConfiguration<InnerNode, ?> any(NamedListConfiguration<?, InnerNode, ?> namedConfig) {
+        return (DynamicConfiguration<InnerNode, ?>)namedConfig.any();
+    }
+
+    /**
+     * Get the dynamic configuration of the child node.
+     *
+     * @param namedConfig Named dynamic configuration.
+     * @param nodeName Name of the child node.
+     * @return Dynamic configuration of the child node.
+     */
+    private static DynamicConfiguration<InnerNode, ?> dynamicConfig(

Review comment:
       How's this different from "DynamicConfiguration<InnerNode, ?> dynamicConfig(..."?

##########
File path: modules/configuration/src/test/java/org/apache/ignite/internal/configuration/notifications/ConfigurationListenerTest.java
##########
@@ -644,4 +667,366 @@ public void dataRace() throws Exception {
 
         assertEquals(List.of("deleted"), log);
     }
+
+    /** */
+    @Test
+    void testNoGetOrUpdateConfigValueForAny() throws Exception {
+        ChildConfiguration any0 = configuration.elements().any();
+
+        assertThrows(ConfigurationException.class, () -> any0.value());
+        assertThrows(ConfigurationException.class, () -> any0.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any0.str().value());
+        assertThrows(ConfigurationException.class, () -> any0.str().update(""));
+
+        assertThrows(ConfigurationException.class, () -> any0.child2().value());
+        assertThrows(ConfigurationException.class, () -> any0.child2().change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any0.child2().i().value());
+        assertThrows(ConfigurationException.class, () -> any0.child2().i().update(100));
+
+        assertThrows(ConfigurationException.class, () -> any0.elements2().value());
+        assertThrows(ConfigurationException.class, () -> any0.elements2().change(doNothingConsumer()));
+        assertThrows(ConfigurationException.class, () -> any0.elements2().get("test"));
+
+        Child2Configuration any1 = any0.elements2().any();
+
+        assertThrows(ConfigurationException.class, () -> any1.value());
+        assertThrows(ConfigurationException.class, () -> any1.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any1.i().value());
+        assertThrows(ConfigurationException.class, () -> any1.i().update(200));
+
+        configuration.elements().change(c0 -> c0.create("test", c1 -> c1.changeStr("foo"))).get(1, SECONDS);
+
+        Child2Configuration any2 = configuration.elements().get("test").elements2().any();
+
+        assertThrows(ConfigurationException.class, () -> any2.value());
+        assertThrows(ConfigurationException.class, () -> any2.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any2.i().value());
+        assertThrows(ConfigurationException.class, () -> any2.i().update(300));
+    }
+
+    /** */
+    @Test
+    void testAnyListeners() throws Exception {
+        List<String> events = new ArrayList<>();
+
+        // Add "regular" listeners.
+        configuration.listen(configListener(ctx -> events.add("root")));
+
+        configuration.child().listen(configListener(ctx -> events.add("root.child")));
+        configuration.child().str().listen(configListener(ctx -> events.add("root.child.str")));
+        configuration.child().child2().listen(configListener(ctx -> events.add("root.child.child2")));
+        configuration.child().child2().i().listen(configListener(ctx -> events.add("root.child.child2.i")));
+
+        configuration.elements().listen(configListener(ctx -> events.add("root.elements")));
+        configuration.elements().listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.onCrt")));
+        configuration.elements().listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.onUpd")));
+        configuration.elements().listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.onRen")));
+        configuration.elements().listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.onDel")));
+
+        configuration.elements().change(c -> c.create("0", doNothingConsumer())).get(1, SECONDS);
+
+        ChildConfiguration childCfg = this.configuration.elements().get("0");
+
+        childCfg.listen(configListener(ctx -> events.add("root.elements.0")));
+        childCfg.str().listen(configListener(ctx -> events.add("root.elements.0.str")));
+        childCfg.child2().listen(configListener(ctx -> events.add("root.elements.0.child2")));
+        childCfg.child2().i().listen(configListener(ctx -> events.add("root.elements.0.child2.i")));
+
+        NamedConfigurationTree<Child2Configuration, Child2View, Child2Change> elements2 = childCfg.elements2();
+
+        elements2.listen(configListener(ctx -> events.add("root.elements.0.elements2")));
+        elements2.listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.0.elements2.onCrt")));
+        elements2.listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.0.elements2.onUpd")));
+        elements2.listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.0.elements2.onRen")));
+        elements2.listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.0.elements2.onDel")));
+
+        elements2.change(c -> c.create("0", doNothingConsumer())).get(1, SECONDS);
+
+        Child2Configuration child2 = elements2.get("0");
+
+        child2.listen(configListener(ctx -> events.add("root.elements.0.elements2.0")));
+        child2.i().listen(configListener(ctx -> events.add("root.elements.0.elements2.0.i")));
+
+        // Adding "any" listeners.
+        ChildConfiguration anyChild = configuration.elements().any();
+
+        anyChild.listen(configListener(ctx -> events.add("root.elements.any")));
+        anyChild.str().listen(configListener(ctx -> events.add("root.elements.any.str")));
+        anyChild.child2().listen(configListener(ctx -> events.add("root.elements.any.child2")));
+        anyChild.child2().i().listen(configListener(ctx -> events.add("root.elements.any.child2.i")));
+
+        NamedConfigurationTree<Child2Configuration, Child2View, Child2Change> anyEl2 = anyChild.elements2();
+
+        anyEl2.listen(configListener(ctx -> events.add("root.elements.any.elements2")));
+        anyEl2.listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.any.elements2.onCrt")));
+        anyEl2.listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.any.elements2.onUpd")));
+        anyEl2.listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.any.elements2.onRen")));
+        anyEl2.listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.any.elements2.onDel")));
+
+        Child2Configuration anyChild2 = anyEl2.any();
+
+        anyChild2.listen(configListener(ctx -> events.add("root.elements.any.elements2.any")));
+        anyChild2.i().listen(configListener(ctx -> events.add("root.elements.any.elements2.any.i")));
+
+        childCfg.elements2().any().listen(configListener(ctx -> events.add("root.elements.0.elements2.any")));
+        childCfg.elements2().any().i().listen(configListener(ctx -> events.add("root.elements.0.elements2.any.i")));
+
+        // Tests.
+        checkListeners(
+            () -> configuration.child().change(c -> c.changeStr("x").changeChild2(c0 -> c0.changeI(100))),
+            List.of("root", "root.child", "root.child.str", "root.child.child2", "root.child.child2.i"),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").str().update("x"),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.str",
+                "root.elements.0.str"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").elements2().get("0").i().update(200),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.elements2",
+                "root.elements.0.elements2",
+                "root.elements.any.elements2.onUpd",
+                "root.elements.0.elements2.onUpd",
+                //
+                "root.elements.any.elements2.any",
+                "root.elements.0.elements2.any",
+                "root.elements.0.elements2.0",
+                //
+                "root.elements.any.elements2.any.i",
+                "root.elements.0.elements2.any.i",
+                "root.elements.0.elements2.0.i"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").elements2().change(c -> c.create("1", doNothingConsumer())),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.elements2",
+                "root.elements.0.elements2",
+                "root.elements.any.elements2.onCrt",
+                "root.elements.0.elements2.onCrt",
+                //
+                "root.elements.any.elements2.onUpd",
+                "root.elements.0.elements2.onUpd"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements()
+                .change(c -> c.create("1", c0 -> c0.changeElements2(c1 -> c1.create("2", doNothingConsumer())))),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onCrt",
+                "root.elements.onUpd"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("1").elements2().get("2").i().update(200),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.any.elements2",
+                "root.elements.any.elements2.onUpd",
+                "root.elements.any.elements2.any",
+                "root.elements.any.elements2.any.i"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("1").elements2().change(c -> c.rename("2", "2x")),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.any.elements2",
+                "root.elements.any.elements2.onRen"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("1").elements2().change(c -> c.delete("2x")),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.any.elements2",
+                "root.elements.any.elements2.onDel",
+                "root.elements.any.elements2.any"
+            ),
+            events
+        );
+    }
+
+    /**
+     * Helper method for testing listeners.
+     *
+     * @param changeFun Configuration change function.
+     * @param exp Expected list of executing listeners.
+     * @param act Reference to the list of executing listeners that is filled after the {@code changeFun} is executed.
+     * @throws Exception If failed.
+     */
+    private static void checkListeners(
+        Supplier<CompletableFuture<Void>> changeFun,
+        List<String> exp,
+        List<String> act
+    ) throws Exception {
+        act.clear();
+
+        changeFun.get().get(1, SECONDS);
+
+        assertEquals(exp, act);
+    }
+
+    /**
+     * @param consumer Consumer of the notification context.
+     * @return Config value change listener.
+     */
+    private static <T> ConfigurationListener<T> configListener(Consumer<ConfigurationNotificationEvent<T>> consumer) {
+        return ctx -> {
+            consumer.accept(ctx);
+
+            return completedFuture(null);
+        };
+    }
+
+    /**
+     * @param consumer Consumer of the notification context.
+     * @return Named config value change listener.
+     */
+    private static <T> ConfigurationNamedListListener<T> configNamedListenerOnUpdate(
+        Consumer<ConfigurationNotificationEvent<T>> consumer
+    ) {
+        return ctx -> {

Review comment:
       Why is ConfigurationNamedListListener suddenly became FunctionalInterface? I think you should provide default implementation for "onUpdate" as well

##########
File path: modules/configuration/src/test/java/org/apache/ignite/internal/configuration/notifications/ConfigurationListenerTest.java
##########
@@ -644,4 +667,366 @@ public void dataRace() throws Exception {
 
         assertEquals(List.of("deleted"), log);
     }
+
+    /** */
+    @Test
+    void testNoGetOrUpdateConfigValueForAny() throws Exception {
+        ChildConfiguration any0 = configuration.elements().any();
+
+        assertThrows(ConfigurationException.class, () -> any0.value());
+        assertThrows(ConfigurationException.class, () -> any0.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any0.str().value());
+        assertThrows(ConfigurationException.class, () -> any0.str().update(""));
+
+        assertThrows(ConfigurationException.class, () -> any0.child2().value());
+        assertThrows(ConfigurationException.class, () -> any0.child2().change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any0.child2().i().value());
+        assertThrows(ConfigurationException.class, () -> any0.child2().i().update(100));
+
+        assertThrows(ConfigurationException.class, () -> any0.elements2().value());
+        assertThrows(ConfigurationException.class, () -> any0.elements2().change(doNothingConsumer()));
+        assertThrows(ConfigurationException.class, () -> any0.elements2().get("test"));
+
+        Child2Configuration any1 = any0.elements2().any();
+
+        assertThrows(ConfigurationException.class, () -> any1.value());
+        assertThrows(ConfigurationException.class, () -> any1.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any1.i().value());
+        assertThrows(ConfigurationException.class, () -> any1.i().update(200));
+
+        configuration.elements().change(c0 -> c0.create("test", c1 -> c1.changeStr("foo"))).get(1, SECONDS);
+
+        Child2Configuration any2 = configuration.elements().get("test").elements2().any();
+
+        assertThrows(ConfigurationException.class, () -> any2.value());
+        assertThrows(ConfigurationException.class, () -> any2.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any2.i().value());
+        assertThrows(ConfigurationException.class, () -> any2.i().update(300));
+    }
+
+    /** */
+    @Test
+    void testAnyListeners() throws Exception {
+        List<String> events = new ArrayList<>();
+
+        // Add "regular" listeners.
+        configuration.listen(configListener(ctx -> events.add("root")));
+
+        configuration.child().listen(configListener(ctx -> events.add("root.child")));
+        configuration.child().str().listen(configListener(ctx -> events.add("root.child.str")));
+        configuration.child().child2().listen(configListener(ctx -> events.add("root.child.child2")));
+        configuration.child().child2().i().listen(configListener(ctx -> events.add("root.child.child2.i")));
+
+        configuration.elements().listen(configListener(ctx -> events.add("root.elements")));
+        configuration.elements().listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.onCrt")));
+        configuration.elements().listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.onUpd")));
+        configuration.elements().listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.onRen")));
+        configuration.elements().listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.onDel")));
+
+        configuration.elements().change(c -> c.create("0", doNothingConsumer())).get(1, SECONDS);
+
+        ChildConfiguration childCfg = this.configuration.elements().get("0");
+
+        childCfg.listen(configListener(ctx -> events.add("root.elements.0")));
+        childCfg.str().listen(configListener(ctx -> events.add("root.elements.0.str")));
+        childCfg.child2().listen(configListener(ctx -> events.add("root.elements.0.child2")));
+        childCfg.child2().i().listen(configListener(ctx -> events.add("root.elements.0.child2.i")));
+
+        NamedConfigurationTree<Child2Configuration, Child2View, Child2Change> elements2 = childCfg.elements2();
+
+        elements2.listen(configListener(ctx -> events.add("root.elements.0.elements2")));
+        elements2.listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.0.elements2.onCrt")));
+        elements2.listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.0.elements2.onUpd")));
+        elements2.listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.0.elements2.onRen")));
+        elements2.listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.0.elements2.onDel")));
+
+        elements2.change(c -> c.create("0", doNothingConsumer())).get(1, SECONDS);
+
+        Child2Configuration child2 = elements2.get("0");
+
+        child2.listen(configListener(ctx -> events.add("root.elements.0.elements2.0")));
+        child2.i().listen(configListener(ctx -> events.add("root.elements.0.elements2.0.i")));
+
+        // Adding "any" listeners.
+        ChildConfiguration anyChild = configuration.elements().any();
+
+        anyChild.listen(configListener(ctx -> events.add("root.elements.any")));
+        anyChild.str().listen(configListener(ctx -> events.add("root.elements.any.str")));
+        anyChild.child2().listen(configListener(ctx -> events.add("root.elements.any.child2")));
+        anyChild.child2().i().listen(configListener(ctx -> events.add("root.elements.any.child2.i")));
+
+        NamedConfigurationTree<Child2Configuration, Child2View, Child2Change> anyEl2 = anyChild.elements2();
+
+        anyEl2.listen(configListener(ctx -> events.add("root.elements.any.elements2")));
+        anyEl2.listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.any.elements2.onCrt")));
+        anyEl2.listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.any.elements2.onUpd")));
+        anyEl2.listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.any.elements2.onRen")));
+        anyEl2.listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.any.elements2.onDel")));
+
+        Child2Configuration anyChild2 = anyEl2.any();
+
+        anyChild2.listen(configListener(ctx -> events.add("root.elements.any.elements2.any")));
+        anyChild2.i().listen(configListener(ctx -> events.add("root.elements.any.elements2.any.i")));
+
+        childCfg.elements2().any().listen(configListener(ctx -> events.add("root.elements.0.elements2.any")));
+        childCfg.elements2().any().i().listen(configListener(ctx -> events.add("root.elements.0.elements2.any.i")));
+
+        // Tests.
+        checkListeners(
+            () -> configuration.child().change(c -> c.changeStr("x").changeChild2(c0 -> c0.changeI(100))),
+            List.of("root", "root.child", "root.child.str", "root.child.child2", "root.child.child2.i"),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").str().update("x"),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.str",
+                "root.elements.0.str"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").elements2().get("0").i().update(200),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.elements2",
+                "root.elements.0.elements2",
+                "root.elements.any.elements2.onUpd",
+                "root.elements.0.elements2.onUpd",
+                //
+                "root.elements.any.elements2.any",
+                "root.elements.0.elements2.any",
+                "root.elements.0.elements2.0",
+                //
+                "root.elements.any.elements2.any.i",
+                "root.elements.0.elements2.any.i",
+                "root.elements.0.elements2.0.i"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").elements2().change(c -> c.create("1", doNothingConsumer())),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.elements2",
+                "root.elements.0.elements2",
+                "root.elements.any.elements2.onCrt",
+                "root.elements.0.elements2.onCrt",
+                //
+                "root.elements.any.elements2.onUpd",
+                "root.elements.0.elements2.onUpd"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements()
+                .change(c -> c.create("1", c0 -> c0.changeElements2(c1 -> c1.create("2", doNothingConsumer())))),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onCrt",
+                "root.elements.onUpd"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("1").elements2().get("2").i().update(200),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.any.elements2",
+                "root.elements.any.elements2.onUpd",
+                "root.elements.any.elements2.any",
+                "root.elements.any.elements2.any.i"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("1").elements2().change(c -> c.rename("2", "2x")),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                "root.elements.onUpd",

Review comment:
       Same here and later in this file

##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/asm/ConfigurationAsmGenerator.java
##########
@@ -1027,37 +1039,41 @@ private void addConfigurationImplConstructor(
                 ParameterizedType cfgImplParameterizedType = typeFromJavaClassName(fieldInfo.cfgImplClassName);
 
                 if (isConfigValue(schemaField)) {
-                    // newValue = new MyConfigurationImpl(super.keys, fieldName, rootKey, changer);
+                    // newValue = new MyConfigurationImpl(super.keys, fieldName, rootKey, changer, boolean);
                     newValue = newInstance(
                         cfgImplParameterizedType,
-                        ctor.getThis().getField("keys", List.class),
+                        thisKeysVar,
                         constantString(schemaField.getName()),
-                        ctor.getScope().getVariable("rootKey"),
-                        ctor.getScope().getVariable("changer")
+                        rootKeyVar,
+                        changerVar,
+                        listenOnlyVar
                     );
                 }
                 else {
                     // We have to create method "$new$<idx>" to reference it in lambda expression. That's the way it
-                    // works, it'll invoke constructor with all 4 arguments, not just 2 as in BiFunction.
+                    // works, it'll invoke constructor with all 5 arguments, not just 2 as in BiFunction.
                     MethodDefinition newMtd = classDef.declareMethod(
                         of(PRIVATE, STATIC, SYNTHETIC),
                         "$new$" + newIdx++,
                         typeFromJavaClassName(fieldInfo.cfgClassName),
                         arg("rootKey", RootKey.class),
                         arg("changer", DynamicConfigurationChanger.class),
+                        arg("listenOnly", boolean.class),
                         arg("prefix", List.class),
                         arg("key", String.class)
                     );
 
-                    // newValue = new NamedListConfiguration(super.keys, fieldName, rootKey, changer, (p, k) ->
-                    //     new ValueConfigurationImpl(p, k, rootKey, changer)
+                    // newValue = new NamedListConfiguration(this.keys, fieldName, rootKey, changer, listenOnly,
+                    //      (p, k) -> new ValueConfigurationImpl(p, k, rootKey, changer, listenOnly),
+                    //      new ValueConfigurationImpl(this.keys, "any", rootKey, changer, true)

Review comment:
       Can the string "any" be publicly seen anywhere?

##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/asm/ConfigurationAsmGenerator.java
##########
@@ -1027,37 +1039,41 @@ private void addConfigurationImplConstructor(
                 ParameterizedType cfgImplParameterizedType = typeFromJavaClassName(fieldInfo.cfgImplClassName);
 
                 if (isConfigValue(schemaField)) {
-                    // newValue = new MyConfigurationImpl(super.keys, fieldName, rootKey, changer);
+                    // newValue = new MyConfigurationImpl(super.keys, fieldName, rootKey, changer, boolean);

Review comment:
       ```suggestion
                       // newValue = new MyConfigurationImpl(super.keys, fieldName, rootKey, changer, listenOnly);
   ```

##########
File path: modules/configuration/src/test/java/org/apache/ignite/internal/configuration/notifications/ConfigurationListenerTest.java
##########
@@ -644,4 +667,366 @@ public void dataRace() throws Exception {
 
         assertEquals(List.of("deleted"), log);
     }
+
+    /** */
+    @Test
+    void testNoGetOrUpdateConfigValueForAny() throws Exception {
+        ChildConfiguration any0 = configuration.elements().any();
+
+        assertThrows(ConfigurationException.class, () -> any0.value());
+        assertThrows(ConfigurationException.class, () -> any0.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any0.str().value());
+        assertThrows(ConfigurationException.class, () -> any0.str().update(""));
+
+        assertThrows(ConfigurationException.class, () -> any0.child2().value());
+        assertThrows(ConfigurationException.class, () -> any0.child2().change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any0.child2().i().value());
+        assertThrows(ConfigurationException.class, () -> any0.child2().i().update(100));
+
+        assertThrows(ConfigurationException.class, () -> any0.elements2().value());
+        assertThrows(ConfigurationException.class, () -> any0.elements2().change(doNothingConsumer()));
+        assertThrows(ConfigurationException.class, () -> any0.elements2().get("test"));
+
+        Child2Configuration any1 = any0.elements2().any();
+
+        assertThrows(ConfigurationException.class, () -> any1.value());
+        assertThrows(ConfigurationException.class, () -> any1.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any1.i().value());
+        assertThrows(ConfigurationException.class, () -> any1.i().update(200));
+
+        configuration.elements().change(c0 -> c0.create("test", c1 -> c1.changeStr("foo"))).get(1, SECONDS);
+
+        Child2Configuration any2 = configuration.elements().get("test").elements2().any();
+
+        assertThrows(ConfigurationException.class, () -> any2.value());
+        assertThrows(ConfigurationException.class, () -> any2.change(doNothingConsumer()));
+
+        assertThrows(ConfigurationException.class, () -> any2.i().value());
+        assertThrows(ConfigurationException.class, () -> any2.i().update(300));
+    }
+
+    /** */
+    @Test
+    void testAnyListeners() throws Exception {
+        List<String> events = new ArrayList<>();
+
+        // Add "regular" listeners.
+        configuration.listen(configListener(ctx -> events.add("root")));
+
+        configuration.child().listen(configListener(ctx -> events.add("root.child")));
+        configuration.child().str().listen(configListener(ctx -> events.add("root.child.str")));
+        configuration.child().child2().listen(configListener(ctx -> events.add("root.child.child2")));
+        configuration.child().child2().i().listen(configListener(ctx -> events.add("root.child.child2.i")));
+
+        configuration.elements().listen(configListener(ctx -> events.add("root.elements")));
+        configuration.elements().listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.onCrt")));
+        configuration.elements().listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.onUpd")));
+        configuration.elements().listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.onRen")));
+        configuration.elements().listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.onDel")));
+
+        configuration.elements().change(c -> c.create("0", doNothingConsumer())).get(1, SECONDS);
+
+        ChildConfiguration childCfg = this.configuration.elements().get("0");
+
+        childCfg.listen(configListener(ctx -> events.add("root.elements.0")));
+        childCfg.str().listen(configListener(ctx -> events.add("root.elements.0.str")));
+        childCfg.child2().listen(configListener(ctx -> events.add("root.elements.0.child2")));
+        childCfg.child2().i().listen(configListener(ctx -> events.add("root.elements.0.child2.i")));
+
+        NamedConfigurationTree<Child2Configuration, Child2View, Child2Change> elements2 = childCfg.elements2();
+
+        elements2.listen(configListener(ctx -> events.add("root.elements.0.elements2")));
+        elements2.listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.0.elements2.onCrt")));
+        elements2.listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.0.elements2.onUpd")));
+        elements2.listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.0.elements2.onRen")));
+        elements2.listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.0.elements2.onDel")));
+
+        elements2.change(c -> c.create("0", doNothingConsumer())).get(1, SECONDS);
+
+        Child2Configuration child2 = elements2.get("0");
+
+        child2.listen(configListener(ctx -> events.add("root.elements.0.elements2.0")));
+        child2.i().listen(configListener(ctx -> events.add("root.elements.0.elements2.0.i")));
+
+        // Adding "any" listeners.
+        ChildConfiguration anyChild = configuration.elements().any();
+
+        anyChild.listen(configListener(ctx -> events.add("root.elements.any")));
+        anyChild.str().listen(configListener(ctx -> events.add("root.elements.any.str")));
+        anyChild.child2().listen(configListener(ctx -> events.add("root.elements.any.child2")));
+        anyChild.child2().i().listen(configListener(ctx -> events.add("root.elements.any.child2.i")));
+
+        NamedConfigurationTree<Child2Configuration, Child2View, Child2Change> anyEl2 = anyChild.elements2();
+
+        anyEl2.listen(configListener(ctx -> events.add("root.elements.any.elements2")));
+        anyEl2.listenElements(configNamedListenerOnCreate(ctx -> events.add("root.elements.any.elements2.onCrt")));
+        anyEl2.listenElements(configNamedListenerOnUpdate(ctx -> events.add("root.elements.any.elements2.onUpd")));
+        anyEl2.listenElements(configNamedListenerOnRename(ctx -> events.add("root.elements.any.elements2.onRen")));
+        anyEl2.listenElements(configNamedListenerOnDelete(ctx -> events.add("root.elements.any.elements2.onDel")));
+
+        Child2Configuration anyChild2 = anyEl2.any();
+
+        anyChild2.listen(configListener(ctx -> events.add("root.elements.any.elements2.any")));
+        anyChild2.i().listen(configListener(ctx -> events.add("root.elements.any.elements2.any.i")));
+
+        childCfg.elements2().any().listen(configListener(ctx -> events.add("root.elements.0.elements2.any")));
+        childCfg.elements2().any().i().listen(configListener(ctx -> events.add("root.elements.0.elements2.any.i")));
+
+        // Tests.
+        checkListeners(
+            () -> configuration.child().change(c -> c.changeStr("x").changeChild2(c0 -> c0.changeI(100))),
+            List.of("root", "root.child", "root.child.str", "root.child.child2", "root.child.child2.i"),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").str().update("x"),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.str",
+                "root.elements.0.str"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").elements2().get("0").i().update(200),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.elements2",
+                "root.elements.0.elements2",
+                "root.elements.any.elements2.onUpd",
+                "root.elements.0.elements2.onUpd",
+                //
+                "root.elements.any.elements2.any",
+                "root.elements.0.elements2.any",
+                "root.elements.0.elements2.0",
+                //
+                "root.elements.any.elements2.any.i",
+                "root.elements.0.elements2.any.i",
+                "root.elements.0.elements2.0.i"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("0").elements2().change(c -> c.create("1", doNothingConsumer())),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                //
+                "root.elements.any",
+                "root.elements.0",
+                //
+                "root.elements.any.elements2",
+                "root.elements.0.elements2",
+                "root.elements.any.elements2.onCrt",
+                "root.elements.0.elements2.onCrt",
+                //
+                "root.elements.any.elements2.onUpd",
+                "root.elements.0.elements2.onUpd"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements()
+                .change(c -> c.create("1", c0 -> c0.changeElements2(c1 -> c1.create("2", doNothingConsumer())))),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onCrt",
+                "root.elements.onUpd"
+            ),
+            events
+        );
+
+        checkListeners(
+            () -> configuration.elements().get("1").elements2().get("2").i().update(200),
+            List.of(
+                "root",
+                "root.elements",
+                "root.elements.onUpd",
+                "root.elements.onUpd",

Review comment:
       This one's called twice, why? Looks like a bug




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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



##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/util/ConfigurationNotificationsUtil.java
##########
@@ -173,19 +240,41 @@ public static void notifyListeners(
                     }
 
                     for (String name : deleted) {
+                        for (DynamicConfiguration<InnerNode, ?> anyConfig : anyConfigs) {
+                            notifyPublicListeners(
+                                namedDynamicConfig(anyConfig, key).extendedListeners(),
+                                oldNamedList.get(name),
+                                null,
+                                storageRevision,
+                                futures,
+                                ConfigurationNamedListListener::onDelete
+                            );
+                        }
+
                         notifyPublicListeners(
-                            extListeners,
+                            namedDynamicConfig(cfgNode, key).extendedListeners(),
                             oldNamedList.get(name),
                             null,
                             storageRevision,
                             futures,
                             ConfigurationNamedListListener::onDelete
                         );
 
-                        var deletedProp = (ConfigurationNode<N, ?>)namedListCfgMembers.get(name);
+                        // Notification for deleted configuration.
+
+                        for (var anyConfig : view(anyConfigs, cfg -> any(namedDynamicConfig(cfg, key)))) {

Review comment:
       Please don't use "var" this way




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] tkalkirill commented on a change in pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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



##########
File path: modules/configuration-api/src/main/java/org/apache/ignite/configuration/ConfigurationListenOnlyException.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+/**
+ * This exception is used if an attempt was made to get/update a configuration value in listen-only mode.
+ */
+public class ConfigurationListenOnlyException extends RuntimeException {

Review comment:
       Let's leave it as it is.




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] tkalkirill commented on a change in pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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



##########
File path: modules/configuration-api/src/main/java/org/apache/ignite/configuration/ConfigurationListenOnlyException.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+/**
+ * This exception is used if an attempt was made to get/update a configuration value in listen-only mode.
+ */
+public class ConfigurationListenOnlyException extends RuntimeException {

Review comment:
       Then all the `Configuration*Exception` need to be done this way, I think we need to ask @ibessonov opinion.




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] tkalkirill commented on a change in pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/CollectionUtils.java
##########
@@ -142,6 +143,41 @@ else if (ts == null || ts.length == 0)
         }
     }
 
+    /**
+     * Create a view for an iterable with conversion of elements.
+     *
+     * @param iterable Iterable.
+     * @param mapper Conversion function.
+     * @param <T1> Type of the elements.
+     * @param <T2> Converted type of the elements.
+     * @return View of iterable.
+     */
+    public static <T1, T2> Iterable<T2> view(

Review comment:
       Replaced with `CollectionUtils#viewReadOnly`




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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



##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/util/ConfigurationNotificationsUtil.java
##########
@@ -250,23 +366,25 @@ public static void notifyListeners(
         List<CompletableFuture<?>> futures,
         BiFunction<L, ConfigurationNotificationEvent<V>, CompletableFuture<?>> updater
     ) {
-        ConfigurationNotificationEvent<V> evt = new ConfigurationNotificationEventImpl<>(
-            oldVal,
-            newVal,
-            storageRevision
-        );
+        if (!listeners.isEmpty()) {

Review comment:
       Why not just add this?:
   if (listeners.isEmpty())
       return;
   This way code will much easier to read




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] ibessonov merged pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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


   


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] tkalkirill commented on a change in pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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



##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/util/ConfigurationNotificationsUtil.java
##########
@@ -30,66 +32,115 @@
 import org.apache.ignite.configuration.notifications.ConfigurationListener;
 import org.apache.ignite.configuration.notifications.ConfigurationNamedListListener;
 import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
-import org.apache.ignite.internal.configuration.ConfigurationNode;
 import org.apache.ignite.internal.configuration.DynamicConfiguration;
 import org.apache.ignite.internal.configuration.DynamicProperty;
 import org.apache.ignite.internal.configuration.NamedListConfiguration;
 import org.apache.ignite.internal.configuration.tree.ConfigurationVisitor;
 import org.apache.ignite.internal.configuration.tree.InnerNode;
 import org.apache.ignite.internal.configuration.tree.NamedListNode;
+import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.innerNodeVisitor;
 import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.leafNodeVisitor;
 import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.namedListNodeVisitor;
+import static org.apache.ignite.internal.util.CollectionUtils.viewReadOnly;
 
 /** */
 public class ConfigurationNotificationsUtil {
     /**
      * Recursively notifies all public configuration listeners, accumulating resulting futures in {@code futures} list.
+     *
      * @param oldInnerNode Old configuration values root.
      * @param newInnerNode New configuration values root.
      * @param cfgNode Public configuration tree node corresponding to the current inner nodes.
      * @param storageRevision Storage revision.
      * @param futures Write-only list of futures to accumulate results.
      */
     public static void notifyListeners(
-        InnerNode oldInnerNode,
+        @Nullable InnerNode oldInnerNode,
         InnerNode newInnerNode,
         DynamicConfiguration<InnerNode, ?> cfgNode,
         long storageRevision,
         List<CompletableFuture<?>> futures
+    ) {
+        notifyListeners(oldInnerNode, newInnerNode, cfgNode, storageRevision, futures, List.of(), new HashMap<>());
+    }
+
+    /**
+     * Recursively notifies all public configuration listeners, accumulating resulting futures in {@code futures} list.
+     *
+     * @param oldInnerNode Old configuration values root.
+     * @param newInnerNode New configuration values root.
+     * @param cfgNode Public configuration tree node corresponding to the current inner nodes.
+     * @param storageRevision Storage revision.
+     * @param futures Write-only list of futures to accumulate results.
+     * @param anyConfigs Current {@link NamedListConfiguration#any "any"} configurations.
+     * @param eventConfigs Configuration containers for {@link ConfigurationNotificationEvent}.
+     */
+    private static void notifyListeners(

Review comment:
       I suggest leaving it as it is.




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/CollectionUtils.java
##########
@@ -142,6 +143,41 @@ else if (ts == null || ts.length == 0)
         }
     }
 
+    /**
+     * Create a view for an iterable with conversion of elements.
+     *
+     * @param iterable Iterable.
+     * @param mapper Conversion function.
+     * @param <T1> Type of the elements.
+     * @param <T2> Converted type of the elements.
+     * @return View of iterable.
+     */
+    public static <T1, T2> Iterable<T2> view(

Review comment:
       Why do you need this if Java has streams?




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #336: IGNITE-15409 Adding NamedConfigurationTree#any to listen for changes in any configuration

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



##########
File path: modules/configuration-api/src/main/java/org/apache/ignite/configuration/NamedConfigurationTree.java
##########
@@ -45,7 +45,14 @@
     void listenElements(ConfigurationNamedListListener<VIEW> listener);
 
     /**
-     * Get a placeholder that allows you to add listeners for changing configuration value
+     * Remove named-list-specific configuration values listener.

Review comment:
       ```suggestion
        * Removes named-list-specific configuration values listener.
   ```




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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