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