You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "ibessonov (via GitHub)" <gi...@apache.org> on 2023/04/14 08:16:47 UTC

[GitHub] [ignite-3] ibessonov commented on a diff in pull request #1929: IGNITE-19152 Use schema information in LocalFileConfigurationStorage

ibessonov commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1166420335


##########
modules/configuration/src/main/java/org/apache/ignite/internal/configuration/hocon/HoconConverter.java:
##########
@@ -47,6 +53,26 @@ public static ConfigValue represent(
         return ConfigImpl.fromAnyRef(res, null);
     }
 
+    /**
+     * TBD.
+     */
+    public static <T> T represent(SuperRoot superRoot, ConfigurationVisitor<T> visitor) throws IllegalArgumentException {
+        Object node;
+        try {
+            node = ConfigurationUtil.find(List.of(), superRoot, false);

Review Comment:
   Finding by an empty path gets you predictable result. It's the parameter that you have passed.
   This entire method could be simplified to `return superRoot.accept(null, visitor);`.
   I propose removing it and calling `accept` directly where you need it.



##########
modules/configuration/src/test/java/org/apache/ignite/internal/configuration/hocon/HoconConverterTest.java:
##########
@@ -352,7 +352,7 @@ public void testHoconArraysSerialization() throws Exception {
     private static String asHoconStr(List<String> basePath, String... path) {
         List<String> fullPath = Stream.concat(basePath.stream(), Arrays.stream(path)).collect(Collectors.toList());
 
-        ConfigValue hoconCfg = HoconConverter.represent(registry, fullPath);
+        ConfigValue hoconCfg = HoconConverter.represent(registry.superRoot(), fullPath);

Review Comment:
   I propose calling the old method with "registry" parameter, and remove the getter for "superRoot". It's an internal object and it should remain internal



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -40,114 +53,187 @@
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.stream.Collectors;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.ignite.configuration.RootKey;
 import org.apache.ignite.configuration.annotation.ConfigurationType;
 import org.apache.ignite.internal.configuration.NodeConfigCreateException;
 import org.apache.ignite.internal.configuration.NodeConfigWriteException;
+import org.apache.ignite.internal.configuration.RootInnerNode;
+import org.apache.ignite.internal.configuration.SuperRoot;
+import org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator;
+import org.apache.ignite.internal.configuration.hocon.HoconConverter;
+import org.apache.ignite.internal.configuration.tree.ConverterToMapVisitor;
+import org.apache.ignite.internal.configuration.tree.InnerNode;
 import org.apache.ignite.internal.future.InFlightFutures;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.thread.NamedThreadFactory;
 import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
 
 /**
  * Implementation of {@link ConfigurationStorage} based on local file configuration storage.
  */
 public class LocalFileConfigurationStorage implements ConfigurationStorage {
     private static final IgniteLogger LOG = Loggers.forClass(LocalFileConfigurationStorage.class);
 
-    /**
-     * Path to config file.
-     */
+    /** Path to config file. */
     private final Path configPath;
 
-    /**
-     * Path to temporary configuration storage.
-     */
+    /** Path to temporary configuration storage. */
     private final Path tempConfigPath;
 
+    /** R/W lock to guard the latest configuration and config file. */
     private final ReadWriteLock lock = new ReentrantReadWriteLock();
 
-    /**
-     * Latest state of last applied configuration.
-     */
+    /** Latest state of last applied configuration. */
     private final Map<String, Serializable> latest = new ConcurrentHashMap<>();
 
-    /**
-     *  Configuration changes listener.
-     *  */
+    /** Configuration nodes generator. */
+    private final ConfigurationAsmGenerator cgen = new ConfigurationAsmGenerator();

Review Comment:
   This is interesting. It would be nice to use the same object that we have in `ConfigurationRegistry`. Can we drill that hole?



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -40,114 +53,187 @@
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.stream.Collectors;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.ignite.configuration.RootKey;
 import org.apache.ignite.configuration.annotation.ConfigurationType;
 import org.apache.ignite.internal.configuration.NodeConfigCreateException;
 import org.apache.ignite.internal.configuration.NodeConfigWriteException;
+import org.apache.ignite.internal.configuration.RootInnerNode;
+import org.apache.ignite.internal.configuration.SuperRoot;
+import org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator;
+import org.apache.ignite.internal.configuration.hocon.HoconConverter;
+import org.apache.ignite.internal.configuration.tree.ConverterToMapVisitor;
+import org.apache.ignite.internal.configuration.tree.InnerNode;
 import org.apache.ignite.internal.future.InFlightFutures;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.thread.NamedThreadFactory;
 import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
 
 /**
  * Implementation of {@link ConfigurationStorage} based on local file configuration storage.
  */
 public class LocalFileConfigurationStorage implements ConfigurationStorage {
     private static final IgniteLogger LOG = Loggers.forClass(LocalFileConfigurationStorage.class);
 
-    /**
-     * Path to config file.
-     */
+    /** Path to config file. */
     private final Path configPath;
 
-    /**
-     * Path to temporary configuration storage.
-     */
+    /** Path to temporary configuration storage. */
     private final Path tempConfigPath;
 
+    /** R/W lock to guard the latest configuration and config file. */
     private final ReadWriteLock lock = new ReentrantReadWriteLock();
 
-    /**
-     * Latest state of last applied configuration.
-     */
+    /** Latest state of last applied configuration. */
     private final Map<String, Serializable> latest = new ConcurrentHashMap<>();
 
-    /**
-     *  Configuration changes listener.
-     *  */
+    /** Configuration nodes generator. */
+    private final ConfigurationAsmGenerator cgen = new ConfigurationAsmGenerator();
+
+    /** Map of root keys that are needed to generate configuration tree. */
+    private final Map<String, RootKey<?, ?>> rootKeys;
+
+    /** Configuration changes listener. */
     private final AtomicReference<ConfigurationStorageListener> lsnrRef = new AtomicReference<>();
 
-    private final ExecutorService threadPool = Executors.newFixedThreadPool(2, new NamedThreadFactory("loc-cfg-file", LOG));
+    /** Thread pool for configuration updates notifications. */
+    private final ExecutorService threadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("loc-cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates notifications. */
+    private final ExecutorService workerThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file-worker", LOG)
+    );
 
+    /** Tracks all running futures. */
     private final InFlightFutures futureTracker = new InFlightFutures();
 
+    /** Last revision for configuration. */
     private long lastRevision = 0L;
 
+    /**
+     * Constructor without configuration extensions.
+     *
+     * @param configPath Path to node bootstrap configuration file.
+     * @param rootKeys Configuration root keys.
+     */
+    public LocalFileConfigurationStorage(Path configPath, Collection<RootKey<?, ?>> rootKeys) {
+        this(configPath, rootKeys, Collections.emptyList(), Collections.emptyList());
+    }
+
     /**
      * Constructor.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param rootKeys Configuration root keys.
+     * @param internalSchemaExtensions Internal schema extensions.
+     * @param polymorphicSchemaExtensions Polymorphic schema extensions.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, Collection<RootKey<?, ?>> rootKeys,
+            Collection<Class<?>> internalSchemaExtensions, Collection<Class<?>> polymorphicSchemaExtensions) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.rootKeys = rootKeys.stream().collect(toMap(RootKey::key, identity()));
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
+        Map<Class<?>, Set<Class<?>>> internalExtensions = internalSchemaExtensions(internalSchemaExtensions);
+        Map<Class<?>, Set<Class<?>>> polymorphicExtensions = polymorphicSchemaExtensions(polymorphicSchemaExtensions);
+
+        rootKeys.forEach(key -> cgen.compileRootSchema(key.schemaClass(), internalExtensions, polymorphicExtensions));
+
         checkAndRestoreConfigFile();
     }
 
+    private static void setValues(SuperRoot target, Config source) {
+        HoconConverter.hoconSource(source.root()).descend(target);
+    }
+
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return writeLockAsync(() -> {
+            SuperRoot superRoot = createSuperRoot();
+            SuperRoot copiedSuperRoot = superRoot.copy();
+
+            Config hocon = readHoconFromFile();
+            setValues(copiedSuperRoot, hocon);
+
+            addDefaults(copiedSuperRoot);

Review Comment:
   Right now I would avoid doing this



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -40,114 +53,187 @@
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.stream.Collectors;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.ignite.configuration.RootKey;
 import org.apache.ignite.configuration.annotation.ConfigurationType;
 import org.apache.ignite.internal.configuration.NodeConfigCreateException;
 import org.apache.ignite.internal.configuration.NodeConfigWriteException;
+import org.apache.ignite.internal.configuration.RootInnerNode;
+import org.apache.ignite.internal.configuration.SuperRoot;
+import org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator;
+import org.apache.ignite.internal.configuration.hocon.HoconConverter;
+import org.apache.ignite.internal.configuration.tree.ConverterToMapVisitor;
+import org.apache.ignite.internal.configuration.tree.InnerNode;
 import org.apache.ignite.internal.future.InFlightFutures;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.thread.NamedThreadFactory;
 import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
 
 /**
  * Implementation of {@link ConfigurationStorage} based on local file configuration storage.
  */
 public class LocalFileConfigurationStorage implements ConfigurationStorage {
     private static final IgniteLogger LOG = Loggers.forClass(LocalFileConfigurationStorage.class);
 
-    /**
-     * Path to config file.
-     */
+    /** Path to config file. */
     private final Path configPath;
 
-    /**
-     * Path to temporary configuration storage.
-     */
+    /** Path to temporary configuration storage. */
     private final Path tempConfigPath;
 
+    /** R/W lock to guard the latest configuration and config file. */
     private final ReadWriteLock lock = new ReentrantReadWriteLock();
 
-    /**
-     * Latest state of last applied configuration.
-     */
+    /** Latest state of last applied configuration. */
     private final Map<String, Serializable> latest = new ConcurrentHashMap<>();
 
-    /**
-     *  Configuration changes listener.
-     *  */
+    /** Configuration nodes generator. */
+    private final ConfigurationAsmGenerator cgen = new ConfigurationAsmGenerator();
+
+    /** Map of root keys that are needed to generate configuration tree. */
+    private final Map<String, RootKey<?, ?>> rootKeys;
+
+    /** Configuration changes listener. */
     private final AtomicReference<ConfigurationStorageListener> lsnrRef = new AtomicReference<>();
 
-    private final ExecutorService threadPool = Executors.newFixedThreadPool(2, new NamedThreadFactory("loc-cfg-file", LOG));
+    /** Thread pool for configuration updates notifications. */
+    private final ExecutorService threadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("loc-cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates notifications. */
+    private final ExecutorService workerThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file-worker", LOG)
+    );
 
+    /** Tracks all running futures. */
     private final InFlightFutures futureTracker = new InFlightFutures();
 
+    /** Last revision for configuration. */
     private long lastRevision = 0L;
 
+    /**
+     * Constructor without configuration extensions.
+     *
+     * @param configPath Path to node bootstrap configuration file.
+     * @param rootKeys Configuration root keys.
+     */
+    public LocalFileConfigurationStorage(Path configPath, Collection<RootKey<?, ?>> rootKeys) {
+        this(configPath, rootKeys, Collections.emptyList(), Collections.emptyList());
+    }
+
     /**
      * Constructor.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param rootKeys Configuration root keys.
+     * @param internalSchemaExtensions Internal schema extensions.
+     * @param polymorphicSchemaExtensions Polymorphic schema extensions.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, Collection<RootKey<?, ?>> rootKeys,
+            Collection<Class<?>> internalSchemaExtensions, Collection<Class<?>> polymorphicSchemaExtensions) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.rootKeys = rootKeys.stream().collect(toMap(RootKey::key, identity()));
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
+        Map<Class<?>, Set<Class<?>>> internalExtensions = internalSchemaExtensions(internalSchemaExtensions);
+        Map<Class<?>, Set<Class<?>>> polymorphicExtensions = polymorphicSchemaExtensions(polymorphicSchemaExtensions);
+
+        rootKeys.forEach(key -> cgen.compileRootSchema(key.schemaClass(), internalExtensions, polymorphicExtensions));
+
         checkAndRestoreConfigFile();
     }
 
+    private static void setValues(SuperRoot target, Config source) {
+        HoconConverter.hoconSource(source.root()).descend(target);
+    }
+
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return writeLockAsync(() -> {
+            SuperRoot superRoot = createSuperRoot();
+            SuperRoot copiedSuperRoot = superRoot.copy();
+
+            Config hocon = readHoconFromFile();
+            setValues(copiedSuperRoot, hocon);
+
+            addDefaults(copiedSuperRoot);
+
+            Map<String, Serializable> flattenedUpdatesMap = createFlattenedUpdatesMap(superRoot, copiedSuperRoot);
+
+            latest.putAll(flattenedUpdatesMap);
+
+            return new Data(flattenedUpdatesMap, lastRevision);
+        });
+    }
+

Review Comment:
   Too many empty lines



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -175,47 +261,66 @@ public CompletableFuture<Void> writeConfigurationRevision(long prevRevision, lon
 
     @Override
     public void close() {
+        IgniteUtils.shutdownAndAwaitTermination(workerThreadPool, 10, TimeUnit.SECONDS);
         IgniteUtils.shutdownAndAwaitTermination(threadPool, 10, TimeUnit.SECONDS);
 
         futureTracker.cancelInFlightFutures();
     }
 
-    private void saveValues(Map<String, ? extends Serializable> values) {
+    private void saveConfigFile() {
         try {
-            Files.write(tempConfigPath, renderHoconString(values).getBytes(StandardCharsets.UTF_8),
-                    StandardOpenOption.SYNC, StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
-            Files.move(tempConfigPath, configPath, StandardCopyOption.ATOMIC_MOVE, StandardCopyOption.REPLACE_EXISTING);
+            Files.write(
+                    tempConfigPath,
+                    renderHoconString().getBytes(StandardCharsets.UTF_8),
+                    StandardOpenOption.SYNC, StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING
+            );
+
+            Files.move(
+                    tempConfigPath,
+                    configPath,
+                    StandardCopyOption.ATOMIC_MOVE, StandardCopyOption.REPLACE_EXISTING
+            );
         } catch (IOException e) {
             throw new NodeConfigWriteException(
-                    "Failed to write values " + values + " to config file.", e);
+                    "Failed to write values to config file.", e);
         }
     }
 
     /**
      * Convert provided map to Hocon String representation.
      *
-     * @param values Values of configuration.
      * @return Configuration file string representation in HOCON format.
      */
-    private String renderHoconString(Map<String, ? extends Serializable> values) {
-        Map<String, Object> map = values.entrySet().stream().collect(Collectors.toMap(Entry::getKey, stringEntry -> {
-            Serializable value = stringEntry.getValue();
-            if (value.getClass().isArray()) {
-                return Arrays.asList((Object[]) value);
-            }
-            return value;
-        }));
-        Config other = ConfigFactory.parseMap(map);
-        Config newConfig = other.withFallback(parseConfigOptions()).resolve();
+    private String renderHoconString() {
+        // Super root that'll be filled from the storage data.
+        SuperRoot rootNode = new SuperRoot(rootCreator());
+
+        fillFromPrefixMap(rootNode, toPrefixMap(latest));
+
+        addDefaults(rootNode);

Review Comment:
   Why did you add defaults here? I guess it's a copied code, right? And you didn't tests whether you write default values or not.



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -175,47 +261,66 @@ public CompletableFuture<Void> writeConfigurationRevision(long prevRevision, lon
 
     @Override
     public void close() {
+        IgniteUtils.shutdownAndAwaitTermination(workerThreadPool, 10, TimeUnit.SECONDS);
         IgniteUtils.shutdownAndAwaitTermination(threadPool, 10, TimeUnit.SECONDS);
 
         futureTracker.cancelInFlightFutures();
     }
 
-    private void saveValues(Map<String, ? extends Serializable> values) {
+    private void saveConfigFile() {
         try {
-            Files.write(tempConfigPath, renderHoconString(values).getBytes(StandardCharsets.UTF_8),
-                    StandardOpenOption.SYNC, StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
-            Files.move(tempConfigPath, configPath, StandardCopyOption.ATOMIC_MOVE, StandardCopyOption.REPLACE_EXISTING);
+            Files.write(
+                    tempConfigPath,
+                    renderHoconString().getBytes(StandardCharsets.UTF_8),
+                    StandardOpenOption.SYNC, StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING
+            );
+
+            Files.move(
+                    tempConfigPath,
+                    configPath,
+                    StandardCopyOption.ATOMIC_MOVE, StandardCopyOption.REPLACE_EXISTING
+            );
         } catch (IOException e) {
             throw new NodeConfigWriteException(
-                    "Failed to write values " + values + " to config file.", e);
+                    "Failed to write values to config file.", e);
         }
     }
 
     /**
      * Convert provided map to Hocon String representation.
      *
-     * @param values Values of configuration.
      * @return Configuration file string representation in HOCON format.
      */
-    private String renderHoconString(Map<String, ? extends Serializable> values) {
-        Map<String, Object> map = values.entrySet().stream().collect(Collectors.toMap(Entry::getKey, stringEntry -> {
-            Serializable value = stringEntry.getValue();
-            if (value.getClass().isArray()) {
-                return Arrays.asList((Object[]) value);
-            }
-            return value;
-        }));
-        Config other = ConfigFactory.parseMap(map);
-        Config newConfig = other.withFallback(parseConfigOptions()).resolve();
+    private String renderHoconString() {
+        // Super root that'll be filled from the storage data.
+        SuperRoot rootNode = new SuperRoot(rootCreator());
+
+        fillFromPrefixMap(rootNode, toPrefixMap(latest));
+
+        addDefaults(rootNode);
+
+        ConfigValue conf = ConfigImpl.fromAnyRef(HoconConverter.represent(rootNode, new ConverterToMapVisitor(false)), null);
+
+        Config newConfig = ((ConfigObject) conf).toConfig().resolve();
         return newConfig.isEmpty()
                 ? ""
                 : newConfig.root().render(ConfigRenderOptions.concise().setFormatted(true).setJson(false));
     }
 
+    private Function<String, RootInnerNode> rootCreator() {
+        return key -> {
+            RootKey<?, ?> rootKey = rootKeys.get(key);
+
+            return rootKey == null ? null : new RootInnerNode(rootKey, createRootNode(rootKey));
+        };
+    }
+
+    private InnerNode createRootNode(RootKey<?, ?> rootKey) {
+        return cgen.instantiateNode(rootKey.schemaClass());
+    }
+
     private Config parseConfigOptions() {
-        return ConfigFactory.parseFile(
-                configPath.toFile(),
-                ConfigParseOptions.defaults().setAllowMissing(false));
+        return readHoconFromFile();

Review Comment:
   Please inline this method



##########
modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorageTest.java:
##########
@@ -17,75 +17,401 @@
 
 package org.apache.ignite.internal.configuration.storage;
 
-import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.aMapWithSize;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.emptyString;
+import static org.hamcrest.Matchers.equalToCompressingWhiteSpace;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.hasValue;
 import static org.hamcrest.Matchers.is;
 
 import java.io.IOException;
+import java.io.Serializable;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.ConfigurationRoot;
+import org.apache.ignite.configuration.annotation.NamedConfigValue;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.internal.configuration.TestConfigurationChanger;
+import org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
-import org.junit.jupiter.api.Disabled;
+import org.hamcrest.Matchers;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.DisplayName;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
-/**
- * Tests for the {@link LocalFileConfigurationStorage}.
- */
+/** Test for local file configurations storage. */
 @ExtendWith(WorkDirectoryExtension.class)
-@Disabled("https://issues.apache.org/jira/browse/IGNITE-19152")
-public class LocalFileConfigurationStorageTest extends ConfigurationStorageTest {
+public class LocalFileConfigurationStorageTest {
 
     private static final String CONFIG_NAME = "ignite-config.conf";
 
+    private static ConfigurationAsmGenerator cgen;
+
     @WorkDirectory
     private Path tmpDir;
 
-    @Override
-    public ConfigurationStorage getStorage() {
-        return new LocalFileConfigurationStorage(getConfigFile());
+    /** Test configuration storage. */
+    private LocalFileConfigurationStorage storage;
+
+    /** Test configuration changer. */
+    private TestConfigurationChanger changer;
+
+    /** Instantiates {@link #cgen}. */
+    @BeforeAll
+    public static void beforeAll() {
+        cgen = new ConfigurationAsmGenerator();
+    }
+
+    /** Nullifies {@link #cgen} to prevent memory leak from having runtime ClassLoader accessible from GC root. */
+    @AfterAll
+    public static void afterAll() {
+        cgen = null;
+    }
+
+    private Path getConfigFile() {
+        return tmpDir.resolve(CONFIG_NAME);
+    }
+
+    @BeforeEach
+    void before() {
+        storage = new LocalFileConfigurationStorage(getConfigFile(), List.of(TopConfiguration.KEY));
+
+        changer = new TestConfigurationChanger(
+                cgen,
+                List.of(TopConfiguration.KEY),
+                Set.of(),
+                storage,
+                List.of(),
+                List.of()
+        );
+
+        changer.start();
+    }
+
+    @AfterEach
+    void after() {
+        changer.stop();
     }
 
     @Test
-    void testHocon() throws IOException {
-        // All of this is needed because write expects serializable values and only concrete classes are serializable
-        HashMap<String, ArrayList<String>> map = new HashMap<>(Map.of("list", new ArrayList<>(List.of("val1", "val2"))));
-        var data = Map.of("foo1", "bar1", "foo2", "bar2", "map", map);
-
-        assertThat(storage.write(data, 0), willBe(true));
-
-        String contents = Files.readString(getConfigFile());
-
-        // \n instead of System.lineSeparator because Config library writes \n only
-        assertThat(contents, is("foo1=bar1\n"
-                + "foo2=bar2\n"
-                + "map {\n"
-                + "    list=[\n"
-                + "        val1,\n"
-                + "        val2\n"
-                + "    ]\n"
-                + "}\n"));
+    @DisplayName("Default values are not added enriched on read when the config file is empty")
+    void empty() throws IOException {
+        // Given
+        assertThat(configFileContent(), emptyString());
+
+        // When
+        var storageValues = readAllLatest();
+
+        // Then storage data only contains top level defaults
+        assertThat(storageValues.entrySet(), hasSize(1));

Review Comment:
   Can be replaced with `assertThat(storageValues, is(aMapWithSize(1)))`, it would look better.
   By the way, what's a `top level defaults`? Configuration is empty, and we shouldn't store defaults in current implementation (I guess).
   Also, why don't you explicitly assert the content of the map in this test? It's easy to do



##########
modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorageTest.java:
##########
@@ -17,75 +17,401 @@
 
 package org.apache.ignite.internal.configuration.storage;
 
-import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.aMapWithSize;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.emptyString;
+import static org.hamcrest.Matchers.equalToCompressingWhiteSpace;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.hasValue;
 import static org.hamcrest.Matchers.is;
 
 import java.io.IOException;
+import java.io.Serializable;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.ConfigurationRoot;
+import org.apache.ignite.configuration.annotation.NamedConfigValue;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.internal.configuration.TestConfigurationChanger;
+import org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
-import org.junit.jupiter.api.Disabled;
+import org.hamcrest.Matchers;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.DisplayName;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
-/**
- * Tests for the {@link LocalFileConfigurationStorage}.
- */
+/** Test for local file configurations storage. */
 @ExtendWith(WorkDirectoryExtension.class)
-@Disabled("https://issues.apache.org/jira/browse/IGNITE-19152")
-public class LocalFileConfigurationStorageTest extends ConfigurationStorageTest {
+public class LocalFileConfigurationStorageTest {
 
     private static final String CONFIG_NAME = "ignite-config.conf";
 
+    private static ConfigurationAsmGenerator cgen;
+
     @WorkDirectory
     private Path tmpDir;
 
-    @Override
-    public ConfigurationStorage getStorage() {
-        return new LocalFileConfigurationStorage(getConfigFile());
+    /** Test configuration storage. */
+    private LocalFileConfigurationStorage storage;
+
+    /** Test configuration changer. */
+    private TestConfigurationChanger changer;
+
+    /** Instantiates {@link #cgen}. */
+    @BeforeAll
+    public static void beforeAll() {
+        cgen = new ConfigurationAsmGenerator();
+    }
+
+    /** Nullifies {@link #cgen} to prevent memory leak from having runtime ClassLoader accessible from GC root. */
+    @AfterAll
+    public static void afterAll() {
+        cgen = null;
+    }
+
+    private Path getConfigFile() {
+        return tmpDir.resolve(CONFIG_NAME);
+    }
+
+    @BeforeEach
+    void before() {
+        storage = new LocalFileConfigurationStorage(getConfigFile(), List.of(TopConfiguration.KEY));
+
+        changer = new TestConfigurationChanger(
+                cgen,
+                List.of(TopConfiguration.KEY),
+                Set.of(),
+                storage,
+                List.of(),
+                List.of()
+        );
+
+        changer.start();
+    }
+
+    @AfterEach
+    void after() {
+        changer.stop();
     }
 
     @Test
-    void testHocon() throws IOException {
-        // All of this is needed because write expects serializable values and only concrete classes are serializable
-        HashMap<String, ArrayList<String>> map = new HashMap<>(Map.of("list", new ArrayList<>(List.of("val1", "val2"))));
-        var data = Map.of("foo1", "bar1", "foo2", "bar2", "map", map);
-
-        assertThat(storage.write(data, 0), willBe(true));
-
-        String contents = Files.readString(getConfigFile());
-
-        // \n instead of System.lineSeparator because Config library writes \n only
-        assertThat(contents, is("foo1=bar1\n"
-                + "foo2=bar2\n"
-                + "map {\n"
-                + "    list=[\n"
-                + "        val1,\n"
-                + "        val2\n"
-                + "    ]\n"
-                + "}\n"));
+    @DisplayName("Default values are not added enriched on read when the config file is empty")

Review Comment:
   "not added enriched" - not added or not enriched?
   Why is it a display name, is that convenient? You better give the method a proper name and move this description into a javadoc, because that's how most of our tests are written



##########
modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorageTest.java:
##########
@@ -17,75 +17,401 @@
 
 package org.apache.ignite.internal.configuration.storage;
 
-import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.aMapWithSize;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.emptyString;
+import static org.hamcrest.Matchers.equalToCompressingWhiteSpace;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.hasValue;
 import static org.hamcrest.Matchers.is;
 
 import java.io.IOException;
+import java.io.Serializable;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.ConfigurationRoot;
+import org.apache.ignite.configuration.annotation.NamedConfigValue;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.internal.configuration.TestConfigurationChanger;
+import org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
-import org.junit.jupiter.api.Disabled;
+import org.hamcrest.Matchers;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.DisplayName;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
-/**
- * Tests for the {@link LocalFileConfigurationStorage}.
- */
+/** Test for local file configurations storage. */
 @ExtendWith(WorkDirectoryExtension.class)
-@Disabled("https://issues.apache.org/jira/browse/IGNITE-19152")
-public class LocalFileConfigurationStorageTest extends ConfigurationStorageTest {
+public class LocalFileConfigurationStorageTest {
 
     private static final String CONFIG_NAME = "ignite-config.conf";
 
+    private static ConfigurationAsmGenerator cgen;
+
     @WorkDirectory
     private Path tmpDir;
 
-    @Override
-    public ConfigurationStorage getStorage() {
-        return new LocalFileConfigurationStorage(getConfigFile());
+    /** Test configuration storage. */
+    private LocalFileConfigurationStorage storage;
+
+    /** Test configuration changer. */
+    private TestConfigurationChanger changer;
+
+    /** Instantiates {@link #cgen}. */
+    @BeforeAll
+    public static void beforeAll() {
+        cgen = new ConfigurationAsmGenerator();
+    }
+
+    /** Nullifies {@link #cgen} to prevent memory leak from having runtime ClassLoader accessible from GC root. */
+    @AfterAll
+    public static void afterAll() {
+        cgen = null;
+    }
+
+    private Path getConfigFile() {
+        return tmpDir.resolve(CONFIG_NAME);
+    }
+
+    @BeforeEach
+    void before() {
+        storage = new LocalFileConfigurationStorage(getConfigFile(), List.of(TopConfiguration.KEY));
+
+        changer = new TestConfigurationChanger(
+                cgen,
+                List.of(TopConfiguration.KEY),
+                Set.of(),
+                storage,
+                List.of(),
+                List.of()
+        );
+
+        changer.start();
+    }
+
+    @AfterEach
+    void after() {
+        changer.stop();
     }
 
     @Test
-    void testHocon() throws IOException {
-        // All of this is needed because write expects serializable values and only concrete classes are serializable
-        HashMap<String, ArrayList<String>> map = new HashMap<>(Map.of("list", new ArrayList<>(List.of("val1", "val2"))));
-        var data = Map.of("foo1", "bar1", "foo2", "bar2", "map", map);
-
-        assertThat(storage.write(data, 0), willBe(true));
-
-        String contents = Files.readString(getConfigFile());
-
-        // \n instead of System.lineSeparator because Config library writes \n only
-        assertThat(contents, is("foo1=bar1\n"
-                + "foo2=bar2\n"
-                + "map {\n"
-                + "    list=[\n"
-                + "        val1,\n"
-                + "        val2\n"
-                + "    ]\n"
-                + "}\n"));
+    @DisplayName("Default values are not added enriched on read when the config file is empty")
+    void empty() throws IOException {
+        // Given
+        assertThat(configFileContent(), emptyString());
+
+        // When
+        var storageValues = readAllLatest();
+
+        // Then storage data only contains top level defaults
+        assertThat(storageValues.entrySet(), hasSize(1));
     }
 
     @Test
-    void testMergeHocon() throws IOException {
-        var data = Map.of("foo1", "bar");
-        assertThat(storage.write(data, 0), willBe(true));
+    @DisplayName("Named list entities can be added")
+    void add() throws Exception {
+        // Given
+        assertThat(configFileContent(), emptyString());
+        // And
+        var topConfiguration = (TopConfiguration) cgen.instantiateCfg(TopConfiguration.KEY, changer);
+        topConfiguration.namedList().change(b -> b.create("name1", x -> {
+            x.changeStrVal("strVal1");
+            x.changeIntVal(-1);
+        })).get();
+
+        // When
+        var storageValues = readAllLatest();
 
-        var append = Map.of("foo1", "baz", "foo2", "bar");
-        assertThat(storage.write(append, 1), willBe(true));
+        // Then
+        assertThat(storageValues, allOf(aMapWithSize(6), hasValue(-1)));
+        assertThat(storageValues, allOf(aMapWithSize(6), hasValue("strVal1")));
+        // And
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[\n"
+                        + "        {\n"
+                        + "            intVal=-1\n"
+                        + "            name=name1\n"
+                        + "            strVal=strVal1\n"
+                        + "        }\n"
+                        + "    ]\n"
+                        + "    shortVal=1\n"
+                        + "}"
+        ));
 
-        String contents = Files.readString(getConfigFile());
-        assertThat(contents, is("foo1=baz\n"
-                + "foo2=bar\n"));
+        // When
+        topConfiguration.namedList().change(b -> b.create("name2", x -> {
+            x.changeStrVal("strVal2");
+            x.changeIntVal(-2);
+        })).get();
+        // And
+        storageValues = readAllLatest();
+
+        // Then
+        assertThat(storageValues, allOf(aMapWithSize(11), hasValue(-2)));
+        assertThat(storageValues, allOf(aMapWithSize(11), hasValue("strVal2")));
+        // And
+        assertThat(storageValues, allOf(aMapWithSize(11), hasValue(-1)));
+        assertThat(storageValues, allOf(aMapWithSize(11), hasValue("strVal1")));
+        // And
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[\n"
+                        + "        {\n"
+                        + "            intVal=-1\n"
+                        + "            name=name1\n"
+                        + "            strVal=strVal1\n"
+                        + "        },\n"
+                        + "        {\n"
+                        + "            intVal=-2\n"
+                        + "            name=name2\n"
+                        + "            strVal=strVal2\n"
+                        + "        }\n"
+                        + "    ]\n"
+                        + "    shortVal=1\n"
+                        + "}\n"
+        ));
     }
 
-    private Path getConfigFile() {
-        return tmpDir.resolve(CONFIG_NAME);
+    @DisplayName("Update values")
+    @Test
+    void update() throws Exception {
+        // Given
+        assertThat(configFileContent(), emptyString());
+
+        // When
+        var topConfiguration = (TopConfiguration) cgen.instantiateCfg(TopConfiguration.KEY, changer);
+        topConfiguration.shortVal().update((short) 3).get();
+        // And
+        var storageValues = readAllLatest();
+
+        // Then
+        assertThat(storageValues, allOf(aMapWithSize(1), hasValue((short) 3)));
+        // And
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[]\n"
+                        + "    shortVal=3\n"
+                        + "}\n"
+        ));
+
+        // When create named list entity with defaults
+        topConfiguration.namedList().change(b -> b.create("name1", x -> {
+        })).get();
+        // And
+        storageValues = readAllLatest();
+
+        // Then
+        assertThat(storageValues, allOf(aMapWithSize(6), hasValue(1)));
+        assertThat(storageValues, allOf(aMapWithSize(6), hasValue("foo")));
+        // And
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[\n"
+                        + "        {\n"
+                        + "            intVal=1\n"
+                        + "            name=name1\n"
+                        + "            strVal=foo\n"
+                        + "        }\n"
+                        + "    ]\n"
+                        + "    shortVal=3\n"
+                        + "}"
+        ));
+
+        // When update named list entity
+        topConfiguration.namedList().change(b -> b.update("name1", x -> {
+            x.changeStrVal("strVal1");
+            x.changeIntVal(-1);
+        })).get();
+        // And
+        storageValues = readAllLatest();
+
+        // Then
+        assertThat(storageValues, allOf(aMapWithSize(6), hasValue(-1)));
+        assertThat(storageValues, allOf(aMapWithSize(6), hasValue("strVal1")));
+        // And
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[\n"
+                        + "        {\n"
+                        + "            intVal=-1\n"
+                        + "            name=name1\n"
+                        + "            strVal=strVal1\n"
+                        + "        }\n"
+                        + "    ]\n"
+                        + "    shortVal=3\n"
+                        + "}"
+        ));
+    }
+
+    @DisplayName("Remove values")
+    @Test
+    void remove() throws Exception {
+        // Given
+        var topConfiguration = (TopConfiguration) cgen.instantiateCfg(TopConfiguration.KEY, changer);
+        topConfiguration.namedList().change(b -> {
+            b.create("name1", x -> {
+                x.changeStrVal("strVal1");
+                x.changeIntVal(-1);
+            });
+            b.create("name2", x -> {
+                x.changeStrVal("strVal2");
+                x.changeIntVal(-2);
+            });
+        }).get();
+        topConfiguration.shortVal().update((short) 3).get();
+        // And values are saved to file
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[\n"
+                        + "        {\n"
+                        + "            intVal=-1\n"
+                        + "            name=name1\n"
+                        + "            strVal=strVal1\n"
+                        + "        },\n"
+                        + "        {\n"
+                        + "            intVal=-2\n"
+                        + "            name=name2\n"
+                        + "            strVal=strVal2\n"
+                        + "        }\n"
+                        + "    ]\n"
+                        + "    shortVal=3\n"
+                        + "}\n"
+        ));
+
+        // When remove named list entity
+        topConfiguration.namedList().change(b -> b.delete("name1")).get();
+        // And
+        var storageValues = readAllLatest();
+
+        // Then
+        assertThat(storageValues, allOf(aMapWithSize(6), Matchers.not(hasValue("strVal1"))));
+        // And entity removed from file
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[\n"
+                        + "        {\n"
+                        + "            intVal=-2\n"
+                        + "            name=name2\n"
+                        + "            strVal=strVal2\n"
+                        + "        }\n"
+                        + "    ]\n"
+                        + "    shortVal=3\n"
+                        + "}\n"
+        ));
+
+        // When remove the last entity
+        topConfiguration.namedList().change(b -> b.delete("name2")).get();
+        // And
+        storageValues = readAllLatest();
+
+        // Then
+        assertThat(storageValues, allOf(aMapWithSize(1), hasValue((short) 3)));
+        // And entity removed from file
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[]\n"
+                        + "    shortVal=3\n"
+                        + "}\n"
+        ));
+    }
+
+    @DisplayName("Delete file before read on recovery")
+    @Test
+    void deleteFileBeforeReadOnRecovery() throws IOException {
+        // Given
+        Files.delete(getConfigFile());
+
+        // When
+        var storageValues = storage.readDataOnRecovery().join().values();
+
+        // Then storage data only contains top level defaults
+        assertThat(storageValues.entrySet(), hasSize(1));
+        // And
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[]\n"

Review Comment:
   Can we avoid rendering empty lists?



##########
modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorageTest.java:
##########
@@ -17,75 +17,401 @@
 
 package org.apache.ignite.internal.configuration.storage;
 
-import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.aMapWithSize;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.emptyString;
+import static org.hamcrest.Matchers.equalToCompressingWhiteSpace;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.hasValue;
 import static org.hamcrest.Matchers.is;
 
 import java.io.IOException;
+import java.io.Serializable;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.ConfigurationRoot;
+import org.apache.ignite.configuration.annotation.NamedConfigValue;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.internal.configuration.TestConfigurationChanger;
+import org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
-import org.junit.jupiter.api.Disabled;
+import org.hamcrest.Matchers;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.DisplayName;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
-/**
- * Tests for the {@link LocalFileConfigurationStorage}.
- */
+/** Test for local file configurations storage. */
 @ExtendWith(WorkDirectoryExtension.class)
-@Disabled("https://issues.apache.org/jira/browse/IGNITE-19152")
-public class LocalFileConfigurationStorageTest extends ConfigurationStorageTest {
+public class LocalFileConfigurationStorageTest {
 
     private static final String CONFIG_NAME = "ignite-config.conf";
 
+    private static ConfigurationAsmGenerator cgen;
+
     @WorkDirectory
     private Path tmpDir;
 
-    @Override
-    public ConfigurationStorage getStorage() {
-        return new LocalFileConfigurationStorage(getConfigFile());
+    /** Test configuration storage. */
+    private LocalFileConfigurationStorage storage;
+
+    /** Test configuration changer. */
+    private TestConfigurationChanger changer;
+
+    /** Instantiates {@link #cgen}. */
+    @BeforeAll
+    public static void beforeAll() {
+        cgen = new ConfigurationAsmGenerator();
+    }
+
+    /** Nullifies {@link #cgen} to prevent memory leak from having runtime ClassLoader accessible from GC root. */
+    @AfterAll
+    public static void afterAll() {
+        cgen = null;
+    }
+
+    private Path getConfigFile() {
+        return tmpDir.resolve(CONFIG_NAME);
+    }
+
+    @BeforeEach
+    void before() {
+        storage = new LocalFileConfigurationStorage(getConfigFile(), List.of(TopConfiguration.KEY));
+
+        changer = new TestConfigurationChanger(
+                cgen,
+                List.of(TopConfiguration.KEY),
+                Set.of(),
+                storage,
+                List.of(),
+                List.of()
+        );
+
+        changer.start();
+    }
+
+    @AfterEach
+    void after() {
+        changer.stop();
     }
 
     @Test
-    void testHocon() throws IOException {
-        // All of this is needed because write expects serializable values and only concrete classes are serializable
-        HashMap<String, ArrayList<String>> map = new HashMap<>(Map.of("list", new ArrayList<>(List.of("val1", "val2"))));
-        var data = Map.of("foo1", "bar1", "foo2", "bar2", "map", map);
-
-        assertThat(storage.write(data, 0), willBe(true));
-
-        String contents = Files.readString(getConfigFile());
-
-        // \n instead of System.lineSeparator because Config library writes \n only
-        assertThat(contents, is("foo1=bar1\n"
-                + "foo2=bar2\n"
-                + "map {\n"
-                + "    list=[\n"
-                + "        val1,\n"
-                + "        val2\n"
-                + "    ]\n"
-                + "}\n"));
+    @DisplayName("Default values are not added enriched on read when the config file is empty")
+    void empty() throws IOException {
+        // Given
+        assertThat(configFileContent(), emptyString());
+
+        // When
+        var storageValues = readAllLatest();
+
+        // Then storage data only contains top level defaults
+        assertThat(storageValues.entrySet(), hasSize(1));
     }
 
     @Test
-    void testMergeHocon() throws IOException {
-        var data = Map.of("foo1", "bar");
-        assertThat(storage.write(data, 0), willBe(true));
+    @DisplayName("Named list entities can be added")
+    void add() throws Exception {
+        // Given
+        assertThat(configFileContent(), emptyString());
+        // And
+        var topConfiguration = (TopConfiguration) cgen.instantiateCfg(TopConfiguration.KEY, changer);
+        topConfiguration.namedList().change(b -> b.create("name1", x -> {
+            x.changeStrVal("strVal1");
+            x.changeIntVal(-1);
+        })).get();
+
+        // When
+        var storageValues = readAllLatest();
 
-        var append = Map.of("foo1", "baz", "foo2", "bar");
-        assertThat(storage.write(append, 1), willBe(true));
+        // Then
+        assertThat(storageValues, allOf(aMapWithSize(6), hasValue(-1)));
+        assertThat(storageValues, allOf(aMapWithSize(6), hasValue("strVal1")));
+        // And
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[\n"
+                        + "        {\n"
+                        + "            intVal=-1\n"
+                        + "            name=name1\n"
+                        + "            strVal=strVal1\n"
+                        + "        }\n"
+                        + "    ]\n"
+                        + "    shortVal=1\n"
+                        + "}"
+        ));
 
-        String contents = Files.readString(getConfigFile());
-        assertThat(contents, is("foo1=baz\n"
-                + "foo2=bar\n"));
+        // When
+        topConfiguration.namedList().change(b -> b.create("name2", x -> {
+            x.changeStrVal("strVal2");
+            x.changeIntVal(-2);
+        })).get();
+        // And
+        storageValues = readAllLatest();
+
+        // Then
+        assertThat(storageValues, allOf(aMapWithSize(11), hasValue(-2)));
+        assertThat(storageValues, allOf(aMapWithSize(11), hasValue("strVal2")));
+        // And
+        assertThat(storageValues, allOf(aMapWithSize(11), hasValue(-1)));
+        assertThat(storageValues, allOf(aMapWithSize(11), hasValue("strVal1")));
+        // And
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[\n"
+                        + "        {\n"
+                        + "            intVal=-1\n"
+                        + "            name=name1\n"
+                        + "            strVal=strVal1\n"
+                        + "        },\n"
+                        + "        {\n"
+                        + "            intVal=-2\n"
+                        + "            name=name2\n"
+                        + "            strVal=strVal2\n"
+                        + "        }\n"
+                        + "    ]\n"
+                        + "    shortVal=1\n"
+                        + "}\n"
+        ));
     }
 
-    private Path getConfigFile() {
-        return tmpDir.resolve(CONFIG_NAME);
+    @DisplayName("Update values")
+    @Test
+    void update() throws Exception {
+        // Given
+        assertThat(configFileContent(), emptyString());
+
+        // When
+        var topConfiguration = (TopConfiguration) cgen.instantiateCfg(TopConfiguration.KEY, changer);
+        topConfiguration.shortVal().update((short) 3).get();
+        // And
+        var storageValues = readAllLatest();
+
+        // Then
+        assertThat(storageValues, allOf(aMapWithSize(1), hasValue((short) 3)));
+        // And
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[]\n"
+                        + "    shortVal=3\n"
+                        + "}\n"
+        ));
+
+        // When create named list entity with defaults
+        topConfiguration.namedList().change(b -> b.create("name1", x -> {
+        })).get();
+        // And
+        storageValues = readAllLatest();
+
+        // Then
+        assertThat(storageValues, allOf(aMapWithSize(6), hasValue(1)));
+        assertThat(storageValues, allOf(aMapWithSize(6), hasValue("foo")));
+        // And
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[\n"
+                        + "        {\n"
+                        + "            intVal=1\n"
+                        + "            name=name1\n"
+                        + "            strVal=foo\n"
+                        + "        }\n"
+                        + "    ]\n"
+                        + "    shortVal=3\n"
+                        + "}"
+        ));
+
+        // When update named list entity
+        topConfiguration.namedList().change(b -> b.update("name1", x -> {
+            x.changeStrVal("strVal1");
+            x.changeIntVal(-1);
+        })).get();
+        // And
+        storageValues = readAllLatest();
+
+        // Then
+        assertThat(storageValues, allOf(aMapWithSize(6), hasValue(-1)));
+        assertThat(storageValues, allOf(aMapWithSize(6), hasValue("strVal1")));
+        // And
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[\n"
+                        + "        {\n"
+                        + "            intVal=-1\n"
+                        + "            name=name1\n"
+                        + "            strVal=strVal1\n"
+                        + "        }\n"
+                        + "    ]\n"
+                        + "    shortVal=3\n"
+                        + "}"
+        ));
+    }
+
+    @DisplayName("Remove values")
+    @Test
+    void remove() throws Exception {
+        // Given
+        var topConfiguration = (TopConfiguration) cgen.instantiateCfg(TopConfiguration.KEY, changer);
+        topConfiguration.namedList().change(b -> {
+            b.create("name1", x -> {
+                x.changeStrVal("strVal1");
+                x.changeIntVal(-1);
+            });
+            b.create("name2", x -> {
+                x.changeStrVal("strVal2");
+                x.changeIntVal(-2);
+            });
+        }).get();
+        topConfiguration.shortVal().update((short) 3).get();
+        // And values are saved to file
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[\n"
+                        + "        {\n"
+                        + "            intVal=-1\n"
+                        + "            name=name1\n"
+                        + "            strVal=strVal1\n"
+                        + "        },\n"
+                        + "        {\n"
+                        + "            intVal=-2\n"
+                        + "            name=name2\n"
+                        + "            strVal=strVal2\n"
+                        + "        }\n"
+                        + "    ]\n"
+                        + "    shortVal=3\n"
+                        + "}\n"
+        ));
+
+        // When remove named list entity
+        topConfiguration.namedList().change(b -> b.delete("name1")).get();
+        // And
+        var storageValues = readAllLatest();
+
+        // Then
+        assertThat(storageValues, allOf(aMapWithSize(6), Matchers.not(hasValue("strVal1"))));
+        // And entity removed from file
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[\n"
+                        + "        {\n"
+                        + "            intVal=-2\n"
+                        + "            name=name2\n"
+                        + "            strVal=strVal2\n"
+                        + "        }\n"
+                        + "    ]\n"
+                        + "    shortVal=3\n"
+                        + "}\n"
+        ));
+
+        // When remove the last entity
+        topConfiguration.namedList().change(b -> b.delete("name2")).get();
+        // And
+        storageValues = readAllLatest();
+
+        // Then
+        assertThat(storageValues, allOf(aMapWithSize(1), hasValue((short) 3)));
+        // And entity removed from file
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[]\n"
+                        + "    shortVal=3\n"
+                        + "}\n"
+        ));
+    }
+
+    @DisplayName("Delete file before read on recovery")
+    @Test
+    void deleteFileBeforeReadOnRecovery() throws IOException {
+        // Given
+        Files.delete(getConfigFile());
+
+        // When
+        var storageValues = storage.readDataOnRecovery().join().values();
+
+        // Then storage data only contains top level defaults
+        assertThat(storageValues.entrySet(), hasSize(1));
+        // And
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[]\n"
+                        + "    shortVal=1\n"
+                        + "}\n"
+        ));
+    }
+
+    @DisplayName("Delete file before read all")
+    @Test
+    void deleteFileBeforeReadAll() throws Exception {
+        // Given
+        Files.delete(getConfigFile());
+
+        // When
+        var storageValues = readAllLatest();
+
+        // Then storage data only contains top level defaults
+        assertThat(storageValues.entrySet(), hasSize(1));
+        // And there is no file
+        assertThat(Files.exists(getConfigFile()), is(false));
+
+        // When update configuration
+        var topConfiguration = (TopConfiguration) cgen.instantiateCfg(TopConfiguration.KEY, changer);
+        topConfiguration.namedList().change(b -> b.create("name1", x -> {
+                x.changeStrVal("strVal1");
+                x.changeIntVal(-1);
+            })).get();
+
+        // Then file is created
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[\n"
+                        + "        {\n"
+                        + "            intVal=-1\n"
+                        + "            name=name1\n"
+                        + "            strVal=strVal1\n"
+                        + "        }\n"
+                        + "    ]\n"
+                        + "    shortVal=1\n"
+                        + "}\n"
+        ));
+    }
+
+    private String configFileContent() throws IOException {
+        return Files.readString(getConfigFile());
+    }
+
+    private Map<String, ? extends Serializable> readAllLatest() {
+        return storage.readAllLatest("").join();
+    }
+
+    // null == remove
+
+    // when read file you can see all values
+    //

Review Comment:
   What's this?



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java:
##########
@@ -606,7 +606,9 @@ private class Node {
                             RestConfiguration.KEY,
                             ClientConnectorConfiguration.KEY),
                     Set.of(),
-                    new LocalFileConfigurationStorage(configPath),
+                    new LocalFileConfigurationStorage(configPath, List.of(NetworkConfiguration.KEY,

Review Comment:
   Why don't you use `ConfigurationModule` inside of the storage? Explicit list is not very flexible.
   I recommend returning back to the old constructor.



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -175,47 +261,66 @@ public CompletableFuture<Void> writeConfigurationRevision(long prevRevision, lon
 
     @Override
     public void close() {
+        IgniteUtils.shutdownAndAwaitTermination(workerThreadPool, 10, TimeUnit.SECONDS);
         IgniteUtils.shutdownAndAwaitTermination(threadPool, 10, TimeUnit.SECONDS);
 
         futureTracker.cancelInFlightFutures();
     }
 
-    private void saveValues(Map<String, ? extends Serializable> values) {
+    private void saveConfigFile() {
         try {
-            Files.write(tempConfigPath, renderHoconString(values).getBytes(StandardCharsets.UTF_8),
-                    StandardOpenOption.SYNC, StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
-            Files.move(tempConfigPath, configPath, StandardCopyOption.ATOMIC_MOVE, StandardCopyOption.REPLACE_EXISTING);
+            Files.write(
+                    tempConfigPath,
+                    renderHoconString().getBytes(StandardCharsets.UTF_8),
+                    StandardOpenOption.SYNC, StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING
+            );
+
+            Files.move(
+                    tempConfigPath,
+                    configPath,
+                    StandardCopyOption.ATOMIC_MOVE, StandardCopyOption.REPLACE_EXISTING
+            );
         } catch (IOException e) {
             throw new NodeConfigWriteException(
-                    "Failed to write values " + values + " to config file.", e);
+                    "Failed to write values to config file.", e);
         }
     }
 
     /**
      * Convert provided map to Hocon String representation.
      *
-     * @param values Values of configuration.
      * @return Configuration file string representation in HOCON format.
      */
-    private String renderHoconString(Map<String, ? extends Serializable> values) {
-        Map<String, Object> map = values.entrySet().stream().collect(Collectors.toMap(Entry::getKey, stringEntry -> {
-            Serializable value = stringEntry.getValue();
-            if (value.getClass().isArray()) {
-                return Arrays.asList((Object[]) value);
-            }
-            return value;
-        }));
-        Config other = ConfigFactory.parseMap(map);
-        Config newConfig = other.withFallback(parseConfigOptions()).resolve();
+    private String renderHoconString() {
+        // Super root that'll be filled from the storage data.
+        SuperRoot rootNode = new SuperRoot(rootCreator());

Review Comment:
   I think you should call `createSuperRoot()` instead. Right now this code may not work. Have you tested it?



##########
modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorageTest.java:
##########
@@ -17,75 +17,401 @@
 
 package org.apache.ignite.internal.configuration.storage;
 
-import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.aMapWithSize;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.emptyString;
+import static org.hamcrest.Matchers.equalToCompressingWhiteSpace;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.hasValue;
 import static org.hamcrest.Matchers.is;
 
 import java.io.IOException;
+import java.io.Serializable;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.ConfigurationRoot;
+import org.apache.ignite.configuration.annotation.NamedConfigValue;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.internal.configuration.TestConfigurationChanger;
+import org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
-import org.junit.jupiter.api.Disabled;
+import org.hamcrest.Matchers;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.DisplayName;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
-/**
- * Tests for the {@link LocalFileConfigurationStorage}.
- */
+/** Test for local file configurations storage. */
 @ExtendWith(WorkDirectoryExtension.class)
-@Disabled("https://issues.apache.org/jira/browse/IGNITE-19152")
-public class LocalFileConfigurationStorageTest extends ConfigurationStorageTest {
+public class LocalFileConfigurationStorageTest {
 
     private static final String CONFIG_NAME = "ignite-config.conf";
 
+    private static ConfigurationAsmGenerator cgen;
+
     @WorkDirectory
     private Path tmpDir;
 
-    @Override
-    public ConfigurationStorage getStorage() {
-        return new LocalFileConfigurationStorage(getConfigFile());
+    /** Test configuration storage. */
+    private LocalFileConfigurationStorage storage;
+
+    /** Test configuration changer. */
+    private TestConfigurationChanger changer;
+
+    /** Instantiates {@link #cgen}. */
+    @BeforeAll
+    public static void beforeAll() {
+        cgen = new ConfigurationAsmGenerator();
+    }
+
+    /** Nullifies {@link #cgen} to prevent memory leak from having runtime ClassLoader accessible from GC root. */
+    @AfterAll
+    public static void afterAll() {
+        cgen = null;
+    }
+
+    private Path getConfigFile() {
+        return tmpDir.resolve(CONFIG_NAME);
+    }
+
+    @BeforeEach
+    void before() {
+        storage = new LocalFileConfigurationStorage(getConfigFile(), List.of(TopConfiguration.KEY));
+
+        changer = new TestConfigurationChanger(
+                cgen,
+                List.of(TopConfiguration.KEY),
+                Set.of(),
+                storage,
+                List.of(),
+                List.of()
+        );
+
+        changer.start();
+    }
+
+    @AfterEach
+    void after() {
+        changer.stop();
     }
 
     @Test
-    void testHocon() throws IOException {
-        // All of this is needed because write expects serializable values and only concrete classes are serializable
-        HashMap<String, ArrayList<String>> map = new HashMap<>(Map.of("list", new ArrayList<>(List.of("val1", "val2"))));
-        var data = Map.of("foo1", "bar1", "foo2", "bar2", "map", map);
-
-        assertThat(storage.write(data, 0), willBe(true));
-
-        String contents = Files.readString(getConfigFile());
-
-        // \n instead of System.lineSeparator because Config library writes \n only
-        assertThat(contents, is("foo1=bar1\n"
-                + "foo2=bar2\n"
-                + "map {\n"
-                + "    list=[\n"
-                + "        val1,\n"
-                + "        val2\n"
-                + "    ]\n"
-                + "}\n"));
+    @DisplayName("Default values are not added enriched on read when the config file is empty")
+    void empty() throws IOException {
+        // Given
+        assertThat(configFileContent(), emptyString());
+
+        // When
+        var storageValues = readAllLatest();
+
+        // Then storage data only contains top level defaults
+        assertThat(storageValues.entrySet(), hasSize(1));
     }
 
     @Test
-    void testMergeHocon() throws IOException {
-        var data = Map.of("foo1", "bar");
-        assertThat(storage.write(data, 0), willBe(true));
+    @DisplayName("Named list entities can be added")
+    void add() throws Exception {
+        // Given
+        assertThat(configFileContent(), emptyString());
+        // And

Review Comment:
   Please add empty lines here and there, this code looks too cramped



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -40,114 +53,187 @@
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.stream.Collectors;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.ignite.configuration.RootKey;
 import org.apache.ignite.configuration.annotation.ConfigurationType;
 import org.apache.ignite.internal.configuration.NodeConfigCreateException;
 import org.apache.ignite.internal.configuration.NodeConfigWriteException;
+import org.apache.ignite.internal.configuration.RootInnerNode;
+import org.apache.ignite.internal.configuration.SuperRoot;
+import org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator;
+import org.apache.ignite.internal.configuration.hocon.HoconConverter;
+import org.apache.ignite.internal.configuration.tree.ConverterToMapVisitor;
+import org.apache.ignite.internal.configuration.tree.InnerNode;
 import org.apache.ignite.internal.future.InFlightFutures;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.thread.NamedThreadFactory;
 import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
 
 /**
  * Implementation of {@link ConfigurationStorage} based on local file configuration storage.
  */
 public class LocalFileConfigurationStorage implements ConfigurationStorage {
     private static final IgniteLogger LOG = Loggers.forClass(LocalFileConfigurationStorage.class);
 
-    /**
-     * Path to config file.
-     */
+    /** Path to config file. */
     private final Path configPath;
 
-    /**
-     * Path to temporary configuration storage.
-     */
+    /** Path to temporary configuration storage. */
     private final Path tempConfigPath;
 
+    /** R/W lock to guard the latest configuration and config file. */
     private final ReadWriteLock lock = new ReentrantReadWriteLock();
 
-    /**
-     * Latest state of last applied configuration.
-     */
+    /** Latest state of last applied configuration. */
     private final Map<String, Serializable> latest = new ConcurrentHashMap<>();
 
-    /**
-     *  Configuration changes listener.
-     *  */
+    /** Configuration nodes generator. */
+    private final ConfigurationAsmGenerator cgen = new ConfigurationAsmGenerator();
+
+    /** Map of root keys that are needed to generate configuration tree. */
+    private final Map<String, RootKey<?, ?>> rootKeys;
+
+    /** Configuration changes listener. */
     private final AtomicReference<ConfigurationStorageListener> lsnrRef = new AtomicReference<>();
 
-    private final ExecutorService threadPool = Executors.newFixedThreadPool(2, new NamedThreadFactory("loc-cfg-file", LOG));
+    /** Thread pool for configuration updates notifications. */
+    private final ExecutorService threadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("loc-cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates notifications. */
+    private final ExecutorService workerThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file-worker", LOG)
+    );
 
+    /** Tracks all running futures. */
     private final InFlightFutures futureTracker = new InFlightFutures();
 
+    /** Last revision for configuration. */
     private long lastRevision = 0L;
 
+    /**
+     * Constructor without configuration extensions.
+     *
+     * @param configPath Path to node bootstrap configuration file.
+     * @param rootKeys Configuration root keys.
+     */
+    public LocalFileConfigurationStorage(Path configPath, Collection<RootKey<?, ?>> rootKeys) {
+        this(configPath, rootKeys, Collections.emptyList(), Collections.emptyList());
+    }
+
     /**
      * Constructor.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param rootKeys Configuration root keys.
+     * @param internalSchemaExtensions Internal schema extensions.
+     * @param polymorphicSchemaExtensions Polymorphic schema extensions.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, Collection<RootKey<?, ?>> rootKeys,
+            Collection<Class<?>> internalSchemaExtensions, Collection<Class<?>> polymorphicSchemaExtensions) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.rootKeys = rootKeys.stream().collect(toMap(RootKey::key, identity()));
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
+        Map<Class<?>, Set<Class<?>>> internalExtensions = internalSchemaExtensions(internalSchemaExtensions);
+        Map<Class<?>, Set<Class<?>>> polymorphicExtensions = polymorphicSchemaExtensions(polymorphicSchemaExtensions);
+
+        rootKeys.forEach(key -> cgen.compileRootSchema(key.schemaClass(), internalExtensions, polymorphicExtensions));
+
         checkAndRestoreConfigFile();
     }
 
+    private static void setValues(SuperRoot target, Config source) {
+        HoconConverter.hoconSource(source.root()).descend(target);
+    }
+
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return writeLockAsync(() -> {

Review Comment:
   Considering that now we really read data on recovery, we should remove the step of applying this very configuration file as a change on top of itself. Please fix that part.



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -40,114 +53,187 @@
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.stream.Collectors;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.ignite.configuration.RootKey;
 import org.apache.ignite.configuration.annotation.ConfigurationType;
 import org.apache.ignite.internal.configuration.NodeConfigCreateException;
 import org.apache.ignite.internal.configuration.NodeConfigWriteException;
+import org.apache.ignite.internal.configuration.RootInnerNode;
+import org.apache.ignite.internal.configuration.SuperRoot;
+import org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator;
+import org.apache.ignite.internal.configuration.hocon.HoconConverter;
+import org.apache.ignite.internal.configuration.tree.ConverterToMapVisitor;
+import org.apache.ignite.internal.configuration.tree.InnerNode;
 import org.apache.ignite.internal.future.InFlightFutures;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.thread.NamedThreadFactory;
 import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
 
 /**
  * Implementation of {@link ConfigurationStorage} based on local file configuration storage.
  */
 public class LocalFileConfigurationStorage implements ConfigurationStorage {
     private static final IgniteLogger LOG = Loggers.forClass(LocalFileConfigurationStorage.class);
 
-    /**
-     * Path to config file.
-     */
+    /** Path to config file. */
     private final Path configPath;
 
-    /**
-     * Path to temporary configuration storage.
-     */
+    /** Path to temporary configuration storage. */
     private final Path tempConfigPath;
 
+    /** R/W lock to guard the latest configuration and config file. */
     private final ReadWriteLock lock = new ReentrantReadWriteLock();
 
-    /**
-     * Latest state of last applied configuration.
-     */
+    /** Latest state of last applied configuration. */
     private final Map<String, Serializable> latest = new ConcurrentHashMap<>();
 
-    /**
-     *  Configuration changes listener.
-     *  */
+    /** Configuration nodes generator. */
+    private final ConfigurationAsmGenerator cgen = new ConfigurationAsmGenerator();
+
+    /** Map of root keys that are needed to generate configuration tree. */
+    private final Map<String, RootKey<?, ?>> rootKeys;
+
+    /** Configuration changes listener. */
     private final AtomicReference<ConfigurationStorageListener> lsnrRef = new AtomicReference<>();
 
-    private final ExecutorService threadPool = Executors.newFixedThreadPool(2, new NamedThreadFactory("loc-cfg-file", LOG));
+    /** Thread pool for configuration updates notifications. */
+    private final ExecutorService threadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("loc-cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates notifications. */
+    private final ExecutorService workerThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file-worker", LOG)
+    );
 
+    /** Tracks all running futures. */
     private final InFlightFutures futureTracker = new InFlightFutures();
 
+    /** Last revision for configuration. */
     private long lastRevision = 0L;
 
+    /**
+     * Constructor without configuration extensions.
+     *
+     * @param configPath Path to node bootstrap configuration file.
+     * @param rootKeys Configuration root keys.
+     */
+    public LocalFileConfigurationStorage(Path configPath, Collection<RootKey<?, ?>> rootKeys) {
+        this(configPath, rootKeys, Collections.emptyList(), Collections.emptyList());
+    }
+
     /**
      * Constructor.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param rootKeys Configuration root keys.
+     * @param internalSchemaExtensions Internal schema extensions.
+     * @param polymorphicSchemaExtensions Polymorphic schema extensions.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, Collection<RootKey<?, ?>> rootKeys,
+            Collection<Class<?>> internalSchemaExtensions, Collection<Class<?>> polymorphicSchemaExtensions) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.rootKeys = rootKeys.stream().collect(toMap(RootKey::key, identity()));
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
+        Map<Class<?>, Set<Class<?>>> internalExtensions = internalSchemaExtensions(internalSchemaExtensions);
+        Map<Class<?>, Set<Class<?>>> polymorphicExtensions = polymorphicSchemaExtensions(polymorphicSchemaExtensions);
+
+        rootKeys.forEach(key -> cgen.compileRootSchema(key.schemaClass(), internalExtensions, polymorphicExtensions));
+
         checkAndRestoreConfigFile();
     }
 
+    private static void setValues(SuperRoot target, Config source) {
+        HoconConverter.hoconSource(source.root()).descend(target);
+    }
+
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return writeLockAsync(() -> {
+            SuperRoot superRoot = createSuperRoot();
+            SuperRoot copiedSuperRoot = superRoot.copy();
+
+            Config hocon = readHoconFromFile();
+            setValues(copiedSuperRoot, hocon);
+
+            addDefaults(copiedSuperRoot);
+
+            Map<String, Serializable> flattenedUpdatesMap = createFlattenedUpdatesMap(superRoot, copiedSuperRoot);
+
+            latest.putAll(flattenedUpdatesMap);
+
+            return new Data(flattenedUpdatesMap, lastRevision);
+        });
+    }
+
+
+    private Config readHoconFromFile() {
+        checkAndRestoreConfigFile();
+        return ConfigFactory.parseFile(configPath.toFile(), ConfigParseOptions.defaults().setAllowMissing(false));
+    }
+
+    @NotNull

Review Comment:
   We don't use `@NotNull`



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -40,114 +53,187 @@
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.stream.Collectors;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.ignite.configuration.RootKey;
 import org.apache.ignite.configuration.annotation.ConfigurationType;
 import org.apache.ignite.internal.configuration.NodeConfigCreateException;
 import org.apache.ignite.internal.configuration.NodeConfigWriteException;
+import org.apache.ignite.internal.configuration.RootInnerNode;
+import org.apache.ignite.internal.configuration.SuperRoot;
+import org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator;
+import org.apache.ignite.internal.configuration.hocon.HoconConverter;
+import org.apache.ignite.internal.configuration.tree.ConverterToMapVisitor;
+import org.apache.ignite.internal.configuration.tree.InnerNode;
 import org.apache.ignite.internal.future.InFlightFutures;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.thread.NamedThreadFactory;
 import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
 
 /**
  * Implementation of {@link ConfigurationStorage} based on local file configuration storage.
  */
 public class LocalFileConfigurationStorage implements ConfigurationStorage {
     private static final IgniteLogger LOG = Loggers.forClass(LocalFileConfigurationStorage.class);
 
-    /**
-     * Path to config file.
-     */
+    /** Path to config file. */
     private final Path configPath;
 
-    /**
-     * Path to temporary configuration storage.
-     */
+    /** Path to temporary configuration storage. */
     private final Path tempConfigPath;
 
+    /** R/W lock to guard the latest configuration and config file. */
     private final ReadWriteLock lock = new ReentrantReadWriteLock();
 
-    /**
-     * Latest state of last applied configuration.
-     */
+    /** Latest state of last applied configuration. */
     private final Map<String, Serializable> latest = new ConcurrentHashMap<>();
 
-    /**
-     *  Configuration changes listener.
-     *  */
+    /** Configuration nodes generator. */
+    private final ConfigurationAsmGenerator cgen = new ConfigurationAsmGenerator();
+
+    /** Map of root keys that are needed to generate configuration tree. */
+    private final Map<String, RootKey<?, ?>> rootKeys;
+
+    /** Configuration changes listener. */
     private final AtomicReference<ConfigurationStorageListener> lsnrRef = new AtomicReference<>();
 
-    private final ExecutorService threadPool = Executors.newFixedThreadPool(2, new NamedThreadFactory("loc-cfg-file", LOG));
+    /** Thread pool for configuration updates notifications. */
+    private final ExecutorService threadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("loc-cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates notifications. */
+    private final ExecutorService workerThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file-worker", LOG)
+    );
 
+    /** Tracks all running futures. */
     private final InFlightFutures futureTracker = new InFlightFutures();
 
+    /** Last revision for configuration. */
     private long lastRevision = 0L;
 
+    /**
+     * Constructor without configuration extensions.
+     *
+     * @param configPath Path to node bootstrap configuration file.
+     * @param rootKeys Configuration root keys.
+     */
+    public LocalFileConfigurationStorage(Path configPath, Collection<RootKey<?, ?>> rootKeys) {
+        this(configPath, rootKeys, Collections.emptyList(), Collections.emptyList());
+    }
+
     /**
      * Constructor.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param rootKeys Configuration root keys.
+     * @param internalSchemaExtensions Internal schema extensions.
+     * @param polymorphicSchemaExtensions Polymorphic schema extensions.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, Collection<RootKey<?, ?>> rootKeys,
+            Collection<Class<?>> internalSchemaExtensions, Collection<Class<?>> polymorphicSchemaExtensions) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.rootKeys = rootKeys.stream().collect(toMap(RootKey::key, identity()));
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
+        Map<Class<?>, Set<Class<?>>> internalExtensions = internalSchemaExtensions(internalSchemaExtensions);
+        Map<Class<?>, Set<Class<?>>> polymorphicExtensions = polymorphicSchemaExtensions(polymorphicSchemaExtensions);
+
+        rootKeys.forEach(key -> cgen.compileRootSchema(key.schemaClass(), internalExtensions, polymorphicExtensions));
+
         checkAndRestoreConfigFile();
     }
 
+    private static void setValues(SuperRoot target, Config source) {
+        HoconConverter.hoconSource(source.root()).descend(target);
+    }
+
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return writeLockAsync(() -> {
+            SuperRoot superRoot = createSuperRoot();
+            SuperRoot copiedSuperRoot = superRoot.copy();
+
+            Config hocon = readHoconFromFile();
+            setValues(copiedSuperRoot, hocon);
+
+            addDefaults(copiedSuperRoot);
+
+            Map<String, Serializable> flattenedUpdatesMap = createFlattenedUpdatesMap(superRoot, copiedSuperRoot);
+
+            latest.putAll(flattenedUpdatesMap);
+
+            return new Data(flattenedUpdatesMap, lastRevision);
+        });
+    }
+
+
+    private Config readHoconFromFile() {
+        checkAndRestoreConfigFile();
+        return ConfigFactory.parseFile(configPath.toFile(), ConfigParseOptions.defaults().setAllowMissing(false));
+    }
+
+    @NotNull
+    private SuperRoot createSuperRoot() {
+        SuperRoot superRoot = new SuperRoot(rootCreator());
+        for (RootKey<?, ?> rootKey : rootKeys.values()) {
+            superRoot.addRoot(rootKey, createRootNode(rootKey));
+        }
+
+        return superRoot;
     }
 
     @Override
     public CompletableFuture<Map<String, ? extends Serializable>> readAllLatest(String prefix) {
-        lock.readLock().lock();
-        try {
-            checkAndRestoreConfigFile();
-            Map<String, Serializable> map = latest.entrySet()
-                    .stream()
-                    .filter(entry -> entry.getKey().startsWith(prefix))
-                    .collect(Collectors.toMap(Entry::getKey, Entry::getValue));
-            return CompletableFuture.completedFuture(map);
-        } finally {
-            lock.readLock().unlock();
-        }
+        return readLockAsync(() ->
+                latest.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getKey().startsWith(prefix))
+                        .collect(toMap(Entry::getKey, Entry::getValue))
+        );
     }
 
     @Override
     public CompletableFuture<Serializable> readLatest(String key) {
-        lock.readLock().lock();
-        try {
-            checkAndRestoreConfigFile();
-            return CompletableFuture.completedFuture(latest.get(key));
-        } finally {
-            lock.readLock().unlock();
-        }
+        return readLockAsync(() -> latest.get(key));
     }
 
     @Override
     public CompletableFuture<Boolean> write(Map<String, ? extends Serializable> newValues, long ver) {
-        lock.writeLock().lock();
-        try {
+        return writeLockAsync(() -> {
             if (ver != lastRevision) {
-                return CompletableFuture.completedFuture(false);
+                return false;
             }
-            checkAndRestoreConfigFile();
-            // TODO: https://issues.apache.org/jira/browse/IGNITE-19152
-            //saveValues(newValues);
-            latest.putAll(newValues);
-            lastRevision++;
+
+            mergeAndSave(newValues);
+
             runAsync(() -> lsnrRef.get().onEntriesChanged(new Data(newValues, lastRevision)));
-            return CompletableFuture.completedFuture(true);
-        } finally {
-            lock.writeLock().unlock();
-        }
+
+            return true;
+        });
+    }
+
+    private void mergeAndSave(Map<String, ? extends Serializable> newValues) {
+        updateLatestState(newValues);
+        saveConfigFile();
+        lastRevision++;
+    }
+
+    private void updateLatestState(Map<String, ? extends Serializable> newValues) {
+        newValues.forEach((key, value) -> {
+            if (value == null) { // null means that we should remove this entry
+                latest.remove(key);
+            } else {
+                latest.put(key, value);
+            }
+        });
     }
 
     private void runAsync(Runnable runnable) {
         CompletableFuture<Void> future = CompletableFuture.runAsync(runnable, threadPool);
-

Review Comment:
   Why did you remove this empty line? What's the reason?
   For longest time we have specific code convention, we use empty lines throughout the code, it gives Ignite's code certain look and feel. I would say that you don't follow it enough, but that's ok.
   What's not ok to me is simply removing such formatting from the code, that has nothing to do with your changes.



##########
modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorageTest.java:
##########
@@ -17,75 +17,401 @@
 
 package org.apache.ignite.internal.configuration.storage;
 
-import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.aMapWithSize;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.emptyString;
+import static org.hamcrest.Matchers.equalToCompressingWhiteSpace;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.hasValue;
 import static org.hamcrest.Matchers.is;
 
 import java.io.IOException;
+import java.io.Serializable;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.ConfigurationRoot;
+import org.apache.ignite.configuration.annotation.NamedConfigValue;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.internal.configuration.TestConfigurationChanger;
+import org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
-import org.junit.jupiter.api.Disabled;
+import org.hamcrest.Matchers;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.DisplayName;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
-/**
- * Tests for the {@link LocalFileConfigurationStorage}.
- */
+/** Test for local file configurations storage. */
 @ExtendWith(WorkDirectoryExtension.class)
-@Disabled("https://issues.apache.org/jira/browse/IGNITE-19152")
-public class LocalFileConfigurationStorageTest extends ConfigurationStorageTest {
+public class LocalFileConfigurationStorageTest {
 
     private static final String CONFIG_NAME = "ignite-config.conf";
 
+    private static ConfigurationAsmGenerator cgen;
+
     @WorkDirectory
     private Path tmpDir;
 
-    @Override
-    public ConfigurationStorage getStorage() {
-        return new LocalFileConfigurationStorage(getConfigFile());
+    /** Test configuration storage. */
+    private LocalFileConfigurationStorage storage;
+
+    /** Test configuration changer. */
+    private TestConfigurationChanger changer;
+
+    /** Instantiates {@link #cgen}. */
+    @BeforeAll
+    public static void beforeAll() {
+        cgen = new ConfigurationAsmGenerator();
+    }
+
+    /** Nullifies {@link #cgen} to prevent memory leak from having runtime ClassLoader accessible from GC root. */
+    @AfterAll
+    public static void afterAll() {
+        cgen = null;
+    }
+
+    private Path getConfigFile() {
+        return tmpDir.resolve(CONFIG_NAME);
+    }
+
+    @BeforeEach
+    void before() {
+        storage = new LocalFileConfigurationStorage(getConfigFile(), List.of(TopConfiguration.KEY));
+
+        changer = new TestConfigurationChanger(
+                cgen,
+                List.of(TopConfiguration.KEY),
+                Set.of(),
+                storage,
+                List.of(),
+                List.of()
+        );
+
+        changer.start();
+    }
+
+    @AfterEach
+    void after() {
+        changer.stop();
     }
 
     @Test
-    void testHocon() throws IOException {
-        // All of this is needed because write expects serializable values and only concrete classes are serializable
-        HashMap<String, ArrayList<String>> map = new HashMap<>(Map.of("list", new ArrayList<>(List.of("val1", "val2"))));
-        var data = Map.of("foo1", "bar1", "foo2", "bar2", "map", map);
-
-        assertThat(storage.write(data, 0), willBe(true));
-
-        String contents = Files.readString(getConfigFile());
-
-        // \n instead of System.lineSeparator because Config library writes \n only
-        assertThat(contents, is("foo1=bar1\n"
-                + "foo2=bar2\n"
-                + "map {\n"
-                + "    list=[\n"
-                + "        val1,\n"
-                + "        val2\n"
-                + "    ]\n"
-                + "}\n"));
+    @DisplayName("Default values are not added enriched on read when the config file is empty")
+    void empty() throws IOException {
+        // Given
+        assertThat(configFileContent(), emptyString());
+
+        // When
+        var storageValues = readAllLatest();
+
+        // Then storage data only contains top level defaults
+        assertThat(storageValues.entrySet(), hasSize(1));
     }
 
     @Test
-    void testMergeHocon() throws IOException {
-        var data = Map.of("foo1", "bar");
-        assertThat(storage.write(data, 0), willBe(true));
+    @DisplayName("Named list entities can be added")
+    void add() throws Exception {
+        // Given
+        assertThat(configFileContent(), emptyString());
+        // And
+        var topConfiguration = (TopConfiguration) cgen.instantiateCfg(TopConfiguration.KEY, changer);
+        topConfiguration.namedList().change(b -> b.create("name1", x -> {
+            x.changeStrVal("strVal1");
+            x.changeIntVal(-1);
+        })).get();
+
+        // When
+        var storageValues = readAllLatest();
 
-        var append = Map.of("foo1", "baz", "foo2", "bar");
-        assertThat(storage.write(append, 1), willBe(true));
+        // Then
+        assertThat(storageValues, allOf(aMapWithSize(6), hasValue(-1)));

Review Comment:
   Please add a comment, what are the other 4 entries in the map. There are asserts only for 2 of them, this is not trivial



##########
modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorageTest.java:
##########
@@ -17,75 +17,401 @@
 
 package org.apache.ignite.internal.configuration.storage;
 
-import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.aMapWithSize;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.emptyString;
+import static org.hamcrest.Matchers.equalToCompressingWhiteSpace;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.hasValue;
 import static org.hamcrest.Matchers.is;
 
 import java.io.IOException;
+import java.io.Serializable;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.ConfigurationRoot;
+import org.apache.ignite.configuration.annotation.NamedConfigValue;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.internal.configuration.TestConfigurationChanger;
+import org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
-import org.junit.jupiter.api.Disabled;
+import org.hamcrest.Matchers;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.DisplayName;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
-/**
- * Tests for the {@link LocalFileConfigurationStorage}.
- */
+/** Test for local file configurations storage. */
 @ExtendWith(WorkDirectoryExtension.class)
-@Disabled("https://issues.apache.org/jira/browse/IGNITE-19152")
-public class LocalFileConfigurationStorageTest extends ConfigurationStorageTest {
+public class LocalFileConfigurationStorageTest {
 
     private static final String CONFIG_NAME = "ignite-config.conf";
 
+    private static ConfigurationAsmGenerator cgen;
+
     @WorkDirectory
     private Path tmpDir;
 
-    @Override
-    public ConfigurationStorage getStorage() {
-        return new LocalFileConfigurationStorage(getConfigFile());
+    /** Test configuration storage. */
+    private LocalFileConfigurationStorage storage;
+
+    /** Test configuration changer. */
+    private TestConfigurationChanger changer;
+
+    /** Instantiates {@link #cgen}. */
+    @BeforeAll
+    public static void beforeAll() {
+        cgen = new ConfigurationAsmGenerator();
+    }
+
+    /** Nullifies {@link #cgen} to prevent memory leak from having runtime ClassLoader accessible from GC root. */
+    @AfterAll
+    public static void afterAll() {
+        cgen = null;
+    }
+
+    private Path getConfigFile() {
+        return tmpDir.resolve(CONFIG_NAME);
+    }
+
+    @BeforeEach
+    void before() {
+        storage = new LocalFileConfigurationStorage(getConfigFile(), List.of(TopConfiguration.KEY));
+
+        changer = new TestConfigurationChanger(
+                cgen,
+                List.of(TopConfiguration.KEY),
+                Set.of(),
+                storage,
+                List.of(),
+                List.of()
+        );
+
+        changer.start();
+    }
+
+    @AfterEach
+    void after() {
+        changer.stop();
     }
 
     @Test
-    void testHocon() throws IOException {
-        // All of this is needed because write expects serializable values and only concrete classes are serializable
-        HashMap<String, ArrayList<String>> map = new HashMap<>(Map.of("list", new ArrayList<>(List.of("val1", "val2"))));
-        var data = Map.of("foo1", "bar1", "foo2", "bar2", "map", map);
-
-        assertThat(storage.write(data, 0), willBe(true));
-
-        String contents = Files.readString(getConfigFile());
-
-        // \n instead of System.lineSeparator because Config library writes \n only
-        assertThat(contents, is("foo1=bar1\n"
-                + "foo2=bar2\n"
-                + "map {\n"
-                + "    list=[\n"
-                + "        val1,\n"
-                + "        val2\n"
-                + "    ]\n"
-                + "}\n"));
+    @DisplayName("Default values are not added enriched on read when the config file is empty")
+    void empty() throws IOException {
+        // Given
+        assertThat(configFileContent(), emptyString());
+
+        // When
+        var storageValues = readAllLatest();
+
+        // Then storage data only contains top level defaults
+        assertThat(storageValues.entrySet(), hasSize(1));
     }
 
     @Test
-    void testMergeHocon() throws IOException {
-        var data = Map.of("foo1", "bar");
-        assertThat(storage.write(data, 0), willBe(true));
+    @DisplayName("Named list entities can be added")
+    void add() throws Exception {
+        // Given
+        assertThat(configFileContent(), emptyString());
+        // And
+        var topConfiguration = (TopConfiguration) cgen.instantiateCfg(TopConfiguration.KEY, changer);
+        topConfiguration.namedList().change(b -> b.create("name1", x -> {
+            x.changeStrVal("strVal1");
+            x.changeIntVal(-1);
+        })).get();
+
+        // When
+        var storageValues = readAllLatest();
 
-        var append = Map.of("foo1", "baz", "foo2", "bar");
-        assertThat(storage.write(append, 1), willBe(true));
+        // Then
+        assertThat(storageValues, allOf(aMapWithSize(6), hasValue(-1)));
+        assertThat(storageValues, allOf(aMapWithSize(6), hasValue("strVal1")));
+        // And
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[\n"
+                        + "        {\n"
+                        + "            intVal=-1\n"
+                        + "            name=name1\n"
+                        + "            strVal=strVal1\n"
+                        + "        }\n"

Review Comment:
   Can we have a TODO that would tell us to fix the rendering syntax for named lists?
   Also, I would expect (in the future) the following (when there's only a single property to the list entry):
   ```
   namedList {
       foo.attribute = a
       bar.attribute = b
   }
   ```
   instead of
   ```
   namedList {
       foo {
           attribute = a
       }
       ...
   }
   ```
   Maybe we would require a custom rendering code. This topic must be discussed with somebody. Syntax matters, it's a user-facing part of Ignite.



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