You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ib...@apache.org on 2023/01/18 14:10:27 UTC

[ignite-3] branch main updated: IGNITE-18516 Allowed safe configuration read from configuration update closures & other enhancements (#1540)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 9934564b0e IGNITE-18516 Allowed safe configuration read from configuration update closures & other enhancements (#1540)
9934564b0e is described below

commit 9934564b0ee86b896a912f226a3b9409d422b2eb
Author: Ivan Bessonov <be...@gmail.com>
AuthorDate: Wed Jan 18 17:10:21 2023 +0300

    IGNITE-18516 Allowed safe configuration read from configuration update closures & other enhancements (#1540)
---
 .../processor/ConfigurationProcessor.java          |   9 ++
 modules/configuration/README.md                    |   9 +-
 .../configuration/ConfigurationChanger.java        |  92 +++++++------
 .../configuration/ConfigurationRegistry.java       |  36 ++++++
 .../internal/configuration/SuperRootChange.java    |  36 ++++++
 .../configuration/asm/InnerNodeAsmGenerator.java   | 144 +++++++++++++--------
 .../configuration/ConfigurationChangerTest.java    |  20 +++
 .../configuration/ConfigurationRegistryTest.java   |  49 +++++++
 .../asm/ConfigurationAsmGeneratorTest.java         |  29 ++++-
 .../distributionzones/DistributionZoneManager.java |   1 -
 .../DistributedConfigurationCatchUpTest.java       |  22 ++--
 .../sql/engine/exec/ddl/DdlCommandHandler.java     |   1 -
 12 files changed, 339 insertions(+), 109 deletions(-)

diff --git a/modules/configuration-annotation-processor/src/main/java/org/apache/ignite/internal/configuration/processor/ConfigurationProcessor.java b/modules/configuration-annotation-processor/src/main/java/org/apache/ignite/internal/configuration/processor/ConfigurationProcessor.java
index cc501368e3..45d529f3f3 100644
--- a/modules/configuration-annotation-processor/src/main/java/org/apache/ignite/internal/configuration/processor/ConfigurationProcessor.java
+++ b/modules/configuration-annotation-processor/src/main/java/org/apache/ignite/internal/configuration/processor/ConfigurationProcessor.java
@@ -518,6 +518,15 @@ public class ConfigurationProcessor extends AbstractProcessor {
             }
 
             changeClsBuilder.addMethod(changeMtdBuilder.build());
+
+            // Create "FooChange changeFoo()" method with no parameters, if it's a config value or named list value.
+            if (valAnnotation == null) {
+                MethodSpec.Builder shortChangeMtdBuilder = MethodSpec.methodBuilder(changeMtdName)
+                        .addModifiers(PUBLIC, ABSTRACT)
+                        .returns(changeFieldType);
+
+                changeClsBuilder.addMethod(shortChangeMtdBuilder.build());
+            }
         }
 
         if (isPolymorphicConfig) {
diff --git a/modules/configuration/README.md b/modules/configuration/README.md
index 3803736bfa..6eceb06ab1 100644
--- a/modules/configuration/README.md
+++ b/modules/configuration/README.md
@@ -284,10 +284,13 @@ For the example above, the following interfaces would be generated:
 ```java
 public interface ParentChange extends ParentView { 
     ParentChange changeElements(Consumer<NamedListChange<NamedElementChange>> elements);
+    NamedListChange<NamedElementChange> changeElements();
 
     ParentChange changeChild(Consumer<ChildChange> child);
+    ChildChange changeChild();
 
     ParentChange changePolymorphicChild(Consumer<PolymorphicChange> polymorphicChild);
+    PolymorphicChange changePolymorphicChild();
 }
 
 public interface ChildChange extends ChildView {
@@ -314,9 +317,13 @@ parentCfg.change(parent ->
     )
 ).get();
 
+parentCfg.change(parent ->
+    parent.changeChild().changeStr("newStr2")
+).get();
+
 ChildConfiguration childCfg = parentCfg.child();
 
-childCfg.changeStr("newStr2").get();
+childCfg.changeStr("newStr3").get();
 ```
 
 Example of changing the type of a polymorphic configuration:
diff --git a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationChanger.java b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationChanger.java
index a86447cba4..e018ee70d8 100644
--- a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationChanger.java
+++ b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationChanger.java
@@ -53,6 +53,8 @@ import java.util.concurrent.ForkJoinPool;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.Function;
 import org.apache.ignite.configuration.ConfigurationChangeException;
 import org.apache.ignite.configuration.RootKey;
@@ -102,6 +104,9 @@ public abstract class ConfigurationChanger implements DynamicConfigurationChange
     /** Configuration listener notification counter, must be incremented before each use of {@link #notificator}. */
     private final AtomicLong notificationListenerCnt = new AtomicLong();
 
+    /** Lock for reading/updating the {@link #storageRoots}. Fair, to give a higher priority to external updates. */
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock(true);
+
     /**
      * Closure interface to be used by the configuration changer. An instance of this closure is passed into the constructor and invoked
      * every time when there's an update from any of the storages.
@@ -482,23 +487,16 @@ public abstract class ConfigurationChanger implements DynamicConfigurationChange
      * @throws ComponentNotStartedException if changer is not started.
      */
     private CompletableFuture<Void> changeInternally(ConfigurationSource src) {
-        StorageRoots localRoots = storageRoots;
-
-        if (localRoots == null) {
+        if (storageRoots == null) {
             throw new ComponentNotStartedException();
         }
 
         return storage.lastRevision()
-            .thenCompose(storageRevision -> {
+            .thenComposeAsync(storageRevision -> {
                 assert storageRevision != null;
 
-                if (localRoots.version < storageRevision) {
-                    // Need to wait for the configuration updates from the storage, then try to update again (loop).
-                    return localRoots.changeFuture.thenCompose(v -> changeInternally(src));
-                } else {
-                    return changeInternally0(localRoots, src);
-                }
-            })
+                return changeInternally0(src, storageRevision);
+            }, pool)
             .exceptionally(throwable -> {
                 Throwable cause = throwable.getCause();
 
@@ -514,41 +512,51 @@ public abstract class ConfigurationChanger implements DynamicConfigurationChange
      * Internal configuration change method that completes provided future.
      *
      * @param src Configuration source.
+     * @param storageRevision Latest storage revision from the metastorage.
      * @return Future that will be completed after changes are written to the storage.
      */
-    private CompletableFuture<Void> changeInternally0(StorageRoots localRoots, ConfigurationSource src) {
-        return CompletableFuture
-            .supplyAsync(() -> {
-                SuperRoot curRoots = localRoots.roots;
+    private CompletableFuture<Void> changeInternally0(ConfigurationSource src, long storageRevision) {
+        // Read lock protects "storageRoots" field from being updated, thus guaranteeing a thread-safe read of configuration inside of the
+        // change closure.
+        rwLock.readLock().lock();
 
-                SuperRoot changes = curRoots.copy();
+        try {
+            // Put it into a variable to avoid volatile reads.
+            // This read and the following comparison MUST be performed while holding read lock.
+            StorageRoots localRoots = storageRoots;
 
-                src.reset();
+            if (localRoots.version < storageRevision) {
+                // Need to wait for the configuration updates from the storage, then try to update again (loop).
+                return localRoots.changeFuture.thenCompose(v -> changeInternally(src));
+            }
 
-                src.descend(changes);
+            SuperRoot curRoots = localRoots.roots;
 
-                addDefaults(changes);
+            SuperRoot changes = curRoots.copy();
 
-                Map<String, Serializable> allChanges = createFlattenedUpdatesMap(curRoots, changes);
+            src.reset();
 
-                dropNulls(changes);
+            src.descend(changes);
 
-                List<ValidationIssue> validationIssues = ValidationUtil.validate(
-                        curRoots,
-                        changes,
-                        this::getRootNode,
-                        cachedAnnotations,
-                        validators
-                );
+            addDefaults(changes);
 
-                if (!validationIssues.isEmpty()) {
-                    throw new ConfigurationValidationException(validationIssues);
-                }
+            Map<String, Serializable> allChanges = createFlattenedUpdatesMap(curRoots, changes);
 
-                return allChanges;
-            }, pool)
-            .thenCompose(allChanges ->
-                storage.write(allChanges, localRoots.version)
+            dropNulls(changes);
+
+            List<ValidationIssue> validationIssues = ValidationUtil.validate(
+                    curRoots,
+                    changes,
+                    this::getRootNode,
+                    cachedAnnotations,
+                    validators
+            );
+
+            if (!validationIssues.isEmpty()) {
+                throw new ConfigurationValidationException(validationIssues);
+            }
+
+            return storage.write(allChanges, localRoots.version)
                     .thenCompose(casWroteSuccessfully -> {
                         if (casWroteSuccessfully) {
                             return localRoots.changeFuture;
@@ -557,8 +565,10 @@ public abstract class ConfigurationChanger implements DynamicConfigurationChange
                             // because we work with async code (futures).
                             return localRoots.changeFuture.thenCompose(v -> changeInternally(src));
                         }
-                    })
-            );
+                    });
+        } finally {
+            rwLock.readLock().unlock();
+        }
     }
 
     /**
@@ -581,7 +591,13 @@ public abstract class ConfigurationChanger implements DynamicConfigurationChange
 
         long newChangeId = changedEntries.changeId();
 
-        storageRoots = new StorageRoots(newSuperRoot, newChangeId);
+        rwLock.writeLock().lock();
+
+        try {
+            storageRoots = new StorageRoots(newSuperRoot, newChangeId);
+        } finally {
+            rwLock.writeLock().unlock();
+        }
 
         // Save revisions for recovery.
         return storage.writeConfigurationRevision(oldStorageRoots.version, storageRoots.version)
diff --git a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationRegistry.java b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationRegistry.java
index 82724c9d5e..781ed6be9a 100644
--- a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationRegistry.java
+++ b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationRegistry.java
@@ -42,8 +42,10 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
 import java.util.function.Function;
 import org.apache.ignite.configuration.ConfigurationTree;
 import org.apache.ignite.configuration.RootKey;
@@ -62,6 +64,7 @@ import org.apache.ignite.internal.configuration.notifications.ConfigurationStora
 import org.apache.ignite.internal.configuration.storage.ConfigurationStorage;
 import org.apache.ignite.internal.configuration.tree.ConfigurationSource;
 import org.apache.ignite.internal.configuration.tree.ConfigurationVisitor;
+import org.apache.ignite.internal.configuration.tree.ConstructableTreeNode;
 import org.apache.ignite.internal.configuration.tree.InnerNode;
 import org.apache.ignite.internal.configuration.tree.TraversableTreeNode;
 import org.apache.ignite.internal.configuration.util.ConfigurationUtil;
@@ -267,6 +270,39 @@ public class ConfigurationRegistry implements IgniteComponent, ConfigurationStor
         return changer.change(changesSrc);
     }
 
+    /**
+     * Change configuration. Gives the possibility to atomically update several root trees.
+     *
+     * @param change Closure that would consume a mutable super root instance.
+     * @return Future that is completed on change completion.
+     */
+    public CompletableFuture<Void> change(Consumer<SuperRootChange> change) {
+        return change(new ConfigurationSource() {
+            @Override
+            public void descend(ConstructableTreeNode node) {
+                assert node instanceof SuperRoot : "Descending always starts with super root: " + node;
+
+                SuperRoot superRoot = (SuperRoot) node;
+
+                change.accept(new SuperRootChange() {
+                    @Override
+                    public <V> V viewRoot(RootKey<? extends ConfigurationTree<V, ?>, V> rootKey) {
+                        return Objects.requireNonNull(superRoot.getRoot(rootKey)).specificNode();
+                    }
+
+                    @Override
+                    public <C> C changeRoot(RootKey<? extends ConfigurationTree<?, C>, ?> rootKey) {
+                        // "construct" does a field copying, which is what we need before mutating it.
+                        superRoot.construct(rootKey.key(), ConfigurationUtil.EMPTY_CFG_SRC, true);
+
+                        // "rootView" is not re-used here because of return type incompatibility, although code is the same.
+                        return Objects.requireNonNull(superRoot.getRoot(rootKey)).specificNode();
+                    }
+                });
+            }
+        });
+    }
+
     /**
      * Configuration change notifier.
      *
diff --git a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/SuperRootChange.java b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/SuperRootChange.java
new file mode 100644
index 0000000000..f622c9b563
--- /dev/null
+++ b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/SuperRootChange.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.configuration;
+
+import org.apache.ignite.configuration.ConfigurationTree;
+import org.apache.ignite.configuration.RootKey;
+
+/**
+ * Interface that represent a "change" for the conjunction of all roots in the configuration.
+ */
+public interface SuperRootChange {
+    /**
+     * Returns a root view for the root key.
+     */
+    <V> V viewRoot(RootKey<? extends ConfigurationTree<V, ?>, V> rootKey);
+
+    /**
+     * Returns a root change for the root key.
+     */
+    <C> C changeRoot(RootKey<? extends ConfigurationTree<?, C>, ?> rootKey);
+}
diff --git a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/asm/InnerNodeAsmGenerator.java b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/asm/InnerNodeAsmGenerator.java
index 94afc231dc..fd6ac4a357 100644
--- a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/asm/InnerNodeAsmGenerator.java
+++ b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/asm/InnerNodeAsmGenerator.java
@@ -91,6 +91,7 @@ import java.util.function.BiFunction;
 import java.util.function.Consumer;
 import java.util.function.Function;
 import org.apache.ignite.configuration.ConfigurationWrongPolymorphicTypeIdException;
+import org.apache.ignite.configuration.NamedListChange;
 import org.apache.ignite.configuration.NamedListView;
 import org.apache.ignite.configuration.annotation.AbstractConfiguration;
 import org.apache.ignite.configuration.annotation.InjectedName;
@@ -345,7 +346,7 @@ class InnerNodeAsmGenerator extends AbstractAsmGenerator {
             }
 
             // Add change methods.
-            MethodDefinition changeMtd0 = addNodeChangeMethod(
+            List<MethodDefinition> changeMethods = addNodeChangeMethod(
                     innerNodeClassDef,
                     schemaField,
                     changeMtd -> getThisFieldCode(changeMtd, fieldDef),
@@ -353,7 +354,8 @@ class InnerNodeAsmGenerator extends AbstractAsmGenerator {
                     null
             );
 
-            addNodeChangeBridgeMethod(innerNodeClassDef, changeClassName(schemaField.getDeclaringClass()), changeMtd0);
+            // Only first element requires a bridge. Please refer to "addNodeChangeMethod" for explanation.
+            addNodeChangeBridgeMethod(innerNodeClassDef, changeClassName(schemaField.getDeclaringClass()), changeMethods.get(0));
         }
 
         Map<Class<?>, List<Field>> polymorphicFieldsByExtension = Map.of();
@@ -659,24 +661,7 @@ class InnerNodeAsmGenerator extends AbstractAsmGenerator {
         // return result;
         bytecodeBlock.ret(schemaFieldType);
 
-        if (getPolymorphicTypeIdFieldFun != null) {
-            assert isPolymorphicConfigInstance(schemaField.getDeclaringClass()) : schemaField;
-
-            // tmpVar = this.typeId; OR this.field.typeId.
-            BytecodeExpression getPolymorphicTypeIdFieldValue = getPolymorphicTypeIdFieldFun.apply(viewMtd);
-            String polymorphicInstanceId = polymorphicInstanceId(schemaField.getDeclaringClass());
-
-            // if (!"first".equals(tmpVar)) throw Ex;
-            // else return value;
-            viewMtd.getBody().append(
-                    new IfStatement()
-                            .condition(not(constantString(polymorphicInstanceId).invoke(STRING_EQUALS_MTD, getPolymorphicTypeIdFieldValue)))
-                            .ifTrue(throwException(ConfigurationWrongPolymorphicTypeIdException.class, getPolymorphicTypeIdFieldValue))
-                            .ifFalse(bytecodeBlock)
-            );
-        } else {
-            viewMtd.getBody().append(bytecodeBlock);
-        }
+        enrichWithPolymorphicTypeCheck(schemaField, getPolymorphicTypeIdFieldFun, viewMtd, bytecodeBlock);
     }
 
     /**
@@ -684,9 +669,10 @@ class InnerNodeAsmGenerator extends AbstractAsmGenerator {
      *
      * @param classDef    Node class definition.
      * @param schemaField Configuration schema class field.
-     * @return Definition of change method.
+     * @return List of method definition. First element is the "default" change method that accepts closure or a value. Second, optional
+     *      element, is a change methods with no parameters.
      */
-    private MethodDefinition addNodeChangeMethod(
+    private List<MethodDefinition> addNodeChangeMethod(
             ClassDefinition classDef,
             Field schemaField,
             Function<MethodDefinition, BytecodeExpression> getFieldCodeFun,
@@ -703,6 +689,8 @@ class InnerNodeAsmGenerator extends AbstractAsmGenerator {
                 arg("change", isValue(schemaField) ? type(schemaFieldType) : type(Consumer.class))
         );
 
+        MethodDefinition shortChangeMtd = null;
+
         // var change;
         BytecodeExpression changeVar = changeMtd.getScope().getVariable("change");
 
@@ -731,56 +719,106 @@ class InnerNodeAsmGenerator extends AbstractAsmGenerator {
             // this.field = newValue;
             bytecodeBlock.append(setFieldCodeFun.apply(changeMtd, newValue));
         } else {
-            BytecodeExpression newValue;
+            shortChangeMtd = createShortChangeMethod(classDef, schemaField, getFieldCodeFun, setFieldCodeFun, getPolymorphicTypeIdFieldFun);
 
-            if (isConfigValue(schemaField)) {
-                // newValue = (this.field == null) ? new ValueNode() : (ValueNode)this.field.copy();
-                newValue = cgen.newOrCopyNodeField(schemaField, getFieldCodeFun.apply(changeMtd));
-            } else {
-                assert isNamedConfigValue(schemaField) : schemaField;
+            // change.accept(this.field); OR change.accept(this.field.specificNode());
+            bytecodeBlock.append(changeVar.invoke(ACCEPT, changeMtd.getThis().invoke(shortChangeMtd, List.of())));
+        }
 
-                // newValue = (ValueNode)this.field.copy();
-                newValue = cgen.copyNodeField(schemaField, getFieldCodeFun.apply(changeMtd));
-            }
+        // return this;
+        bytecodeBlock.append(changeMtd.getThis()).retObject();
 
-            // this.field = newValue;
-            bytecodeBlock.append(setFieldCodeFun.apply(changeMtd, newValue));
+        enrichWithPolymorphicTypeCheck(schemaField, getPolymorphicTypeIdFieldFun, changeMtd, bytecodeBlock);
 
-            // this.field;
-            BytecodeExpression getFieldCode = getFieldCodeFun.apply(changeMtd);
+        if (shortChangeMtd == null) {
+            return List.of(changeMtd);
+        }
 
-            if (isPolymorphicConfig(schemaFieldType) && isConfigValue(schemaField)) {
-                // this.field.specificNode();
-                getFieldCode = getFieldCode.invoke(SPECIFIC_NODE_MTD);
-            }
+        return List.of(changeMtd, shortChangeMtd);
+    }
 
-            // change.accept(this.field); OR change.accept(this.field.specificNode());
-            bytecodeBlock.append(changeVar.invoke(ACCEPT, getFieldCode));
+    /**
+     * Creates a "short" method to return a changed field instance. Name is the same as for {@code changeFoo(Consumer<FooChange> change)"}.
+     * This method will be reused to create the value that is passed to "default" change method.
+     * Method's signature is {@code FooChange changeFoo()}, it returns a mutable configuration value to be used for configuration updates.
+     */
+    private MethodDefinition createShortChangeMethod(
+            ClassDefinition classDef,
+            Field schemaField,
+            Function<MethodDefinition, BytecodeExpression> getFieldCodeFun,
+            BiFunction<MethodDefinition, BytecodeExpression, BytecodeExpression> setFieldCodeFun,
+            @Nullable Function<MethodDefinition, BytecodeExpression> getPolymorphicTypeIdFieldFun
+    ) {
+        MethodDefinition shortChangeMtd = classDef.declareMethod(
+                EnumSet.of(PUBLIC),
+                changeMethodName(schemaField.getName()),
+                isConfigValue(schemaField)
+                    ? typeFromJavaClassName(changeClassName(schemaField.getType()))
+                    : type(NamedListChange.class)
+        );
+
+        BytecodeBlock shortBytecodeBlock = new BytecodeBlock();
+
+        BytecodeExpression newValue;
+
+        if (isConfigValue(schemaField)) {
+            // newValue = (this.field == null) ? new ValueNode() : (ValueNode)this.field.copy();
+            newValue = cgen.newOrCopyNodeField(schemaField, getFieldCodeFun.apply(shortChangeMtd));
+        } else {
+            assert isNamedConfigValue(schemaField) : schemaField;
+
+            // newValue = (ValueNode)this.field.copy();
+            newValue = cgen.copyNodeField(schemaField, getFieldCodeFun.apply(shortChangeMtd));
         }
 
-        // return this;
-        bytecodeBlock.append(changeMtd.getThis()).retObject();
+        // this.field = newValue;
+        shortBytecodeBlock.append(setFieldCodeFun.apply(shortChangeMtd, newValue));
+
+        // this.field;
+        BytecodeExpression getFieldCode = getFieldCodeFun.apply(shortChangeMtd);
+
+        if (isPolymorphicConfig(schemaField.getType()) && isConfigValue(schemaField)) {
+            // this.field.specificNode();
+            getFieldCode = getFieldCode.invoke(SPECIFIC_NODE_MTD);
+        }
+
+        shortBytecodeBlock.append(getFieldCode).retObject();
+
+        enrichWithPolymorphicTypeCheck(schemaField, getPolymorphicTypeIdFieldFun, shortChangeMtd, shortBytecodeBlock);
+
+        shortChangeMtd.getBody().append(shortBytecodeBlock);
 
+        return shortChangeMtd;
+    }
+
+    /**
+     * Adds a check that expected polymorphic type ID matches the real one. Throws exception otherwise. Simply adds {@code bytecodeBlock}
+     * into methods body if {@code getPolymorphicTypeIdFieldFun} is {@code null} (this means that class is not polymorphic).
+     */
+    private void enrichWithPolymorphicTypeCheck(
+            Field schemaField,
+            @Nullable Function<MethodDefinition, BytecodeExpression> getPolymorphicTypeIdFieldFun,
+            MethodDefinition method,
+            BytecodeBlock bytecodeBlock
+    ) {
         if (getPolymorphicTypeIdFieldFun != null) {
             assert isPolymorphicConfigInstance(schemaField.getDeclaringClass()) : schemaField;
 
             // tmpVar = this.typeId; OR this.field.typeId.
-            BytecodeExpression getPolymorphicTypeIdFieldValue = getPolymorphicTypeIdFieldFun.apply(changeMtd);
+            BytecodeExpression getPolymorphicTypeIdFieldValue = getPolymorphicTypeIdFieldFun.apply(method);
             String polymorphicInstanceId = polymorphicInstanceId(schemaField.getDeclaringClass());
 
             // if (!"first".equals(tmpVar)) throw Ex;
             // else change_value;
-            changeMtd.getBody().append(
+            method.getBody().append(
                     new IfStatement()
                             .condition(not(constantString(polymorphicInstanceId).invoke(STRING_EQUALS_MTD, getPolymorphicTypeIdFieldValue)))
                             .ifTrue(throwException(ConfigurationWrongPolymorphicTypeIdException.class, getPolymorphicTypeIdFieldValue))
                             .ifFalse(bytecodeBlock)
             );
         } else {
-            changeMtd.getBody().append(bytecodeBlock);
+            method.getBody().append(bytecodeBlock);
         }
-
-        return changeMtd;
     }
 
     /**
@@ -1586,7 +1624,7 @@ class InnerNodeAsmGenerator extends AbstractAsmGenerator {
                 continue;
             }
 
-            MethodDefinition changeMtd0 = addNodeChangeMethod(
+            List<MethodDefinition> changeMethods = addNodeChangeMethod(
                     classDef,
                     schemaField,
                     changeMtd -> getThisFieldCode(changeMtd, parentInnerNodeFieldDef, schemaFieldDef),
@@ -1594,7 +1632,8 @@ class InnerNodeAsmGenerator extends AbstractAsmGenerator {
                     null
             );
 
-            addNodeChangeBridgeMethod(classDef, schemaClassInfo.changeClassName, changeMtd0);
+            // Only first element requires a bridge. Please refer to "addNodeChangeMethod" for explanation.
+            addNodeChangeBridgeMethod(classDef, schemaClassInfo.changeClassName, changeMethods.get(0));
         }
 
         FieldDefinition polymorphicTypeIdFieldDef = fieldDefs.get(polymorphicIdField(schemaClass).getName());
@@ -1610,7 +1649,7 @@ class InnerNodeAsmGenerator extends AbstractAsmGenerator {
                     viewMtd -> getThisFieldCode(viewMtd, parentInnerNodeFieldDef, polymorphicTypeIdFieldDef)
             );
 
-            MethodDefinition changeMtd0 = addNodeChangeMethod(
+            List<MethodDefinition> changeMethods = addNodeChangeMethod(
                     classDef,
                     polymorphicField,
                     changeMtd -> getThisFieldCode(changeMtd, parentInnerNodeFieldDef, polymorphicFieldDef),
@@ -1618,7 +1657,8 @@ class InnerNodeAsmGenerator extends AbstractAsmGenerator {
                     changeMtd -> getThisFieldCode(changeMtd, parentInnerNodeFieldDef, polymorphicTypeIdFieldDef)
             );
 
-            addNodeChangeBridgeMethod(classDef, polymorphicExtensionClassInfo.changeClassName, changeMtd0);
+            // Only first element requires a bridge. Please refer to "addNodeChangeMethod" for explanation.
+            addNodeChangeBridgeMethod(classDef, polymorphicExtensionClassInfo.changeClassName, changeMethods.get(0));
         }
 
         ParameterizedType returnType = typeFromJavaClassName(schemaClassInfo.changeClassName);
diff --git a/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/ConfigurationChangerTest.java b/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/ConfigurationChangerTest.java
index a65ad892d9..59b1be5292 100644
--- a/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/ConfigurationChangerTest.java
+++ b/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/ConfigurationChangerTest.java
@@ -142,6 +142,26 @@ public class ConfigurationChangerTest {
         assertEquals("1", newRoot.elements().get("a").strCfg());
     }
 
+    /**
+     * Test simple change of configuration.
+     */
+    @Test
+    public void testSimpleConfigurationChangeWithoutExtraLambdas() throws Exception {
+        ConfigurationChanger changer = createChanger(KEY);
+        changer.start();
+
+        changer.change(source(KEY, (FirstChange parent) -> {
+            parent.changeChild().changeIntCfg(1).changeStrCfg("1");
+            parent.changeElements().create("a", element -> element.changeStrCfg("1"));
+        })).get(1, SECONDS);
+
+        FirstView newRoot = (FirstView) changer.getRootNode(KEY);
+
+        assertEquals(1, newRoot.child().intCfg());
+        assertEquals("1", newRoot.child().strCfg());
+        assertEquals("1", newRoot.elements().get("a").strCfg());
+    }
+
     /**
      * Test subsequent change of configuration via different changers.
      */
diff --git a/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/ConfigurationRegistryTest.java b/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/ConfigurationRegistryTest.java
index e88356d1f7..bb6ca661ff 100644
--- a/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/ConfigurationRegistryTest.java
+++ b/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/ConfigurationRegistryTest.java
@@ -19,12 +19,18 @@ package org.apache.ignite.internal.configuration;
 
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.ignite.configuration.annotation.ConfigurationType.LOCAL;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.is;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertSame;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.CompletableFuture;
 import java.util.function.Consumer;
 import org.apache.ignite.configuration.annotation.Config;
 import org.apache.ignite.configuration.annotation.ConfigValue;
@@ -156,6 +162,49 @@ public class ConfigurationRegistryTest {
         }
     }
 
+    @Test
+    void testChangeSuperRoot() throws Exception {
+        TestConfigurationStorage storage = new TestConfigurationStorage(LOCAL);
+
+        var registry = new ConfigurationRegistry(
+                List.of(FirstRootConfiguration.KEY, SecondRootConfiguration.KEY),
+                Set.of(),
+                storage,
+                List.of(),
+                List.of()
+        );
+
+        registry.start();
+
+        try {
+            FirstRootConfiguration firstConfiguration = registry.getConfiguration(FirstRootConfiguration.KEY);
+            SecondRootConfiguration secondConfiguration = registry.getConfiguration(SecondRootConfiguration.KEY);
+
+            CompletableFuture<Void> changeFuture = registry.change(superRootChange -> {
+                assertNotNull(superRootChange);
+
+                // Check that originally we have the same value for the root.
+                assertSame(firstConfiguration.value(), superRootChange.viewRoot(FirstRootConfiguration.KEY));
+
+                FirstRootChange firstRootChange = superRootChange.changeRoot(FirstRootConfiguration.KEY);
+
+                // Check that the value of the root has changed.
+                assertNotSame(firstConfiguration.value(), firstRootChange);
+                assertSame(firstRootChange, superRootChange.viewRoot(FirstRootConfiguration.KEY));
+
+                firstRootChange.changeStr("foo");
+                superRootChange.changeRoot(SecondRootConfiguration.KEY).changeStr("bar");
+            });
+
+            assertThat(changeFuture, willCompleteSuccessfully());
+
+            assertEquals("foo", firstConfiguration.str().value());
+            assertEquals("bar", secondConfiguration.str().value());
+        } finally {
+            registry.stop();
+        }
+    }
+
     private Consumer<FourthPolymorphicChange> toFirst0Polymorphic(int v) {
         return c -> c.convert(Fourth0PolymorphicChange.class).changeIntVal(v).changeStrVal(Integer.toString(v));
     }
diff --git a/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/asm/ConfigurationAsmGeneratorTest.java b/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/asm/ConfigurationAsmGeneratorTest.java
index 53d59b324c..36efcd9a66 100644
--- a/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/asm/ConfigurationAsmGeneratorTest.java
+++ b/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/asm/ConfigurationAsmGeneratorTest.java
@@ -203,6 +203,12 @@ public class ConfigurationAsmGeneratorTest {
             ((ExtendedTestChange) c).changeStr3("str3").changeStr2("str2");
             ((ExtendedSecondTestChange) c).changeI1(200).changeStr2("str2");
         }).get(1, SECONDS);
+
+        rootConfig.change(r -> {
+            TestChange subCfg = r.changeSubCfg().changeStr2("str3");
+
+            assertTrue(subCfg instanceof ExtendedTestChange);
+        }).get(1, SECONDS);
     }
 
     @Test
@@ -305,6 +311,13 @@ public class ConfigurationAsmGeneratorTest {
         assertEquals("strVal3", firstCfg.strVal().value());
         assertEquals(3, firstCfg.intVal().value());
 
+        rootConfig.change(c -> ((FirstPolymorphicInstanceTestChange) c.changePolymorphicSubCfg()).changeIntVal(4).changeStrVal("strVal4"))
+                .get(1, SECONDS);
+
+        assertEquals("first", firstCfg.typeId().value());
+        assertEquals("strVal4", firstCfg.strVal().value());
+        assertEquals(4, firstCfg.intVal().value());
+
         // Check convert.
 
         rootConfig.polymorphicSubCfg()
@@ -335,13 +348,13 @@ public class ConfigurationAsmGeneratorTest {
 
         SecondPolymorphicInstanceTestConfiguration secondCfg = (SecondPolymorphicInstanceTestConfiguration) rootConfig.polymorphicSubCfg();
         assertEquals("second", secondCfg.typeId().value());
-        assertEquals("strVal3", secondCfg.strVal().value());
+        assertEquals("strVal4", secondCfg.strVal().value());
         assertEquals(0, secondCfg.intVal().value());
         assertEquals(0L, secondCfg.longVal().value());
 
         SecondPolymorphicInstanceTestView secondView = (SecondPolymorphicInstanceTestView) secondCfg.value();
         assertEquals("second", secondView.typeId());
-        assertEquals("strVal3", secondView.strVal());
+        assertEquals("strVal4", secondView.strVal());
         assertEquals(0, secondView.intVal());
         assertEquals(0L, secondView.longVal());
 
@@ -349,7 +362,7 @@ public class ConfigurationAsmGeneratorTest {
 
         firstCfg = (FirstPolymorphicInstanceTestConfiguration) rootConfig.polymorphicSubCfg();
         assertEquals("first", firstCfg.typeId().value());
-        assertEquals("strVal3", firstCfg.strVal().value());
+        assertEquals("strVal4", firstCfg.strVal().value());
         assertEquals(0, firstCfg.intVal().value());
     }
 
@@ -644,6 +657,16 @@ public class ConfigurationAsmGeneratorTest {
 
         assertEquals(2, rootFromAbstractConfig.configFromAbstract().intVal().value());
         assertTrue(rootFromAbstractConfig.configFromAbstract().booleanVal().value());
+
+        // Check "short" version of change method.
+        rootFromAbstractConfig.change(ch0 -> ch0
+                .changeConfigFromAbstract().changeBooleanVal(true).changeIntVal(3)
+        ).get(1, SECONDS);
+
+        RootFromAbstractView fromAbstractView2 = rootFromAbstractConfig.value();
+
+        assertEquals(3, fromAbstractView2.configFromAbstract().intVal());
+        assertTrue(fromAbstractView2.configFromAbstract().booleanVal());
     }
 
     @Test
diff --git a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java
index bd830de458..5d82f6cfae 100644
--- a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java
+++ b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java
@@ -368,7 +368,6 @@ public class DistributionZoneManager implements IgniteComponent {
                     throw new DistributionZoneNotFoundException(name);
                 }
 
-                //TODO: IGNITE-18516 Access to other configuration must be thread safe.
                 NamedConfigurationTree<TableConfiguration, TableView, TableChange> tables = tablesConfiguration.tables();
 
                 for (int i = 0; i < tables.value().size(); i++) {
diff --git a/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationCatchUpTest.java b/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationCatchUpTest.java
index 81cf5d61c7..c7b17fb09a 100644
--- a/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationCatchUpTest.java
+++ b/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationCatchUpTest.java
@@ -190,18 +190,10 @@ public class DistributedConfigurationCatchUpTest {
 
             // On any invocation - trigger storage listener.
             when(mock.invoke(any(), anyCollection(), any()))
-                    .then(invocation -> {
-                        triggerStorageListener();
-
-                        return CompletableFuture.completedFuture(true);
-                    });
+                    .then(invocation -> triggerStorageListener());
 
             when(mock.invoke(any(), any(Operation.class), any()))
-                    .then(invocation -> {
-                        triggerStorageListener();
-
-                        return CompletableFuture.completedFuture(true);
-                    });
+                    .then(invocation -> triggerStorageListener());
 
             // This captures the listener.
             when(mock.registerPrefixWatch(any(), any())).then(invocation -> {
@@ -214,9 +206,13 @@ public class DistributedConfigurationCatchUpTest {
         /**
          * Triggers MetaStorage listener incrementing master key revision.
          */
-        private void triggerStorageListener() {
-            EntryEvent entryEvent = new EntryEvent(null, new EntryImpl(MASTER_KEY.bytes(), null, ++masterKeyRevision, -1));
-            lsnr.onUpdate(new WatchEvent(entryEvent));
+        private CompletableFuture<Boolean> triggerStorageListener() {
+            return CompletableFuture.supplyAsync(() -> {
+                EntryEvent entryEvent = new EntryEvent(null, new EntryImpl(MASTER_KEY.bytes(), null, ++masterKeyRevision, -1));
+                lsnr.onUpdate(new WatchEvent(entryEvent));
+
+                return true;
+            });
         }
 
         private MetaStorageManager metaStorageManager() {
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandler.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandler.java
index c79b5616e6..dc33133f40 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandler.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandler.java
@@ -210,7 +210,6 @@ public class DdlCommandHandler {
             }
 
             if (cmd.zone() != null) {
-                //TODO: IGNITE-18516 Access to other configuration must be thread safe.
                 tableChange.changeZoneId(distributionZoneManager.getZoneId(cmd.zone()));
             } else {
                 tableChange.changeZoneId(DEFAULT_ZONE_ID);