You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "Mmuzaf (via GitHub)" <gi...@apache.org> on 2023/02/02 22:31:11 UTC

[GitHub] [cassandra] Mmuzaf opened a new pull request, #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Mmuzaf opened a new pull request, #2133:
URL: https://github.com/apache/cassandra/pull/2133

   
   Thanks for sending a pull request! Here are some tips if you're new here:
    
    * Ensure you have added or run the [appropriate tests](https://cassandra.apache.org/_/development/testing.html) for your PR.
    * Be sure to keep the PR description updated to reflect all changes.
    * Write your PR title to summarize what this PR proposes.
    * If possible, provide a concise example to reproduce the issue for a faster review.
    * Read our [contributor guidelines](https://cassandra.apache.org/_/development/index.html)
    * If you're making a documentation change, see our [guide to documentation contribution](https://cassandra.apache.org/_/development/documentation.html)
    
   Commit messages should follow the following format:
   
   ```
   <One sentence description, usually Jira title or CHANGES.txt summary>
   
   <Optional lengthier description (context on patch)>
   
   patch by <Authors>; reviewed by <Reviewers> for CASSANDRA-#####
   
   Co-authored-by: Name1 <email1>
   Co-authored-by: Name2 <email2>
   
   ```
   
   The [Cassandra Jira](https://issues.apache.org/jira/projects/CASSANDRA/issues/)
   
   


-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1146393791


##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -80,45 +82,54 @@ public static Set<String> splitCommaDelimited(String src)
     public String authorizer;
     public String role_manager;
     public String network_authorizer;
-    @Replaces(oldName = "permissions_validity_in_ms", converter = Converters.MILLIS_DURATION_INT, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound permissions_validity = new DurationSpec.IntMillisecondsBound("2s");
-    public volatile int permissions_cache_max_entries = 1000;
-    @Replaces(oldName = "permissions_update_interval_in_ms", converter = Converters.MILLIS_CUSTOM_DURATION, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound permissions_update_interval = null;
-    public volatile boolean permissions_cache_active_update = false;
-    @Replaces(oldName = "roles_validity_in_ms", converter = Converters.MILLIS_DURATION_INT, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound roles_validity = new DurationSpec.IntMillisecondsBound("2s");
-    public volatile int roles_cache_max_entries = 1000;
-    @Replaces(oldName = "roles_update_interval_in_ms", converter = Converters.MILLIS_CUSTOM_DURATION, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound roles_update_interval = null;
-    public volatile boolean roles_cache_active_update = false;
-    @Replaces(oldName = "credentials_validity_in_ms", converter = Converters.MILLIS_DURATION_INT, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound credentials_validity = new DurationSpec.IntMillisecondsBound("2s");
-    public volatile int credentials_cache_max_entries = 1000;
-    @Replaces(oldName = "credentials_update_interval_in_ms", converter = Converters.MILLIS_CUSTOM_DURATION, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound credentials_update_interval = null;
-    public volatile boolean credentials_cache_active_update = false;
+    @Mutable @Replaces(oldName = "permissions_validity_in_ms", converter = Converters.MILLIS_DURATION_INT, deprecated = true)
+    public DurationSpec.IntMillisecondsBound permissions_validity = new DurationSpec.IntMillisecondsBound("2s");

Review Comment:
   why did you remove `volatile`?  this is a regression



##########
src/java/org/apache/cassandra/config/Mutable.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.cassandra.config;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Annotation denoting that a configuration field in the {@link Config} is mutable with a live update.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.FIELD)

Review Comment:
   we do support methods as well, think maybe 1-2 use it and not sure if they are mutable... but it is supported



##########
src/java/org/apache/cassandra/config/StringConverters.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.cassandra.config;
+
+import org.apache.cassandra.config.registry.TypeConverter;
+import org.apache.cassandra.db.ConsistencyLevel;
+
+import static org.apache.cassandra.config.registry.PrimitiveUnaryConverter.convertSafe;
+
+/**
+ * String converters for Cassandra configuration types used in {@link Config}. These are used to convert
+ * configuration values from their string representation to their actual type and back. The converters
+ * are uses the {@link TypeConverter} interface.
+ * <p>
+ * Take care when adding new converters, only one converter per type is allowed. If a converter for a
+ * type already exists, the new converter will be ignored since the first converter for a type is used.
+ *
+ * @see Config
+ * @see TypeConverter
+ */
+public enum StringConverters
+{
+    PRIMITIVE_BOOLEAN(Boolean.TYPE, s -> Boolean.parseBoolean((String) s), b -> Boolean.toString((Boolean) b)),
+    PRIMITIVE_DOUBLE(Double.TYPE, s -> Double.parseDouble((String) s),  d -> Double.toString((Double) d)),
+    PRIMITIVE_INTEGER(Integer.TYPE, s -> Integer.parseInt((String) s), i -> Integer.toString((Integer) i)),
+    PRIMITIVE_LONG(Long.TYPE, s -> Long.parseLong((String) s), l -> Long.toString((Long) l)),
+    BOOLEAN(Boolean.class, s -> Boolean.parseBoolean((String) s), b -> Boolean.toString((Boolean) b)),
+    DOUBLE(Double.class, s -> Double.parseDouble((String) s),  d -> Double.toString((Double) d)),
+    INTEGER(Integer.class, s -> Integer.parseInt((String) s), i -> Integer.toString((Integer) i)),
+    LONG(Long.class, s -> Long.parseLong((String) s), l -> Long.toString((Long) l)),
+    STRING(String.class, s -> (String) s, s -> (String) s),
+    // Cassandra specific configuration types.
+    LONG_NANOSECONDS_BOUND(DurationSpec.LongNanosecondsBound.class, s -> new DurationSpec.LongNanosecondsBound((String) s), TypeConverter.DEFAULT),
+    LONG_MILLISECONDS_BOUND(DurationSpec.LongMillisecondsBound.class, s -> new DurationSpec.LongMillisecondsBound((String) s), TypeConverter.DEFAULT),
+    LONG_SECONDS_BOUND(DurationSpec.LongSecondsBound.class, s -> new DurationSpec.LongSecondsBound((String) s), TypeConverter.DEFAULT),
+    INT_MINUSTES_BOUND(DurationSpec.IntMinutesBound.class, s -> new DurationSpec.IntMinutesBound((String) s), TypeConverter.DEFAULT),
+    INT_SECONDS_BOUND(DurationSpec.IntSecondsBound.class, s -> new DurationSpec.IntSecondsBound((String) s), TypeConverter.DEFAULT),
+    INT_MILLISECONDS_BOUND(DurationSpec.IntMillisecondsBound.class, s -> new DurationSpec.IntMillisecondsBound((String) s), TypeConverter.DEFAULT),
+    LONG_BYTES_BOUND(DataStorageSpec.LongBytesBound.class, s -> new DataStorageSpec.LongBytesBound((String) s), TypeConverter.DEFAULT),
+    INT_BYTES_BOUND(DataStorageSpec.IntBytesBound.class, s -> new DataStorageSpec.IntBytesBound((String) s), TypeConverter.DEFAULT),
+    INT_KIBIBYTES_BOUND(DataStorageSpec.IntKibibytesBound.class, s -> new DataStorageSpec.IntKibibytesBound((String) s), TypeConverter.DEFAULT),
+    LONG_MEBIBYTES_BOUND(DataStorageSpec.LongMebibytesBound.class, s -> new DataStorageSpec.LongMebibytesBound((String) s), TypeConverter.DEFAULT),
+    INT_MEBIBYTES_BOUND(DataStorageSpec.IntMebibytesBound.class, s -> new DataStorageSpec.IntMebibytesBound((String) s), TypeConverter.DEFAULT),
+    CONSYSTENCY_LEVEL(ConsistencyLevel.class, s -> ConsistencyLevel.fromStringIgnoreCase((String) s), c -> ((ConsistencyLevel) c).name());
+
+    private final Class<?> type;
+    private final TypeConverter<?> forward;
+    private final TypeConverter<String> reverse;
+
+    /**
+     * Creates a new converter for the given type and using the given converters.
+     * @param type the type this converter converts to and from.
+     * @param forward the forward converter to use.
+     * @param reverse the reverse converter to use.
+     */
+    <T> StringConverters(Class<T> type, TypeConverter<T> forward, TypeConverter<String> reverse)
+    {
+        this.type = type;
+        this.forward = forward;
+        this.reverse = reverse;
+    }
+    public <T> T fromString(String value, Class<T> target)
+    {
+        if (target.equals(type))
+            return convertSafe(type, forward.convertNullable(value));
+        throw new IllegalArgumentException(String.format("Invalid target type '%s' for converter '%s'", target, this));
+    }
+
+    public String toString(Object value)
+    {
+        return reverse.convertNullable(value);
+    }
+
+    /**
+     * Returns the converter for the given type.
+     * @param type the type to return the converter for.
+     * @return the converter for the given type or {@code null} if no converter exists for the given type
+     * or default conversion should be used instead.
+     */
+    public static StringConverters fromType(Class<?> type)
+    {
+        for (StringConverters converter : values())
+            if (converter.type.equals(type))
+                return converter;
+
+        return null;
+    }

Review Comment:
   this highlights to me that we don't want an enum, we really want a map.  `O(n)` for every config accessed is a problem as we would need to do this on the write side as well, so its `O(n * m)` due to this... 
   
   We need a `Map<Class<?>, Converter>`



##########
src/java/org/apache/cassandra/config/registry/ConfigurationConstraint.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.cassandra.config.registry;
+
+/**
+ * Interface validating configuration property's value.
+ */
+public interface ConfigurationConstraint<T>

Review Comment:
   what is this?  looks like dead code?



##########
src/java/org/apache/cassandra/db/virtual/SettingsTable.java:
##########
@@ -68,54 +91,48 @@ public DataSet data(DecoratedKey partitionKey)
         SimpleDataSet result = new SimpleDataSet(metadata());
         String name = UTF8Type.instance.compose(partitionKey.getKey());
         if (BACKWARDS_COMPATABLE_NAMES.containsKey(name))
-            ClientWarn.instance.warn("key '" + name + "' is deprecated; should switch to '" + BACKWARDS_COMPATABLE_NAMES.get(name) + "'");
-        if (PROPERTIES.containsKey(name))
-            result.row(name).column(VALUE, getValue(PROPERTIES.get(name)));
+            ClientWarn.instance.warn("key '" + name + "' is deprecated; should switch to '" + BACKWARDS_COMPATABLE_NAMES.get(name) + '\'');
+        if (registry.contains(name))

Review Comment:
   how does the registry know about names defined in `BACKWARDS_COMPATABLE_NAMES`?  Those are not valid configs, those are bugs with this specific API, and must only live with this API



##########
src/java/org/apache/cassandra/config/registry/TypeConverter.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.cassandra.config.registry;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+/**
+ * Converts configuration value from one type to another, you can use {@link org.apache.cassandra.config.StringConverters}
+ * if your input type is String and you want to convert it to an appropriate configuration object type.
+ *
+ * @param <T> Type to convert to.
+ *
+ * @see Registry
+ * @see org.apache.cassandra.config.StringConverters
+ */
+public interface TypeConverter<T>

Review Comment:
   this API isn't what we need, we need
   
   ```
   T1 -> T2
   T2 -> T1
   ```
   
   The reason is that w/e we do when we return *must* be accepted as valid input when the user writes, so both directions are required.
   
   Also, using `Object` is a problem as we can use the wrong type and wouldn't ever know



##########
src/java/org/apache/cassandra/db/virtual/SettingsTable.java:
##########
@@ -68,54 +91,48 @@ public DataSet data(DecoratedKey partitionKey)
         SimpleDataSet result = new SimpleDataSet(metadata());
         String name = UTF8Type.instance.compose(partitionKey.getKey());
         if (BACKWARDS_COMPATABLE_NAMES.containsKey(name))
-            ClientWarn.instance.warn("key '" + name + "' is deprecated; should switch to '" + BACKWARDS_COMPATABLE_NAMES.get(name) + "'");
-        if (PROPERTIES.containsKey(name))
-            result.row(name).column(VALUE, getValue(PROPERTIES.get(name)));
+            ClientWarn.instance.warn("key '" + name + "' is deprecated; should switch to '" + BACKWARDS_COMPATABLE_NAMES.get(name) + '\'');
+        if (registry.contains(name))
+            runExceptionally(() -> result.row(name).column(VALUE, registry.getString(name)),
+                                         e -> invalidRequest("Invalid configuration request during searching by key; cause: '%s'", e.getMessage()));
         return result;
     }
 
     @Override
     public DataSet data()
     {
         SimpleDataSet result = new SimpleDataSet(metadata());
-        for (Map.Entry<String, Property> e : PROPERTIES.entrySet())
-            result.row(e.getKey()).column(VALUE, getValue(e.getValue()));
+        for (String name : registry.keys())
+            runExceptionally(() -> result.row(name).column(VALUE, registry.getString(name)),
+                                         e -> invalidRequest("Invalid configuration request; cause: '%s'", e.getMessage()));

Review Comment:
   missing config key



##########
src/java/org/apache/cassandra/config/registry/ConfigurationListener.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.cassandra.config.registry;
+
+/**
+ * Interface for listening to configuration property changes.
+ */
+public interface ConfigurationListener<T>
+{
+    /**
+     * Called on configuration change property event occurr.
+     *
+     * @param name     the name of the property.
+     * @param oldValue the old value of the property.
+     * @param newValue the new value of the property.
+     */
+    void onUpdate(String name, T oldValue, T newValue);
+
+    /** Type of property change. */
+    enum ChangeType

Review Comment:
   this is dead code



##########
src/java/org/apache/cassandra/db/virtual/SettingsTable.java:
##########
@@ -157,4 +174,85 @@ private static Map<String, String> getBackwardsCompatableNames()
 
         return names;
     }
+
+    /**
+     * This class is used to provide backwards compatable support for the settings table in case the {@link Config}
+     * metadata changes. This class will provide the old names for the properties, but will use the new name to
+     * get the value from the {@link Config} object.
+     * <p>
+     * Updating a configuration property object will throw an exception if you will try to update a deprecated property.
+     */
+    private static class BackwardsCompatableRegistry implements Registry

Review Comment:
   this code is more verbose than just handling the `Replacement`... before we would create a valid `Property` for each `Replacement` and that would handle the conversions and call the "real" `Property`
   
   It looks like this class makes invalid assumptions, such as old names are not allowed... they are 100% supported, which makes this more verbose it seems...
   
   Is there a reason to migrate away from the existing logic that makes old/new properties look the same?



##########
src/java/org/apache/cassandra/db/commitlog/CommitLog.java:
##########
@@ -137,7 +150,13 @@ synchronized public CommitLog start()
             segmentManager.start();
             executor.start();
             started = true;
-        } catch (Throwable t)
+
+            DatabaseDescriptor.getConfigRegistry().addPropertyConstraint(CDC_BLOCK_WRITES, Boolean.TYPE, (newVal) -> ensureCDCEnabled());
+            DatabaseDescriptor.getConfigRegistry().addPropertyConstraint(CDC_ON_REPAIR_ENABLED, Boolean.TYPE, (newVal) -> ensureCDCEnabled());

Review Comment:
   nit: cleaner to save `DatabaseDescriptor.getConfigRegistry()` to a variable, makes the code less verbose



##########
src/java/org/apache/cassandra/service/StorageService.java:
##########
@@ -1536,7 +1540,6 @@ else if (tokens == null)
     public void setRpcTimeout(long value)
     {
         DatabaseDescriptor.setRpcTimeout(value);
-        logger.info("set rpc timeout to {} ms", value);

Review Comment:
   this is a regression, people may depend on these logs



##########
src/java/org/apache/cassandra/db/ConsistencyLevel.java:
##########
@@ -264,4 +264,9 @@ private void requireNetworkTopologyStrategy(AbstractReplicationStrategy replicat
             throw new InvalidRequestException(String.format("consistency level %s not compatible with replication strategy (%s)",
                                                             this, replicationStrategy.getClass().getName()));
     }
+
+    public static ConsistencyLevel fromStringIgnoreCase(String s)

Review Comment:
   why?  Is this now trying to make even more things valid?  Is this needed for this patch?



##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.Mutable;
+import org.apache.cassandra.config.StringConverters;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.yaml.snakeyaml.introspector.Property;
+
+import static org.apache.cassandra.config.Properties.defaultLoader;
+import static org.apache.cassandra.config.registry.PrimitiveUnaryConverter.convertSafe;
+import static org.apache.commons.lang3.ClassUtils.primitiveToWrapper;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings, it doesn't
+ * take into account any configuration changes that might happen during properties replacement between releases.
+ */
+public class ConfigurationRegistry implements Registry
+{
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Supplier<Config> configProvider;
+    private final Map<ConfigurationListener.ChangeType, ConfigurationListenerList> propertyChangeListeners = new EnumMap<>(ConfigurationListener.ChangeType.class);
+    private final Map<String, List<TypedConstraintAdapter<?>>> constraints = new HashMap<>();
+    private volatile boolean initialized;
+    private Map<String, PropertyAdapter> properties = Collections.emptyMap();
+
+    public ConfigurationRegistry(Supplier<Config> configProvider)
+    {
+        this.configProvider = configProvider;
+        // Initialize the property change listeners.
+        for (ConfigurationListener.ChangeType type : ConfigurationListener.ChangeType.values())
+            propertyChangeListeners.put(type, new ConfigurationListenerList());
+    }
+
+    private void lazyInit()
+    {
+        if (initialized)
+            return;
+
+        rwLock.writeLock().lock();
+        try
+        {
+            if (initialized)
+                return;
+            properties = ImmutableMap.copyOf(defaultLoader()
+                                             .flatten(Config.class)
+                                             .entrySet()
+                                             .stream()
+                                             .map(e -> new AbstractMap.SimpleEntry<>(e.getKey(), new PropertyAdapter(configProvider, e.getValue())))
+                                             .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));

Review Comment:
   why is this in `lazyInit`? This is static and can be done during class loading... we shouldn't need to make the locks this costly



##########
src/java/org/apache/cassandra/auth/AuthConfig.java:
##########
@@ -56,9 +60,9 @@ public static void applyAuth()
         // work with PasswordAuthenticator, so log a message if some other authenticator
         // is in use and non-default values are detected
         if (!(authenticator instanceof PasswordAuthenticator)
-            && (conf.credentials_update_interval != null
-                || conf.credentials_validity.toMilliseconds() != 2000
-                || conf.credentials_cache_max_entries != 1000))
+            && (registry.get(DurationSpec.IntMillisecondsBound.class, ConfigFields.CREDENTIALS_UPDATE_INTERVAL) != null

Review Comment:
   why is config validation going through `registry`?



##########
src/java/org/apache/cassandra/db/virtual/SettingsTable.java:
##########
@@ -157,4 +174,85 @@ private static Map<String, String> getBackwardsCompatableNames()
 
         return names;
     }
+
+    /**
+     * This class is used to provide backwards compatable support for the settings table in case the {@link Config}
+     * metadata changes. This class will provide the old names for the properties, but will use the new name to
+     * get the value from the {@link Config} object.
+     * <p>
+     * Updating a configuration property object will throw an exception if you will try to update a deprecated property.
+     */
+    private static class BackwardsCompatableRegistry implements Registry
+    {
+        private final Registry registry;
+        private final Map<String, Replacement> replacements;
+        private final Set<String> uniquePropertyKeys;
+        public BackwardsCompatableRegistry(Registry registry)
+        {
+            this.registry = registry;
+            this.replacements = replacements(registry);
+            // Some configs kept the same name, but changed the type, so we need to make sure we don't return the same name twice.
+            this.uniquePropertyKeys = ImmutableSet.<String>builder().addAll(registry.keys()).addAll(replacements.keySet()).build();
+        }
+
+        @Override
+        public void set(String name, @Nullable Object value)
+        {
+            Replacement replacement = replacements.get(name);
+            if (replacement == null)
+                registry.set(name, value);
+            else
+                throw new ConfigurationException(String.format("Unable to set '%s' as it is deprecated and is read only; use '%s' instead", name, replacement.newName));

Review Comment:
   this is a breaking change, so can't allow this... the point of `Replacement` is that we can make changes without forcing users, this logic is now forcing users.



##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.Mutable;
+import org.apache.cassandra.config.StringConverters;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.yaml.snakeyaml.introspector.Property;
+
+import static org.apache.cassandra.config.Properties.defaultLoader;
+import static org.apache.cassandra.config.registry.PrimitiveUnaryConverter.convertSafe;
+import static org.apache.commons.lang3.ClassUtils.primitiveToWrapper;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings, it doesn't
+ * take into account any configuration changes that might happen during properties replacement between releases.
+ */
+public class ConfigurationRegistry implements Registry
+{
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Supplier<Config> configProvider;
+    private final Map<ConfigurationListener.ChangeType, ConfigurationListenerList> propertyChangeListeners = new EnumMap<>(ConfigurationListener.ChangeType.class);
+    private final Map<String, List<TypedConstraintAdapter<?>>> constraints = new HashMap<>();
+    private volatile boolean initialized;
+    private Map<String, PropertyAdapter> properties = Collections.emptyMap();
+
+    public ConfigurationRegistry(Supplier<Config> configProvider)
+    {
+        this.configProvider = configProvider;
+        // Initialize the property change listeners.
+        for (ConfigurationListener.ChangeType type : ConfigurationListener.ChangeType.values())
+            propertyChangeListeners.put(type, new ConfigurationListenerList());
+    }
+
+    private void lazyInit()
+    {
+        if (initialized)
+            return;
+
+        rwLock.writeLock().lock();
+        try
+        {
+            if (initialized)
+                return;
+            properties = ImmutableMap.copyOf(defaultLoader()
+                                             .flatten(Config.class)
+                                             .entrySet()
+                                             .stream()
+                                             .map(e -> new AbstractMap.SimpleEntry<>(e.getKey(), new PropertyAdapter(configProvider, e.getValue())))
+                                             .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
+            Set<String> leftConstraints = new HashSet<>(constraints.keySet());
+            leftConstraints.removeAll(properties.keySet());
+            if (!leftConstraints.isEmpty())
+                throw new ConfigurationException("Constraints are defined for non-existing properties:" + leftConstraints);

Review Comment:
   properties are static and immutable, so why even allow this to happen?  This feels like trying to be lazy caused this, but there is no reason to do that and you can just make properties static and loaded during class loading, then you can block such cases from happening *at the call site*, which is far easier to work with



##########
src/java/org/apache/cassandra/db/virtual/SettingsTable.java:
##########
@@ -68,54 +91,48 @@ public DataSet data(DecoratedKey partitionKey)
         SimpleDataSet result = new SimpleDataSet(metadata());
         String name = UTF8Type.instance.compose(partitionKey.getKey());
         if (BACKWARDS_COMPATABLE_NAMES.containsKey(name))
-            ClientWarn.instance.warn("key '" + name + "' is deprecated; should switch to '" + BACKWARDS_COMPATABLE_NAMES.get(name) + "'");
-        if (PROPERTIES.containsKey(name))
-            result.row(name).column(VALUE, getValue(PROPERTIES.get(name)));
+            ClientWarn.instance.warn("key '" + name + "' is deprecated; should switch to '" + BACKWARDS_COMPATABLE_NAMES.get(name) + '\'');
+        if (registry.contains(name))
+            runExceptionally(() -> result.row(name).column(VALUE, registry.getString(name)),
+                                         e -> invalidRequest("Invalid configuration request during searching by key; cause: '%s'", e.getMessage()));

Review Comment:
   should include the config key



##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.Mutable;
+import org.apache.cassandra.config.StringConverters;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.yaml.snakeyaml.introspector.Property;
+
+import static org.apache.cassandra.config.Properties.defaultLoader;
+import static org.apache.cassandra.config.registry.PrimitiveUnaryConverter.convertSafe;
+import static org.apache.commons.lang3.ClassUtils.primitiveToWrapper;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings, it doesn't
+ * take into account any configuration changes that might happen during properties replacement between releases.
+ */
+public class ConfigurationRegistry implements Registry
+{
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Supplier<Config> configProvider;
+    private final Map<ConfigurationListener.ChangeType, ConfigurationListenerList> propertyChangeListeners = new EnumMap<>(ConfigurationListener.ChangeType.class);
+    private final Map<String, List<TypedConstraintAdapter<?>>> constraints = new HashMap<>();
+    private volatile boolean initialized;
+    private Map<String, PropertyAdapter> properties = Collections.emptyMap();
+
+    public ConfigurationRegistry(Supplier<Config> configProvider)
+    {
+        this.configProvider = configProvider;
+        // Initialize the property change listeners.
+        for (ConfigurationListener.ChangeType type : ConfigurationListener.ChangeType.values())
+            propertyChangeListeners.put(type, new ConfigurationListenerList());
+    }
+
+    private void lazyInit()
+    {
+        if (initialized)
+            return;
+
+        rwLock.writeLock().lock();
+        try
+        {
+            if (initialized)
+                return;
+            properties = ImmutableMap.copyOf(defaultLoader()
+                                             .flatten(Config.class)
+                                             .entrySet()
+                                             .stream()
+                                             .map(e -> new AbstractMap.SimpleEntry<>(e.getKey(), new PropertyAdapter(configProvider, e.getValue())))
+                                             .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
+            Set<String> leftConstraints = new HashSet<>(constraints.keySet());
+            leftConstraints.removeAll(properties.keySet());
+            if (!leftConstraints.isEmpty())
+                throw new ConfigurationException("Constraints are defined for non-existing properties:" + leftConstraints);
+            Set<String> leftListeners = propertyChangeListeners.values().stream()
+                                                               .map(l -> l.wrappers.keySet())
+                                                               .flatMap(Collection::stream)
+                                                               .collect(Collectors.toSet());
+            leftListeners.removeAll(properties.keySet());
+            if (!leftListeners.isEmpty())
+                throw new ConfigurationException("Listeners are defined for non-existing properties:" + leftListeners);
+            initialized = true;
+        }
+        finally
+        {
+            rwLock.writeLock().unlock();
+        }
+    }
+
+    @Override public void set(String name, @Nullable Object value)
+    {
+        lazyInit();
+        PropertyAdapter property = properties.get(name);
+        validatePropertyExists(property, name);
+        setInternal(property, value);
+    }
+
+    /**
+     * Setter for the property with the given name. Can accept {@code null} value.
+     * @param property The property.
+     * @param value The value to set.
+     */
+    private void setInternal(PropertyAdapter property, @Nullable Object value)
+    {
+        rwLock.writeLock().lock();
+        try
+        {
+            Class<?> originalType = property.getType();
+            Class<?> sourceType = value == null ? null : value.getClass();
+            Object convertedValue = value;
+            // Do conversion if the value is not null and the type is not the same as the property type.
+            if (sourceType != null && !primitiveToWrapper(originalType).equals(sourceType))
+            {
+                StringConverters converter;
+                if (sourceType.equals(String.class) && (converter = StringConverters.fromType(originalType)) != null)
+                    convertedValue = converter.fromString((String) value, originalType);
+                else
+                    throw new IllegalArgumentException(String.format("No converter found for type '%s'", originalType.getName()));
+            }
+            // Do validation first for converted new value.
+            List<TypedConstraintAdapter<?>> constraintsList = constraints.getOrDefault(property.getName(), Collections.emptyList());
+            for (TypedConstraintAdapter<?> typed : constraintsList)
+                typed.validateTypeCast(convertedValue);

Review Comment:
   Why do we need this?  We have been moving towards types doing their own validation, so custom validators adds more complexity if they are "lazy".
   
   I am actually ok if this exists and is done via the following
   
   ```
   // Config.java
   @Mutable
   @Min(42) @Max(57)
   public volatile int some_config = 42;
   ```
   
   but custom validation has been something we are trying to move away from... I don't see this style in this patch, so questioning...
   
   Also, this concept duplicates `propertyChangeListeners.get(ConfigurationListener.ChangeType.BEFORE).fireTypeCast(property.getName(), oldValue, convertedValue);`, so you could create a much simpler model by having any custom validation done as a before listener



##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.Mutable;
+import org.apache.cassandra.config.StringConverters;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.yaml.snakeyaml.introspector.Property;
+
+import static org.apache.cassandra.config.Properties.defaultLoader;
+import static org.apache.cassandra.config.registry.PrimitiveUnaryConverter.convertSafe;
+import static org.apache.commons.lang3.ClassUtils.primitiveToWrapper;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings, it doesn't
+ * take into account any configuration changes that might happen during properties replacement between releases.
+ */
+public class ConfigurationRegistry implements Registry
+{
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Supplier<Config> configProvider;
+    private final Map<ConfigurationListener.ChangeType, ConfigurationListenerList> propertyChangeListeners = new EnumMap<>(ConfigurationListener.ChangeType.class);
+    private final Map<String, List<TypedConstraintAdapter<?>>> constraints = new HashMap<>();
+    private volatile boolean initialized;
+    private Map<String, PropertyAdapter> properties = Collections.emptyMap();
+
+    public ConfigurationRegistry(Supplier<Config> configProvider)
+    {
+        this.configProvider = configProvider;
+        // Initialize the property change listeners.
+        for (ConfigurationListener.ChangeType type : ConfigurationListener.ChangeType.values())
+            propertyChangeListeners.put(type, new ConfigurationListenerList());
+    }
+
+    private void lazyInit()
+    {
+        if (initialized)
+            return;
+
+        rwLock.writeLock().lock();
+        try
+        {
+            if (initialized)
+                return;
+            properties = ImmutableMap.copyOf(defaultLoader()
+                                             .flatten(Config.class)
+                                             .entrySet()
+                                             .stream()
+                                             .map(e -> new AbstractMap.SimpleEntry<>(e.getKey(), new PropertyAdapter(configProvider, e.getValue())))
+                                             .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
+            Set<String> leftConstraints = new HashSet<>(constraints.keySet());
+            leftConstraints.removeAll(properties.keySet());
+            if (!leftConstraints.isEmpty())
+                throw new ConfigurationException("Constraints are defined for non-existing properties:" + leftConstraints);
+            Set<String> leftListeners = propertyChangeListeners.values().stream()
+                                                               .map(l -> l.wrappers.keySet())
+                                                               .flatMap(Collection::stream)
+                                                               .collect(Collectors.toSet());
+            leftListeners.removeAll(properties.keySet());
+            if (!leftListeners.isEmpty())
+                throw new ConfigurationException("Listeners are defined for non-existing properties:" + leftListeners);
+            initialized = true;
+        }
+        finally
+        {
+            rwLock.writeLock().unlock();
+        }
+    }
+
+    @Override public void set(String name, @Nullable Object value)
+    {
+        lazyInit();
+        PropertyAdapter property = properties.get(name);
+        validatePropertyExists(property, name);
+        setInternal(property, value);
+    }
+
+    /**
+     * Setter for the property with the given name. Can accept {@code null} value.
+     * @param property The property.
+     * @param value The value to set.
+     */
+    private void setInternal(PropertyAdapter property, @Nullable Object value)
+    {
+        rwLock.writeLock().lock();
+        try
+        {
+            Class<?> originalType = property.getType();
+            Class<?> sourceType = value == null ? null : value.getClass();
+            Object convertedValue = value;
+            // Do conversion if the value is not null and the type is not the same as the property type.
+            if (sourceType != null && !primitiveToWrapper(originalType).equals(sourceType))
+            {
+                StringConverters converter;
+                if (sourceType.equals(String.class) && (converter = StringConverters.fromType(originalType)) != null)
+                    convertedValue = converter.fromString((String) value, originalType);
+                else
+                    throw new IllegalArgumentException(String.format("No converter found for type '%s'", originalType.getName()));
+            }

Review Comment:
   this section isn't needed.  Why are we trying to support saving the wrong type?  This adds a lot of complexity and possible bugs, we should keep this super simple.
   
   ```
   Object oldValue = property.getValue();
                propertyChangeListeners.get(ConfigurationListener.ChangeType.BEFORE).fireTypeCast(property.getName(), oldValue, convertedValue);
                property.setValue(convertedValue);
                propertyChangeListeners.get(ConfigurationListener.ChangeType.AFTER).fireTypeCast(property.getName(), oldValue, convertedValue);
                // This potentially may expose the values that are not safe to see in logs on production.
                logger.info("Property '{}' updated from '{}' to '{}'.", property.getName(), oldValue, convertedValue);
   ```



##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.Mutable;
+import org.apache.cassandra.config.StringConverters;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.yaml.snakeyaml.introspector.Property;
+
+import static org.apache.cassandra.config.Properties.defaultLoader;
+import static org.apache.cassandra.config.registry.PrimitiveUnaryConverter.convertSafe;
+import static org.apache.commons.lang3.ClassUtils.primitiveToWrapper;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings, it doesn't
+ * take into account any configuration changes that might happen during properties replacement between releases.
+ */
+public class ConfigurationRegistry implements Registry
+{
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Supplier<Config> configProvider;
+    private final Map<ConfigurationListener.ChangeType, ConfigurationListenerList> propertyChangeListeners = new EnumMap<>(ConfigurationListener.ChangeType.class);
+    private final Map<String, List<TypedConstraintAdapter<?>>> constraints = new HashMap<>();
+    private volatile boolean initialized;
+    private Map<String, PropertyAdapter> properties = Collections.emptyMap();
+
+    public ConfigurationRegistry(Supplier<Config> configProvider)
+    {
+        this.configProvider = configProvider;
+        // Initialize the property change listeners.
+        for (ConfigurationListener.ChangeType type : ConfigurationListener.ChangeType.values())
+            propertyChangeListeners.put(type, new ConfigurationListenerList());
+    }
+
+    private void lazyInit()
+    {
+        if (initialized)
+            return;
+
+        rwLock.writeLock().lock();
+        try
+        {
+            if (initialized)
+                return;
+            properties = ImmutableMap.copyOf(defaultLoader()
+                                             .flatten(Config.class)
+                                             .entrySet()
+                                             .stream()
+                                             .map(e -> new AbstractMap.SimpleEntry<>(e.getKey(), new PropertyAdapter(configProvider, e.getValue())))
+                                             .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
+            Set<String> leftConstraints = new HashSet<>(constraints.keySet());
+            leftConstraints.removeAll(properties.keySet());
+            if (!leftConstraints.isEmpty())
+                throw new ConfigurationException("Constraints are defined for non-existing properties:" + leftConstraints);
+            Set<String> leftListeners = propertyChangeListeners.values().stream()
+                                                               .map(l -> l.wrappers.keySet())
+                                                               .flatMap(Collection::stream)
+                                                               .collect(Collectors.toSet());
+            leftListeners.removeAll(properties.keySet());
+            if (!leftListeners.isEmpty())
+                throw new ConfigurationException("Listeners are defined for non-existing properties:" + leftListeners);
+            initialized = true;
+        }
+        finally
+        {
+            rwLock.writeLock().unlock();
+        }
+    }
+
+    @Override public void set(String name, @Nullable Object value)
+    {
+        lazyInit();
+        PropertyAdapter property = properties.get(name);

Review Comment:
   concurrency bug.. because you are trying to use these locks to defer logic, you need to access these via the locks as well.



##########
src/java/org/apache/cassandra/db/virtual/SettingsTable.java:
##########
@@ -68,54 +91,48 @@ public DataSet data(DecoratedKey partitionKey)
         SimpleDataSet result = new SimpleDataSet(metadata());
         String name = UTF8Type.instance.compose(partitionKey.getKey());
         if (BACKWARDS_COMPATABLE_NAMES.containsKey(name))
-            ClientWarn.instance.warn("key '" + name + "' is deprecated; should switch to '" + BACKWARDS_COMPATABLE_NAMES.get(name) + "'");
-        if (PROPERTIES.containsKey(name))
-            result.row(name).column(VALUE, getValue(PROPERTIES.get(name)));
+            ClientWarn.instance.warn("key '" + name + "' is deprecated; should switch to '" + BACKWARDS_COMPATABLE_NAMES.get(name) + '\'');
+        if (registry.contains(name))
+            runExceptionally(() -> result.row(name).column(VALUE, registry.getString(name)),

Review Comment:
   nit: `try/catch` is easier to fallow.



##########
src/java/org/apache/cassandra/utils/ConstantFieldsGenerateUtil.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.cassandra.utils;
+
+import java.io.File; //checkstyle: permit this import
+import java.io.FileWriter; //checkstyle: permit this import
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.net.URL;
+import java.nio.file.Paths; //checkstyle: permit this import
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.regex.Pattern;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.Mutable;
+
+/**
+ * This class is used to generate constants class of provided class.
+ */
+public class ConstantFieldsGenerateUtil

Review Comment:
   this is dead code.
   
   Also, adding in code gen is a problem as it makes this solution far too complex.  Can we remove and avoid code gen all together?  I personally want to remove the `ConfigFields` class as well as that is a problem to maintain.



##########
src/java/org/apache/cassandra/service/StorageService.java:
##########
@@ -972,6 +974,8 @@ public void runMayThrow() throws InterruptedException, ExecutionException, IOExc
             logger.info("Not joining ring as requested. Use JMX (StorageService->joinRing()) to initiate ring joining");
         }
 
+        getConfigRegistry().addPropertyConstraint(REPAIR_REQUEST_TIMEOUT,
+                                                  DurationSpec.LongMillisecondsBound.class, (newVal) -> Preconditions.checkState(newVal.toMilliseconds() > 0));

Review Comment:
   why does this exist?
   
   `DurationSpec.LongMillisecondsBound` adds restrictions on the input, so your logic looks to duplicate?
   
   You are also doing it *AFTER* configs have been accepted, so this is far too late to add.
   
   You also do not add a useful error message, so this gets hard for users



##########
src/java/org/apache/cassandra/db/commitlog/CommitLog.java:
##########
@@ -137,7 +150,13 @@ synchronized public CommitLog start()
             segmentManager.start();
             executor.start();
             started = true;
-        } catch (Throwable t)
+
+            DatabaseDescriptor.getConfigRegistry().addPropertyConstraint(CDC_BLOCK_WRITES, Boolean.TYPE, (newVal) -> ensureCDCEnabled());
+            DatabaseDescriptor.getConfigRegistry().addPropertyConstraint(CDC_ON_REPAIR_ENABLED, Boolean.TYPE, (newVal) -> ensureCDCEnabled());
+            DatabaseDescriptor.getConfigRegistry().addPropertyChangeListener(CDC_BLOCK_WRITES, ConfigurationListener.ChangeType.BEFORE,
+                                                                             this::cdcBlockWritesBeforeChangeListener, Boolean.TYPE);

Review Comment:
   why does this API flip type and function?  others are type, function; this is function, type



##########
src/java/org/apache/cassandra/config/registry/PrimitiveUnaryConverter.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.cassandra.config.registry;
+
+import javax.annotation.Nonnull;
+
+import org.apache.cassandra.exceptions.ConfigurationException;
+
+/**
+ * Converter do type conversion from the specified Object  value to the given {@code T}. If the class is
+ * a primitive type (e.g. Boolean.TYPE, Long.TYPE etc), the value returned will use the corresponding
+ * wrapper type (Long.class, Boolean.class, etc).
+ *
+ * @see TypeConverter
+ * @see org.apache.cassandra.config.StringConverters
+ */
+public class PrimitiveUnaryConverter
+{
+    private static Object to(Class<?> cls, @Nonnull Object value)

Review Comment:
   this feels unneeded, if you follow the feedback I gave and use typed classes, then any violation is blocked during the cast that java adds...



##########
src/java/org/apache/cassandra/db/commitlog/CommitLog.java:
##########
@@ -137,7 +150,13 @@ synchronized public CommitLog start()
             segmentManager.start();
             executor.start();
             started = true;
-        } catch (Throwable t)
+
+            DatabaseDescriptor.getConfigRegistry().addPropertyConstraint(CDC_BLOCK_WRITES, Boolean.TYPE, (newVal) -> ensureCDCEnabled());

Review Comment:
   if you add this to start don't you need to remove on shutdown?



##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.Mutable;
+import org.apache.cassandra.config.StringConverters;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.yaml.snakeyaml.introspector.Property;
+
+import static org.apache.cassandra.config.Properties.defaultLoader;
+import static org.apache.cassandra.config.registry.PrimitiveUnaryConverter.convertSafe;
+import static org.apache.commons.lang3.ClassUtils.primitiveToWrapper;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings, it doesn't
+ * take into account any configuration changes that might happen during properties replacement between releases.
+ */
+public class ConfigurationRegistry implements Registry
+{
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Supplier<Config> configProvider;
+    private final Map<ConfigurationListener.ChangeType, ConfigurationListenerList> propertyChangeListeners = new EnumMap<>(ConfigurationListener.ChangeType.class);
+    private final Map<String, List<TypedConstraintAdapter<?>>> constraints = new HashMap<>();
+    private volatile boolean initialized;
+    private Map<String, PropertyAdapter> properties = Collections.emptyMap();
+
+    public ConfigurationRegistry(Supplier<Config> configProvider)
+    {
+        this.configProvider = configProvider;
+        // Initialize the property change listeners.
+        for (ConfigurationListener.ChangeType type : ConfigurationListener.ChangeType.values())
+            propertyChangeListeners.put(type, new ConfigurationListenerList());
+    }
+
+    private void lazyInit()
+    {
+        if (initialized)
+            return;
+
+        rwLock.writeLock().lock();
+        try
+        {
+            if (initialized)
+                return;
+            properties = ImmutableMap.copyOf(defaultLoader()
+                                             .flatten(Config.class)
+                                             .entrySet()
+                                             .stream()
+                                             .map(e -> new AbstractMap.SimpleEntry<>(e.getKey(), new PropertyAdapter(configProvider, e.getValue())))
+                                             .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
+            Set<String> leftConstraints = new HashSet<>(constraints.keySet());

Review Comment:
   `com.google.common.collect.Sets#difference` is cheaper



##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.Mutable;
+import org.apache.cassandra.config.StringConverters;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.yaml.snakeyaml.introspector.Property;
+
+import static org.apache.cassandra.config.Properties.defaultLoader;
+import static org.apache.cassandra.config.registry.PrimitiveUnaryConverter.convertSafe;
+import static org.apache.commons.lang3.ClassUtils.primitiveToWrapper;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings, it doesn't
+ * take into account any configuration changes that might happen during properties replacement between releases.
+ */
+public class ConfigurationRegistry implements Registry
+{
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Supplier<Config> configProvider;
+    private final Map<ConfigurationListener.ChangeType, ConfigurationListenerList> propertyChangeListeners = new EnumMap<>(ConfigurationListener.ChangeType.class);
+    private final Map<String, List<TypedConstraintAdapter<?>>> constraints = new HashMap<>();
+    private volatile boolean initialized;
+    private Map<String, PropertyAdapter> properties = Collections.emptyMap();
+
+    public ConfigurationRegistry(Supplier<Config> configProvider)
+    {
+        this.configProvider = configProvider;
+        // Initialize the property change listeners.
+        for (ConfigurationListener.ChangeType type : ConfigurationListener.ChangeType.values())
+            propertyChangeListeners.put(type, new ConfigurationListenerList());
+    }
+
+    private void lazyInit()
+    {
+        if (initialized)
+            return;
+
+        rwLock.writeLock().lock();
+        try
+        {
+            if (initialized)
+                return;
+            properties = ImmutableMap.copyOf(defaultLoader()
+                                             .flatten(Config.class)
+                                             .entrySet()
+                                             .stream()
+                                             .map(e -> new AbstractMap.SimpleEntry<>(e.getKey(), new PropertyAdapter(configProvider, e.getValue())))
+                                             .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
+            Set<String> leftConstraints = new HashSet<>(constraints.keySet());
+            leftConstraints.removeAll(properties.keySet());
+            if (!leftConstraints.isEmpty())
+                throw new ConfigurationException("Constraints are defined for non-existing properties:" + leftConstraints);
+            Set<String> leftListeners = propertyChangeListeners.values().stream()
+                                                               .map(l -> l.wrappers.keySet())
+                                                               .flatMap(Collection::stream)
+                                                               .collect(Collectors.toSet());
+            leftListeners.removeAll(properties.keySet());
+            if (!leftListeners.isEmpty())
+                throw new ConfigurationException("Listeners are defined for non-existing properties:" + leftListeners);

Review Comment:
   properties are static and immutable, so why even allow this to happen?  This feels like trying to be lazy caused this, but there is no reason to do that and you can just make properties static and loaded during class loading, then you can block such cases from happening *at the call site*, which is far easier to work with



##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.Mutable;
+import org.apache.cassandra.config.StringConverters;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.yaml.snakeyaml.introspector.Property;
+
+import static org.apache.cassandra.config.Properties.defaultLoader;
+import static org.apache.cassandra.config.registry.PrimitiveUnaryConverter.convertSafe;
+import static org.apache.commons.lang3.ClassUtils.primitiveToWrapper;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings, it doesn't
+ * take into account any configuration changes that might happen during properties replacement between releases.
+ */
+public class ConfigurationRegistry implements Registry
+{
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();

Review Comment:
   this class is more complex than it needs to due to these locks... I strongly believe that we know all properties/"validations" at object creation, so those should be final and immutable... this makes it so the only *mutable* field is the listeners, which can use standard concurrent collections to avoid needing to manage our locks ourselves; they also make it so we can have stronger concurrent access to this class, right now contention can add up slowing down the system.



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Mmuzaf commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "Mmuzaf (via GitHub)" <gi...@apache.org>.
Mmuzaf commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1149871406


##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -4676,4 +4729,41 @@ public static void setClientRequestSizeMetricsEnabled(boolean enabled)
     {
         return Objects.requireNonNull(sstableFormatFactories, "Forgot to initialize DatabaseDescriptor?");
     }
+
+    /**
+     * Set configuration property for the given name to {@link #confRegistry} if a safe manner
+     * with handling internal Cassandra exceptions.
+     *
+     * @param name Property name.
+     * @param value Property value.
+     */
+    private static void setProperty(String name, Object value)
+    {
+        runExceptionally(() -> confRegistry.set(name, value), new SearchInternalCauseForPublicAPI());
+    }
+
+    private static <T> T getProperty(Class<T> cls, String name)
+    {
+        return callExceptionally(() -> confRegistry.get(cls, name), new SearchInternalCauseForPublicAPI());
+    }
+
+    private static class SearchInternalCauseForPublicAPI implements Function<Exception, RuntimeException>
+    {
+        @Override
+        public RuntimeException apply(Exception e)
+        {
+            RuntimeException rt;
+            if ((rt = cause(e, IllegalArgumentException.class)) != null)
+                return new IllegalArgumentException(rt.getMessage());
+            else if ((rt = cause(e, IllegalStateException.class)) != null)
+                return new IllegalStateException(rt.getMessage());
+            else if ((rt = cause(e, UnsupportedOperationException.class)) != null)
+                return new UnsupportedOperationException(rt.getMessage());
+            else
+            {
+                logger.error("Unexpected exception", e);
+                return new RuntimeException(e.getMessage());
+            }
+        }
+    }

Review Comment:
   The reason for this exception conversion is that, for example, the JMX client may not know about Cassandra classes (missing in its classpath), so it will fail if the `ConfigurationRegistry` throws an exception that has them. 
   
   Here is an example of a potential issues:
   https://issues.apache.org/jira/browse/CASSANDRA-17527
   
   I just re-think the usage once will find a common ground for the properties validation process on the ML. 



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Mmuzaf commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "Mmuzaf (via GitHub)" <gi...@apache.org>.
Mmuzaf commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1149839378


##########
src/java/org/apache/cassandra/config/registry/ConfigurationConstraint.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.cassandra.config.registry;
+
+/**
+ * Interface validating configuration property's value.
+ */
+public interface ConfigurationConstraint<T>

Review Comment:
   As we have discussed, we need to find a robust way of validating the configuration fields. This interface will provide a way to build a custom validation for each field that will be used to check the input, regardless of what type of user interface is being used: JMX, Yaml, SettingsTable. 
   I will start a public discussion on the dev list and try to find common ground.



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Mmuzaf commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "Mmuzaf (via GitHub)" <gi...@apache.org>.
Mmuzaf commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1149864738


##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.Mutable;
+import org.apache.cassandra.config.StringConverters;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.yaml.snakeyaml.introspector.Property;
+
+import static org.apache.cassandra.config.Properties.defaultLoader;
+import static org.apache.cassandra.config.registry.PrimitiveUnaryConverter.convertSafe;
+import static org.apache.commons.lang3.ClassUtils.primitiveToWrapper;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings, it doesn't
+ * take into account any configuration changes that might happen during properties replacement between releases.
+ */
+public class ConfigurationRegistry implements Registry
+{
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Supplier<Config> configProvider;
+    private final Map<ConfigurationListener.ChangeType, ConfigurationListenerList> propertyChangeListeners = new EnumMap<>(ConfigurationListener.ChangeType.class);
+    private final Map<String, List<TypedConstraintAdapter<?>>> constraints = new HashMap<>();
+    private volatile boolean initialized;
+    private Map<String, PropertyAdapter> properties = Collections.emptyMap();
+
+    public ConfigurationRegistry(Supplier<Config> configProvider)
+    {
+        this.configProvider = configProvider;
+        // Initialize the property change listeners.
+        for (ConfigurationListener.ChangeType type : ConfigurationListener.ChangeType.values())
+            propertyChangeListeners.put(type, new ConfigurationListenerList());
+    }
+
+    private void lazyInit()
+    {
+        if (initialized)
+            return;
+
+        rwLock.writeLock().lock();
+        try
+        {
+            if (initialized)
+                return;
+            properties = ImmutableMap.copyOf(defaultLoader()
+                                             .flatten(Config.class)
+                                             .entrySet()
+                                             .stream()
+                                             .map(e -> new AbstractMap.SimpleEntry<>(e.getKey(), new PropertyAdapter(configProvider, e.getValue())))
+                                             .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
+            Set<String> leftConstraints = new HashSet<>(constraints.keySet());
+            leftConstraints.removeAll(properties.keySet());
+            if (!leftConstraints.isEmpty())
+                throw new ConfigurationException("Constraints are defined for non-existing properties:" + leftConstraints);
+            Set<String> leftListeners = propertyChangeListeners.values().stream()
+                                                               .map(l -> l.wrappers.keySet())
+                                                               .flatMap(Collection::stream)
+                                                               .collect(Collectors.toSet());
+            leftListeners.removeAll(properties.keySet());
+            if (!leftListeners.isEmpty())
+                throw new ConfigurationException("Listeners are defined for non-existing properties:" + leftListeners);
+            initialized = true;
+        }
+        finally
+        {
+            rwLock.writeLock().unlock();
+        }
+    }
+
+    @Override public void set(String name, @Nullable Object value)
+    {
+        lazyInit();
+        PropertyAdapter property = properties.get(name);
+        validatePropertyExists(property, name);
+        setInternal(property, value);
+    }
+
+    /**
+     * Setter for the property with the given name. Can accept {@code null} value.
+     * @param property The property.
+     * @param value The value to set.
+     */
+    private void setInternal(PropertyAdapter property, @Nullable Object value)
+    {
+        rwLock.writeLock().lock();
+        try
+        {
+            Class<?> originalType = property.getType();
+            Class<?> sourceType = value == null ? null : value.getClass();
+            Object convertedValue = value;
+            // Do conversion if the value is not null and the type is not the same as the property type.
+            if (sourceType != null && !primitiveToWrapper(originalType).equals(sourceType))
+            {
+                StringConverters converter;
+                if (sourceType.equals(String.class) && (converter = StringConverters.fromType(originalType)) != null)
+                    convertedValue = converter.fromString((String) value, originalType);
+                else
+                    throw new IllegalArgumentException(String.format("No converter found for type '%s'", originalType.getName()));
+            }
+            // Do validation first for converted new value.
+            List<TypedConstraintAdapter<?>> constraintsList = constraints.getOrDefault(property.getName(), Collections.emptyList());
+            for (TypedConstraintAdapter<?> typed : constraintsList)
+                typed.validateTypeCast(convertedValue);
+            // Do set the value only if the validation passes.
+            Object oldValue = property.getValue();
+            propertyChangeListeners.get(ConfigurationListener.ChangeType.BEFORE).fireTypeCast(property.getName(), oldValue, convertedValue);
+            property.setValue(convertedValue);
+            propertyChangeListeners.get(ConfigurationListener.ChangeType.AFTER).fireTypeCast(property.getName(), oldValue, convertedValue);
+            // This potentially may expose the values that are not safe to see in logs on production.
+            logger.info("Property '{}' updated from '{}' to '{}'.", property.getName(), oldValue, convertedValue);
+        }
+        catch (Exception e)
+        {
+            if (e instanceof ConfigurationException)
+                throw (ConfigurationException) e;
+            else
+                throw new ConfigurationException(String.format("Error updating property '%s'; cause: %s", property.getName(), e.getMessage()), e);
+        }
+        finally
+        {
+            rwLock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * @param cls Class to cast the property value to.
+     * @param name the property name to get.
+     * @return The value of the property with the given name.
+     */
+    public <T> T get(Class<T> cls, String name)
+    {
+        lazyInit();
+        rwLock.readLock().lock();
+        try
+        {
+            validatePropertyExists(properties.get(name), name);
+            Class<?> propertyType = type(name);
+            Object value = properties.get(name).getValue();
+            if (cls.equals(propertyType))
+                return convertSafe(cls, value);
+            else if (cls.equals(String.class))
+            {
+                StringConverters converter = StringConverters.fromType(propertyType);
+                return cls.cast(converter == null ? TypeConverter.DEFAULT.convertNullable(value) : converter.toString(value));
+            }
+            else
+                throw new ConfigurationException(String.format("Property '%s' is of type '%s' and cannot be cast to '%s'",
+                                                               name, propertyType.getCanonicalName(), cls.getCanonicalName()));
+        }
+        finally
+        {
+            rwLock.readLock().unlock();
+        }
+    }
+
+    @Override public String getString(String name)
+    {
+        return get(String.class, name);
+    }
+
+    /**
+     * @param name the property name to check.
+     * @return {@code true} if the property with the given name is available, {@code false} otherwise.
+     */
+    @Override public boolean contains(String name)
+    {
+        lazyInit();
+        return properties.containsKey(name);
+    }
+
+    /**
+     * Returns a set of all the property names.
+     * @return set of all the property names.
+     */
+    @Override public Iterable<String> keys()
+    {
+        lazyInit();
+        return properties.keySet();
+    }
+
+    /**
+     * @param name The property name to get the type for.
+     * @return Property type for the property with the given name.
+     */
+    @Override public Class<?> type(String name)
+    {
+        lazyInit();
+        validatePropertyExists(properties.get(name), name);
+        return properties.get(name).getType();
+    }
+
+    /**
+     * @return The number of properties.
+     */
+    @Override public int size()
+    {
+        lazyInit();
+        return properties.size();
+    }
+
+    /**
+     * @param name The property name to get the type for.
+     * @return Property type for the property with the given name.
+     */
+    public boolean isWritable(String name)
+    {
+        validatePropertyExists(properties.get(name), name);
+        return properties.get(name).isWritable();
+    }
+
+    /**
+     * Adds a listener for the property with the given name.
+     * @param name property name to listen to.
+     * @param listener listener to add.
+     * @param <T> type of the property.
+     */
+    public <T> void addPropertyChangeListener(String name, ConfigurationListener.ChangeType type, ConfigurationListener<T> listener, Class<T> listenerType)

Review Comment:
   In my view, if you change the name of a property, you must also update the corresponding listener usage. I propose to enforce this rule and avoid supporting different names for the same properties in internal code, so only limited subsystems will work with the `Replaced` annotation and know the old names.
   
   The `fail fast` rule will work once the `addPropertyChangeListener` is fired with an incorrect property name.



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Mmuzaf commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "Mmuzaf (via GitHub)" <gi...@apache.org>.
Mmuzaf commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1149865189


##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1504,137 +1599,131 @@ public static void setRoleManager(IRoleManager roleManager)
 
     public static int getPermissionsValidity()
     {
-        return conf.permissions_validity.toMilliseconds();
+        return getProperty(DurationSpec.IntMillisecondsBound.class, ConfigFields.PERMISSIONS_VALIDITY).toMilliseconds();

Review Comment:
   Agree, let's move this and all such changes out of the scope. 



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Mmuzaf commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "Mmuzaf (via GitHub)" <gi...@apache.org>.
Mmuzaf commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1149840265


##########
src/java/org/apache/cassandra/config/StringConverters.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.cassandra.config;
+
+import org.apache.cassandra.config.registry.TypeConverter;
+import org.apache.cassandra.db.ConsistencyLevel;
+
+import static org.apache.cassandra.config.registry.PrimitiveUnaryConverter.convertSafe;
+
+/**
+ * String converters for Cassandra configuration types used in {@link Config}. These are used to convert
+ * configuration values from their string representation to their actual type and back. The converters
+ * are uses the {@link TypeConverter} interface.
+ * <p>
+ * Take care when adding new converters, only one converter per type is allowed. If a converter for a
+ * type already exists, the new converter will be ignored since the first converter for a type is used.
+ *
+ * @see Config
+ * @see TypeConverter
+ */
+public enum StringConverters
+{
+    PRIMITIVE_BOOLEAN(Boolean.TYPE, s -> Boolean.parseBoolean((String) s), b -> Boolean.toString((Boolean) b)),
+    PRIMITIVE_DOUBLE(Double.TYPE, s -> Double.parseDouble((String) s),  d -> Double.toString((Double) d)),
+    PRIMITIVE_INTEGER(Integer.TYPE, s -> Integer.parseInt((String) s), i -> Integer.toString((Integer) i)),
+    PRIMITIVE_LONG(Long.TYPE, s -> Long.parseLong((String) s), l -> Long.toString((Long) l)),
+    BOOLEAN(Boolean.class, s -> Boolean.parseBoolean((String) s), b -> Boolean.toString((Boolean) b)),
+    DOUBLE(Double.class, s -> Double.parseDouble((String) s),  d -> Double.toString((Double) d)),
+    INTEGER(Integer.class, s -> Integer.parseInt((String) s), i -> Integer.toString((Integer) i)),
+    LONG(Long.class, s -> Long.parseLong((String) s), l -> Long.toString((Long) l)),
+    STRING(String.class, s -> (String) s, s -> (String) s),
+    // Cassandra specific configuration types.
+    LONG_NANOSECONDS_BOUND(DurationSpec.LongNanosecondsBound.class, s -> new DurationSpec.LongNanosecondsBound((String) s), TypeConverter.DEFAULT),
+    LONG_MILLISECONDS_BOUND(DurationSpec.LongMillisecondsBound.class, s -> new DurationSpec.LongMillisecondsBound((String) s), TypeConverter.DEFAULT),
+    LONG_SECONDS_BOUND(DurationSpec.LongSecondsBound.class, s -> new DurationSpec.LongSecondsBound((String) s), TypeConverter.DEFAULT),
+    INT_MINUSTES_BOUND(DurationSpec.IntMinutesBound.class, s -> new DurationSpec.IntMinutesBound((String) s), TypeConverter.DEFAULT),
+    INT_SECONDS_BOUND(DurationSpec.IntSecondsBound.class, s -> new DurationSpec.IntSecondsBound((String) s), TypeConverter.DEFAULT),
+    INT_MILLISECONDS_BOUND(DurationSpec.IntMillisecondsBound.class, s -> new DurationSpec.IntMillisecondsBound((String) s), TypeConverter.DEFAULT),
+    LONG_BYTES_BOUND(DataStorageSpec.LongBytesBound.class, s -> new DataStorageSpec.LongBytesBound((String) s), TypeConverter.DEFAULT),
+    INT_BYTES_BOUND(DataStorageSpec.IntBytesBound.class, s -> new DataStorageSpec.IntBytesBound((String) s), TypeConverter.DEFAULT),
+    INT_KIBIBYTES_BOUND(DataStorageSpec.IntKibibytesBound.class, s -> new DataStorageSpec.IntKibibytesBound((String) s), TypeConverter.DEFAULT),
+    LONG_MEBIBYTES_BOUND(DataStorageSpec.LongMebibytesBound.class, s -> new DataStorageSpec.LongMebibytesBound((String) s), TypeConverter.DEFAULT),
+    INT_MEBIBYTES_BOUND(DataStorageSpec.IntMebibytesBound.class, s -> new DataStorageSpec.IntMebibytesBound((String) s), TypeConverter.DEFAULT),
+    CONSYSTENCY_LEVEL(ConsistencyLevel.class, s -> ConsistencyLevel.fromStringIgnoreCase((String) s), c -> ((ConsistencyLevel) c).name());
+
+    private final Class<?> type;
+    private final TypeConverter<?> forward;
+    private final TypeConverter<String> reverse;
+
+    /**
+     * Creates a new converter for the given type and using the given converters.
+     * @param type the type this converter converts to and from.
+     * @param forward the forward converter to use.
+     * @param reverse the reverse converter to use.
+     */
+    <T> StringConverters(Class<T> type, TypeConverter<T> forward, TypeConverter<String> reverse)
+    {
+        this.type = type;
+        this.forward = forward;
+        this.reverse = reverse;
+    }
+    public <T> T fromString(String value, Class<T> target)
+    {
+        if (target.equals(type))
+            return convertSafe(type, forward.convertNullable(value));
+        throw new IllegalArgumentException(String.format("Invalid target type '%s' for converter '%s'", target, this));
+    }
+
+    public String toString(Object value)
+    {
+        return reverse.convertNullable(value);
+    }
+
+    /**
+     * Returns the converter for the given type.
+     * @param type the type to return the converter for.
+     * @return the converter for the given type or {@code null} if no converter exists for the given type
+     * or default conversion should be used instead.
+     */
+    public static StringConverters fromType(Class<?> type)
+    {
+        for (StringConverters converter : values())
+            if (converter.type.equals(type))
+                return converter;
+
+        return null;
+    }

Review Comment:
   I think you're right here, we don't need such complexity with `Enum`s and the `Map` is the easiest solution for us.



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Mmuzaf commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "Mmuzaf (via GitHub)" <gi...@apache.org>.
Mmuzaf commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1149857695


##########
src/java/org/apache/cassandra/service/StorageService.java:
##########
@@ -972,6 +974,8 @@ public void runMayThrow() throws InterruptedException, ExecutionException, IOExc
             logger.info("Not joining ring as requested. Use JMX (StorageService->joinRing()) to initiate ring joining");
         }
 
+        getConfigRegistry().addPropertyConstraint(REPAIR_REQUEST_TIMEOUT,
+                                                  DurationSpec.LongMillisecondsBound.class, (newVal) -> Preconditions.checkState(newVal.toMilliseconds() > 0));

Review Comment:
   I agree with you - this is not the right place to have property constraints, so we will move it to the validation subsystem once the discussion on the ML is finished.
   
   The reason why it exists is simple - the type `DurationSpec.LongMillisecondsBound` itself provides only `<= 0` guarantees, however, the property requires strickly `> 0`.



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1146470588


##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.Mutable;
+import org.apache.cassandra.config.StringConverters;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.yaml.snakeyaml.introspector.Property;
+
+import static org.apache.cassandra.config.Properties.defaultLoader;
+import static org.apache.cassandra.config.registry.PrimitiveUnaryConverter.convertSafe;
+import static org.apache.commons.lang3.ClassUtils.primitiveToWrapper;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings, it doesn't
+ * take into account any configuration changes that might happen during properties replacement between releases.
+ */
+public class ConfigurationRegistry implements Registry
+{
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Supplier<Config> configProvider;
+    private final Map<ConfigurationListener.ChangeType, ConfigurationListenerList> propertyChangeListeners = new EnumMap<>(ConfigurationListener.ChangeType.class);
+    private final Map<String, List<TypedConstraintAdapter<?>>> constraints = new HashMap<>();
+    private volatile boolean initialized;
+    private Map<String, PropertyAdapter> properties = Collections.emptyMap();
+
+    public ConfigurationRegistry(Supplier<Config> configProvider)
+    {
+        this.configProvider = configProvider;
+        // Initialize the property change listeners.
+        for (ConfigurationListener.ChangeType type : ConfigurationListener.ChangeType.values())
+            propertyChangeListeners.put(type, new ConfigurationListenerList());
+    }
+
+    private void lazyInit()
+    {
+        if (initialized)
+            return;
+
+        rwLock.writeLock().lock();
+        try
+        {
+            if (initialized)
+                return;
+            properties = ImmutableMap.copyOf(defaultLoader()
+                                             .flatten(Config.class)
+                                             .entrySet()
+                                             .stream()
+                                             .map(e -> new AbstractMap.SimpleEntry<>(e.getKey(), new PropertyAdapter(configProvider, e.getValue())))
+                                             .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
+            Set<String> leftConstraints = new HashSet<>(constraints.keySet());
+            leftConstraints.removeAll(properties.keySet());
+            if (!leftConstraints.isEmpty())
+                throw new ConfigurationException("Constraints are defined for non-existing properties:" + leftConstraints);
+            Set<String> leftListeners = propertyChangeListeners.values().stream()
+                                                               .map(l -> l.wrappers.keySet())
+                                                               .flatMap(Collection::stream)
+                                                               .collect(Collectors.toSet());
+            leftListeners.removeAll(properties.keySet());
+            if (!leftListeners.isEmpty())
+                throw new ConfigurationException("Listeners are defined for non-existing properties:" + leftListeners);
+            initialized = true;
+        }
+        finally
+        {
+            rwLock.writeLock().unlock();
+        }
+    }
+
+    @Override public void set(String name, @Nullable Object value)
+    {
+        lazyInit();
+        PropertyAdapter property = properties.get(name);
+        validatePropertyExists(property, name);
+        setInternal(property, value);
+    }
+
+    /**
+     * Setter for the property with the given name. Can accept {@code null} value.
+     * @param property The property.
+     * @param value The value to set.
+     */
+    private void setInternal(PropertyAdapter property, @Nullable Object value)
+    {
+        rwLock.writeLock().lock();
+        try
+        {
+            Class<?> originalType = property.getType();
+            Class<?> sourceType = value == null ? null : value.getClass();
+            Object convertedValue = value;
+            // Do conversion if the value is not null and the type is not the same as the property type.
+            if (sourceType != null && !primitiveToWrapper(originalType).equals(sourceType))
+            {
+                StringConverters converter;
+                if (sourceType.equals(String.class) && (converter = StringConverters.fromType(originalType)) != null)
+                    convertedValue = converter.fromString((String) value, originalType);
+                else
+                    throw new IllegalArgumentException(String.format("No converter found for type '%s'", originalType.getName()));
+            }
+            // Do validation first for converted new value.
+            List<TypedConstraintAdapter<?>> constraintsList = constraints.getOrDefault(property.getName(), Collections.emptyList());
+            for (TypedConstraintAdapter<?> typed : constraintsList)
+                typed.validateTypeCast(convertedValue);
+            // Do set the value only if the validation passes.
+            Object oldValue = property.getValue();
+            propertyChangeListeners.get(ConfigurationListener.ChangeType.BEFORE).fireTypeCast(property.getName(), oldValue, convertedValue);
+            property.setValue(convertedValue);
+            propertyChangeListeners.get(ConfigurationListener.ChangeType.AFTER).fireTypeCast(property.getName(), oldValue, convertedValue);
+            // This potentially may expose the values that are not safe to see in logs on production.
+            logger.info("Property '{}' updated from '{}' to '{}'.", property.getName(), oldValue, convertedValue);
+        }
+        catch (Exception e)
+        {
+            if (e instanceof ConfigurationException)
+                throw (ConfigurationException) e;
+            else
+                throw new ConfigurationException(String.format("Error updating property '%s'; cause: %s", property.getName(), e.getMessage()), e);
+        }
+        finally
+        {
+            rwLock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * @param cls Class to cast the property value to.
+     * @param name the property name to get.
+     * @return The value of the property with the given name.
+     */
+    public <T> T get(Class<T> cls, String name)
+    {
+        lazyInit();
+        rwLock.readLock().lock();
+        try
+        {
+            validatePropertyExists(properties.get(name), name);
+            Class<?> propertyType = type(name);
+            Object value = properties.get(name).getValue();
+            if (cls.equals(propertyType))
+                return convertSafe(cls, value);
+            else if (cls.equals(String.class))
+            {
+                StringConverters converter = StringConverters.fromType(propertyType);
+                return cls.cast(converter == null ? TypeConverter.DEFAULT.convertNullable(value) : converter.toString(value));
+            }
+            else
+                throw new ConfigurationException(String.format("Property '%s' is of type '%s' and cannot be cast to '%s'",
+                                                               name, propertyType.getCanonicalName(), cls.getCanonicalName()));
+        }
+        finally
+        {
+            rwLock.readLock().unlock();
+        }
+    }
+
+    @Override public String getString(String name)
+    {
+        return get(String.class, name);
+    }
+
+    /**
+     * @param name the property name to check.
+     * @return {@code true} if the property with the given name is available, {@code false} otherwise.
+     */
+    @Override public boolean contains(String name)
+    {
+        lazyInit();
+        return properties.containsKey(name);
+    }
+
+    /**
+     * Returns a set of all the property names.
+     * @return set of all the property names.
+     */
+    @Override public Iterable<String> keys()
+    {
+        lazyInit();
+        return properties.keySet();
+    }
+
+    /**
+     * @param name The property name to get the type for.
+     * @return Property type for the property with the given name.
+     */
+    @Override public Class<?> type(String name)
+    {
+        lazyInit();
+        validatePropertyExists(properties.get(name), name);
+        return properties.get(name).getType();
+    }
+
+    /**
+     * @return The number of properties.
+     */
+    @Override public int size()
+    {
+        lazyInit();
+        return properties.size();
+    }
+
+    /**
+     * @param name The property name to get the type for.
+     * @return Property type for the property with the given name.
+     */
+    public boolean isWritable(String name)
+    {
+        validatePropertyExists(properties.get(name), name);
+        return properties.get(name).isWritable();
+    }
+
+    /**
+     * Adds a listener for the property with the given name.
+     * @param name property name to listen to.
+     * @param listener listener to add.
+     * @param <T> type of the property.
+     */
+    public <T> void addPropertyChangeListener(String name, ConfigurationListener.ChangeType type, ConfigurationListener<T> listener, Class<T> listenerType)

Review Comment:
   if you listen to a property that is `@Replaced` what *should* happen?  The common case would be a bug (we forgot to update), but there *might* be a case where you want to see if this is happening (don't know of one)...
   
   Maybe we should detect this and fail fast, force the caller to say "I know what I am doing, I want the old name!"?



##########
src/java/org/apache/cassandra/config/DataStorageSpec.java:
##########
@@ -207,6 +207,14 @@ public long toBytes()
         {
             return unit().toBytes(quantity());
         }
+
+        /**
+         * @return the amount of data storage in mebibytes.
+         */
+        public long toMebibytes()
+        {
+            return unit().toMebibytes(quantity());
+        }

Review Comment:
   unrelated changes must be removed as per our style guide



##########
src/java/org/apache/cassandra/config/registry/Registry.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.cassandra.config.registry;
+
+import javax.annotation.Nullable;
+
+/**
+ * A registry of Cassandra's configuration properties that can be updated at runtime. The {@link org.apache.cassandra.config.Config}
+ * class is the source of configuration fields, types and other metadata available to the registry. The registry is used to
+ * handle configuration properties that are loaded from the configuration file, and are set via JMX or updated through
+ * the settings virtual table.
+ * <p>
+ * You can use {@link #set(String, Object)} to update a property, in case the property is not present in the registry,
+ * an exception will be thrown. If the property is present, the registry will try to convert given value to the property's
+ * type, and if the conversion fails, an exception will be thrown. You can use the {@code String} as a value to be converted,
+ * or you can use the property's type as a value. In the latter case, no conversion will be performed.
+ * <p>
+ * You can use {@link #get(Class, String)} to get a property's value, to read the value, the registry will try to convert the
+ * property's value if the {@link #getString(String)} to String type (the converter is called to convert the value to String).
+ */
+public interface Registry
+{
+    /**
+     * Update configuration property with the given name to the given value. The value may be the same
+     * as the property's value, or it may be represented as a string. In the latter case a corresponding
+     * will be called to get the property's value matching type.
+     *
+     * @param name Property name.
+     * @param value Value to set.
+     */
+    void set(String name, @Nullable Object value);
+
+    /**
+     * Get property's value by name, The exception will be thrown if the property is not present in the registry or
+     * the property's value cannot be converted to given generic type.
+     *
+     * @param <T>  Type to convert to.
+     * @param cls Class to convert to.
+     * @param name Property name.
+     * @return Property's value matching the property's type in the Config.
+     */
+    <T> T get(Class<T> cls, String name);
+
+    /**
+     * Get property's value by name and convert it to the String type. The exception will be thrown if the property
+     * is not present in the registry.
+     *
+     * @param name Property name.
+     * @return Property's value converted to String.
+     */
+    String getString(String name);

Review Comment:
   we should remove this API as it is 100% only for SettingsTable, leave this problem for the SettingsTable



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Mmuzaf commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "Mmuzaf (via GitHub)" <gi...@apache.org>.
Mmuzaf commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1149856093


##########
src/java/org/apache/cassandra/service/StorageService.java:
##########
@@ -1536,7 +1540,6 @@ else if (tokens == null)
     public void setRpcTimeout(long value)
     {
         DatabaseDescriptor.setRpcTimeout(value);
-        logger.info("set rpc timeout to {} ms", value);

Review Comment:
   I tend to agree here, but my concern here is that someone is doing something wrong if someone relies on log messages. 
   
   The `ConfigurationRegistry` will log any changes to the property's values, but the log message will look slightly different. Anyway, a user will still be able to see it in the logs.



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Mmuzaf commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "Mmuzaf (via GitHub)" <gi...@apache.org>.
Mmuzaf commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1149837141


##########
src/java/org/apache/cassandra/config/Mutable.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.cassandra.config;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Annotation denoting that a configuration field in the {@link Config} is mutable with a live update.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.FIELD)

Review Comment:
   We can do it, but without real use it might look strange, so I suggest we extend it when we come up with such methods.



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Mmuzaf commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "Mmuzaf (via GitHub)" <gi...@apache.org>.
Mmuzaf commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1149853885


##########
src/java/org/apache/cassandra/db/virtual/SettingsTable.java:
##########
@@ -157,4 +174,85 @@ private static Map<String, String> getBackwardsCompatableNames()
 
         return names;
     }
+
+    /**
+     * This class is used to provide backwards compatable support for the settings table in case the {@link Config}
+     * metadata changes. This class will provide the old names for the properties, but will use the new name to
+     * get the value from the {@link Config} object.
+     * <p>
+     * Updating a configuration property object will throw an exception if you will try to update a deprecated property.
+     */
+    private static class BackwardsCompatableRegistry implements Registry

Review Comment:
   There are a few reasons under the hood to move away from the `ForwardingProperty` logic for the `SettingsTable`:
   - The design assumes that we have a central point of access to the `Config' instance - the `ConfigurationRegistry' - to perform mutate configuration operations, ensuring that they are changed in the same way for each public interface;
   - We want to restrict the places where the `Config' instance is accessed - the `DatabaseDescriptor' and the `ConfigurationRegistry' are enough to force the `SettingsTable` to do what it needs to do;
   - the `Replacement` provides us with the name and type conversions for properties and as there are no usages of the old names in the source code we are able to perform name conversions for the `SettingsTable` needs;
   
   Maybe I'm missing something, but I see the following access path as more verbose than giving short cuts from the `SettingsTable' directly to the `Config':
   Config -> DatabaseDescription -> ConfigurationRegistry -> ReplacementRegistry -> SettingsTable;
   
   
   



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1095171823


##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -3711,7 +3718,13 @@ public static boolean getCDCBlockWrites()
         return conf.cdc_block_writes;
     }
 
+    @Deprecated

Review Comment:
   why?  This isn't a public class, we can make breaking changes here



##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.AbstractSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.introspector.Property;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.DatabaseDescriptorWalker;
+import org.apache.cassandra.config.Loader;
+import org.apache.cassandra.config.Properties;
+import org.apache.cassandra.config.Replacement;
+import org.apache.cassandra.config.Replacements;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.utils.AbstractIterator;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings.
+ * It is a singleton and can be accessed via {@link #instance} field.
+ */
+public class ConfigurationRegistry extends AbstractMap<String, Object>
+{
+    public static final List<PropertySetterWalker> PROPERTY_SETTERS_LIST = ImmutableList.of(new DatabaseDescriptorWalker());
+    private static final Map<String, String> BACKWARDS_COMPATABLE_NAMES = ImmutableMap.copyOf(getBackwardsCompatableNames());
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    public static final ConfigurationRegistry instance = new ConfigurationRegistry();
+    private final Map<String, ConfigurationUnit<?>> registry;
+    private final TransformSet<Map.Entry<String, ConfigurationUnit<?>>, Map.Entry<String, Object>> transformSet;
+    private final Map<String, PropertyChangeListenerWrapper<?>> listeners = new HashMap<>();
+
+    public ConfigurationRegistry()
+    {
+        this(DatabaseDescriptor.getRawConfig());
+    }
+
+    public ConfigurationRegistry(Config config)
+    {
+        registry = getProperties().entrySet()
+                                  .stream()
+                                  .collect(Collectors.toMap(Map.Entry::getKey, e -> new ConfigurationUnit<>(config, e.getKey(), e.getValue())));
+
+        ConfigurationSettersEnricher enricher = new ConfigurationSettersEnricher(registry);
+        PROPERTY_SETTERS_LIST.forEach(w -> w.walk(enricher));
+        transformSet = new TransformSet<>(registry.entrySet(),
+                                          e -> new AbstractMap.SimpleEntry<>(e.getKey(), e.getValue().getValue()));
+    }
+
+    /**
+     * Setter for the property with the given name. Can accept {@code null} value.
+     * @param name the name of the property.
+     * @param value the value to set.
+     */
+    public void update(String name, Object value)
+    {
+        ConfigurationUnit<?> unit = registry.get(name);
+
+        if (unit == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not availabe.", name));
+
+        if (unit.setter == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not writable.", name));
+
+        unit.lock.lock();

Review Comment:
   why do we need locks?



##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.AbstractSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.introspector.Property;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.DatabaseDescriptorWalker;
+import org.apache.cassandra.config.Loader;
+import org.apache.cassandra.config.Properties;
+import org.apache.cassandra.config.Replacement;
+import org.apache.cassandra.config.Replacements;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.utils.AbstractIterator;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings.
+ * It is a singleton and can be accessed via {@link #instance} field.
+ */
+public class ConfigurationRegistry extends AbstractMap<String, Object>
+{
+    public static final List<PropertySetterWalker> PROPERTY_SETTERS_LIST = ImmutableList.of(new DatabaseDescriptorWalker());
+    private static final Map<String, String> BACKWARDS_COMPATABLE_NAMES = ImmutableMap.copyOf(getBackwardsCompatableNames());
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    public static final ConfigurationRegistry instance = new ConfigurationRegistry();
+    private final Map<String, ConfigurationUnit<?>> registry;
+    private final TransformSet<Map.Entry<String, ConfigurationUnit<?>>, Map.Entry<String, Object>> transformSet;
+    private final Map<String, PropertyChangeListenerWrapper<?>> listeners = new HashMap<>();
+
+    public ConfigurationRegistry()
+    {
+        this(DatabaseDescriptor.getRawConfig());
+    }
+
+    public ConfigurationRegistry(Config config)
+    {
+        registry = getProperties().entrySet()
+                                  .stream()
+                                  .collect(Collectors.toMap(Map.Entry::getKey, e -> new ConfigurationUnit<>(config, e.getKey(), e.getValue())));

Review Comment:
   you shouldn't hold the ref, this can change on you causing this class to drift.  `DD` is what owns `Config`, we should reference it and not cache



##########
src/java/org/apache/cassandra/service/StorageService.java:
##########
@@ -384,6 +386,8 @@ public StorageService()
         jmxObjectName = "org.apache.cassandra.db:type=StorageService";
 
         sstablesTracker = new SSTablesGlobalTracker(SSTableFormat.Type.current());
+
+        ConfigurationRegistry.instance.addPropertyChangeListener(REPAIR_REQUEST_TIMEOUT, this::repairRequestTimeoutListener, DurationSpec.LongMillisecondsBound.class);

Review Comment:
   this doesn't need to exist... All "mutators" via JMX log the mutation, this may be in DD or in the Mean itself (such as this case), there is no reason to trigger this as all this does is update `Config`.
   
   the vtable can produce a similar log, so don't need to duplicate



##########
src/java/org/apache/cassandra/db/virtual/SettingsTable.java:
##########
@@ -59,102 +56,83 @@ final class SettingsTable extends AbstractVirtualTable
                            .addPartitionKeyColumn(NAME, UTF8Type.instance)
                            .addRegularColumn(VALUE, UTF8Type.instance)
                            .build());
-        this.config = config;
+        registerConverters();
+        this.configurationRegistry = configurationRegistry;
     }
 
-    @Override
-    public DataSet data(DecoratedKey partitionKey)
+    private void registerConverters()
     {
-        SimpleDataSet result = new SimpleDataSet(metadata());
-        String name = UTF8Type.instance.compose(partitionKey.getKey());
-        if (BACKWARDS_COMPATABLE_NAMES.containsKey(name))
-            ClientWarn.instance.warn("key '" + name + "' is deprecated; should switch to '" + BACKWARDS_COMPATABLE_NAMES.get(name) + "'");
-        if (PROPERTIES.containsKey(name))
-            result.row(name).column(VALUE, getValue(PROPERTIES.get(name)));
-        return result;
+        converterRegistry.put(Boolean.class, CassandraRelevantProperties.BOOLEAN_CONVERTER);
+        converterRegistry.put(boolean.class, CassandraRelevantProperties.BOOLEAN_CONVERTER);
+        converterRegistry.put(Integer.class, CassandraRelevantProperties.INTEGER_CONVERTER);
+        converterRegistry.put(int.class, CassandraRelevantProperties.INTEGER_CONVERTER);
+        converterRegistry.put(DurationSpec.LongMillisecondsBound.class,
+                              (PropertyConverter<DurationSpec.LongMillisecondsBound>) value -> DurationSpec.from(value, DurationSpec.LongMillisecondsBound.class));
     }
 
-    @Override
-    public DataSet data()
+    /**
+     * Setter for the property.
+     * @param name the name of the property.
+     * @param value the string representation of the value of the property to set.
+     */
+    private void setProperty(String name, String value)
     {
-        SimpleDataSet result = new SimpleDataSet(metadata());
-        for (Map.Entry<String, Property> e : PROPERTIES.entrySet())
-            result.row(e.getKey()).column(VALUE, getValue(e.getValue()));
-        return result;
+        Class<?> setterType = configurationRegistry.getPropertyType(name);
+        PropertyConverter<?> converter = converterRegistry.get(setterType);
+        if (converter == null)
+            throw invalidRequest("Unknown converter for property with name '%s' and type '%s'", name, setterType);
+
+        configurationRegistry.update(name, value == null ? null : converter.convert(value));
     }
 
-    private String getValue(Property prop)
+    private static @Nullable String getProperty(ConfigurationRegistry registry, String name)
     {
-        Object value = prop.get(config);
+        Object value = registry.get(name);
         return value == null ? null : value.toString();
     }
 
-    private static Map<String, Property> getProperties()
+    @Override
+    protected void applyColumnDeletion(ColumnValues partitionKey, ColumnValues clusteringColumns, String columnName)
     {
-        Loader loader = Properties.defaultLoader();
-        Map<String, Property> properties = loader.flatten(Config.class);
-        // only handling top-level replacements for now, previous logic was only top level so not a regression
-        Map<String, Replacement> replacements = Replacements.getNameReplacements(Config.class).get(Config.class);
-        if (replacements != null)
-        {
-            for (Replacement r : replacements.values())
-            {
-                Property latest = properties.get(r.newName);
-                assert latest != null : "Unable to find replacement new name: " + r.newName;
-                Property conflict = properties.put(r.oldName, r.toProperty(latest));
-                // some configs kept the same name, but changed the type, if this is detected then rely on the replaced property
-                assert conflict == null || r.oldName.equals(r.newName) : String.format("New property %s attempted to replace %s, but this property already exists", latest.getName(), conflict.getName());
-            }
-        }
-        for (Map.Entry<String, String> e : BACKWARDS_COMPATABLE_NAMES.entrySet())
-        {
-            String oldName = e.getKey();
-            if (properties.containsKey(oldName))
-                throw new AssertionError("Name " + oldName + " is present in Config, this adds a conflict as this name had a different meaning in " + SettingsTable.class.getSimpleName());
-            String newName = e.getValue();
-            Property prop = Objects.requireNonNull(properties.get(newName), newName + " cant be found for " + oldName);
-            properties.put(oldName, Properties.rename(oldName, prop));
-        }
-        return properties;
+        String key = partitionKey.value(0);
+        setProperty(key, null);
     }
 
-    /**
-     * settings table was released in 4.0 and attempted to support nested properties for a few hand selected properties.
-     * The issue is that 4.0 used '_' to seperate the names, which makes it hard to map back to the yaml names; to solve
-     * this 4.1+ uses '.' to avoid possible conflicts, this class provides mappings from old names to the '.' names.
-     *
-     * There were a handle full of properties which had custom names, names not present in the yaml, this map also
-     * fixes this and returns the proper (what is accessable via yaml) names.
-     */
-    private static Map<String, String> getBackwardsCompatableNames()
+    @Override
+    protected void applyColumnUpdate(ColumnValues partitionKey,
+                                     ColumnValues clusteringColumns,
+                                     Optional<ColumnValue> columnValue)
     {
-        Map<String, String> names = new HashMap<>();
-        // Names that dont match yaml
-        names.put("audit_logging_options_logger", "audit_logging_options.logger.class_name");
-        names.put("server_encryption_options_client_auth", "server_encryption_options.require_client_auth");
-        names.put("server_encryption_options_endpoint_verification", "server_encryption_options.require_endpoint_verification");
-        names.put("server_encryption_options_legacy_ssl_storage_port", "server_encryption_options.legacy_ssl_storage_port_enabled");
-        names.put("server_encryption_options_protocol", "server_encryption_options.accepted_protocols");
-
-        // matching names
-        names.put("audit_logging_options_audit_logs_dir", "audit_logging_options.audit_logs_dir");
-        names.put("audit_logging_options_enabled", "audit_logging_options.enabled");
-        names.put("audit_logging_options_excluded_categories", "audit_logging_options.excluded_categories");
-        names.put("audit_logging_options_excluded_keyspaces", "audit_logging_options.excluded_keyspaces");
-        names.put("audit_logging_options_excluded_users", "audit_logging_options.excluded_users");
-        names.put("audit_logging_options_included_categories", "audit_logging_options.included_categories");
-        names.put("audit_logging_options_included_keyspaces", "audit_logging_options.included_keyspaces");
-        names.put("audit_logging_options_included_users", "audit_logging_options.included_users");
-        names.put("server_encryption_options_algorithm", "server_encryption_options.algorithm");
-        names.put("server_encryption_options_cipher_suites", "server_encryption_options.cipher_suites");
-        names.put("server_encryption_options_enabled", "server_encryption_options.enabled");
-        names.put("server_encryption_options_internode_encryption", "server_encryption_options.internode_encryption");
-        names.put("server_encryption_options_optional", "server_encryption_options.optional");
-        names.put("transparent_data_encryption_options_chunk_length_kb", "transparent_data_encryption_options.chunk_length_kb");
-        names.put("transparent_data_encryption_options_cipher", "transparent_data_encryption_options.cipher");
-        names.put("transparent_data_encryption_options_enabled", "transparent_data_encryption_options.enabled");
-        names.put("transparent_data_encryption_options_iv_length", "transparent_data_encryption_options.iv_length");
-
-        return names;

Review Comment:
   why are you removing?  These strings only make sense to this class, so pulling out to a config registry is not safe, only this class should know about config names it created in the past



##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.AbstractSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.introspector.Property;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.DatabaseDescriptorWalker;
+import org.apache.cassandra.config.Loader;
+import org.apache.cassandra.config.Properties;
+import org.apache.cassandra.config.Replacement;
+import org.apache.cassandra.config.Replacements;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.utils.AbstractIterator;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings.
+ * It is a singleton and can be accessed via {@link #instance} field.
+ */
+public class ConfigurationRegistry extends AbstractMap<String, Object>

Review Comment:
   should *not* extend `AbstractMap<String, Object>`, this can be error prone as there are 2 other maps and from an API everything takes a `String`, so what does `ConfigurationRegistry.get` actually mean?  What about `ConfigurationRegistry.put` or `ConfigurationRegistry.removeKey`?



##########
src/java/org/apache/cassandra/config/DurationSpec.java:
##########
@@ -135,6 +135,25 @@ public TimeUnit unit()
         return unit;
     }
 
+    public static <T extends DurationSpec> T from(String value, Class<T> clazz)

Review Comment:
   can we remove?  The only usage can be replaced with `DurationSpec.IntMillisecondsBound::new`



##########
src/java/org/apache/cassandra/config/registry/PropertyChangeListener.java:
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.cassandra.config.registry;
+
+/**
+ * Interface for listening to configuration property changes.
+ */
+public interface PropertyChangeListener<T>
+{
+    /**
+     * Called before a configuration property has been changed.
+     *
+     * @param name the name of the property.
+     * @param oldValue the old value of the property.
+     * @param newValue the new value of the property.
+     */
+    void onBeforeChange(String name, T oldValue, T newValue);

Review Comment:
   based off name, should there also be `onAfterChange`?



##########
src/java/org/apache/cassandra/db/commitlog/CommitLog.java:
##########
@@ -122,6 +135,9 @@ private static CommitLog construct()
 
         // register metrics
         metrics.attach(executor, segmentManager);
+
+        ConfigurationRegistry.instance.addPropertyChangeListener(CDC_BLOCK_WRITES, this::cdcBlockWritesChangeListener, Boolean.class);

Review Comment:
   Looks like the majority of complexity of this patch is to add listeners for this single listener?  There maybe more cases as you flesh this out, but at least in the current PR this is the only real listener (the others just log you mutated, so don't need to listen).
   
   I feel we could simplify to
   
   ```ConfigurationRegistry.instance.register("cdc_block_writes", this:: setCDCBlockWrites0)```
   
   Where `setCDCBlockWrites0` is just the logic in `cdcBlockWritesChangeListener`
   
   I don't like the `Boolean.class` as it's brittle and can drift; in fact this patch provides the wrong type!  You want `Boolean.TYPE`.
   



##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.AbstractSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.introspector.Property;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.DatabaseDescriptorWalker;
+import org.apache.cassandra.config.Loader;
+import org.apache.cassandra.config.Properties;
+import org.apache.cassandra.config.Replacement;
+import org.apache.cassandra.config.Replacements;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.utils.AbstractIterator;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings.
+ * It is a singleton and can be accessed via {@link #instance} field.
+ */
+public class ConfigurationRegistry extends AbstractMap<String, Object>
+{
+    public static final List<PropertySetterWalker> PROPERTY_SETTERS_LIST = ImmutableList.of(new DatabaseDescriptorWalker());
+    private static final Map<String, String> BACKWARDS_COMPATABLE_NAMES = ImmutableMap.copyOf(getBackwardsCompatableNames());
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    public static final ConfigurationRegistry instance = new ConfigurationRegistry();
+    private final Map<String, ConfigurationUnit<?>> registry;
+    private final TransformSet<Map.Entry<String, ConfigurationUnit<?>>, Map.Entry<String, Object>> transformSet;
+    private final Map<String, PropertyChangeListenerWrapper<?>> listeners = new HashMap<>();
+
+    public ConfigurationRegistry()
+    {
+        this(DatabaseDescriptor.getRawConfig());
+    }
+
+    public ConfigurationRegistry(Config config)
+    {
+        registry = getProperties().entrySet()
+                                  .stream()
+                                  .collect(Collectors.toMap(Map.Entry::getKey, e -> new ConfigurationUnit<>(config, e.getKey(), e.getValue())));
+
+        ConfigurationSettersEnricher enricher = new ConfigurationSettersEnricher(registry);
+        PROPERTY_SETTERS_LIST.forEach(w -> w.walk(enricher));
+        transformSet = new TransformSet<>(registry.entrySet(),
+                                          e -> new AbstractMap.SimpleEntry<>(e.getKey(), e.getValue().getValue()));
+    }
+
+    /**
+     * Setter for the property with the given name. Can accept {@code null} value.
+     * @param name the name of the property.
+     * @param value the value to set.
+     */
+    public void update(String name, Object value)
+    {
+        ConfigurationUnit<?> unit = registry.get(name);
+
+        if (unit == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not availabe.", name));
+
+        if (unit.setter == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not writable.", name));
+
+        unit.lock.lock();
+        try
+        {
+            Object oldValue = unit.getValue();
+            if (listeners.get(name) != null)
+                listeners.get(name).onBeforeChange(name, oldValue, value);
+
+            unit.setValue(value);
+            // This potentially may expose the values that are not safe to see in logs on production.
+            logger.info("Updated property {} from {} to {}", name, oldValue, value);
+        }
+        catch (Exception e)
+        {
+            throw new ConfigurationException(String.format("Error updating property with name '%s', cause: %s", name, e.getMessage()), e);
+        }
+        finally
+        {
+            unit.lock.unlock();
+        }
+    }
+
+    public Class<?> getPropertyType(String key)
+    {
+        if (!registry.containsKey(key))
+            throw new ConfigurationException(String.format("Property with name '%s' is not availabe.", key));
+
+        return registry.get(key).getter.getType();
+    }
+
+    public String getBackwardCompatibleKey(String key)
+    {
+        return BACKWARDS_COMPATABLE_NAMES.get(key);
+    }
+
+    /**
+     * Adds a listener for the property with the given name.
+     * @param name property name to listen to.
+     * @param listener listener to add.
+     * @param <T> type of the property.
+     */
+    public <T> void addPropertyChangeListener(String name, PropertyChangeListener<T> listener, Class<T> listenerType)
+    {
+        ConfigurationUnit<?> unit = registry.get(name);
+        if (unit == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not availabe for resitering handler.", name));
+
+        if (unit.setterType == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not writable.", name));
+
+//        Class<T> type = (Class<T>) ((ParameterizedType) listener.getClass().getGenericInterfaces()[0]).getActualTypeArguments()[0];
+        if (!unit.setterType.equals(listenerType))
+            throw new ConfigurationException(String.format("Property with name '%s' expects type '%s', but got '%s'.", name, unit.setterType, listenerType));
+
+        listeners.put(name, new PropertyChangeListenerWrapper<>(listener, listenerType));
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public Set<Entry<String, Object>> entrySet()
+    {
+        return transformSet;
+    }
+
+    private static Map<String, Property> getProperties()
+    {
+        Loader loader = Properties.defaultLoader();
+
+        Map<String, Property> properties = loader.flatten(Config.class);
+        // only handling top-level replacements for now, previous logic was only top level so not a regression
+        Map<String, Replacement> replacements = Replacements.getNameReplacements(Config.class).get(Config.class);
+        if (replacements != null)
+        {
+            for (Replacement r : replacements.values())
+            {
+                Property latest = properties.get(r.newName);
+                assert latest != null : "Unable to find replacement new name: " + r.newName;
+                Property conflict = properties.put(r.oldName, r.toProperty(latest));
+                // some configs kept the same name, but changed the type, if this is detected then rely on the replaced property
+                assert conflict == null || r.oldName.equals(r.newName) : String.format("New property %s attempted to replace %s, but this property already exists", latest.getName(), conflict.getName());
+            }
+        }
+
+        for (Map.Entry<String, String> e : BACKWARDS_COMPATABLE_NAMES.entrySet())
+        {
+            String oldName = e.getKey();
+            if (properties.containsKey(oldName))
+                throw new AssertionError("Name " + oldName + " is present in Config, this adds a conflict as this name had a different meaning.");
+            String newName = e.getValue();
+            Property prop = Objects.requireNonNull(properties.get(newName), newName + " cant be found for " + oldName);
+            properties.put(oldName, Properties.rename(oldName, prop));
+        }
+        return properties;
+    }
+
+    /**
+     * settings table was released in 4.0 and attempted to support nested properties for a few hand selected properties.
+     * The issue is that 4.0 used '_' to seperate the names, which makes it hard to map back to the yaml names; to solve
+     * this 4.1+ uses '.' to avoid possible conflicts, this class provides mappings from old names to the '.' names.
+     *
+     * There were a handle full of properties which had custom names, names not present in the yaml, this map also
+     * fixes this and returns the proper (what is accessable via yaml) names.
+     */
+    private static Map<String, String> getBackwardsCompatableNames()
+    {
+        Map<String, String> names = new HashMap<>();
+        // Names that don't match yaml
+        names.put("audit_logging_options_logger", "audit_logging_options.logger.class_name");
+        names.put("server_encryption_options_client_auth", "server_encryption_options.require_client_auth");
+        names.put("server_encryption_options_endpoint_verification", "server_encryption_options.require_endpoint_verification");
+        names.put("server_encryption_options_legacy_ssl_storage_port", "server_encryption_options.legacy_ssl_storage_port_enabled");
+        names.put("server_encryption_options_protocol", "server_encryption_options.accepted_protocols");
+
+        // matching names
+        names.put("audit_logging_options_audit_logs_dir", "audit_logging_options.audit_logs_dir");
+        names.put("audit_logging_options_enabled", "audit_logging_options.enabled");
+        names.put("audit_logging_options_excluded_categories", "audit_logging_options.excluded_categories");
+        names.put("audit_logging_options_excluded_keyspaces", "audit_logging_options.excluded_keyspaces");
+        names.put("audit_logging_options_excluded_users", "audit_logging_options.excluded_users");
+        names.put("audit_logging_options_included_categories", "audit_logging_options.included_categories");
+        names.put("audit_logging_options_included_keyspaces", "audit_logging_options.included_keyspaces");
+        names.put("audit_logging_options_included_users", "audit_logging_options.included_users");
+        names.put("server_encryption_options_algorithm", "server_encryption_options.algorithm");
+        names.put("server_encryption_options_cipher_suites", "server_encryption_options.cipher_suites");
+        names.put("server_encryption_options_enabled", "server_encryption_options.enabled");
+        names.put("server_encryption_options_internode_encryption", "server_encryption_options.internode_encryption");
+        names.put("server_encryption_options_optional", "server_encryption_options.optional");
+        names.put("transparent_data_encryption_options_chunk_length_kb", "transparent_data_encryption_options.chunk_length_kb");
+        names.put("transparent_data_encryption_options_cipher", "transparent_data_encryption_options.cipher");
+        names.put("transparent_data_encryption_options_enabled", "transparent_data_encryption_options.enabled");
+        names.put("transparent_data_encryption_options_iv_length", "transparent_data_encryption_options.iv_length");
+
+        return names;
+    }
+
+    private static class TransformSet<T, R> extends AbstractSet<R>
+    {
+        private final Set<T> set;
+        private final Function<T, R> transform;
+
+        public TransformSet(Set<T> set, Function<T, R> transform)
+        {
+            this.set = set;
+            this.transform = transform;
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public Iterator<R> iterator()
+        {
+            Iterator<T> iter = set.iterator();
+            return new AbstractIterator<R>()
+            {
+                @Override
+                protected R computeNext()
+                {
+                    if (!iter.hasNext())
+                        return endOfData();
+
+                    T entry = iter.next();
+                    return transform.apply(entry);
+                }
+            };
+        }
+        /** {@inheritDoc} */
+        @Override
+        public int size()
+        {
+            return set.size();
+        }
+    }
+
+    private static class PropertyChangeListenerWrapper<T> implements PropertyChangeListener<T>
+    {
+        private final PropertyChangeListener<T> listener;
+        private final Class<T> type;
+
+        public PropertyChangeListenerWrapper(PropertyChangeListener<T> listener, Class<T> type)
+        {
+            this.listener = listener;
+            this.type = type;
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public void onBeforeChange(String name, Object oldValue, Object newValue)
+        {
+            T castedOldValue = type.cast(oldValue);
+            T castedNewValue = type.cast(newValue);
+            listener.onBeforeChange(name, castedOldValue, castedNewValue);
+        }
+    }

Review Comment:
   is this needed?  If the user defines the call site as `boolean` the JVM will add the cost for you, so should be able to drop this whole class



##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.AbstractSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.introspector.Property;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.DatabaseDescriptorWalker;
+import org.apache.cassandra.config.Loader;
+import org.apache.cassandra.config.Properties;
+import org.apache.cassandra.config.Replacement;
+import org.apache.cassandra.config.Replacements;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.utils.AbstractIterator;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings.
+ * It is a singleton and can be accessed via {@link #instance} field.
+ */
+public class ConfigurationRegistry extends AbstractMap<String, Object>
+{
+    public static final List<PropertySetterWalker> PROPERTY_SETTERS_LIST = ImmutableList.of(new DatabaseDescriptorWalker());
+    private static final Map<String, String> BACKWARDS_COMPATABLE_NAMES = ImmutableMap.copyOf(getBackwardsCompatableNames());
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    public static final ConfigurationRegistry instance = new ConfigurationRegistry();
+    private final Map<String, ConfigurationUnit<?>> registry;
+    private final TransformSet<Map.Entry<String, ConfigurationUnit<?>>, Map.Entry<String, Object>> transformSet;
+    private final Map<String, PropertyChangeListenerWrapper<?>> listeners = new HashMap<>();
+
+    public ConfigurationRegistry()
+    {
+        this(DatabaseDescriptor.getRawConfig());
+    }
+
+    public ConfigurationRegistry(Config config)
+    {
+        registry = getProperties().entrySet()
+                                  .stream()
+                                  .collect(Collectors.toMap(Map.Entry::getKey, e -> new ConfigurationUnit<>(config, e.getKey(), e.getValue())));
+
+        ConfigurationSettersEnricher enricher = new ConfigurationSettersEnricher(registry);
+        PROPERTY_SETTERS_LIST.forEach(w -> w.walk(enricher));
+        transformSet = new TransformSet<>(registry.entrySet(),
+                                          e -> new AbstractMap.SimpleEntry<>(e.getKey(), e.getValue().getValue()));
+    }
+
+    /**
+     * Setter for the property with the given name. Can accept {@code null} value.
+     * @param name the name of the property.
+     * @param value the value to set.
+     */
+    public void update(String name, Object value)
+    {
+        ConfigurationUnit<?> unit = registry.get(name);
+
+        if (unit == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not availabe.", name));
+
+        if (unit.setter == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not writable.", name));
+
+        unit.lock.lock();
+        try
+        {
+            Object oldValue = unit.getValue();
+            if (listeners.get(name) != null)
+                listeners.get(name).onBeforeChange(name, oldValue, value);
+
+            unit.setValue(value);
+            // This potentially may expose the values that are not safe to see in logs on production.
+            logger.info("Updated property {} from {} to {}", name, oldValue, value);
+        }
+        catch (Exception e)
+        {
+            throw new ConfigurationException(String.format("Error updating property with name '%s', cause: %s", name, e.getMessage()), e);
+        }
+        finally
+        {
+            unit.lock.unlock();
+        }
+    }
+
+    public Class<?> getPropertyType(String key)
+    {
+        if (!registry.containsKey(key))
+            throw new ConfigurationException(String.format("Property with name '%s' is not availabe.", key));
+
+        return registry.get(key).getter.getType();
+    }
+
+    public String getBackwardCompatibleKey(String key)
+    {
+        return BACKWARDS_COMPATABLE_NAMES.get(key);
+    }
+
+    /**
+     * Adds a listener for the property with the given name.
+     * @param name property name to listen to.
+     * @param listener listener to add.
+     * @param <T> type of the property.
+     */
+    public <T> void addPropertyChangeListener(String name, PropertyChangeListener<T> listener, Class<T> listenerType)
+    {
+        ConfigurationUnit<?> unit = registry.get(name);
+        if (unit == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not availabe for resitering handler.", name));
+
+        if (unit.setterType == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not writable.", name));
+
+//        Class<T> type = (Class<T>) ((ParameterizedType) listener.getClass().getGenericInterfaces()[0]).getActualTypeArguments()[0];
+        if (!unit.setterType.equals(listenerType))
+            throw new ConfigurationException(String.format("Property with name '%s' expects type '%s', but got '%s'.", name, unit.setterType, listenerType));
+
+        listeners.put(name, new PropertyChangeListenerWrapper<>(listener, listenerType));
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public Set<Entry<String, Object>> entrySet()
+    {
+        return transformSet;
+    }
+
+    private static Map<String, Property> getProperties()
+    {
+        Loader loader = Properties.defaultLoader();
+
+        Map<String, Property> properties = loader.flatten(Config.class);
+        // only handling top-level replacements for now, previous logic was only top level so not a regression
+        Map<String, Replacement> replacements = Replacements.getNameReplacements(Config.class).get(Config.class);
+        if (replacements != null)
+        {
+            for (Replacement r : replacements.values())
+            {
+                Property latest = properties.get(r.newName);
+                assert latest != null : "Unable to find replacement new name: " + r.newName;
+                Property conflict = properties.put(r.oldName, r.toProperty(latest));
+                // some configs kept the same name, but changed the type, if this is detected then rely on the replaced property
+                assert conflict == null || r.oldName.equals(r.newName) : String.format("New property %s attempted to replace %s, but this property already exists", latest.getName(), conflict.getName());
+            }
+        }
+
+        for (Map.Entry<String, String> e : BACKWARDS_COMPATABLE_NAMES.entrySet())
+        {
+            String oldName = e.getKey();
+            if (properties.containsKey(oldName))
+                throw new AssertionError("Name " + oldName + " is present in Config, this adds a conflict as this name had a different meaning.");
+            String newName = e.getValue();
+            Property prop = Objects.requireNonNull(properties.get(newName), newName + " cant be found for " + oldName);
+            properties.put(oldName, Properties.rename(oldName, prop));
+        }
+        return properties;
+    }
+
+    /**
+     * settings table was released in 4.0 and attempted to support nested properties for a few hand selected properties.
+     * The issue is that 4.0 used '_' to seperate the names, which makes it hard to map back to the yaml names; to solve
+     * this 4.1+ uses '.' to avoid possible conflicts, this class provides mappings from old names to the '.' names.
+     *
+     * There were a handle full of properties which had custom names, names not present in the yaml, this map also
+     * fixes this and returns the proper (what is accessable via yaml) names.
+     */
+    private static Map<String, String> getBackwardsCompatableNames()
+    {
+        Map<String, String> names = new HashMap<>();
+        // Names that don't match yaml
+        names.put("audit_logging_options_logger", "audit_logging_options.logger.class_name");
+        names.put("server_encryption_options_client_auth", "server_encryption_options.require_client_auth");
+        names.put("server_encryption_options_endpoint_verification", "server_encryption_options.require_endpoint_verification");
+        names.put("server_encryption_options_legacy_ssl_storage_port", "server_encryption_options.legacy_ssl_storage_port_enabled");
+        names.put("server_encryption_options_protocol", "server_encryption_options.accepted_protocols");
+
+        // matching names
+        names.put("audit_logging_options_audit_logs_dir", "audit_logging_options.audit_logs_dir");
+        names.put("audit_logging_options_enabled", "audit_logging_options.enabled");
+        names.put("audit_logging_options_excluded_categories", "audit_logging_options.excluded_categories");
+        names.put("audit_logging_options_excluded_keyspaces", "audit_logging_options.excluded_keyspaces");
+        names.put("audit_logging_options_excluded_users", "audit_logging_options.excluded_users");
+        names.put("audit_logging_options_included_categories", "audit_logging_options.included_categories");
+        names.put("audit_logging_options_included_keyspaces", "audit_logging_options.included_keyspaces");
+        names.put("audit_logging_options_included_users", "audit_logging_options.included_users");
+        names.put("server_encryption_options_algorithm", "server_encryption_options.algorithm");
+        names.put("server_encryption_options_cipher_suites", "server_encryption_options.cipher_suites");
+        names.put("server_encryption_options_enabled", "server_encryption_options.enabled");
+        names.put("server_encryption_options_internode_encryption", "server_encryption_options.internode_encryption");
+        names.put("server_encryption_options_optional", "server_encryption_options.optional");
+        names.put("transparent_data_encryption_options_chunk_length_kb", "transparent_data_encryption_options.chunk_length_kb");
+        names.put("transparent_data_encryption_options_cipher", "transparent_data_encryption_options.cipher");
+        names.put("transparent_data_encryption_options_enabled", "transparent_data_encryption_options.enabled");
+        names.put("transparent_data_encryption_options_iv_length", "transparent_data_encryption_options.iv_length");
+
+        return names;
+    }
+
+    private static class TransformSet<T, R> extends AbstractSet<R>
+    {
+        private final Set<T> set;
+        private final Function<T, R> transform;
+
+        public TransformSet(Set<T> set, Function<T, R> transform)
+        {
+            this.set = set;
+            this.transform = transform;
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public Iterator<R> iterator()
+        {
+            Iterator<T> iter = set.iterator();
+            return new AbstractIterator<R>()
+            {
+                @Override
+                protected R computeNext()
+                {
+                    if (!iter.hasNext())
+                        return endOfData();
+
+                    T entry = iter.next();
+                    return transform.apply(entry);
+                }
+            };
+        }
+        /** {@inheritDoc} */
+        @Override
+        public int size()
+        {
+            return set.size();
+        }
+    }
+
+    private static class PropertyChangeListenerWrapper<T> implements PropertyChangeListener<T>
+    {
+        private final PropertyChangeListener<T> listener;
+        private final Class<T> type;
+
+        public PropertyChangeListenerWrapper(PropertyChangeListener<T> listener, Class<T> type)
+        {
+            this.listener = listener;
+            this.type = type;
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public void onBeforeChange(String name, Object oldValue, Object newValue)
+        {
+            T castedOldValue = type.cast(oldValue);
+            T castedNewValue = type.cast(newValue);
+            listener.onBeforeChange(name, castedOldValue, castedNewValue);
+        }
+    }
+
+    /**
+     * Configuration enricher is used to add new property setters to the configuration.
+     */
+    private static class ConfigurationSettersEnricher implements PropertySetterWalker.SetterVisitor
+    {
+        private final Map<String, ConfigurationUnit<?>> registry;
+
+        ConfigurationSettersEnricher(Map<String, ConfigurationUnit<?>> registry)
+        {
+            this.registry = registry;
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public <T> void visit(String name, Class<T> clazz, PropertySetter<T> setter)
+        {
+            ConfigurationUnit<?> old = registry.get(name);
+            registry.replace(name, new ConfigurationUnit<>(old.source, name, old.getter, clazz, setter));
+        }
+    }
+
+    /**
+     * Configuration unit is a single configuration entry that can be read and written.
+     * @param <V> type of the configuration setter value.
+     */
+    private static class ConfigurationUnit<V> implements Map.Entry<String, V>

Review Comment:
   im not seeing why we have this class... it looks like it just duplicates `Property`. 



##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.AbstractSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.introspector.Property;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.DatabaseDescriptorWalker;
+import org.apache.cassandra.config.Loader;
+import org.apache.cassandra.config.Properties;
+import org.apache.cassandra.config.Replacement;
+import org.apache.cassandra.config.Replacements;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.utils.AbstractIterator;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings.
+ * It is a singleton and can be accessed via {@link #instance} field.
+ */
+public class ConfigurationRegistry extends AbstractMap<String, Object>
+{
+    public static final List<PropertySetterWalker> PROPERTY_SETTERS_LIST = ImmutableList.of(new DatabaseDescriptorWalker());
+    private static final Map<String, String> BACKWARDS_COMPATABLE_NAMES = ImmutableMap.copyOf(getBackwardsCompatableNames());
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    public static final ConfigurationRegistry instance = new ConfigurationRegistry();
+    private final Map<String, ConfigurationUnit<?>> registry;
+    private final TransformSet<Map.Entry<String, ConfigurationUnit<?>>, Map.Entry<String, Object>> transformSet;
+    private final Map<String, PropertyChangeListenerWrapper<?>> listeners = new HashMap<>();
+
+    public ConfigurationRegistry()
+    {
+        this(DatabaseDescriptor.getRawConfig());
+    }
+
+    public ConfigurationRegistry(Config config)
+    {
+        registry = getProperties().entrySet()
+                                  .stream()
+                                  .collect(Collectors.toMap(Map.Entry::getKey, e -> new ConfigurationUnit<>(config, e.getKey(), e.getValue())));
+
+        ConfigurationSettersEnricher enricher = new ConfigurationSettersEnricher(registry);
+        PROPERTY_SETTERS_LIST.forEach(w -> w.walk(enricher));
+        transformSet = new TransformSet<>(registry.entrySet(),
+                                          e -> new AbstractMap.SimpleEntry<>(e.getKey(), e.getValue().getValue()));
+    }
+
+    /**
+     * Setter for the property with the given name. Can accept {@code null} value.
+     * @param name the name of the property.
+     * @param value the value to set.
+     */
+    public void update(String name, Object value)
+    {
+        ConfigurationUnit<?> unit = registry.get(name);
+
+        if (unit == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not availabe.", name));
+
+        if (unit.setter == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not writable.", name));
+
+        unit.lock.lock();
+        try
+        {
+            Object oldValue = unit.getValue();
+            if (listeners.get(name) != null)
+                listeners.get(name).onBeforeChange(name, oldValue, value);
+
+            unit.setValue(value);
+            // This potentially may expose the values that are not safe to see in logs on production.
+            logger.info("Updated property {} from {} to {}", name, oldValue, value);
+        }
+        catch (Exception e)
+        {
+            throw new ConfigurationException(String.format("Error updating property with name '%s', cause: %s", name, e.getMessage()), e);
+        }
+        finally
+        {
+            unit.lock.unlock();
+        }
+    }
+
+    public Class<?> getPropertyType(String key)
+    {
+        if (!registry.containsKey(key))
+            throw new ConfigurationException(String.format("Property with name '%s' is not availabe.", key));
+
+        return registry.get(key).getter.getType();
+    }
+
+    public String getBackwardCompatibleKey(String key)
+    {
+        return BACKWARDS_COMPATABLE_NAMES.get(key);
+    }
+
+    /**
+     * Adds a listener for the property with the given name.
+     * @param name property name to listen to.
+     * @param listener listener to add.
+     * @param <T> type of the property.
+     */
+    public <T> void addPropertyChangeListener(String name, PropertyChangeListener<T> listener, Class<T> listenerType)
+    {
+        ConfigurationUnit<?> unit = registry.get(name);
+        if (unit == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not availabe for resitering handler.", name));
+
+        if (unit.setterType == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not writable.", name));
+
+//        Class<T> type = (Class<T>) ((ParameterizedType) listener.getClass().getGenericInterfaces()[0]).getActualTypeArguments()[0];
+        if (!unit.setterType.equals(listenerType))
+            throw new ConfigurationException(String.format("Property with name '%s' expects type '%s', but got '%s'.", name, unit.setterType, listenerType));
+
+        listeners.put(name, new PropertyChangeListenerWrapper<>(listener, listenerType));
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public Set<Entry<String, Object>> entrySet()
+    {
+        return transformSet;
+    }
+
+    private static Map<String, Property> getProperties()
+    {
+        Loader loader = Properties.defaultLoader();
+
+        Map<String, Property> properties = loader.flatten(Config.class);
+        // only handling top-level replacements for now, previous logic was only top level so not a regression
+        Map<String, Replacement> replacements = Replacements.getNameReplacements(Config.class).get(Config.class);
+        if (replacements != null)
+        {
+            for (Replacement r : replacements.values())
+            {
+                Property latest = properties.get(r.newName);
+                assert latest != null : "Unable to find replacement new name: " + r.newName;
+                Property conflict = properties.put(r.oldName, r.toProperty(latest));
+                // some configs kept the same name, but changed the type, if this is detected then rely on the replaced property
+                assert conflict == null || r.oldName.equals(r.newName) : String.format("New property %s attempted to replace %s, but this property already exists", latest.getName(), conflict.getName());
+            }
+        }
+
+        for (Map.Entry<String, String> e : BACKWARDS_COMPATABLE_NAMES.entrySet())
+        {
+            String oldName = e.getKey();
+            if (properties.containsKey(oldName))
+                throw new AssertionError("Name " + oldName + " is present in Config, this adds a conflict as this name had a different meaning.");
+            String newName = e.getValue();
+            Property prop = Objects.requireNonNull(properties.get(newName), newName + " cant be found for " + oldName);
+            properties.put(oldName, Properties.rename(oldName, prop));
+        }
+        return properties;
+    }
+
+    /**
+     * settings table was released in 4.0 and attempted to support nested properties for a few hand selected properties.
+     * The issue is that 4.0 used '_' to seperate the names, which makes it hard to map back to the yaml names; to solve
+     * this 4.1+ uses '.' to avoid possible conflicts, this class provides mappings from old names to the '.' names.
+     *
+     * There were a handle full of properties which had custom names, names not present in the yaml, this map also
+     * fixes this and returns the proper (what is accessable via yaml) names.
+     */
+    private static Map<String, String> getBackwardsCompatableNames()

Review Comment:
   should not rely on copy/paste, the 2 may drift and will be hard to maintain



##########
src/java/org/apache/cassandra/config/ConfigFields.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.cassandra.config;
+
+/**
+ * This class is autogenerated bases on the {@link Config} class. It contains all the configuration property

Review Comment:
   > This class is autogenerated
   
   Where?  I don't see anything in this patch that generates it, `build.xml` wasn't updated and this is not in a `gen-java` dir



##########
src/java/org/apache/cassandra/config/DatabaseDescriptorWalker.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.cassandra.config;
+
+import org.apache.cassandra.config.registry.PropertySetterWalker;
+
+/**
+ * This class is autogenerated bases on given {@link DatabaseDescriptor} class. It contains all the configuration

Review Comment:
   > This class is autogenerated
   
   Where?  I don't see such a thing in this patch



##########
src/java/org/apache/cassandra/db/virtual/SettingsTable.java:
##########
@@ -59,102 +56,83 @@ final class SettingsTable extends AbstractVirtualTable
                            .addPartitionKeyColumn(NAME, UTF8Type.instance)
                            .addRegularColumn(VALUE, UTF8Type.instance)
                            .build());
-        this.config = config;
+        registerConverters();
+        this.configurationRegistry = configurationRegistry;
     }
 
-    @Override
-    public DataSet data(DecoratedKey partitionKey)
+    private void registerConverters()
     {
-        SimpleDataSet result = new SimpleDataSet(metadata());
-        String name = UTF8Type.instance.compose(partitionKey.getKey());
-        if (BACKWARDS_COMPATABLE_NAMES.containsKey(name))
-            ClientWarn.instance.warn("key '" + name + "' is deprecated; should switch to '" + BACKWARDS_COMPATABLE_NAMES.get(name) + "'");
-        if (PROPERTIES.containsKey(name))
-            result.row(name).column(VALUE, getValue(PROPERTIES.get(name)));
-        return result;
+        converterRegistry.put(Boolean.class, CassandraRelevantProperties.BOOLEAN_CONVERTER);
+        converterRegistry.put(boolean.class, CassandraRelevantProperties.BOOLEAN_CONVERTER);
+        converterRegistry.put(Integer.class, CassandraRelevantProperties.INTEGER_CONVERTER);
+        converterRegistry.put(int.class, CassandraRelevantProperties.INTEGER_CONVERTER);
+        converterRegistry.put(DurationSpec.LongMillisecondsBound.class,
+                              (PropertyConverter<DurationSpec.LongMillisecondsBound>) value -> DurationSpec.from(value, DurationSpec.LongMillisecondsBound.class));
     }
 
-    @Override
-    public DataSet data()
+    /**
+     * Setter for the property.
+     * @param name the name of the property.
+     * @param value the string representation of the value of the property to set.
+     */
+    private void setProperty(String name, String value)
     {
-        SimpleDataSet result = new SimpleDataSet(metadata());
-        for (Map.Entry<String, Property> e : PROPERTIES.entrySet())
-            result.row(e.getKey()).column(VALUE, getValue(e.getValue()));
-        return result;
+        Class<?> setterType = configurationRegistry.getPropertyType(name);
+        PropertyConverter<?> converter = converterRegistry.get(setterType);
+        if (converter == null)
+            throw invalidRequest("Unknown converter for property with name '%s' and type '%s'", name, setterType);
+
+        configurationRegistry.update(name, value == null ? null : converter.convert(value));
     }
 
-    private String getValue(Property prop)
+    private static @Nullable String getProperty(ConfigurationRegistry registry, String name)
     {
-        Object value = prop.get(config);
+        Object value = registry.get(name);
         return value == null ? null : value.toString();

Review Comment:
   an issue this patch doesn't look to address is that the `input` and `output` may not match.
   
   For basic types like int/boolean this is fine as they are simple, but cases like `hinted_handoff_disabled_datacenters` are not correct as that's java collection toString, which we won't have a property converter for...  it's also weird for users to generate java's collection toString.  There are many types that fall into this bucket.
   
   You *should* have a property that w/e is returned from this table is accepted by this table; which is not true with this patch.



##########
src/java/org/apache/cassandra/db/virtual/SettingsTable.java:
##########
@@ -59,102 +56,83 @@ final class SettingsTable extends AbstractVirtualTable
                            .addPartitionKeyColumn(NAME, UTF8Type.instance)
                            .addRegularColumn(VALUE, UTF8Type.instance)
                            .build());
-        this.config = config;
+        registerConverters();
+        this.configurationRegistry = configurationRegistry;
     }
 
-    @Override
-    public DataSet data(DecoratedKey partitionKey)
+    private void registerConverters()
     {
-        SimpleDataSet result = new SimpleDataSet(metadata());
-        String name = UTF8Type.instance.compose(partitionKey.getKey());
-        if (BACKWARDS_COMPATABLE_NAMES.containsKey(name))
-            ClientWarn.instance.warn("key '" + name + "' is deprecated; should switch to '" + BACKWARDS_COMPATABLE_NAMES.get(name) + "'");
-        if (PROPERTIES.containsKey(name))
-            result.row(name).column(VALUE, getValue(PROPERTIES.get(name)));
-        return result;
+        converterRegistry.put(Boolean.class, CassandraRelevantProperties.BOOLEAN_CONVERTER);
+        converterRegistry.put(boolean.class, CassandraRelevantProperties.BOOLEAN_CONVERTER);
+        converterRegistry.put(Integer.class, CassandraRelevantProperties.INTEGER_CONVERTER);
+        converterRegistry.put(int.class, CassandraRelevantProperties.INTEGER_CONVERTER);
+        converterRegistry.put(DurationSpec.LongMillisecondsBound.class,

Review Comment:
   why are you using reflection when you can do `DurationSpec.LongMillisecondsBound::new`?



##########
src/java/org/apache/cassandra/db/virtual/SettingsTable.java:
##########
@@ -59,102 +56,83 @@ final class SettingsTable extends AbstractVirtualTable
                            .addPartitionKeyColumn(NAME, UTF8Type.instance)
                            .addRegularColumn(VALUE, UTF8Type.instance)
                            .build());
-        this.config = config;
+        registerConverters();
+        this.configurationRegistry = configurationRegistry;
     }
 
-    @Override
-    public DataSet data(DecoratedKey partitionKey)
+    private void registerConverters()
     {
-        SimpleDataSet result = new SimpleDataSet(metadata());
-        String name = UTF8Type.instance.compose(partitionKey.getKey());
-        if (BACKWARDS_COMPATABLE_NAMES.containsKey(name))
-            ClientWarn.instance.warn("key '" + name + "' is deprecated; should switch to '" + BACKWARDS_COMPATABLE_NAMES.get(name) + "'");
-        if (PROPERTIES.containsKey(name))
-            result.row(name).column(VALUE, getValue(PROPERTIES.get(name)));
-        return result;
+        converterRegistry.put(Boolean.class, CassandraRelevantProperties.BOOLEAN_CONVERTER);
+        converterRegistry.put(boolean.class, CassandraRelevantProperties.BOOLEAN_CONVERTER);
+        converterRegistry.put(Integer.class, CassandraRelevantProperties.INTEGER_CONVERTER);
+        converterRegistry.put(int.class, CassandraRelevantProperties.INTEGER_CONVERTER);
+        converterRegistry.put(DurationSpec.LongMillisecondsBound.class,
+                              (PropertyConverter<DurationSpec.LongMillisecondsBound>) value -> DurationSpec.from(value, DurationSpec.LongMillisecondsBound.class));
     }
 
-    @Override
-    public DataSet data()
+    /**
+     * Setter for the property.
+     * @param name the name of the property.
+     * @param value the string representation of the value of the property to set.
+     */
+    private void setProperty(String name, String value)
     {
-        SimpleDataSet result = new SimpleDataSet(metadata());
-        for (Map.Entry<String, Property> e : PROPERTIES.entrySet())
-            result.row(e.getKey()).column(VALUE, getValue(e.getValue()));
-        return result;
+        Class<?> setterType = configurationRegistry.getPropertyType(name);
+        PropertyConverter<?> converter = converterRegistry.get(setterType);
+        if (converter == null)
+            throw invalidRequest("Unknown converter for property with name '%s' and type '%s'", name, setterType);
+
+        configurationRegistry.update(name, value == null ? null : converter.convert(value));

Review Comment:
   I don't follow why this is needed.  You have a map of "real" type to "PropertyConverter", so can't you just do
   
   ```
   Property p = PROPERTIES.get(name);
   if (p == null) throw unknownProperty(name);
   p.set(DatabaseDescriptor.getRawConfig(), getConverter(p.getType()).convert(value));
   ```
   
   If there is a missing converter that means we have a bug, we don't support all types we support...



##########
src/java/org/apache/cassandra/config/PropertyConverter.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.cassandra.config;
+
+/**
+ * A converter that converts a string to a specific type.
+ * @param <T> the type to convert to
+ */
+public interface PropertyConverter<T>

Review Comment:
   you moved this out of the `CassandraRelevantProperties`, so are you making a assertion that system properties *must* match the same input as the vtable?  I am not saying they shouldn't, I am calling out that is what this code is asserting.  The current usage are all primitives, so this is fine, but we should call out that we don't actual handle `ConfigurationException` in the vtable, so user will get an error saying C* failed, and not that we had bad input



##########
test/distributed/org/apache/cassandra/distributed/test/jmx/JMXGetterCheckTest.java:
##########
@@ -119,6 +120,16 @@ public void test() throws Exception
                 throw root;
             }
         }
+
+        long ts = currentTimeMillis();
+        while (!Thread.interrupted())
+        {
+            if (ts + 5000 >= currentTimeMillis())
+                continue;
+
+            ts = currentTimeMillis();
+            System.out.println(">>>> Running: " + ts);
+        }

Review Comment:
   why is this here?  Also should avoid `System.out`



##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.AbstractSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.introspector.Property;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.DatabaseDescriptorWalker;
+import org.apache.cassandra.config.Loader;
+import org.apache.cassandra.config.Properties;
+import org.apache.cassandra.config.Replacement;
+import org.apache.cassandra.config.Replacements;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.utils.AbstractIterator;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings.
+ * It is a singleton and can be accessed via {@link #instance} field.
+ */
+public class ConfigurationRegistry extends AbstractMap<String, Object>
+{
+    public static final List<PropertySetterWalker> PROPERTY_SETTERS_LIST = ImmutableList.of(new DatabaseDescriptorWalker());

Review Comment:
   I don't understand the need for `PropertySetterWalker` and this field?  There exists a `Property` for every key (nested and top level), so I think we could remove a lot of code by just calling `Property.set` like we do in YAML



##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.AbstractSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.introspector.Property;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.DatabaseDescriptorWalker;
+import org.apache.cassandra.config.Loader;
+import org.apache.cassandra.config.Properties;
+import org.apache.cassandra.config.Replacement;
+import org.apache.cassandra.config.Replacements;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.utils.AbstractIterator;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings.
+ * It is a singleton and can be accessed via {@link #instance} field.
+ */
+public class ConfigurationRegistry extends AbstractMap<String, Object>
+{
+    public static final List<PropertySetterWalker> PROPERTY_SETTERS_LIST = ImmutableList.of(new DatabaseDescriptorWalker());
+    private static final Map<String, String> BACKWARDS_COMPATABLE_NAMES = ImmutableMap.copyOf(getBackwardsCompatableNames());
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    public static final ConfigurationRegistry instance = new ConfigurationRegistry();
+    private final Map<String, ConfigurationUnit<?>> registry;
+    private final TransformSet<Map.Entry<String, ConfigurationUnit<?>>, Map.Entry<String, Object>> transformSet;
+    private final Map<String, PropertyChangeListenerWrapper<?>> listeners = new HashMap<>();
+
+    public ConfigurationRegistry()
+    {
+        this(DatabaseDescriptor.getRawConfig());
+    }
+
+    public ConfigurationRegistry(Config config)
+    {
+        registry = getProperties().entrySet()
+                                  .stream()
+                                  .collect(Collectors.toMap(Map.Entry::getKey, e -> new ConfigurationUnit<>(config, e.getKey(), e.getValue())));
+
+        ConfigurationSettersEnricher enricher = new ConfigurationSettersEnricher(registry);
+        PROPERTY_SETTERS_LIST.forEach(w -> w.walk(enricher));
+        transformSet = new TransformSet<>(registry.entrySet(),
+                                          e -> new AbstractMap.SimpleEntry<>(e.getKey(), e.getValue().getValue()));
+    }
+
+    /**
+     * Setter for the property with the given name. Can accept {@code null} value.
+     * @param name the name of the property.
+     * @param value the value to set.
+     */
+    public void update(String name, Object value)
+    {
+        ConfigurationUnit<?> unit = registry.get(name);
+
+        if (unit == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not availabe.", name));
+
+        if (unit.setter == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not writable.", name));
+
+        unit.lock.lock();
+        try
+        {
+            Object oldValue = unit.getValue();
+            if (listeners.get(name) != null)
+                listeners.get(name).onBeforeChange(name, oldValue, value);
+
+            unit.setValue(value);
+            // This potentially may expose the values that are not safe to see in logs on production.
+            logger.info("Updated property {} from {} to {}", name, oldValue, value);
+        }
+        catch (Exception e)
+        {
+            throw new ConfigurationException(String.format("Error updating property with name '%s', cause: %s", name, e.getMessage()), e);
+        }
+        finally
+        {
+            unit.lock.unlock();
+        }
+    }
+
+    public Class<?> getPropertyType(String key)
+    {
+        if (!registry.containsKey(key))
+            throw new ConfigurationException(String.format("Property with name '%s' is not availabe.", key));
+
+        return registry.get(key).getter.getType();
+    }
+
+    public String getBackwardCompatibleKey(String key)
+    {
+        return BACKWARDS_COMPATABLE_NAMES.get(key);
+    }
+
+    /**
+     * Adds a listener for the property with the given name.
+     * @param name property name to listen to.
+     * @param listener listener to add.
+     * @param <T> type of the property.
+     */
+    public <T> void addPropertyChangeListener(String name, PropertyChangeListener<T> listener, Class<T> listenerType)
+    {
+        ConfigurationUnit<?> unit = registry.get(name);
+        if (unit == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not availabe for resitering handler.", name));
+
+        if (unit.setterType == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not writable.", name));
+
+//        Class<T> type = (Class<T>) ((ParameterizedType) listener.getClass().getGenericInterfaces()[0]).getActualTypeArguments()[0];
+        if (!unit.setterType.equals(listenerType))
+            throw new ConfigurationException(String.format("Property with name '%s' expects type '%s', but got '%s'.", name, unit.setterType, listenerType));
+
+        listeners.put(name, new PropertyChangeListenerWrapper<>(listener, listenerType));

Review Comment:
   what happens if multiple different places need to listen to the same config key?



##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.AbstractSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.introspector.Property;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.DatabaseDescriptorWalker;
+import org.apache.cassandra.config.Loader;
+import org.apache.cassandra.config.Properties;
+import org.apache.cassandra.config.Replacement;
+import org.apache.cassandra.config.Replacements;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.utils.AbstractIterator;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings.
+ * It is a singleton and can be accessed via {@link #instance} field.
+ */
+public class ConfigurationRegistry extends AbstractMap<String, Object>
+{
+    public static final List<PropertySetterWalker> PROPERTY_SETTERS_LIST = ImmutableList.of(new DatabaseDescriptorWalker());
+    private static final Map<String, String> BACKWARDS_COMPATABLE_NAMES = ImmutableMap.copyOf(getBackwardsCompatableNames());
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    public static final ConfigurationRegistry instance = new ConfigurationRegistry();
+    private final Map<String, ConfigurationUnit<?>> registry;
+    private final TransformSet<Map.Entry<String, ConfigurationUnit<?>>, Map.Entry<String, Object>> transformSet;
+    private final Map<String, PropertyChangeListenerWrapper<?>> listeners = new HashMap<>();
+
+    public ConfigurationRegistry()
+    {
+        this(DatabaseDescriptor.getRawConfig());
+    }
+
+    public ConfigurationRegistry(Config config)
+    {
+        registry = getProperties().entrySet()
+                                  .stream()
+                                  .collect(Collectors.toMap(Map.Entry::getKey, e -> new ConfigurationUnit<>(config, e.getKey(), e.getValue())));
+
+        ConfigurationSettersEnricher enricher = new ConfigurationSettersEnricher(registry);
+        PROPERTY_SETTERS_LIST.forEach(w -> w.walk(enricher));
+        transformSet = new TransformSet<>(registry.entrySet(),
+                                          e -> new AbstractMap.SimpleEntry<>(e.getKey(), e.getValue().getValue()));
+    }
+
+    /**
+     * Setter for the property with the given name. Can accept {@code null} value.
+     * @param name the name of the property.
+     * @param value the value to set.
+     */
+    public void update(String name, Object value)
+    {
+        ConfigurationUnit<?> unit = registry.get(name);
+
+        if (unit == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not availabe.", name));
+
+        if (unit.setter == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not writable.", name));
+
+        unit.lock.lock();
+        try
+        {
+            Object oldValue = unit.getValue();
+            if (listeners.get(name) != null)
+                listeners.get(name).onBeforeChange(name, oldValue, value);
+
+            unit.setValue(value);
+            // This potentially may expose the values that are not safe to see in logs on production.
+            logger.info("Updated property {} from {} to {}", name, oldValue, value);
+        }
+        catch (Exception e)
+        {
+            throw new ConfigurationException(String.format("Error updating property with name '%s', cause: %s", name, e.getMessage()), e);
+        }
+        finally
+        {
+            unit.lock.unlock();
+        }
+    }
+
+    public Class<?> getPropertyType(String key)
+    {
+        if (!registry.containsKey(key))
+            throw new ConfigurationException(String.format("Property with name '%s' is not availabe.", key));
+
+        return registry.get(key).getter.getType();
+    }
+
+    public String getBackwardCompatibleKey(String key)
+    {
+        return BACKWARDS_COMPATABLE_NAMES.get(key);
+    }
+
+    /**
+     * Adds a listener for the property with the given name.
+     * @param name property name to listen to.
+     * @param listener listener to add.
+     * @param <T> type of the property.
+     */
+    public <T> void addPropertyChangeListener(String name, PropertyChangeListener<T> listener, Class<T> listenerType)
+    {
+        ConfigurationUnit<?> unit = registry.get(name);
+        if (unit == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not availabe for resitering handler.", name));
+
+        if (unit.setterType == null)
+            throw new ConfigurationException(String.format("Property with name '%s' is not writable.", name));
+
+//        Class<T> type = (Class<T>) ((ParameterizedType) listener.getClass().getGenericInterfaces()[0]).getActualTypeArguments()[0];
+        if (!unit.setterType.equals(listenerType))
+            throw new ConfigurationException(String.format("Property with name '%s' expects type '%s', but got '%s'.", name, unit.setterType, listenerType));
+
+        listeners.put(name, new PropertyChangeListenerWrapper<>(listener, listenerType));
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public Set<Entry<String, Object>> entrySet()

Review Comment:
   this is very confusing, this boils down to the state in `registry`, but if I do `cr.put("does not exist", null)` I won't see that if I call `entrySet`



##########
src/java/org/apache/cassandra/config/ConfigFields.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.cassandra.config;
+
+/**
+ * This class is autogenerated bases on the {@link Config} class. It contains all the configuration property
+ * names as constants.
+ */
+public class ConfigFields

Review Comment:
   I don't follow this logic... why is there another config system rather than just updating `Config`?



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Mmuzaf commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "Mmuzaf (via GitHub)" <gi...@apache.org>.
Mmuzaf commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1148136674


##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.Mutable;
+import org.apache.cassandra.config.StringConverters;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.yaml.snakeyaml.introspector.Property;
+
+import static org.apache.cassandra.config.Properties.defaultLoader;
+import static org.apache.cassandra.config.registry.PrimitiveUnaryConverter.convertSafe;
+import static org.apache.commons.lang3.ClassUtils.primitiveToWrapper;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings, it doesn't
+ * take into account any configuration changes that might happen during properties replacement between releases.
+ */
+public class ConfigurationRegistry implements Registry
+{
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Supplier<Config> configProvider;
+    private final Map<ConfigurationListener.ChangeType, ConfigurationListenerList> propertyChangeListeners = new EnumMap<>(ConfigurationListener.ChangeType.class);
+    private final Map<String, List<TypedConstraintAdapter<?>>> constraints = new HashMap<>();
+    private volatile boolean initialized;
+    private Map<String, PropertyAdapter> properties = Collections.emptyMap();
+
+    public ConfigurationRegistry(Supplier<Config> configProvider)
+    {
+        this.configProvider = configProvider;
+        // Initialize the property change listeners.
+        for (ConfigurationListener.ChangeType type : ConfigurationListener.ChangeType.values())
+            propertyChangeListeners.put(type, new ConfigurationListenerList());
+    }
+
+    private void lazyInit()
+    {
+        if (initialized)
+            return;
+
+        rwLock.writeLock().lock();
+        try
+        {
+            if (initialized)
+                return;
+            properties = ImmutableMap.copyOf(defaultLoader()
+                                             .flatten(Config.class)
+                                             .entrySet()
+                                             .stream()
+                                             .map(e -> new AbstractMap.SimpleEntry<>(e.getKey(), new PropertyAdapter(configProvider, e.getValue())))
+                                             .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));

Review Comment:
   The problem I was trying to solve with lazy initialization of the _ConfigurationRegistry_ is that when the `DatabaseDescriptor#toolInitialization(boolean)` is called it will initialize almost all the components affected by access through `defaultLoader().flatten(Config.class)` which pulls dozens of classes to be load. 
   This was highlighted by the `DatabaseDescriptorRefTest` test.



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Mmuzaf commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "Mmuzaf (via GitHub)" <gi...@apache.org>.
Mmuzaf commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1148114197


##########
src/java/org/apache/cassandra/config/registry/ConfigurationRegistry.java:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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.cassandra.config.registry;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.Mutable;
+import org.apache.cassandra.config.StringConverters;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.yaml.snakeyaml.introspector.Property;
+
+import static org.apache.cassandra.config.Properties.defaultLoader;
+import static org.apache.cassandra.config.registry.PrimitiveUnaryConverter.convertSafe;
+import static org.apache.commons.lang3.ClassUtils.primitiveToWrapper;
+
+
+/**
+ * This is a simple configuration property registry that stores all the {@link Config} settings, it doesn't
+ * take into account any configuration changes that might happen during properties replacement between releases.
+ */
+public class ConfigurationRegistry implements Registry
+{
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationRegistry.class);
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Supplier<Config> configProvider;
+    private final Map<ConfigurationListener.ChangeType, ConfigurationListenerList> propertyChangeListeners = new EnumMap<>(ConfigurationListener.ChangeType.class);
+    private final Map<String, List<TypedConstraintAdapter<?>>> constraints = new HashMap<>();
+    private volatile boolean initialized;
+    private Map<String, PropertyAdapter> properties = Collections.emptyMap();
+
+    public ConfigurationRegistry(Supplier<Config> configProvider)
+    {
+        this.configProvider = configProvider;
+        // Initialize the property change listeners.
+        for (ConfigurationListener.ChangeType type : ConfigurationListener.ChangeType.values())
+            propertyChangeListeners.put(type, new ConfigurationListenerList());
+    }
+
+    private void lazyInit()
+    {
+        if (initialized)
+            return;
+
+        rwLock.writeLock().lock();
+        try
+        {
+            if (initialized)
+                return;
+            properties = ImmutableMap.copyOf(defaultLoader()
+                                             .flatten(Config.class)
+                                             .entrySet()
+                                             .stream()
+                                             .map(e -> new AbstractMap.SimpleEntry<>(e.getKey(), new PropertyAdapter(configProvider, e.getValue())))
+                                             .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
+            Set<String> leftConstraints = new HashSet<>(constraints.keySet());
+            leftConstraints.removeAll(properties.keySet());
+            if (!leftConstraints.isEmpty())
+                throw new ConfigurationException("Constraints are defined for non-existing properties:" + leftConstraints);
+            Set<String> leftListeners = propertyChangeListeners.values().stream()
+                                                               .map(l -> l.wrappers.keySet())
+                                                               .flatMap(Collection::stream)
+                                                               .collect(Collectors.toSet());
+            leftListeners.removeAll(properties.keySet());
+            if (!leftListeners.isEmpty())
+                throw new ConfigurationException("Listeners are defined for non-existing properties:" + leftListeners);
+            initialized = true;
+        }
+        finally
+        {
+            rwLock.writeLock().unlock();
+        }
+    }
+
+    @Override public void set(String name, @Nullable Object value)
+    {
+        lazyInit();
+        PropertyAdapter property = properties.get(name);

Review Comment:
   It might be true, but according to my understanding the publication of configuration variables is safe for the following reasons:
   - we read the volatile `initialized` variable each time the `lazyInit();` accessed;
   - we do a safe publication for the `properties` variable as the ImmutableMap and double-check locking are used for; 
   
   We have `happens-before` here and will see the non-null `PropertyAdapter property` (if it exists).



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Mmuzaf commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "Mmuzaf (via GitHub)" <gi...@apache.org>.
Mmuzaf commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1149834883


##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -80,45 +82,54 @@ public static Set<String> splitCommaDelimited(String src)
     public String authorizer;
     public String role_manager;
     public String network_authorizer;
-    @Replaces(oldName = "permissions_validity_in_ms", converter = Converters.MILLIS_DURATION_INT, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound permissions_validity = new DurationSpec.IntMillisecondsBound("2s");
-    public volatile int permissions_cache_max_entries = 1000;
-    @Replaces(oldName = "permissions_update_interval_in_ms", converter = Converters.MILLIS_CUSTOM_DURATION, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound permissions_update_interval = null;
-    public volatile boolean permissions_cache_active_update = false;
-    @Replaces(oldName = "roles_validity_in_ms", converter = Converters.MILLIS_DURATION_INT, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound roles_validity = new DurationSpec.IntMillisecondsBound("2s");
-    public volatile int roles_cache_max_entries = 1000;
-    @Replaces(oldName = "roles_update_interval_in_ms", converter = Converters.MILLIS_CUSTOM_DURATION, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound roles_update_interval = null;
-    public volatile boolean roles_cache_active_update = false;
-    @Replaces(oldName = "credentials_validity_in_ms", converter = Converters.MILLIS_DURATION_INT, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound credentials_validity = new DurationSpec.IntMillisecondsBound("2s");
-    public volatile int credentials_cache_max_entries = 1000;
-    @Replaces(oldName = "credentials_update_interval_in_ms", converter = Converters.MILLIS_CUSTOM_DURATION, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound credentials_update_interval = null;
-    public volatile boolean credentials_cache_active_update = false;
+    @Mutable @Replaces(oldName = "permissions_validity_in_ms", converter = Converters.MILLIS_DURATION_INT, deprecated = true)
+    public DurationSpec.IntMillisecondsBound permissions_validity = new DurationSpec.IntMillisecondsBound("2s");

Review Comment:
   As we decided to leave access DatabaseDescriptor's getters 'as is' for performance reasons, no need to remove the `volatile` keyword for configuration properties. I'll do revert these changes.



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#issuecomment-1414520915

   One thing I want to say is thanks for trying to tackle this work, looking forward to the progress!


-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Mmuzaf commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "Mmuzaf (via GitHub)" <gi...@apache.org>.
Mmuzaf commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1148129493


##########
src/java/org/apache/cassandra/db/virtual/SettingsTable.java:
##########
@@ -157,4 +174,85 @@ private static Map<String, String> getBackwardsCompatableNames()
 
         return names;
     }
+
+    /**
+     * This class is used to provide backwards compatable support for the settings table in case the {@link Config}
+     * metadata changes. This class will provide the old names for the properties, but will use the new name to
+     * get the value from the {@link Config} object.
+     * <p>
+     * Updating a configuration property object will throw an exception if you will try to update a deprecated property.
+     */
+    private static class BackwardsCompatableRegistry implements Registry
+    {
+        private final Registry registry;
+        private final Map<String, Replacement> replacements;
+        private final Set<String> uniquePropertyKeys;
+        public BackwardsCompatableRegistry(Registry registry)
+        {
+            this.registry = registry;
+            this.replacements = replacements(registry);
+            // Some configs kept the same name, but changed the type, so we need to make sure we don't return the same name twice.
+            this.uniquePropertyKeys = ImmutableSet.<String>builder().addAll(registry.keys()).addAll(replacements.keySet()).build();
+        }
+
+        @Override
+        public void set(String name, @Nullable Object value)
+        {
+            Replacement replacement = replacements.get(name);
+            if (replacement == null)
+                registry.set(name, value);
+            else
+                throw new ConfigurationException(String.format("Unable to set '%s' as it is deprecated and is read only; use '%s' instead", name, replacement.newName));

Review Comment:
   It can't break anything because the SettingsTable is not updatable now, so it's up to us in this patch to decide which set of properties to make updatable :-)
   
   I thought the `Replacement` is used to support backward compatibility between the releases so that users can use their old configs with new releases smoothly keeping in mind that they have to migrate to a new version of the `cassandra.yaml` (e.g. export the real config from the cluster to the yaml file).
   
   So showing a replaced name in the SettingsTable might be OK, but if we make it updatable it might cause confusion - because both the replaced name and a new name are responsible for the same single thing,  and this does not lead the user to understand which is the _right_ property in the release he uses.
   
   If I'm missing something, just let me know and I'll fix it (it is easy).



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Mmuzaf commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "Mmuzaf (via GitHub)" <gi...@apache.org>.
Mmuzaf commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1148129493


##########
src/java/org/apache/cassandra/db/virtual/SettingsTable.java:
##########
@@ -157,4 +174,85 @@ private static Map<String, String> getBackwardsCompatableNames()
 
         return names;
     }
+
+    /**
+     * This class is used to provide backwards compatable support for the settings table in case the {@link Config}
+     * metadata changes. This class will provide the old names for the properties, but will use the new name to
+     * get the value from the {@link Config} object.
+     * <p>
+     * Updating a configuration property object will throw an exception if you will try to update a deprecated property.
+     */
+    private static class BackwardsCompatableRegistry implements Registry
+    {
+        private final Registry registry;
+        private final Map<String, Replacement> replacements;
+        private final Set<String> uniquePropertyKeys;
+        public BackwardsCompatableRegistry(Registry registry)
+        {
+            this.registry = registry;
+            this.replacements = replacements(registry);
+            // Some configs kept the same name, but changed the type, so we need to make sure we don't return the same name twice.
+            this.uniquePropertyKeys = ImmutableSet.<String>builder().addAll(registry.keys()).addAll(replacements.keySet()).build();
+        }
+
+        @Override
+        public void set(String name, @Nullable Object value)
+        {
+            Replacement replacement = replacements.get(name);
+            if (replacement == null)
+                registry.set(name, value);
+            else
+                throw new ConfigurationException(String.format("Unable to set '%s' as it is deprecated and is read only; use '%s' instead", name, replacement.newName));

Review Comment:
   It can't break anything because the SettingsTable is not updatable now, so it's up to us in this patch to decide which set of properties to make updatable :-)
   
   AFAIU, the `Replacement` is used to support backward compatibility between the releases so that users can use their old configs with new releases smoothly keeping in mind that they have to migrate to a new version of the `cassandra.yaml` (e.g. export the real config from the cluster to the yaml file).
   
   So showing a replaced name in the SettingsTable might be OK, but if we make it updatable it might cause confusion - because both the replaced name and a new name are responsible for the same single thing,  and this does not lead the user to understand which is the _right_ property in the release he uses.
   
   If I'm missing something, just let me know and I'll fix it (it is easy).



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Mmuzaf commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "Mmuzaf (via GitHub)" <gi...@apache.org>.
Mmuzaf commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1148123089


##########
src/java/org/apache/cassandra/config/registry/PrimitiveUnaryConverter.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.cassandra.config.registry;
+
+import javax.annotation.Nonnull;
+
+import org.apache.cassandra.exceptions.ConfigurationException;
+
+/**
+ * Converter do type conversion from the specified Object  value to the given {@code T}. If the class is
+ * a primitive type (e.g. Boolean.TYPE, Long.TYPE etc), the value returned will use the corresponding
+ * wrapper type (Long.class, Boolean.class, etc).
+ *
+ * @see TypeConverter
+ * @see org.apache.cassandra.config.StringConverters
+ */
+public class PrimitiveUnaryConverter
+{
+    private static Object to(Class<?> cls, @Nonnull Object value)

Review Comment:
   We have a collection of `Config` properties. Some of these properties can be of a `boolean` type, while the others are of a `Boolean` (or we can take any pair from primitives and their wrappers). We always return a wrapper object from the `Registry` when the `<T> T get(Class<T> cls, String name)` method is called, regardless of whether the primitive or wrapper for the property is set.
   
   For both of the cases primitive or wrapper, we have to be sure that we can cast to the right type when the `get` method is called  - and this is the issue hiding under the hood and the purpose of this method.



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Mmuzaf commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "Mmuzaf (via GitHub)" <gi...@apache.org>.
Mmuzaf commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1149859859


##########
src/java/org/apache/cassandra/config/registry/Registry.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.cassandra.config.registry;
+
+import javax.annotation.Nullable;
+
+/**
+ * A registry of Cassandra's configuration properties that can be updated at runtime. The {@link org.apache.cassandra.config.Config}
+ * class is the source of configuration fields, types and other metadata available to the registry. The registry is used to
+ * handle configuration properties that are loaded from the configuration file, and are set via JMX or updated through
+ * the settings virtual table.
+ * <p>
+ * You can use {@link #set(String, Object)} to update a property, in case the property is not present in the registry,
+ * an exception will be thrown. If the property is present, the registry will try to convert given value to the property's
+ * type, and if the conversion fails, an exception will be thrown. You can use the {@code String} as a value to be converted,
+ * or you can use the property's type as a value. In the latter case, no conversion will be performed.
+ * <p>
+ * You can use {@link #get(Class, String)} to get a property's value, to read the value, the registry will try to convert the
+ * property's value if the {@link #getString(String)} to String type (the converter is called to convert the value to String).
+ */
+public interface Registry
+{
+    /**
+     * Update configuration property with the given name to the given value. The value may be the same
+     * as the property's value, or it may be represented as a string. In the latter case a corresponding
+     * will be called to get the property's value matching type.
+     *
+     * @param name Property name.
+     * @param value Value to set.
+     */
+    void set(String name, @Nullable Object value);
+
+    /**
+     * Get property's value by name, The exception will be thrown if the property is not present in the registry or
+     * the property's value cannot be converted to given generic type.
+     *
+     * @param <T>  Type to convert to.
+     * @param cls Class to convert to.
+     * @param name Property name.
+     * @return Property's value matching the property's type in the Config.
+     */
+    <T> T get(Class<T> cls, String name);
+
+    /**
+     * Get property's value by name and convert it to the String type. The exception will be thrown if the property
+     * is not present in the registry.
+     *
+     * @param name Property name.
+     * @return Property's value converted to String.
+     */
+    String getString(String name);

Review Comment:
   This is actually not true. The `SettingsTable' is not the only system that does type conversions to and from the `String' type. 
   
   For example, the same is also performed for JMX:
   
   ```
       public String getPaxosOnLinearizabilityViolations()
       {
           return DatabaseDescriptor.paxosOnLinearizabilityViolations().toString();
       }
   
       public void setPaxosOnLinearizabilityViolations(String v)
       {
           DatabaseDescriptor.setPaxosOnLinearizabilityViolations(Config.PaxosOnLinearizabilityViolation.valueOf(v));
           logger.info("paxos on linearizability violations {} via jmx", v);
       }
   ```



##########
src/java/org/apache/cassandra/config/registry/Registry.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.cassandra.config.registry;
+
+import javax.annotation.Nullable;
+
+/**
+ * A registry of Cassandra's configuration properties that can be updated at runtime. The {@link org.apache.cassandra.config.Config}
+ * class is the source of configuration fields, types and other metadata available to the registry. The registry is used to
+ * handle configuration properties that are loaded from the configuration file, and are set via JMX or updated through
+ * the settings virtual table.
+ * <p>
+ * You can use {@link #set(String, Object)} to update a property, in case the property is not present in the registry,
+ * an exception will be thrown. If the property is present, the registry will try to convert given value to the property's
+ * type, and if the conversion fails, an exception will be thrown. You can use the {@code String} as a value to be converted,
+ * or you can use the property's type as a value. In the latter case, no conversion will be performed.
+ * <p>
+ * You can use {@link #get(Class, String)} to get a property's value, to read the value, the registry will try to convert the
+ * property's value if the {@link #getString(String)} to String type (the converter is called to convert the value to String).
+ */
+public interface Registry
+{
+    /**
+     * Update configuration property with the given name to the given value. The value may be the same
+     * as the property's value, or it may be represented as a string. In the latter case a corresponding
+     * will be called to get the property's value matching type.
+     *
+     * @param name Property name.
+     * @param value Value to set.
+     */
+    void set(String name, @Nullable Object value);
+
+    /**
+     * Get property's value by name, The exception will be thrown if the property is not present in the registry or
+     * the property's value cannot be converted to given generic type.
+     *
+     * @param <T>  Type to convert to.
+     * @param cls Class to convert to.
+     * @param name Property name.
+     * @return Property's value matching the property's type in the Config.
+     */
+    <T> T get(Class<T> cls, String name);
+
+    /**
+     * Get property's value by name and convert it to the String type. The exception will be thrown if the property
+     * is not present in the registry.
+     *
+     * @param name Property name.
+     * @return Property's value converted to String.
+     */
+    String getString(String name);

Review Comment:
   This is actually not true. The `SettingsTable` is not the only system that does type conversions to and from the `String' type. 
   
   For example, the same is also performed for JMX:
   
   ```
       public String getPaxosOnLinearizabilityViolations()
       {
           return DatabaseDescriptor.paxosOnLinearizabilityViolations().toString();
       }
   
       public void setPaxosOnLinearizabilityViolations(String v)
       {
           DatabaseDescriptor.setPaxosOnLinearizabilityViolations(Config.PaxosOnLinearizabilityViolation.valueOf(v));
           logger.info("paxos on linearizability violations {} via jmx", v);
       }
   ```



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Mmuzaf commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "Mmuzaf (via GitHub)" <gi...@apache.org>.
Mmuzaf commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1149859859


##########
src/java/org/apache/cassandra/config/registry/Registry.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.cassandra.config.registry;
+
+import javax.annotation.Nullable;
+
+/**
+ * A registry of Cassandra's configuration properties that can be updated at runtime. The {@link org.apache.cassandra.config.Config}
+ * class is the source of configuration fields, types and other metadata available to the registry. The registry is used to
+ * handle configuration properties that are loaded from the configuration file, and are set via JMX or updated through
+ * the settings virtual table.
+ * <p>
+ * You can use {@link #set(String, Object)} to update a property, in case the property is not present in the registry,
+ * an exception will be thrown. If the property is present, the registry will try to convert given value to the property's
+ * type, and if the conversion fails, an exception will be thrown. You can use the {@code String} as a value to be converted,
+ * or you can use the property's type as a value. In the latter case, no conversion will be performed.
+ * <p>
+ * You can use {@link #get(Class, String)} to get a property's value, to read the value, the registry will try to convert the
+ * property's value if the {@link #getString(String)} to String type (the converter is called to convert the value to String).
+ */
+public interface Registry
+{
+    /**
+     * Update configuration property with the given name to the given value. The value may be the same
+     * as the property's value, or it may be represented as a string. In the latter case a corresponding
+     * will be called to get the property's value matching type.
+     *
+     * @param name Property name.
+     * @param value Value to set.
+     */
+    void set(String name, @Nullable Object value);
+
+    /**
+     * Get property's value by name, The exception will be thrown if the property is not present in the registry or
+     * the property's value cannot be converted to given generic type.
+     *
+     * @param <T>  Type to convert to.
+     * @param cls Class to convert to.
+     * @param name Property name.
+     * @return Property's value matching the property's type in the Config.
+     */
+    <T> T get(Class<T> cls, String name);
+
+    /**
+     * Get property's value by name and convert it to the String type. The exception will be thrown if the property
+     * is not present in the registry.
+     *
+     * @param name Property name.
+     * @return Property's value converted to String.
+     */
+    String getString(String name);

Review Comment:
   This is actually not true. The `SettingsTable` is not the only system that does type conversions to and from the `String` type. 
   
   For example, the same is also performed for JMX:
   
   ```
       public String getPaxosOnLinearizabilityViolations()
       {
           return DatabaseDescriptor.paxosOnLinearizabilityViolations().toString();
       }
   
       public void setPaxosOnLinearizabilityViolations(String v)
       {
           DatabaseDescriptor.setPaxosOnLinearizabilityViolations(Config.PaxosOnLinearizabilityViolation.valueOf(v));
           logger.info("paxos on linearizability violations {} via jmx", v);
       }
   ```



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1146462876


##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -80,45 +82,54 @@ public static Set<String> splitCommaDelimited(String src)
     public String authorizer;
     public String role_manager;
     public String network_authorizer;
-    @Replaces(oldName = "permissions_validity_in_ms", converter = Converters.MILLIS_DURATION_INT, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound permissions_validity = new DurationSpec.IntMillisecondsBound("2s");
-    public volatile int permissions_cache_max_entries = 1000;
-    @Replaces(oldName = "permissions_update_interval_in_ms", converter = Converters.MILLIS_CUSTOM_DURATION, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound permissions_update_interval = null;
-    public volatile boolean permissions_cache_active_update = false;
-    @Replaces(oldName = "roles_validity_in_ms", converter = Converters.MILLIS_DURATION_INT, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound roles_validity = new DurationSpec.IntMillisecondsBound("2s");
-    public volatile int roles_cache_max_entries = 1000;
-    @Replaces(oldName = "roles_update_interval_in_ms", converter = Converters.MILLIS_CUSTOM_DURATION, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound roles_update_interval = null;
-    public volatile boolean roles_cache_active_update = false;
-    @Replaces(oldName = "credentials_validity_in_ms", converter = Converters.MILLIS_DURATION_INT, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound credentials_validity = new DurationSpec.IntMillisecondsBound("2s");
-    public volatile int credentials_cache_max_entries = 1000;
-    @Replaces(oldName = "credentials_update_interval_in_ms", converter = Converters.MILLIS_CUSTOM_DURATION, deprecated = true)
-    public volatile DurationSpec.IntMillisecondsBound credentials_update_interval = null;
-    public volatile boolean credentials_cache_active_update = false;
+    @Mutable @Replaces(oldName = "permissions_validity_in_ms", converter = Converters.MILLIS_DURATION_INT, deprecated = true)

Review Comment:
   by putting `@Mutable` on the same line as `@Replaces` this diff gets hard to review as I have to check every character in `@Replaces` to make sure nothing changed... if you keep then in different lines the diff is much cleaner



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #2133: CASSANDRA-15254 Support update statements over the SettingsTable virtual table

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on code in PR #2133:
URL: https://github.com/apache/cassandra/pull/2133#discussion_r1146567885


##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1504,137 +1599,131 @@ public static void setRoleManager(IRoleManager roleManager)
 
     public static int getPermissionsValidity()
     {
-        return conf.permissions_validity.toMilliseconds();
+        return getProperty(DurationSpec.IntMillisecondsBound.class, ConfigFields.PERMISSIONS_VALIDITY).toMilliseconds();

Review Comment:
   please revert, `getProperty` is very expensive, and this is all for internal access...



##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1504,137 +1599,131 @@ public static void setRoleManager(IRoleManager roleManager)
 
     public static int getPermissionsValidity()
     {
-        return conf.permissions_validity.toMilliseconds();
+        return getProperty(DurationSpec.IntMillisecondsBound.class, ConfigFields.PERMISSIONS_VALIDITY).toMilliseconds();
     }
 
     public static void setPermissionsValidity(int timeout)
     {
-        conf.permissions_validity = new DurationSpec.IntMillisecondsBound(timeout);
+        setProperty(ConfigFields.PERMISSIONS_VALIDITY, new DurationSpec.IntMillisecondsBound(timeout));
     }
 
     public static int getPermissionsUpdateInterval()
     {
-        return conf.permissions_update_interval == null
-             ? conf.permissions_validity.toMilliseconds()
-             : conf.permissions_update_interval.toMilliseconds();
+        return getProperty(DurationSpec.IntMillisecondsBound.class, ConfigFields.PERMISSIONS_UPDATE_INTERVAL) == null
+             ? getPermissionsValidity() :
+               getProperty(DurationSpec.IntMillisecondsBound.class, ConfigFields.PERMISSIONS_UPDATE_INTERVAL).toMilliseconds();
     }
 
     public static void setPermissionsUpdateInterval(int updateInterval)
     {
-        if (updateInterval == -1)
-            conf.permissions_update_interval = null;
-        else
-            conf.permissions_update_interval = new DurationSpec.IntMillisecondsBound(updateInterval);
+        DurationSpec.IntMillisecondsBound interval = updateInterval == -1 ? null : new DurationSpec.IntMillisecondsBound(updateInterval);
+        setProperty(ConfigFields.PERMISSIONS_UPDATE_INTERVAL, interval);
     }
 
     public static int getPermissionsCacheMaxEntries()
     {
-        return conf.permissions_cache_max_entries;
+        return getProperty(Integer.TYPE, ConfigFields.PERMISSIONS_CACHE_MAX_ENTRIES);
     }
 
-    public static int setPermissionsCacheMaxEntries(int maxEntries)
+    public static void setPermissionsCacheMaxEntries(int maxEntries)
     {
-        return conf.permissions_cache_max_entries = maxEntries;
+        setProperty(ConfigFields.PERMISSIONS_CACHE_MAX_ENTRIES, maxEntries);
     }
 
     public static boolean getPermissionsCacheActiveUpdate()
     {
-        return conf.permissions_cache_active_update;
+        return getProperty(Boolean.TYPE, ConfigFields.PERMISSIONS_CACHE_ACTIVE_UPDATE);
     }
 
     public static void setPermissionsCacheActiveUpdate(boolean update)
     {
-        conf.permissions_cache_active_update = update;
+        setProperty(ConfigFields.PERMISSIONS_CACHE_ACTIVE_UPDATE, update);
     }
 
     public static int getRolesValidity()
     {
-        return conf.roles_validity.toMilliseconds();
+        return getProperty(DurationSpec.IntMillisecondsBound.class, ConfigFields.ROLES_VALIDITY).toMilliseconds();
     }
 
     public static void setRolesValidity(int validity)
     {
-        conf.roles_validity = new DurationSpec.IntMillisecondsBound(validity);
+        setProperty(ConfigFields.ROLES_VALIDITY, new DurationSpec.IntMillisecondsBound(validity));
     }
 
     public static int getRolesUpdateInterval()
     {
-        return conf.roles_update_interval == null
-             ? conf.roles_validity.toMilliseconds()
-             : conf.roles_update_interval.toMilliseconds();
+        return getProperty(DurationSpec.IntMillisecondsBound.class, ConfigFields.ROLES_UPDATE_INTERVAL) == null
+             ? getRolesValidity() :
+               getProperty(DurationSpec.IntMillisecondsBound.class, ConfigFields.ROLES_UPDATE_INTERVAL).toMilliseconds();
     }
 
     public static void setRolesCacheActiveUpdate(boolean update)
     {
-        conf.roles_cache_active_update = update;
+        setProperty(ConfigFields.ROLES_CACHE_ACTIVE_UPDATE, update);
     }
 
     public static boolean getRolesCacheActiveUpdate()
     {
-        return conf.roles_cache_active_update;
+        return getProperty(Boolean.TYPE, ConfigFields.ROLES_CACHE_ACTIVE_UPDATE);
     }
 
     public static void setRolesUpdateInterval(int interval)
     {
-        if (interval == -1)
-            conf.roles_update_interval = null;
-        else
-            conf.roles_update_interval = new DurationSpec.IntMillisecondsBound(interval);
+        DurationSpec.IntMillisecondsBound updateInterval = interval == -1 ? null : new DurationSpec.IntMillisecondsBound(interval);
+        setProperty(ConfigFields.ROLES_UPDATE_INTERVAL, updateInterval);
     }
 
     public static int getRolesCacheMaxEntries()
     {
-        return conf.roles_cache_max_entries;
+        return getProperty(Integer.TYPE, ConfigFields.ROLES_CACHE_MAX_ENTRIES);
     }
 
-    public static int setRolesCacheMaxEntries(int maxEntries)
+    public static void setRolesCacheMaxEntries(int maxEntries)

Review Comment:
   please revert, unrelated code changes goes against our style guide



##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -4676,4 +4729,41 @@ public static void setClientRequestSizeMetricsEnabled(boolean enabled)
     {
         return Objects.requireNonNull(sstableFormatFactories, "Forgot to initialize DatabaseDescriptor?");
     }
+
+    /**
+     * Set configuration property for the given name to {@link #confRegistry} if a safe manner
+     * with handling internal Cassandra exceptions.
+     *
+     * @param name Property name.
+     * @param value Property value.
+     */
+    private static void setProperty(String name, Object value)
+    {
+        runExceptionally(() -> confRegistry.set(name, value), new SearchInternalCauseForPublicAPI());
+    }
+
+    private static <T> T getProperty(Class<T> cls, String name)
+    {
+        return callExceptionally(() -> confRegistry.get(cls, name), new SearchInternalCauseForPublicAPI());
+    }
+
+    private static class SearchInternalCauseForPublicAPI implements Function<Exception, RuntimeException>
+    {
+        @Override
+        public RuntimeException apply(Exception e)
+        {
+            RuntimeException rt;
+            if ((rt = cause(e, IllegalArgumentException.class)) != null)
+                return new IllegalArgumentException(rt.getMessage());
+            else if ((rt = cause(e, IllegalStateException.class)) != null)
+                return new IllegalStateException(rt.getMessage());
+            else if ((rt = cause(e, UnsupportedOperationException.class)) != null)
+                return new UnsupportedOperationException(rt.getMessage());
+            else
+            {
+                logger.error("Unexpected exception", e);
+                return new RuntimeException(e.getMessage());
+            }
+        }
+    }

Review Comment:
   Why does this exist? 



##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -3105,12 +3193,14 @@ public static Set<String> hintedHandoffDisabledDCs()
 
     public static boolean useDeterministicTableID()
     {
-        return conf != null && conf.use_deterministic_table_id;
+        if (confRegistry == null)

Review Comment:
   please revert



-- 
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: pr-unsubscribe@cassandra.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org