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

[GitHub] [ignite-3] PakhomovAlexander opened a new pull request, #1929: IGNITE-19152 Use schema information in LocalFileConfigurationStorage

PakhomovAlexander opened a new pull request, #1929:
URL: https://github.com/apache/ignite-3/pull/1929

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


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

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

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


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

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1172222071


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/ItSslConfigurationValidationTest.java:
##########
@@ -34,6 +35,7 @@
  * Integration test for checking SSL configuration validation.
  */
 @ExtendWith(WorkDirectoryExtension.class)
+@Disabled("https://issues.apache.org/jira/browse/IGNITE-19315")

Review Comment:
   Sure, it has "Critical" priority. Will work on it asap.



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

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

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


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

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1166648094


##########
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:
   Good point, I've just applied the same approach as we do in `ConfigurationManager`. Do you think we need to use `ConfigurationModule` there as well?



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

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

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


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

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1173870250


##########
modules/configuration/src/main/java/org/apache/ignite/internal/configuration/util/ConfigurationFlattener.java:
##########
@@ -136,6 +136,8 @@ public Void doVisitInnerNode(String key, InnerNode newNode) {
                 return null;
             }
 
+            // in case inner node is null in both trees,

Review Comment:
   I think I mentioned guidelines somewhere, but I'll repeat. First letter is capitalizes, sentence ends with a dot. This is how we should write comments.



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -147,49 +139,40 @@ private Config readHoconFromFile() {
 
     @Override
     public CompletableFuture<Map<String, ? extends Serializable>> readAllLatest(String prefix) {
-        return async(() -> {
-            lock.readLock().lock();
-            try {
-                return latest.entrySet()
-                        .stream()
-                        .filter(entry -> entry.getKey().startsWith(prefix))
-                        .collect(toMap(Entry::getKey, Entry::getValue));
-            } finally {
-                lock.readLock().unlock();
-            }
-        });
+        lock.readLock().lock();
+        try {
+            return CompletableFuture.completedFuture(
+                    latest.entrySet()
+                            .stream()
+                            .filter(entry -> entry.getKey().startsWith(prefix))
+                            .collect(toMap(Entry::getKey, Entry::getValue))
+            );
+        } finally {
+            lock.readLock().unlock();
+        }
     }
 
     @Override
     public CompletableFuture<Serializable> readLatest(String key) {
-        return async(() -> {
-            lock.readLock().lock();
-            try {
-                return latest.get(key);
-            } finally {
-                lock.readLock().unlock();
-            }
-        });
+        return CompletableFuture.completedFuture(latest.get(key));

Review Comment:
   Why have you removed the read-lock? Technically, you lost the property of the data being "latest". By that I mean that two consecutive calls can return newer value first, and older value next, if you ask them not in the order that they are being inserted in "write" at the same time. Read lock would save you from such problems.
   Please be careful in such places and don't forget about broader invariants.



##########
modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorageTest.java:
##########
@@ -413,10 +411,12 @@ void deleteFileBeforeReadAll() throws Exception {
     /** Read configuration when inner node configured with partial content (some fields are empty). */
     @Test
     void innerNodeWithPartialContent() throws Exception {
+        // Given
         String content = "top: { inner.boolVal: true }";
         Files.write(getConfigFile(), content.getBytes(StandardCharsets.UTF_8));
 
-        storage.readDataOnRecovery().get();
+        // Expect
+        assertThat(storage.readDataOnRecovery().get().values(), allOf(aMapWithSize(1)));

Review Comment:
   Any reason to have "allOf" for a single matcher?



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

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

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


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

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1168449309


##########
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:
   Sure I did.



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

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

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


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

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1173869063


##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -56,99 +66,146 @@
 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 tree generator. */
+    private final ConfigurationTreeGenerator generator;
+
+    /** 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 notificationsThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates. */
+    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.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param generator Configuration tree generator.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, ConfigurationTreeGenerator generator) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.generator = generator;
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
         checkAndRestoreConfigFile();
     }
 
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return async(() -> {
+            lock.writeLock().lock();
+            try {
+                SuperRoot superRoot = generator.createSuperRoot();
+                SuperRoot copiedSuperRoot = superRoot.copy();
+
+                Config hocon = readHoconFromFile();
+                HoconConverter.hoconSource(hocon.root()).descend(copiedSuperRoot);
+
+                Map<String, Serializable> flattenedUpdatesMap = createFlattenedUpdatesMap(superRoot, copiedSuperRoot);
+                flattenedUpdatesMap.forEach((key, value) -> {
+                    if (value != null) { // filter defaults
+                        latest.put(key, value);
+                    }
+                });
+
+                return new Data(flattenedUpdatesMap, lastRevision);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        });
+    }
+
+    private Config readHoconFromFile() {
+        checkAndRestoreConfigFile();
+
+        return ConfigFactory.parseFile(configPath.toFile(), ConfigParseOptions.defaults().setAllowMissing(false));
     }
 
     @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 async(() -> {
+            lock.readLock().lock();
+            try {
+                return latest.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getKey().startsWith(prefix))
+                        .collect(toMap(Entry::getKey, Entry::getValue));
+            } finally {
+                lock.readLock().unlock();
+            }
+        });
     }
 
     @Override
     public CompletableFuture<Serializable> readLatest(String key) {
-        lock.readLock().lock();
-        try {
-            checkAndRestoreConfigFile();
-            return CompletableFuture.completedFuture(latest.get(key));
-        } finally {
-            lock.readLock().unlock();
-        }
+        return async(() -> {
+            lock.readLock().lock();
+            try {
+                return latest.get(key);
+            } finally {
+                lock.readLock().unlock();
+            }
+        });
     }
 
     @Override
     public CompletableFuture<Boolean> write(Map<String, ? extends Serializable> newValues, long ver) {
-        lock.writeLock().lock();
-        try {
-            if (ver != lastRevision) {
-                return CompletableFuture.completedFuture(false);
+        return async(() -> {
+            lock.writeLock().lock();
+            try {
+                if (ver != lastRevision) {
+                    return false;
+                }
+
+                mergeAndSave(newValues);
+
+                sendNotificationAsync(new Data(newValues, lastRevision));
+
+                return true;
+            } finally {
+                lock.writeLock().unlock();
             }
-            checkAndRestoreConfigFile();
-            // TODO: https://issues.apache.org/jira/browse/IGNITE-19152
-            //saveValues(newValues);
-            latest.putAll(newValues);
-            lastRevision++;
-            runAsync(() -> lsnrRef.get().onEntriesChanged(new Data(newValues, lastRevision)));
-            return CompletableFuture.completedFuture(true);
-        } finally {
-            lock.writeLock().unlock();
-        }
+        });
     }
 
-    private void runAsync(Runnable runnable) {
-        CompletableFuture<Void> future = CompletableFuture.runAsync(runnable, threadPool);
+    private void mergeAndSave(Map<String, ? extends Serializable> newValues) {
+        updateLatestState(newValues);
+        saveConfigFile();
+        lastRevision++;

Review Comment:
   Ok, thank you, this means that I'm stupid, there are no problems here.



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

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

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


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

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1172396136


##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -56,99 +66,146 @@
 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 tree generator. */
+    private final ConfigurationTreeGenerator generator;
+
+    /** 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 notificationsThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates. */
+    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.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param generator Configuration tree generator.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, ConfigurationTreeGenerator generator) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.generator = generator;
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
         checkAndRestoreConfigFile();
     }
 
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return async(() -> {
+            lock.writeLock().lock();
+            try {
+                SuperRoot superRoot = generator.createSuperRoot();
+                SuperRoot copiedSuperRoot = superRoot.copy();
+
+                Config hocon = readHoconFromFile();
+                HoconConverter.hoconSource(hocon.root()).descend(copiedSuperRoot);
+
+                Map<String, Serializable> flattenedUpdatesMap = createFlattenedUpdatesMap(superRoot, copiedSuperRoot);
+                flattenedUpdatesMap.forEach((key, value) -> {
+                    if (value != null) { // filter defaults
+                        latest.put(key, value);
+                    }
+                });
+
+                return new Data(flattenedUpdatesMap, lastRevision);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        });
+    }
+
+    private Config readHoconFromFile() {
+        checkAndRestoreConfigFile();
+
+        return ConfigFactory.parseFile(configPath.toFile(), ConfigParseOptions.defaults().setAllowMissing(false));
     }
 
     @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 async(() -> {
+            lock.readLock().lock();
+            try {
+                return latest.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getKey().startsWith(prefix))
+                        .collect(toMap(Entry::getKey, Entry::getValue));
+            } finally {
+                lock.readLock().unlock();
+            }
+        });
     }
 
     @Override
     public CompletableFuture<Serializable> readLatest(String key) {
-        lock.readLock().lock();
-        try {
-            checkAndRestoreConfigFile();
-            return CompletableFuture.completedFuture(latest.get(key));
-        } finally {
-            lock.readLock().unlock();
-        }
+        return async(() -> {
+            lock.readLock().lock();
+            try {
+                return latest.get(key);
+            } finally {
+                lock.readLock().unlock();
+            }
+        });
     }
 
     @Override
     public CompletableFuture<Boolean> write(Map<String, ? extends Serializable> newValues, long ver) {
-        lock.writeLock().lock();
-        try {
-            if (ver != lastRevision) {
-                return CompletableFuture.completedFuture(false);
+        return async(() -> {
+            lock.writeLock().lock();
+            try {
+                if (ver != lastRevision) {
+                    return false;
+                }
+
+                mergeAndSave(newValues);
+
+                sendNotificationAsync(new Data(newValues, lastRevision));
+
+                return true;
+            } finally {
+                lock.writeLock().unlock();
             }
-            checkAndRestoreConfigFile();
-            // TODO: https://issues.apache.org/jira/browse/IGNITE-19152
-            //saveValues(newValues);
-            latest.putAll(newValues);
-            lastRevision++;
-            runAsync(() -> lsnrRef.get().onEntriesChanged(new Data(newValues, lastRevision)));
-            return CompletableFuture.completedFuture(true);
-        } finally {
-            lock.writeLock().unlock();
-        }
+        });
     }
 
-    private void runAsync(Runnable runnable) {
-        CompletableFuture<Void> future = CompletableFuture.runAsync(runnable, threadPool);
+    private void mergeAndSave(Map<String, ? extends Serializable> newValues) {
+        updateLatestState(newValues);
+        saveConfigFile();
+        lastRevision++;
+    }
 
-        futureTracker.registerFuture(future);
+    private void updateLatestState(Map<String, ? extends Serializable> newValues) {
+        newValues.forEach((key, value) -> {
+            if (value == null) { // null means that we should remove this entry

Review Comment:
   Sorry, did not mentioned that it is your suggestion (though it is my code).



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

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

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


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

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1173868324


##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -56,99 +66,146 @@
 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 tree generator. */
+    private final ConfigurationTreeGenerator generator;
+
+    /** 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 notificationsThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates. */
+    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.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param generator Configuration tree generator.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, ConfigurationTreeGenerator generator) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.generator = generator;
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
         checkAndRestoreConfigFile();
     }
 
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return async(() -> {

Review Comment:
   Reason for using the future is that there are multiple implementations. Some of them cannot physically be synchronous, otherwise they would block thread for unknown period of time



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

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

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


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

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1170024228


##########
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:
   https://issues.apache.org/jira/browse/IGNITE-19303



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

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

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


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

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1172179670


##########
modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationManager.java:
##########
@@ -40,6 +31,30 @@ public class ConfigurationManager implements IgniteComponent {
     /** Configuration registry. */
     private final ConfigurationRegistry registry;
 
+    /**
+     * Constructor.
+     *
+     * @param rootKeys                    Configuration root keys.
+     * @param validators                  Validators.
+     * @param storage                     Configuration storage.
+     * @param generator                   Configuration tree generator.
+     * @throws IllegalArgumentException If the configuration type of the root keys is not equal to the storage type, or if the schema or its
+     *                                  extensions are not valid.
+     */
+    public ConfigurationManager(

Review Comment:
   Is there a reason to leave the old constructor in this class? Seems confusing now, but I don't know some nuances probably



##########
modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationTreeGenerator.java:
##########
@@ -0,0 +1,281 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.configuration;
+
+
+import static java.util.function.Function.identity;
+import static java.util.function.Predicate.not;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.collectSchemas;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.internalSchemaExtensions;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isPolymorphicId;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.polymorphicInstanceId;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.polymorphicSchemaExtensions;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.schemaFields;
+import static org.apache.ignite.internal.util.CollectionUtils.difference;
+import static org.apache.ignite.internal.util.CollectionUtils.viewReadOnly;
+
+import java.lang.reflect.Field;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import org.apache.ignite.configuration.RootKey;
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.ConfigurationRoot;
+import org.apache.ignite.configuration.annotation.InternalConfiguration;
+import org.apache.ignite.configuration.annotation.PolymorphicConfigInstance;
+import org.apache.ignite.configuration.annotation.PolymorphicId;
+import org.apache.ignite.internal.close.ManuallyCloseable;
+import org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator;
+import org.apache.ignite.internal.configuration.tree.InnerNode;
+import org.apache.ignite.internal.configuration.util.ConfigurationUtil;
+import org.jetbrains.annotations.Nullable;
+
+/** Schema-aware configuration generator. */
+public class ConfigurationTreeGenerator implements ManuallyCloseable {
+
+    private final Map<String, RootKey<?, ?>> rootKeys;
+
+    @Nullable
+    private ConfigurationAsmGenerator generator = new ConfigurationAsmGenerator();
+
+    /**
+     * Constructor that takes a collection of root keys. Internal and polymorphic schema extensions are empty by default.
+     *
+     * @param rootKeys Root keys.
+     */
+    public ConfigurationTreeGenerator(Collection<RootKey<?, ?>> rootKeys) {
+        this(rootKeys, Set.of(), Set.of());
+    }

Review Comment:
   Is this for tests? Mark it as `@TestOnly` then



##########
modules/configuration/src/main/java/org/apache/ignite/internal/configuration/util/ConfigurationFlattener.java:
##########
@@ -136,6 +136,10 @@ public Void doVisitInnerNode(String key, InnerNode newNode) {
                 return null;
             }
 
+            if (oldNode == null && newNode == null) {

Review Comment:
   Can you please add a comment, explaining how it's possible?



##########
modules/configuration/src/test/java/org/apache/ignite/internal/configuration/ConfigurationChangerTest.java:
##########
@@ -113,13 +112,13 @@ public static class ThirdConfigurationSchema {
         public String strCfg;
     }
 
-    private static ConfigurationAsmGenerator cgen = new ConfigurationAsmGenerator();
+    private static ConfigurationTreeGenerator generator = new ConfigurationTreeGenerator(List.of(KEY, DefaultsConfiguration.KEY));
 
     private final TestConfigurationStorage storage = new TestConfigurationStorage(LOCAL);
 
     @AfterAll
-    public static void afterAll() {
-        cgen = null;
+    public static void afterAll() throws Exception {

Review Comment:
   Does it have to throw an exception? This "close" method in generator simply nullifies the field, there's no way it ends with exception.



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/ItSslConfigurationValidationTest.java:
##########
@@ -34,6 +35,7 @@
  * Integration test for checking SSL configuration validation.
  */
 @ExtendWith(WorkDirectoryExtension.class)
+@Disabled("https://issues.apache.org/jira/browse/IGNITE-19315")

Review Comment:
   Too bad, I expected us to preserve this function.
   Do we prioritize this new feature as the next one that you'll be working on?



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -56,99 +66,146 @@
 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 tree generator. */
+    private final ConfigurationTreeGenerator generator;
+
+    /** 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 notificationsThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates. */
+    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.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param generator Configuration tree generator.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, ConfigurationTreeGenerator generator) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.generator = generator;
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
         checkAndRestoreConfigFile();
     }
 
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return async(() -> {
+            lock.writeLock().lock();
+            try {
+                SuperRoot superRoot = generator.createSuperRoot();
+                SuperRoot copiedSuperRoot = superRoot.copy();
+
+                Config hocon = readHoconFromFile();
+                HoconConverter.hoconSource(hocon.root()).descend(copiedSuperRoot);
+
+                Map<String, Serializable> flattenedUpdatesMap = createFlattenedUpdatesMap(superRoot, copiedSuperRoot);
+                flattenedUpdatesMap.forEach((key, value) -> {
+                    if (value != null) { // filter defaults
+                        latest.put(key, value);
+                    }
+                });
+
+                return new Data(flattenedUpdatesMap, lastRevision);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        });
+    }
+
+    private Config readHoconFromFile() {
+        checkAndRestoreConfigFile();
+
+        return ConfigFactory.parseFile(configPath.toFile(), ConfigParseOptions.defaults().setAllowMissing(false));
     }
 
     @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 async(() -> {

Review Comment:
   Again, I don't get it. We read data from the field. Why do we have to do it in a separate pool? Makes no sense to me, please add a comment or revert this change.



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -56,99 +66,146 @@
 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 tree generator. */
+    private final ConfigurationTreeGenerator generator;
+
+    /** 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 notificationsThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates. */
+    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.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param generator Configuration tree generator.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, ConfigurationTreeGenerator generator) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.generator = generator;
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
         checkAndRestoreConfigFile();
     }
 
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return async(() -> {
+            lock.writeLock().lock();
+            try {
+                SuperRoot superRoot = generator.createSuperRoot();
+                SuperRoot copiedSuperRoot = superRoot.copy();
+
+                Config hocon = readHoconFromFile();
+                HoconConverter.hoconSource(hocon.root()).descend(copiedSuperRoot);
+
+                Map<String, Serializable> flattenedUpdatesMap = createFlattenedUpdatesMap(superRoot, copiedSuperRoot);
+                flattenedUpdatesMap.forEach((key, value) -> {
+                    if (value != null) { // filter defaults

Review Comment:
   ```suggestion
                       if (value != null) { // Filter defaults.
   ```



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -175,56 +232,65 @@ public CompletableFuture<Void> writeConfigurationRevision(long prevRevision, lon
 
     @Override
     public void close() {
-        IgniteUtils.shutdownAndAwaitTermination(threadPool, 10, TimeUnit.SECONDS);
-
         futureTracker.cancelInFlightFutures();
+
+        IgniteUtils.shutdownAndAwaitTermination(workerThreadPool, 10, TimeUnit.SECONDS);
+        IgniteUtils.shutdownAndAwaitTermination(notificationsThreadPool, 10, TimeUnit.SECONDS);
     }
 
-    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 = generator.createSuperRoot();
+
+        fillFromPrefixMap(rootNode, toPrefixMap(latest));
+
+        Object transformed = rootNode.accept(null, new ConverterToMapVisitor(false, true));
+
+        ConfigValue conf = ConfigImpl.fromAnyRef(
+                transformed, null
+        );

Review Comment:
   Could be a single line I guess



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -56,99 +66,146 @@
 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 tree generator. */
+    private final ConfigurationTreeGenerator generator;
+
+    /** 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 notificationsThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates. */
+    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.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param generator Configuration tree generator.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, ConfigurationTreeGenerator generator) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.generator = generator;
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
         checkAndRestoreConfigFile();
     }
 
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return async(() -> {
+            lock.writeLock().lock();
+            try {
+                SuperRoot superRoot = generator.createSuperRoot();
+                SuperRoot copiedSuperRoot = superRoot.copy();
+
+                Config hocon = readHoconFromFile();
+                HoconConverter.hoconSource(hocon.root()).descend(copiedSuperRoot);
+
+                Map<String, Serializable> flattenedUpdatesMap = createFlattenedUpdatesMap(superRoot, copiedSuperRoot);
+                flattenedUpdatesMap.forEach((key, value) -> {
+                    if (value != null) { // filter defaults
+                        latest.put(key, value);
+                    }
+                });
+
+                return new Data(flattenedUpdatesMap, lastRevision);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        });
+    }
+
+    private Config readHoconFromFile() {
+        checkAndRestoreConfigFile();
+
+        return ConfigFactory.parseFile(configPath.toFile(), ConfigParseOptions.defaults().setAllowMissing(false));
     }
 
     @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 async(() -> {
+            lock.readLock().lock();
+            try {
+                return latest.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getKey().startsWith(prefix))
+                        .collect(toMap(Entry::getKey, Entry::getValue));
+            } finally {
+                lock.readLock().unlock();
+            }
+        });
     }
 
     @Override
     public CompletableFuture<Serializable> readLatest(String key) {
-        lock.readLock().lock();
-        try {
-            checkAndRestoreConfigFile();
-            return CompletableFuture.completedFuture(latest.get(key));
-        } finally {
-            lock.readLock().unlock();
-        }
+        return async(() -> {

Review Comment:
   Same here. Literally "Map#get"



##########
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:
   Just in case - I don't mind custom HOCON renderer. This format is kind-of trivial, and converting tree to string would only require a few days of coding, while there are multiple advantages, like the ability to inset comments or to preserve properties order.



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -56,99 +66,146 @@
 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 tree generator. */
+    private final ConfigurationTreeGenerator generator;
+
+    /** 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 notificationsThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates. */
+    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.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param generator Configuration tree generator.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, ConfigurationTreeGenerator generator) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.generator = generator;
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
         checkAndRestoreConfigFile();
     }
 
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return async(() -> {
+            lock.writeLock().lock();
+            try {
+                SuperRoot superRoot = generator.createSuperRoot();
+                SuperRoot copiedSuperRoot = superRoot.copy();
+
+                Config hocon = readHoconFromFile();
+                HoconConverter.hoconSource(hocon.root()).descend(copiedSuperRoot);
+
+                Map<String, Serializable> flattenedUpdatesMap = createFlattenedUpdatesMap(superRoot, copiedSuperRoot);
+                flattenedUpdatesMap.forEach((key, value) -> {
+                    if (value != null) { // filter defaults
+                        latest.put(key, value);
+                    }
+                });
+
+                return new Data(flattenedUpdatesMap, lastRevision);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        });
+    }
+
+    private Config readHoconFromFile() {
+        checkAndRestoreConfigFile();
+
+        return ConfigFactory.parseFile(configPath.toFile(), ConfigParseOptions.defaults().setAllowMissing(false));
     }
 
     @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 async(() -> {
+            lock.readLock().lock();
+            try {
+                return latest.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getKey().startsWith(prefix))
+                        .collect(toMap(Entry::getKey, Entry::getValue));
+            } finally {
+                lock.readLock().unlock();
+            }
+        });
     }
 
     @Override
     public CompletableFuture<Serializable> readLatest(String key) {
-        lock.readLock().lock();
-        try {
-            checkAndRestoreConfigFile();
-            return CompletableFuture.completedFuture(latest.get(key));
-        } finally {
-            lock.readLock().unlock();
-        }
+        return async(() -> {
+            lock.readLock().lock();
+            try {
+                return latest.get(key);
+            } finally {
+                lock.readLock().unlock();
+            }
+        });
     }
 
     @Override
     public CompletableFuture<Boolean> write(Map<String, ? extends Serializable> newValues, long ver) {
-        lock.writeLock().lock();
-        try {
-            if (ver != lastRevision) {
-                return CompletableFuture.completedFuture(false);
+        return async(() -> {
+            lock.writeLock().lock();
+            try {
+                if (ver != lastRevision) {
+                    return false;
+                }
+
+                mergeAndSave(newValues);
+
+                sendNotificationAsync(new Data(newValues, lastRevision));
+
+                return true;
+            } finally {
+                lock.writeLock().unlock();
             }
-            checkAndRestoreConfigFile();
-            // TODO: https://issues.apache.org/jira/browse/IGNITE-19152
-            //saveValues(newValues);
-            latest.putAll(newValues);
-            lastRevision++;
-            runAsync(() -> lsnrRef.get().onEntriesChanged(new Data(newValues, lastRevision)));
-            return CompletableFuture.completedFuture(true);
-        } finally {
-            lock.writeLock().unlock();
-        }
+        });
     }
 
-    private void runAsync(Runnable runnable) {
-        CompletableFuture<Void> future = CompletableFuture.runAsync(runnable, threadPool);
+    private void mergeAndSave(Map<String, ? extends Serializable> newValues) {
+        updateLatestState(newValues);
+        saveConfigFile();
+        lastRevision++;
+    }
 
-        futureTracker.registerFuture(future);
+    private void updateLatestState(Map<String, ? extends Serializable> newValues) {
+        newValues.forEach((key, value) -> {
+            if (value == null) { // null means that we should remove this entry

Review Comment:
   Please don't ignore coding conventions :)



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -56,99 +66,146 @@
 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 tree generator. */
+    private final ConfigurationTreeGenerator generator;
+
+    /** 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 notificationsThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates. */
+    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.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param generator Configuration tree generator.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, ConfigurationTreeGenerator generator) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.generator = generator;
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
         checkAndRestoreConfigFile();
     }
 
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return async(() -> {
+            lock.writeLock().lock();
+            try {
+                SuperRoot superRoot = generator.createSuperRoot();
+                SuperRoot copiedSuperRoot = superRoot.copy();
+
+                Config hocon = readHoconFromFile();
+                HoconConverter.hoconSource(hocon.root()).descend(copiedSuperRoot);
+
+                Map<String, Serializable> flattenedUpdatesMap = createFlattenedUpdatesMap(superRoot, copiedSuperRoot);
+                flattenedUpdatesMap.forEach((key, value) -> {
+                    if (value != null) { // filter defaults
+                        latest.put(key, value);
+                    }
+                });
+
+                return new Data(flattenedUpdatesMap, lastRevision);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        });
+    }
+
+    private Config readHoconFromFile() {
+        checkAndRestoreConfigFile();
+
+        return ConfigFactory.parseFile(configPath.toFile(), ConfigParseOptions.defaults().setAllowMissing(false));
     }
 
     @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 async(() -> {
+            lock.readLock().lock();
+            try {
+                return latest.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getKey().startsWith(prefix))
+                        .collect(toMap(Entry::getKey, Entry::getValue));
+            } finally {
+                lock.readLock().unlock();
+            }
+        });
     }
 
     @Override
     public CompletableFuture<Serializable> readLatest(String key) {
-        lock.readLock().lock();
-        try {
-            checkAndRestoreConfigFile();
-            return CompletableFuture.completedFuture(latest.get(key));
-        } finally {
-            lock.readLock().unlock();
-        }
+        return async(() -> {
+            lock.readLock().lock();
+            try {
+                return latest.get(key);
+            } finally {
+                lock.readLock().unlock();
+            }
+        });
     }
 
     @Override
     public CompletableFuture<Boolean> write(Map<String, ? extends Serializable> newValues, long ver) {
-        lock.writeLock().lock();
-        try {
-            if (ver != lastRevision) {
-                return CompletableFuture.completedFuture(false);
+        return async(() -> {
+            lock.writeLock().lock();
+            try {
+                if (ver != lastRevision) {
+                    return false;
+                }
+
+                mergeAndSave(newValues);
+
+                sendNotificationAsync(new Data(newValues, lastRevision));
+
+                return true;
+            } finally {
+                lock.writeLock().unlock();
             }
-            checkAndRestoreConfigFile();
-            // TODO: https://issues.apache.org/jira/browse/IGNITE-19152
-            //saveValues(newValues);
-            latest.putAll(newValues);
-            lastRevision++;
-            runAsync(() -> lsnrRef.get().onEntriesChanged(new Data(newValues, lastRevision)));
-            return CompletableFuture.completedFuture(true);
-        } finally {
-            lock.writeLock().unlock();
-        }
+        });
     }
 
-    private void runAsync(Runnable runnable) {
-        CompletableFuture<Void> future = CompletableFuture.runAsync(runnable, threadPool);
+    private void mergeAndSave(Map<String, ? extends Serializable> newValues) {
+        updateLatestState(newValues);
+        saveConfigFile();
+        lastRevision++;

Review Comment:
   New revision has been given to you as a parameter of "write" method. Why do you ignore it and increment the old revision instead? Please don't do that.



##########
modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationRegistry.java:
##########
@@ -96,8 +83,11 @@ public class ConfigurationRegistry implements IgniteComponent, ConfigurationStor
     /** Configuration change handler. */
     private final ConfigurationChanger changer;
 
-    /** Configuration generator. */
-    private final ConfigurationAsmGenerator cgen = new ConfigurationAsmGenerator();
+    /** Runtime implementations generator for node classes. */
+    private final ConfigurationTreeGenerator generator;
+
+    /** Flag that indicates if the {@link ConfigurationTreeGenerator} instance is owned by this object or not. */
+    private boolean ownConfigTreeGenerator = false;

Review Comment:
   Why does it matter?
   I believe that generator can always be a constructor parameter, and another component, that instantiated it, should also close it in the end



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -56,99 +66,146 @@
 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 tree generator. */
+    private final ConfigurationTreeGenerator generator;
+
+    /** 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 notificationsThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates. */
+    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.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param generator Configuration tree generator.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, ConfigurationTreeGenerator generator) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.generator = generator;
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
         checkAndRestoreConfigFile();
     }
 
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return async(() -> {

Review Comment:
   What's the deal with doing it asynchronously? It's a node start routine, we can do it in current thread I guess?
   Please comment



##########
modules/configuration/src/test/java/org/apache/ignite/internal/configuration/ConfigurationChangerTest.java:
##########
@@ -113,13 +112,13 @@ public static class ThirdConfigurationSchema {
         public String strCfg;
     }
 
-    private static ConfigurationAsmGenerator cgen = new ConfigurationAsmGenerator();
+    private static ConfigurationTreeGenerator generator = new ConfigurationTreeGenerator(List.of(KEY, DefaultsConfiguration.KEY));

Review Comment:
   Maybe this test-only constructor should use vararg, for convenience



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -56,99 +66,146 @@
 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 tree generator. */
+    private final ConfigurationTreeGenerator generator;
+
+    /** 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 notificationsThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates. */
+    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.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param generator Configuration tree generator.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, ConfigurationTreeGenerator generator) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.generator = generator;
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
         checkAndRestoreConfigFile();
     }
 
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return async(() -> {
+            lock.writeLock().lock();
+            try {
+                SuperRoot superRoot = generator.createSuperRoot();
+                SuperRoot copiedSuperRoot = superRoot.copy();
+
+                Config hocon = readHoconFromFile();
+                HoconConverter.hoconSource(hocon.root()).descend(copiedSuperRoot);
+
+                Map<String, Serializable> flattenedUpdatesMap = createFlattenedUpdatesMap(superRoot, copiedSuperRoot);
+                flattenedUpdatesMap.forEach((key, value) -> {
+                    if (value != null) { // filter defaults
+                        latest.put(key, value);
+                    }
+                });
+
+                return new Data(flattenedUpdatesMap, lastRevision);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        });
+    }
+
+    private Config readHoconFromFile() {
+        checkAndRestoreConfigFile();
+
+        return ConfigFactory.parseFile(configPath.toFile(), ConfigParseOptions.defaults().setAllowMissing(false));
     }
 
     @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 async(() -> {
+            lock.readLock().lock();
+            try {
+                return latest.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getKey().startsWith(prefix))
+                        .collect(toMap(Entry::getKey, Entry::getValue));
+            } finally {
+                lock.readLock().unlock();
+            }
+        });
     }
 
     @Override
     public CompletableFuture<Serializable> readLatest(String key) {
-        lock.readLock().lock();
-        try {
-            checkAndRestoreConfigFile();
-            return CompletableFuture.completedFuture(latest.get(key));
-        } finally {
-            lock.readLock().unlock();
-        }
+        return async(() -> {
+            lock.readLock().lock();
+            try {
+                return latest.get(key);
+            } finally {
+                lock.readLock().unlock();
+            }
+        });
     }
 
     @Override
     public CompletableFuture<Boolean> write(Map<String, ? extends Serializable> newValues, long ver) {
-        lock.writeLock().lock();
-        try {
-            if (ver != lastRevision) {
-                return CompletableFuture.completedFuture(false);
+        return async(() -> {
+            lock.writeLock().lock();
+            try {
+                if (ver != lastRevision) {
+                    return false;
+                }
+
+                mergeAndSave(newValues);
+
+                sendNotificationAsync(new Data(newValues, lastRevision));
+
+                return true;
+            } finally {
+                lock.writeLock().unlock();
             }
-            checkAndRestoreConfigFile();
-            // TODO: https://issues.apache.org/jira/browse/IGNITE-19152
-            //saveValues(newValues);
-            latest.putAll(newValues);
-            lastRevision++;
-            runAsync(() -> lsnrRef.get().onEntriesChanged(new Data(newValues, lastRevision)));
-            return CompletableFuture.completedFuture(true);
-        } finally {
-            lock.writeLock().unlock();
-        }
+        });
     }
 
-    private void runAsync(Runnable runnable) {
-        CompletableFuture<Void> future = CompletableFuture.runAsync(runnable, threadPool);
+    private void mergeAndSave(Map<String, ? extends Serializable> newValues) {
+        updateLatestState(newValues);
+        saveConfigFile();
+        lastRevision++;
+    }
 
-        futureTracker.registerFuture(future);
+    private void updateLatestState(Map<String, ? extends Serializable> newValues) {
+        newValues.forEach((key, value) -> {
+            if (value == null) { // null means that we should remove this entry

Review Comment:
   ```suggestion
               if (value == null) { // Null means that we should remove this entry.
   ```



##########
modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorageTest.java:
##########
@@ -17,75 +17,457 @@
 
 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.equalTo;
+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.charset.StandardCharsets;
 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.ConfigValue;
+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.ConfigurationTreeGenerator;
+import org.apache.ignite.internal.configuration.TestConfigurationChanger;
 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.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 {
+//TODO: https://issues.apache.org/jira/browse/IGNITE-19303
+public class LocalFileConfigurationStorageTest {
 
     private static final String CONFIG_NAME = "ignite-config.conf";
 
+    private static ConfigurationTreeGenerator treeGenerator;
+
     @WorkDirectory
     private Path tmpDir;
 
-    @Override
-    public ConfigurationStorage getStorage() {
-        return new LocalFileConfigurationStorage(getConfigFile());
+    private LocalFileConfigurationStorage storage;
+
+    private TestConfigurationChanger changer;
+
+    @BeforeAll
+    public static void beforeAll() {
+        treeGenerator = new ConfigurationTreeGenerator(
+                List.of(TopConfiguration.KEY, TopEmptyConfiguration.KEY)
+        );
+    }
+
+    @AfterAll
+    public static void afterAll() throws Exception {
+        treeGenerator.close();
+    }
+
+    private Path getConfigFile() {
+        return tmpDir.resolve(CONFIG_NAME);
+    }
+
+    @BeforeEach
+    void before() {
+        storage = new LocalFileConfigurationStorage(getConfigFile(), treeGenerator);
+
+        changer = new TestConfigurationChanger(
+                List.of(TopConfiguration.KEY),
+                Set.of(),
+                storage,
+                treeGenerator
+        );
+
+        changer.start();
+    }
+
+    @AfterEach
+    void after() {
+        changer.stop();
+    }
+
+
+    /** Default values are not enriched on read when the config file is empty. */
+    @Test
+    void empty() throws IOException {
+        // Given
+        assertThat(configFileContent(), emptyString());
+
+        // When
+        var storageValues = readAllLatest();
+
+        // Then
+        assertThat(storageValues.entrySet(), hasSize(0));
+    }
+
+    /** Named list entities can be added. */
+    @Test
+    void add() throws Exception {
+        // Given
+        assertThat(configFileContent(), emptyString());
+
+        // And
+        var topConfiguration = (TopConfiguration) treeGenerator.instantiateCfg(TopConfiguration.KEY, changer);
+
+        topConfiguration.namedList().change(b -> b.create("name1", x -> {
+            x.changeStrVal("strVal1");
+            x.changeIntVal(-1);
+        })).get();
+
+        // When
+        var storageValues = readAllLatest();
+
+        // Then the map has updated values
+        //
+        // top.namedList.<generatedUUID>.strVal  -> strVal1
+        // top.namedList.<generatedUUID>.intVal  -> -1
+        // top.namedList.<generatedUUID>.<name>  -> name1
+        // top.namedList.<ids>.name1             -> "<generatedUUID>"
+        // top.namedList.<generatedUUID>.<order> -> 0
+
+        assertThat(storageValues, allOf(aMapWithSize(5), hasValue(-1)));
+        assertThat(storageValues, allOf(aMapWithSize(5), hasValue("strVal1")));
+
+        // And
+        assertThat(configFileContent(), equalToCompressingWhiteSpace(
+                "top {\n"
+                        + "    namedList=[\n"
+                        + "        {\n"
+                        + "            intVal=-1\n"
+                        + "            name=name1\n"
+                        + "            strVal=strVal1\n"
+                        + "        }\n"
+                        + "    ]\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(10), hasValue(-2)));
+        assertThat(storageValues, allOf(aMapWithSize(10), hasValue("strVal2")));
+        // And
+        assertThat(storageValues, allOf(aMapWithSize(10), hasValue(-1)));
+        assertThat(storageValues, allOf(aMapWithSize(10), 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"
+                        + "}\n"
+        ));
+    }
+
+    /** Update values. */
+    @Test
+    void update() throws Exception {
+        // Given
+        assertThat(configFileContent(), emptyString());
+
+        // When
+        var topConfiguration = (TopConfiguration) treeGenerator.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"
+                        + "    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"
+                        + "}"
+        ));
     }
 
+    /** Remove values. */
     @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"
+    void remove() throws Exception {
+        // Given
+        var topConfiguration = (TopConfiguration) treeGenerator.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"
+                        + "    shortVal=3\n"
+                        + "}\n"
+        ));
+    }
+
+    /** Delete file before read on recovery. */
+    @Test
+    void deleteFileBeforeReadOnRecovery() throws IOException {
+        // Given
+        Files.delete(getConfigFile());
+
+        // When
+        var storageValues = storage.readDataOnRecovery().join().values();
+
+        // Then
+        assertThat(storageValues.entrySet(), hasSize(0));
+        // And empty file was created
+        assertThat(configFileContent(), equalTo(""));
+    }
+
+
+    /** File content is not changed when read data on recovery. */
+    @Test
+    void fileContentIsNotChanged() throws IOException {
+        // Given
+        String fileContent = "top {\n"
+                + "    namedList=[\n"
+                + "        {\n"
+                + "            intVal=-1\n"
+                + "            name=name1\n"
+                + "        }\n"
                 + "    ]\n"
-                + "}\n"));
+                + "}\n";
+
+        Files.write(getConfigFile(), fileContent.getBytes(StandardCharsets.UTF_8));
+
+        // When
+        var storageValues = storage.readDataOnRecovery().join().values();
+        // Then
+        assertThat(storageValues, allOf(aMapWithSize(5), hasValue(-1)));
+        assertThat(storageValues, allOf(aMapWithSize(5), hasValue("foo"))); // default value
+        // And file was not changed
+        assertThat(configFileContent(), equalTo(fileContent));
     }
 
+    /** Delete file before read all. */
     @Test
-    void testMergeHocon() throws IOException {
-        var data = Map.of("foo1", "bar");
-        assertThat(storage.write(data, 0), willBe(true));
+    void deleteFileBeforeReadAll() throws Exception {
+        // Given
+        Files.delete(getConfigFile());
+
+        // When
+        var storageValues = readAllLatest();
 
-        var append = Map.of("foo1", "baz", "foo2", "bar");
-        assertThat(storage.write(append, 1), willBe(true));
+        // Then
+        assertThat(storageValues.entrySet(), hasSize(0));
+        // And there is no file
+        assertThat(Files.exists(getConfigFile()), is(false));
 
-        String contents = Files.readString(getConfigFile());
-        assertThat(contents, is("foo1=baz\n"
-                + "foo2=bar\n"));
+        // When update configuration
+        var topConfiguration = (TopConfiguration) treeGenerator.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"
+                        + "}\n"
+        ));
     }
 
-    private Path getConfigFile() {
-        return tmpDir.resolve(CONFIG_NAME);
+    /** Read configuration when inner node configured with partial content (some fields are empty). */
+    @Test
+    void innerNodeWithPartialContent() throws Exception {
+        String content = "top: { inner.boolVal: true }";
+        Files.write(getConfigFile(), content.getBytes(StandardCharsets.UTF_8));
+
+        storage.readDataOnRecovery().get();

Review Comment:
   Should we assert the size of the map?



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

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

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


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

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1173865056


##########
modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationRegistry.java:
##########
@@ -96,8 +83,11 @@ public class ConfigurationRegistry implements IgniteComponent, ConfigurationStor
     /** Configuration change handler. */
     private final ConfigurationChanger changer;
 
-    /** Configuration generator. */
-    private final ConfigurationAsmGenerator cgen = new ConfigurationAsmGenerator();
+    /** Runtime implementations generator for node classes. */
+    private final ConfigurationTreeGenerator generator;
+
+    /** Flag that indicates if the {@link ConfigurationTreeGenerator} instance is owned by this object or not. */
+    private boolean ownConfigTreeGenerator = false;

Review Comment:
   I don't think I understand the answer. Let me rephrase myself.
   Why do you need a flag "ownConfigTreeGenerator"? It can be removed, right?
   Second question - why do you need a "close" method in the generator? I don't see a reason to have it, other than writing `cgen.close()` instead of `cgen = null` in some tests. Can you please explain your decision?



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

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

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


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

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1166622351


##########
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)
+    );

Review Comment:
   Given an implementation of the configuration listener that freezes the thread, we won't stop processing the configuration updates. So, I decided to separate the notification and main executors. WDYT?



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

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

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


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

Posted by "Pochatkin (via GitHub)" <gi...@apache.org>.
Pochatkin commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1165933231


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

Review Comment:
   Please don't use NotNull annotation



##########
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);
     }
 
+    /**

Review Comment:
   TBD



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -234,4 +339,26 @@ private void checkAndRestoreConfigFile() {
             }
         }
     }
+
+    private <T> CompletableFuture<T> readLockAsync(Supplier<T> supplier) {
+        lock.readLock().lock();
+        try {
+            CompletableFuture<T> future = CompletableFuture.supplyAsync(supplier, workerThreadPool);
+            futureTracker.registerFuture(future);
+            return future;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private <T> CompletableFuture<T> writeLockAsync(Supplier<T> supplier) {
+        lock.writeLock().lock();
+        try {
+            CompletableFuture<T> future = CompletableFuture.supplyAsync(supplier, workerThreadPool);
+            futureTracker.registerFuture(future);
+            return future;
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }

Review Comment:
   As I see all RW operations which you made in supplier has no locking.  So, I think this is not your fault and was previously. I think it would be better to move locking to suppliers implementation 



##########
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)
+    );

Review Comment:
   What the reason to have two thread pools?



##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -234,4 +339,26 @@ private void checkAndRestoreConfigFile() {
             }
         }
     }
+
+    private <T> CompletableFuture<T> readLockAsync(Supplier<T> supplier) {
+        lock.readLock().lock();
+        try {
+            CompletableFuture<T> future = CompletableFuture.supplyAsync(supplier, workerThreadPool);
+            futureTracker.registerFuture(future);
+            return future;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private <T> CompletableFuture<T> writeLockAsync(Supplier<T> supplier) {
+        lock.writeLock().lock();
+        try {
+            CompletableFuture<T> future = CompletableFuture.supplyAsync(supplier, workerThreadPool);
+            futureTracker.registerFuture(future);
+            return future;
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }

Review Comment:
   As I see all RW operations which you made in supplier has no locking.  So, I think this is not your fault and was previously. I think it would be better to move locking to suppliers implementation 



##########
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);
+    }

Review Comment:
   I think this method may be inlined



##########
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();
     }

Review Comment:
   I think that you can introduce RootKeysProvider abstraction and encapsulate all logic about root keys. This constructor should have only provider here as parameter



##########
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());
+    }

Review Comment:
   As I understood correct this is copypasted code from ConfigurationRegistry. May you extract it to separate abstraction layer and avoid code duplicate?



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

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

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


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

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1172392160


##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -56,99 +66,146 @@
 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 tree generator. */
+    private final ConfigurationTreeGenerator generator;
+
+    /** 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 notificationsThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates. */
+    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.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param generator Configuration tree generator.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, ConfigurationTreeGenerator generator) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.generator = generator;
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
         checkAndRestoreConfigFile();
     }
 
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return async(() -> {

Review Comment:
   I am not a fan of declaring the interface as an async (through CompletableFutures) but running the code in the same thread where the future is returned. What is the reason to use futures then? 
   
   But I can agree with you in this particular case.



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

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

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


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

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1172404883


##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -56,99 +66,146 @@
 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 tree generator. */
+    private final ConfigurationTreeGenerator generator;
+
+    /** 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 notificationsThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates. */
+    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.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param generator Configuration tree generator.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, ConfigurationTreeGenerator generator) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.generator = generator;
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
         checkAndRestoreConfigFile();
     }
 
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return async(() -> {
+            lock.writeLock().lock();
+            try {
+                SuperRoot superRoot = generator.createSuperRoot();
+                SuperRoot copiedSuperRoot = superRoot.copy();
+
+                Config hocon = readHoconFromFile();
+                HoconConverter.hoconSource(hocon.root()).descend(copiedSuperRoot);
+
+                Map<String, Serializable> flattenedUpdatesMap = createFlattenedUpdatesMap(superRoot, copiedSuperRoot);
+                flattenedUpdatesMap.forEach((key, value) -> {
+                    if (value != null) { // filter defaults
+                        latest.put(key, value);
+                    }
+                });
+
+                return new Data(flattenedUpdatesMap, lastRevision);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        });
+    }
+
+    private Config readHoconFromFile() {
+        checkAndRestoreConfigFile();
+
+        return ConfigFactory.parseFile(configPath.toFile(), ConfigParseOptions.defaults().setAllowMissing(false));
     }
 
     @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 async(() -> {
+            lock.readLock().lock();
+            try {
+                return latest.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getKey().startsWith(prefix))
+                        .collect(toMap(Entry::getKey, Entry::getValue));
+            } finally {
+                lock.readLock().unlock();
+            }
+        });
     }
 
     @Override
     public CompletableFuture<Serializable> readLatest(String key) {
-        lock.readLock().lock();
-        try {
-            checkAndRestoreConfigFile();
-            return CompletableFuture.completedFuture(latest.get(key));
-        } finally {
-            lock.readLock().unlock();
-        }
+        return async(() -> {
+            lock.readLock().lock();
+            try {
+                return latest.get(key);
+            } finally {
+                lock.readLock().unlock();
+            }
+        });
     }
 
     @Override
     public CompletableFuture<Boolean> write(Map<String, ? extends Serializable> newValues, long ver) {
-        lock.writeLock().lock();
-        try {
-            if (ver != lastRevision) {
-                return CompletableFuture.completedFuture(false);
+        return async(() -> {
+            lock.writeLock().lock();
+            try {
+                if (ver != lastRevision) {
+                    return false;
+                }
+
+                mergeAndSave(newValues);
+
+                sendNotificationAsync(new Data(newValues, lastRevision));
+
+                return true;
+            } finally {
+                lock.writeLock().unlock();
             }
-            checkAndRestoreConfigFile();
-            // TODO: https://issues.apache.org/jira/browse/IGNITE-19152
-            //saveValues(newValues);
-            latest.putAll(newValues);
-            lastRevision++;
-            runAsync(() -> lsnrRef.get().onEntriesChanged(new Data(newValues, lastRevision)));
-            return CompletableFuture.completedFuture(true);
-        } finally {
-            lock.writeLock().unlock();
-        }
+        });
     }
 
-    private void runAsync(Runnable runnable) {
-        CompletableFuture<Void> future = CompletableFuture.runAsync(runnable, threadPool);
+    private void mergeAndSave(Map<String, ? extends Serializable> newValues) {
+        updateLatestState(newValues);
+        saveConfigFile();
+        lastRevision++;

Review Comment:
   I don't get you.  Here is the part of the `write` method that verifies the version is the same. How it can be "new" revision?
   
   ```java
               if (ver != lastRevision) {
                   return CompletableFuture.completedFuture(false);
               }
   ```



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

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

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


[GitHub] [ignite-3] PakhomovAlexander closed pull request #1929: IGNITE-19152 Use schema information in LocalFileConfigurationStorage

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander closed pull request #1929: IGNITE-19152 Use schema information in LocalFileConfigurationStorage
URL: https://github.com/apache/ignite-3/pull/1929


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

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

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


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

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1176136589


##########
modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationRegistry.java:
##########
@@ -96,8 +83,11 @@ public class ConfigurationRegistry implements IgniteComponent, ConfigurationStor
     /** Configuration change handler. */
     private final ConfigurationChanger changer;
 
-    /** Configuration generator. */
-    private final ConfigurationAsmGenerator cgen = new ConfigurationAsmGenerator();
+    /** Runtime implementations generator for node classes. */
+    private final ConfigurationTreeGenerator generator;
+
+    /** Flag that indicates if the {@link ConfigurationTreeGenerator} instance is owned by this object or not. */
+    private boolean ownConfigTreeGenerator = false;

Review Comment:
   Technically you are right, we might write `cgen = null`. But this is something hidden. How developers might know that they should nullify the link because we don't want them to hold the link and produce memory leaks? 
   
   I decided to move this contract into the place where contracts are defined – the interface. I think a clear declaration of `close` method helps developers to understand how to work with the object.



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

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

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


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

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1176140252


##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -147,49 +139,40 @@ private Config readHoconFromFile() {
 
     @Override
     public CompletableFuture<Map<String, ? extends Serializable>> readAllLatest(String prefix) {
-        return async(() -> {
-            lock.readLock().lock();
-            try {
-                return latest.entrySet()
-                        .stream()
-                        .filter(entry -> entry.getKey().startsWith(prefix))
-                        .collect(toMap(Entry::getKey, Entry::getValue));
-            } finally {
-                lock.readLock().unlock();
-            }
-        });
+        lock.readLock().lock();
+        try {
+            return CompletableFuture.completedFuture(
+                    latest.entrySet()
+                            .stream()
+                            .filter(entry -> entry.getKey().startsWith(prefix))
+                            .collect(toMap(Entry::getKey, Entry::getValue))
+            );
+        } finally {
+            lock.readLock().unlock();
+        }
     }
 
     @Override
     public CompletableFuture<Serializable> readLatest(String key) {
-        return async(() -> {
-            lock.readLock().lock();
-            try {
-                return latest.get(key);
-            } finally {
-                lock.readLock().unlock();
-            }
-        });
+        return CompletableFuture.completedFuture(latest.get(key));

Review Comment:
   Thank you for the clarification, I misunderstood the `latest` meaning. My fault.



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

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

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


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

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1166684563


##########
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:
   The question with defaults is a good one. I think you are right and we should not store defaults. 



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

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

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


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

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1173861249


##########
modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationManager.java:
##########
@@ -40,6 +31,30 @@ public class ConfigurationManager implements IgniteComponent {
     /** Configuration registry. */
     private final ConfigurationRegistry registry;
 
+    /**
+     * Constructor.
+     *
+     * @param rootKeys                    Configuration root keys.
+     * @param validators                  Validators.
+     * @param storage                     Configuration storage.
+     * @param generator                   Configuration tree generator.
+     * @throws IllegalArgumentException If the configuration type of the root keys is not equal to the storage type, or if the schema or its
+     *                                  extensions are not valid.
+     */
+    public ConfigurationManager(

Review Comment:
   Ok, I don't mind it, but it's always weird to see multiple constructors



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

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

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


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

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
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


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

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1172368128


##########
modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationRegistry.java:
##########
@@ -96,8 +83,11 @@ public class ConfigurationRegistry implements IgniteComponent, ConfigurationStor
     /** Configuration change handler. */
     private final ConfigurationChanger changer;
 
-    /** Configuration generator. */
-    private final ConfigurationAsmGenerator cgen = new ConfigurationAsmGenerator();
+    /** Runtime implementations generator for node classes. */
+    private final ConfigurationTreeGenerator generator;
+
+    /** Flag that indicates if the {@link ConfigurationTreeGenerator} instance is owned by this object or not. */
+    private boolean ownConfigTreeGenerator = false;

Review Comment:
   "I believe that generator can always be a constructor parameter" -- not really. `ConfigurationRegistry` instance that uses `LocalFileConfigurationStorage` should  NOT create `ConfigurationTreeGenerator ` because it is passed ouside. 
   
   But if `ConfigurationRegistry` instance uses `DistributedConfiguraitonStorage` (that does not know about `ConfigurationTreeGenerator`) that it should create `ConfigurationTreeGenerator` in order to pass it into `Changer`. In this case there is no need to pass `ConfigurationTreeGenerator` from the "parent". 



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

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

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


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

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1172233823


##########
modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java:
##########
@@ -56,99 +66,146 @@
 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 tree generator. */
+    private final ConfigurationTreeGenerator generator;
+
+    /** 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 notificationsThreadPool = Executors.newFixedThreadPool(
+            2, new NamedThreadFactory("cfg-file", LOG)
+    );
+
+    /** Thread pool for configuration updates. */
+    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.
      *
      * @param configPath Path to node bootstrap configuration file.
+     * @param generator Configuration tree generator.
      */
-    public LocalFileConfigurationStorage(Path configPath) {
+    public LocalFileConfigurationStorage(Path configPath, ConfigurationTreeGenerator generator) {
         this.configPath = configPath;
-        tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+        this.generator = generator;
+        this.tempConfigPath = configPath.resolveSibling(configPath.getFileName() + ".tmp");
+
         checkAndRestoreConfigFile();
     }
 
     @Override
     public CompletableFuture<Data> readDataOnRecovery() {
-        return CompletableFuture.completedFuture(new Data(Collections.emptyMap(), 0));
+        return async(() -> {
+            lock.writeLock().lock();
+            try {
+                SuperRoot superRoot = generator.createSuperRoot();
+                SuperRoot copiedSuperRoot = superRoot.copy();
+
+                Config hocon = readHoconFromFile();
+                HoconConverter.hoconSource(hocon.root()).descend(copiedSuperRoot);
+
+                Map<String, Serializable> flattenedUpdatesMap = createFlattenedUpdatesMap(superRoot, copiedSuperRoot);
+                flattenedUpdatesMap.forEach((key, value) -> {
+                    if (value != null) { // filter defaults
+                        latest.put(key, value);
+                    }
+                });
+
+                return new Data(flattenedUpdatesMap, lastRevision);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        });
+    }
+
+    private Config readHoconFromFile() {
+        checkAndRestoreConfigFile();
+
+        return ConfigFactory.parseFile(configPath.toFile(), ConfigParseOptions.defaults().setAllowMissing(false));
     }
 
     @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 async(() -> {
+            lock.readLock().lock();
+            try {
+                return latest.entrySet()
+                        .stream()
+                        .filter(entry -> entry.getKey().startsWith(prefix))
+                        .collect(toMap(Entry::getKey, Entry::getValue));
+            } finally {
+                lock.readLock().unlock();
+            }
+        });
     }
 
     @Override
     public CompletableFuture<Serializable> readLatest(String key) {
-        lock.readLock().lock();
-        try {
-            checkAndRestoreConfigFile();
-            return CompletableFuture.completedFuture(latest.get(key));
-        } finally {
-            lock.readLock().unlock();
-        }
+        return async(() -> {
+            lock.readLock().lock();
+            try {
+                return latest.get(key);
+            } finally {
+                lock.readLock().unlock();
+            }
+        });
     }
 
     @Override
     public CompletableFuture<Boolean> write(Map<String, ? extends Serializable> newValues, long ver) {
-        lock.writeLock().lock();
-        try {
-            if (ver != lastRevision) {
-                return CompletableFuture.completedFuture(false);
+        return async(() -> {
+            lock.writeLock().lock();
+            try {
+                if (ver != lastRevision) {
+                    return false;
+                }
+
+                mergeAndSave(newValues);
+
+                sendNotificationAsync(new Data(newValues, lastRevision));
+
+                return true;
+            } finally {
+                lock.writeLock().unlock();
             }
-            checkAndRestoreConfigFile();
-            // TODO: https://issues.apache.org/jira/browse/IGNITE-19152
-            //saveValues(newValues);
-            latest.putAll(newValues);
-            lastRevision++;
-            runAsync(() -> lsnrRef.get().onEntriesChanged(new Data(newValues, lastRevision)));
-            return CompletableFuture.completedFuture(true);
-        } finally {
-            lock.writeLock().unlock();
-        }
+        });
     }
 
-    private void runAsync(Runnable runnable) {
-        CompletableFuture<Void> future = CompletableFuture.runAsync(runnable, threadPool);
+    private void mergeAndSave(Map<String, ? extends Serializable> newValues) {
+        updateLatestState(newValues);
+        saveConfigFile();
+        lastRevision++;
+    }
 
-        futureTracker.registerFuture(future);
+    private void updateLatestState(Map<String, ? extends Serializable> newValues) {
+        newValues.forEach((key, value) -> {
+            if (value == null) { // null means that we should remove this entry

Review Comment:
   I don't really get you. Here is what I can see in the [Coding Guidelines](https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines#CodingGuidelines-Commentedoutcode)
   
   "All comments should follow English grammar and punctuation including starting with an upper-case letter and ending with '.'."
   
   Could you share the link to the coding conventions you are talking about?



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

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

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


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

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander commented on code in PR #1929:
URL: https://github.com/apache/ignite-3/pull/1929#discussion_r1172363042


##########
modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationManager.java:
##########
@@ -40,6 +31,30 @@ public class ConfigurationManager implements IgniteComponent {
     /** Configuration registry. */
     private final ConfigurationRegistry registry;
 
+    /**
+     * Constructor.
+     *
+     * @param rootKeys                    Configuration root keys.
+     * @param validators                  Validators.
+     * @param storage                     Configuration storage.
+     * @param generator                   Configuration tree generator.
+     * @throws IllegalArgumentException If the configuration type of the root keys is not equal to the storage type, or if the schema or its
+     *                                  extensions are not valid.
+     */
+    public ConfigurationManager(

Review Comment:
   Yes, there is a reason for that. `ConfigurationManager` now gives two constructors like:
   
   1) You can provide `ConfigurationTreeGenerator` with `LocalConfigurationStorage` (that is dependent on `ConfigurationTreeGenerator`)
   2) You can provide lists of roots and extensions with `DistributedConfgurationStorage`(that is NOT dependent on `ConfigurationTreeGenerator`) 
   
   Does it make sense? 



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