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

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

SammyVimes commented on a change in pull request #56:
URL: https://github.com/apache/ignite-3/pull/56#discussion_r582752035



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

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

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

Review comment:
       Doesn't this create default values?

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

Review comment:
       I think this method also throws ConfigurationChangeException

##########
File path: modules/configuration/src/main/java/org/apache/ignite/configuration/internal/ConfigurationNode.java
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.configuration.internal;
+
+import java.util.List;
+import java.util.NoSuchElementException;
+import org.apache.ignite.configuration.ConfigurationChanger;
+import org.apache.ignite.configuration.RootKey;
+import org.apache.ignite.configuration.internal.util.ConfigurationUtil;
+import org.apache.ignite.configuration.internal.util.KeyNotFoundException;
+import org.apache.ignite.configuration.tree.TraversableTreeNode;
+
+/**
+ * Super class for dynamic configuration tree nodes. Has all common data and value retrieving algorithm in it.
+ */
+public abstract class ConfigurationNode<VIEW> {
+    /** Full path to the current node. */
+    protected final List<String> keys;
+
+    /** Name of the current node. Same as last element of {@link #keys}. */
+    protected final String key;
+
+    /** Root key instance for the current trees root. */
+    protected final RootKey<?> rootKey;
+
+    /** Configuration changer instance to get latest value of the root. */
+    protected final ConfigurationChanger changer;
+
+    /**
+     * Cached value of current trees root. Useful to determine whether you have the latest configuration value or not.
+     */
+    private volatile TraversableTreeNode cachedRootNode;
+
+    /** Cached configuration value. Immutable. */
+    private VIEW val;
+
+    /**
+     * Validity flag. Configuration is declared invalid if it's a part of named list configuration and corresponding
+     * entry is already removed.
+     */
+    private boolean invalid;
+
+    /**
+     * Constructor.
+     *
+     * @param prefix Configuration prefix.
+     * @param key Configuration key.
+     * @param rootKey Root key.
+     * @param changer Configuration changer.
+     */
+    protected ConfigurationNode(List<String> prefix, String key, RootKey<?> rootKey, ConfigurationChanger changer) {
+        this.keys = ConfigurationUtil.appendKey(prefix, key);
+        this.key = key;
+        this.rootKey = rootKey;
+        this.changer = changer;
+    }
+
+    /**
+     * Returns latest value of the configuration or throws exception.
+     *
+     * @return Latest configuration value.
+     * @throws NoSuchElementException If configuration is a part of already deleted named list configuration entry.
+     */
+    protected final VIEW refreshValue() throws NoSuchElementException {
+        if (invalid)

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




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

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