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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to