You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by mc...@apache.org on 2020/08/25 18:58:07 UTC

[cassandra] branch trunk updated: Always access system properties and environment variables via the new CassandraRelevantProperties and CassandraRelevantEnv classes

This is an automated email from the ASF dual-hosted git repository.

mck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 7694c1d  Always access system properties and environment variables via the new CassandraRelevantProperties and CassandraRelevantEnv classes
7694c1d is described below

commit 7694c1d191531ac152db55e83bc0db6864a5441e
Author: Ekaterina Dimitrova <ek...@datastax.com>
AuthorDate: Sun Mar 22 15:35:33 2020 -0400

    Always access system properties and environment variables via the new CassandraRelevantProperties and CassandraRelevantEnv classes
    
     patch by Ekaterina Dimitrova; reviewed by David Capwell, Mick Semb Wever for CASSANDRA-15876
---
 CHANGES.txt                                        |   1 +
 .../cassandra/config/CassandraRelevantEnv.java     |  44 ++++
 .../config/CassandraRelevantProperties.java        | 240 +++++++++++++++++++++
 .../cassandra/config/DatabaseDescriptor.java       |   8 +-
 .../db/virtual/SystemPropertiesTable.java          |  63 ++----
 .../org/apache/cassandra/gms/FailureDetector.java  |   4 +-
 .../cassandra/hadoop/cql3/CqlBulkRecordWriter.java |   4 +-
 .../org/apache/cassandra/io/util/FileUtils.java    |   3 +-
 .../apache/cassandra/locator/PendingRangeMaps.java |   5 +-
 .../apache/cassandra/locator/TokenMetadata.java    |  18 +-
 .../apache/cassandra/service/CassandraDaemon.java  |  19 +-
 .../apache/cassandra/service/StartupChecks.java    |  11 +-
 .../org/apache/cassandra/utils/Architecture.java   |   4 +-
 .../org/apache/cassandra/utils/FBUtilities.java    |  30 +--
 .../org/apache/cassandra/utils/GuidGenerator.java  |   4 +-
 src/java/org/apache/cassandra/utils/HeapUtils.java |   5 +-
 .../org/apache/cassandra/utils/JMXServerUtils.java |  26 ++-
 .../org/apache/cassandra/utils/MBeanWrapper.java   |   7 +-
 src/java/org/apache/cassandra/utils/Mx4jTool.java  |   9 +-
 .../org/apache/cassandra/utils/NativeLibrary.java  |   6 +-
 .../config/CassandraRelevantPropertiesTest.java    | 145 +++++++++++++
 21 files changed, 546 insertions(+), 110 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 1e2c64a..b8ace5b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0-beta2
+ * Always access system properties and environment variables via the new CassandraRelevantProperties and CassandraRelevantEnv classes (CASSANDRA-15876)
  * Remove deprecated HintedHandOffManager (CASSANDRA-15939)
  * Prevent repair from overrunning compaction (CASSANDRA-15817)
  * fix cqlsh COPY functions in Python 3.8 on Mac (CASSANDRA-16053)
diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantEnv.java b/src/java/org/apache/cassandra/config/CassandraRelevantEnv.java
new file mode 100644
index 0000000..4960374
--- /dev/null
+++ b/src/java/org/apache/cassandra/config/CassandraRelevantEnv.java
@@ -0,0 +1,44 @@
+/*
+ * 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;
+
+public enum CassandraRelevantEnv
+{
+    /**
+     * Searching in the JAVA_HOME is safer than searching into System.getProperty("java.home") as the Oracle
+     * JVM might use the JRE which do not contains jmap.
+     */
+    JAVA_HOME ("JAVA_HOME");
+
+    CassandraRelevantEnv(String key)
+    {
+        this.key = key;
+    }
+
+    private final String key;
+
+    public String getString()
+    {
+        return System.getenv(key);
+    }
+
+    public String getKey() {
+        return key;
+    }
+}
diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
new file mode 100644
index 0000000..881b7d9
--- /dev/null
+++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
@@ -0,0 +1,240 @@
+/*
+ * 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.exceptions.ConfigurationException;
+
+/** A class that extracts system properties for the cassandra node it runs within. */
+public enum CassandraRelevantProperties
+{
+    //base JVM properties
+    JAVA_HOME("java.home"),
+    CASSANDRA_PID_FILE ("cassandra-pidfile"),
+
+    /**
+     * Indicates the temporary directory used by the Java Virtual Machine (JVM)
+     * to create and store temporary files.
+     */
+    JAVA_IO_TMPDIR ("java.io.tmpdir"),
+
+    /**
+     * Path from which to load native libraries.
+     * Default is absolute path to lib directory.
+     */
+    JAVA_LIBRARY_PATH ("java.library.path"),
+
+    JAVA_SECURITY_EGD ("java.security.egd"),
+
+    /** Java Runtime Environment version */
+    JAVA_VERSION ("java.version"),
+
+    /** Java Virtual Machine implementation name */
+    JAVA_VM_NAME ("java.vm.name"),
+
+    /** Line separator ("\n" on UNIX). */
+    LINE_SEPARATOR ("line.separator"),
+
+    /** Java class path. */
+    JAVA_CLASS_PATH ("java.class.path"),
+
+    /** Operating system architecture. */
+    OS_ARCH ("os.arch"),
+
+    /** Operating system name. */
+    OS_NAME ("os.name"),
+
+    /** User's home directory. */
+    USER_HOME ("user.home"),
+
+    /** Platform word size sun.arch.data.model. Examples: "32", "64", "unknown"*/
+    SUN_ARCH_DATA_MODEL ("sun.arch.data.model"),
+
+    //JMX properties
+    /**
+     * The value of this property represents the host name string
+     * that should be associated with remote stubs for locally created remote objects,
+     * in order to allow clients to invoke methods on the remote object.
+     */
+    JAVA_RMI_SERVER_HOSTNAME ("java.rmi.server.hostname"),
+
+    /**
+     * If this value is true, object identifiers for remote objects exported by this VM will be generated by using
+     * a cryptographically secure random number generator. The default value is false.
+     */
+    JAVA_RMI_SERVER_RANDOM_ID ("java.rmi.server.randomIDs"),
+
+    /**
+     * This property indicates whether password authentication for remote monitoring is
+     * enabled. By default it is disabled - com.sun.management.jmxremote.authenticate
+     */
+    COM_SUN_MANAGEMENT_JMXREMOTE_AUTHENTICATE ("com.sun.management.jmxremote.authenticate"),
+
+    /**
+     * The port number to which the RMI connector will be bound - com.sun.management.jmxremote.rmi.port.
+     * An Integer object that represents the value of the second argument is returned
+     * if there is no port specified, if the port does not have the correct numeric format,
+     * or if the specified name is empty or null.
+     */
+    COM_SUN_MANAGEMENT_JMXREMOTE_RMI_PORT ("com.sun.management.jmxremote.rmi.port", "0"),
+
+    /** Cassandra jmx remote port */
+    CASSANDRA_JMX_REMOTE_PORT("cassandra.jmx.remote.port"),
+
+    /** This property  indicates whether SSL is enabled for monitoring remotely. Default is set to false. */
+    COM_SUN_MANAGEMENT_JMXREMOTE_SSL ("com.sun.management.jmxremote.ssl"),
+
+    /**
+     * This property indicates whether SSL client authentication is enabled - com.sun.management.jmxremote.ssl.need.client.auth.
+     * Default is set to false.
+     */
+    COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH ("com.sun.management.jmxremote.ssl.need.client.auth"),
+
+    /**
+     * This property indicates the location for the access file. If com.sun.management.jmxremote.authenticate is false,
+     * then this property and the password and access files, are ignored. Otherwise, the access file must exist and
+     * be in the valid format. If the access file is empty or nonexistent, then no access is allowed.
+     */
+    COM_SUN_MANAGEMENT_JMXREMOTE_ACCESS_FILE ("com.sun.management.jmxremote.access.file"),
+
+    /** This property indicates the path to the password file - com.sun.management.jmxremote.password.file */
+    COM_SUN_MANAGEMENT_JMXREMOTE_PASSWORD_FILE ("com.sun.management.jmxremote.password.file"),
+
+    /** Port number to enable JMX RMI connections - com.sun.management.jmxremote.port */
+    COM_SUN_MANAGEMENT_JMXREMOTE_PORT ("com.sun.management.jmxremote.port"),
+
+    /**
+     * A comma-delimited list of SSL/TLS protocol versions to enable.
+     * Used in conjunction with com.sun.management.jmxremote.ssl - com.sun.management.jmxremote.ssl.enabled.protocols
+     */
+    COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS ("com.sun.management.jmxremote.ssl.enabled.protocols"),
+
+    /**
+     * A comma-delimited list of SSL/TLS cipher suites to enable.
+     * Used in conjunction with com.sun.management.jmxremote.ssl - com.sun.management.jmxremote.ssl.enabled.cipher.suites
+     */
+    COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES ("com.sun.management.jmxremote.ssl.enabled.cipher.suites"),
+
+    /** mx4jaddress */
+    MX4JADDRESS ("mx4jaddress"),
+
+    /** mx4jport */
+    MX4JPORT ("mx4jport"),
+
+    //cassandra properties (without the "cassandra." prefix)
+
+    /**
+     * The cassandra-foreground option will tell CassandraDaemon whether
+     * to close stdout/stderr, but it's up to us not to background.
+     * yes/null
+     */
+    CASSANDRA_FOREGROUND ("cassandra-foreground"),
+
+    DEFAULT_PROVIDE_OVERLAPPING_TOMBSTONES ("default.provide.overlapping.tombstones"),
+    ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION ("org.apache.cassandra.disable_mbean_registration"),
+    //only for testing
+    ORG_APACHE_CASSANDRA_CONF_CASSANDRA_RELEVANT_PROPERTIES_TEST("org.apache.cassandra.conf.CassandraRelevantPropertiesTest"),
+    ORG_APACHE_CASSANDRA_DB_VIRTUAL_SYSTEM_PROPERTIES_TABLE_TEST("org.apache.cassandra.db.virtual.SystemPropertiesTableTest"),
+
+    /** This property indicates whether disable_mbean_registration is true */
+    IS_DISABLED_MBEAN_REGISTRATION("org.apache.cassandra.disable_mbean_registration");
+
+    CassandraRelevantProperties(String key, String defaultVal)
+    {
+        this.key = key;
+        this.defaultVal = defaultVal;
+    }
+
+    CassandraRelevantProperties(String key)
+    {
+        this.key = key;
+        this.defaultVal = null;
+    }
+
+    private final String key;
+    private final String defaultVal;
+
+    public String getKey()
+    {
+        return key;
+    }
+
+    /**
+     * Gets the value of the indicated system property.
+     * @return system property value if it exists, defaultValue otherwise.
+     */
+    public String getString()
+    {
+        String value = System.getProperty(key);
+
+        return value == null ? defaultVal : STRING_CONVERTER.convert(value);
+    }
+
+    /**
+     * Gets the value of a system property as a boolean.
+     * @return system property boolean value if it exists, false otherwise().
+     */
+    public boolean getBoolean()
+    {
+        String value = System.getProperty(key);
+
+        return BOOLEAN_CONVERTER.convert(value == null ? defaultVal : value);
+    }
+
+    /**
+     * Gets the value of a system property as a int.
+     * @return system property int value if it exists, defaultValue otherwise.
+     */
+    public int getInt()
+    {
+        String value = System.getProperty(key);
+
+        return INTEGER_CONVERTER.convert(value == null ? defaultVal : value);
+    }
+
+    private interface PropertyConverter<T>
+    {
+        T convert(String value);
+    }
+
+    private static final PropertyConverter<String> STRING_CONVERTER = value -> value;
+
+    private static final PropertyConverter<Boolean> BOOLEAN_CONVERTER = Boolean::parseBoolean;
+
+    private static final PropertyConverter<Integer> INTEGER_CONVERTER = value ->
+    {
+        try
+        {
+            return Integer.decode(value);
+        }
+        catch (NumberFormatException e)
+        {
+            throw new ConfigurationException(String.format("Invalid value for system property: " +
+                                                           "expected integer value but got '%s'", value));
+        }
+    };
+
+    /**
+     * @return whether a system property is present or not.
+     */
+    public boolean isPresent()
+    {
+        return System.getProperties().containsKey(key);
+    }
+}
+
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index b47422b..3b5fdfb 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.config;
 
 import java.io.File;
 import java.io.IOException;
-import java.lang.reflect.Constructor;
 import java.net.*;
 import java.nio.file.FileStore;
 import java.nio.file.NoSuchFileException;
@@ -49,7 +48,6 @@ import org.apache.cassandra.auth.IInternodeAuthenticator;
 import org.apache.cassandra.auth.INetworkAuthorizer;
 import org.apache.cassandra.auth.IRoleManager;
 import org.apache.cassandra.config.Config.CommitLogSync;
-import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions.InternodeEncryption;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager;
 import org.apache.cassandra.db.commitlog.CommitLog;
@@ -76,6 +74,8 @@ import org.apache.cassandra.utils.FBUtilities;
 import org.apache.commons.lang3.StringUtils;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.cassandra.config.CassandraRelevantProperties.OS_ARCH;
+import static org.apache.cassandra.config.CassandraRelevantProperties.SUN_ARCH_DATA_MODEL;
 import static org.apache.cassandra.io.util.FileUtils.ONE_GB;
 import static org.apache.cassandra.io.util.FileUtils.ONE_MB;
 
@@ -2692,7 +2692,7 @@ public class DatabaseDescriptor
     public static boolean hasLargeAddressSpace()
     {
         // currently we just check if it's a 64bit arch, but any we only really care if the address space is large
-        String datamodel = System.getProperty("sun.arch.data.model");
+        String datamodel = SUN_ARCH_DATA_MODEL.getString();
         if (datamodel != null)
         {
             switch (datamodel)
@@ -2701,7 +2701,7 @@ public class DatabaseDescriptor
                 case "32": return false;
             }
         }
-        String arch = System.getProperty("os.arch");
+        String arch = OS_ARCH.getString();
         return arch.contains("64") || arch.contains("sparcv9");
     }
 
diff --git a/src/java/org/apache/cassandra/db/virtual/SystemPropertiesTable.java b/src/java/org/apache/cassandra/db/virtual/SystemPropertiesTable.java
index e8c13e7..d08d614 100644
--- a/src/java/org/apache/cassandra/db/virtual/SystemPropertiesTable.java
+++ b/src/java/org/apache/cassandra/db/virtual/SystemPropertiesTable.java
@@ -17,10 +17,10 @@
  */
 package org.apache.cassandra.db.virtual;
 
-import java.util.Set;
-
-import com.google.common.collect.Sets;
+import java.util.Arrays;
 
+import org.apache.cassandra.config.CassandraRelevantEnv;
+import org.apache.cassandra.config.CassandraRelevantProperties;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.marshal.*;
@@ -32,46 +32,6 @@ final class SystemPropertiesTable extends AbstractVirtualTable
     private static final String NAME = "name";
     private static final String VALUE = "value";
 
-    private static final Set<String> CASSANDRA_RELEVANT_PROPERTIES = Sets.newHashSet(
-            // base jvm properties
-            "java.home",
-            "java.io.tmpdir",
-            "java.library.path",
-            "java.security.egd",
-            "java.version",
-            "java.vm.name",
-            "line.separator",
-            "os.arch",
-            "os.name",
-            "user.home",
-            "sun.arch.data.model",
-            // jmx properties
-            "java.rmi.server.hostname",
-            "java.rmi.server.randomID",
-            "com.sun.management.jmxremote.authenticate",
-            "com.sun.management.jmxremote.rmi.port",
-            "com.sun.management.jmxremote.ssl",
-            "com.sun.management.jmxremote.ssl.need.client.auth",
-            "com.sun.management.jmxremote.access.file",
-            "com.sun.management.jmxremote.password.file",
-            "com.sun.management.jmxremote.port",
-            "com.sun.management.jmxremote.ssl.enabled.protocols",
-            "com.sun.management.jmxremote.ssl.enabled.cipher.suites",
-            "mx4jaddress",
-            "mx4jport",
-            // cassandra properties (without the "cassandra." prefix)
-            "cassandra-foreground",
-            "cassandra-pidfile",
-            "default.provide.overlapping.tombstones",
-            "org.apache.cassandra.disable_mbean_registration",
-            // only for testing
-            "org.apache.cassandra.db.virtual.SystemPropertiesTableTest"
-            );
-
-    private static final Set<String> CASSANDRA_RELEVANT_ENVS = Sets.newHashSet(
-            "JAVA_HOME"
-            );
-
     SystemPropertiesTable(String keyspace)
     {
         super(TableMetadata.builder(keyspace, "system_properties")
@@ -88,14 +48,14 @@ final class SystemPropertiesTable extends AbstractVirtualTable
         SimpleDataSet result = new SimpleDataSet(metadata());
 
         System.getenv().keySet()
-                .stream()
-                .filter(SystemPropertiesTable::isCassandraRelevant)
-                .forEach(name -> addRow(result, name, System.getenv(name)));
+              .stream()
+              .filter(SystemPropertiesTable::isCassandraRelevant)
+              .forEach(name -> addRow(result, name, System.getenv(name)));
 
         System.getProperties().stringPropertyNames()
-                .stream()
-                .filter(SystemPropertiesTable::isCassandraRelevant)
-                .forEach(name -> addRow(result, name, System.getProperty(name)));
+              .stream()
+              .filter(SystemPropertiesTable::isCassandraRelevant)
+              .forEach(name -> addRow(result, name, System.getProperty(name)));
 
         return result;
     }
@@ -113,8 +73,9 @@ final class SystemPropertiesTable extends AbstractVirtualTable
 
     static boolean isCassandraRelevant(String name)
     {
-        return name.startsWith(Config.PROPERTY_PREFIX) || CASSANDRA_RELEVANT_PROPERTIES.contains(name)
-                                                       || CASSANDRA_RELEVANT_ENVS.contains(name);
+        return name.startsWith(Config.PROPERTY_PREFIX)
+               || Arrays.stream(CassandraRelevantProperties.values()).anyMatch(p -> p.getKey().equals(name))
+               || Arrays.stream(CassandraRelevantEnv.values()).anyMatch(p -> p.getKey().equals(name));
     }
 
     private static void addRow(SimpleDataSet result, String name, String value)
diff --git a/src/java/org/apache/cassandra/gms/FailureDetector.java b/src/java/org/apache/cassandra/gms/FailureDetector.java
index d3a5f34..7cd14a3 100644
--- a/src/java/org/apache/cassandra/gms/FailureDetector.java
+++ b/src/java/org/apache/cassandra/gms/FailureDetector.java
@@ -29,7 +29,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.function.Predicate;
 import javax.management.openmbean.CompositeData;
 import javax.management.openmbean.*;
-
 import org.apache.cassandra.locator.Replica;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -40,6 +39,7 @@ import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.MBeanWrapper;
 
+import static org.apache.cassandra.config.CassandraRelevantProperties.LINE_SEPARATOR;
 import static org.apache.cassandra.utils.MonotonicClock.preciseTime;
 
 /**
@@ -384,7 +384,7 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
             ArrivalWindow hWnd = arrivalSamples.get(ep);
             sb.append(ep).append(" : ");
             sb.append(hWnd);
-            sb.append(System.getProperty("line.separator"));
+            sb.append(LINE_SEPARATOR.getString());
         }
         sb.append("-----------------------------------------------------------------------");
         return sb.toString();
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
index 77ad95f..45ffa4e 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
@@ -51,6 +51,8 @@ import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.util.Progressable;
 
+import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_IO_TMPDIR;
+
 /**
  * The <code>CqlBulkRecordWriter</code> maps the output &lt;key, value&gt;
  * pairs to a Cassandra column family. In particular, it applies the binded variables
@@ -152,7 +154,7 @@ public class CqlBulkRecordWriter extends RecordWriter<Object, List<ByteBuffer>>
 
     protected String getOutputLocation() throws IOException
     {
-        String dir = conf.get(OUTPUT_LOCATION, System.getProperty("java.io.tmpdir"));
+        String dir = conf.get(OUTPUT_LOCATION, JAVA_IO_TMPDIR.getString());
         if (dir == null)
             throw new IOException("Output directory not defined, if hadoop is not setting java.io.tmpdir then define " + OUTPUT_LOCATION);
         return dir;
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java
index c24ce81..67840c4 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -55,6 +55,7 @@ import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.SyncUtil;
 
 import static com.google.common.base.Throwables.propagate;
+import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_IO_TMPDIR;
 import static org.apache.cassandra.utils.Throwables.maybeFail;
 import static org.apache.cassandra.utils.Throwables.merge;
 
@@ -126,7 +127,7 @@ public final class FileUtils
         }
     }
 
-    private static final File tempDir = new File(System.getProperty("java.io.tmpdir"));
+    private static final File tempDir = new File(JAVA_IO_TMPDIR.getString());
     private static final AtomicLong tempFileNum = new AtomicLong();
 
     public static File getTempDir()
diff --git a/src/java/org/apache/cassandra/locator/PendingRangeMaps.java b/src/java/org/apache/cassandra/locator/PendingRangeMaps.java
index f9e3f66..14870c3 100644
--- a/src/java/org/apache/cassandra/locator/PendingRangeMaps.java
+++ b/src/java/org/apache/cassandra/locator/PendingRangeMaps.java
@@ -21,12 +21,15 @@
 package org.apache.cassandra.locator;
 
 import com.google.common.collect.Iterators;
+
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.locator.ReplicaCollection.Builder.Conflict;
 
 import java.util.*;
 
+import static org.apache.cassandra.config.CassandraRelevantProperties.LINE_SEPARATOR;
+
 public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, EndpointsForRange.Builder>>
 {
     /**
@@ -194,7 +197,7 @@ public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, Endpoi
             for (Replica replica : entry.getValue())
             {
                 sb.append(replica).append(':').append(range);
-                sb.append(System.getProperty("line.separator"));
+                sb.append(LINE_SEPARATOR.getString());
             }
         }
 
diff --git a/src/java/org/apache/cassandra/locator/TokenMetadata.java b/src/java/org/apache/cassandra/locator/TokenMetadata.java
index c16538b..3bb265d 100644
--- a/src/java/org/apache/cassandra/locator/TokenMetadata.java
+++ b/src/java/org/apache/cassandra/locator/TokenMetadata.java
@@ -27,6 +27,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.*;
+
+import org.apache.cassandra.config.CassandraRelevantProperties;
 import org.apache.cassandra.locator.ReplicaCollection.Builder.Conflict;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
@@ -43,6 +45,8 @@ import org.apache.cassandra.utils.BiMultiValMap;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.SortedBiMultiValMap;
 
+import static org.apache.cassandra.config.CassandraRelevantProperties.LINE_SEPARATOR;
+
 public class TokenMetadata
 {
     private static final Logger logger = LoggerFactory.getLogger(TokenMetadata.class);
@@ -1201,42 +1205,42 @@ public class TokenMetadata
             if (!eps.isEmpty())
             {
                 sb.append("Normal Tokens:");
-                sb.append(System.getProperty("line.separator"));
+                sb.append(LINE_SEPARATOR.getString());
                 for (InetAddressAndPort ep : eps)
                 {
                     sb.append(ep);
                     sb.append(':');
                     sb.append(endpointToTokenMap.get(ep));
-                    sb.append(System.getProperty("line.separator"));
+                    sb.append(LINE_SEPARATOR.getString());
                 }
             }
 
             if (!bootstrapTokens.isEmpty())
             {
                 sb.append("Bootstrapping Tokens:" );
-                sb.append(System.getProperty("line.separator"));
+                sb.append(LINE_SEPARATOR.getString());
                 for (Map.Entry<Token, InetAddressAndPort> entry : bootstrapTokens.entrySet())
                 {
                     sb.append(entry.getValue()).append(':').append(entry.getKey());
-                    sb.append(System.getProperty("line.separator"));
+                    sb.append(LINE_SEPARATOR.getString());
                 }
             }
 
             if (!leavingEndpoints.isEmpty())
             {
                 sb.append("Leaving Endpoints:");
-                sb.append(System.getProperty("line.separator"));
+                sb.append(LINE_SEPARATOR.getString());
                 for (InetAddressAndPort ep : leavingEndpoints)
                 {
                     sb.append(ep);
-                    sb.append(System.getProperty("line.separator"));
+                    sb.append(LINE_SEPARATOR.getString());
                 }
             }
 
             if (!pendingRanges.isEmpty())
             {
                 sb.append("Pending Ranges:");
-                sb.append(System.getProperty("line.separator"));
+                sb.append(LINE_SEPARATOR.getString());
                 sb.append(printPendingRanges());
             }
         }
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index 37cb0b1..c8e1e57 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -73,6 +73,13 @@ import org.apache.cassandra.utils.*;
 import org.apache.cassandra.security.ThreadAwareSecurityManager;
 
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_FOREGROUND;
+import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_PID_FILE;
+import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_REMOTE_PORT;
+import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_PORT;
+import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_CLASS_PATH;
+import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VERSION;
+import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VM_NAME;
 
 /**
  * The <code>CassandraDaemon</code> is an abstraction for a Cassandra daemon
@@ -119,7 +126,7 @@ public class CassandraDaemon
         // on it, so log a warning and skip setting up the server with the settings
         // as configured in cassandra-env.(sh|ps1)
         // See: CASSANDRA-11540 & CASSANDRA-11725
-        if (System.getProperty("com.sun.management.jmxremote.port") != null)
+        if (COM_SUN_MANAGEMENT_JMXREMOTE_PORT.isPresent())
         {
             logger.warn("JMX settings in cassandra-env.sh have been bypassed as the JMX connector server is " +
                         "already initialized. Please refer to cassandra-env.(sh|ps1) for JMX configuration info");
@@ -137,7 +144,7 @@ public class CassandraDaemon
         // If neither is remote nor local port is set in cassandra-env.(sh|ps)
         // then JMX is effectively  disabled.
         boolean localOnly = false;
-        String jmxPort = System.getProperty("cassandra.jmx.remote.port");
+        String jmxPort = CASSANDRA_JMX_REMOTE_PORT.getString();
 
         if (jmxPort == null)
         {
@@ -504,7 +511,7 @@ public class CassandraDaemon
 	            logger.info("Could not resolve local host");
 	        }
 
-	        logger.info("JVM vendor/version: {}/{}", System.getProperty("java.vm.name"), System.getProperty("java.version"));
+	        logger.info("JVM vendor/version: {}/{}", JAVA_VM_NAME.getString(), JAVA_VERSION.getString());
 	        logger.info("Heap size: {}/{}",
                         FBUtilities.prettyPrintMemory(Runtime.getRuntime().totalMemory()),
                         FBUtilities.prettyPrintMemory(Runtime.getRuntime().maxMemory()));
@@ -512,7 +519,7 @@ public class CassandraDaemon
 	        for(MemoryPoolMXBean pool: ManagementFactory.getMemoryPoolMXBeans())
 	            logger.info("{} {}: {}", pool.getName(), pool.getType(), pool.getPeakUsage());
 
-	        logger.info("Classpath: {}", System.getProperty("java.class.path"));
+	        logger.info("Classpath: {}", JAVA_CLASS_PATH.getString());
 
             logger.info("JVM Arguments: {}", ManagementFactory.getRuntimeMXBean().getInputArguments());
     	}
@@ -649,14 +656,14 @@ public class CassandraDaemon
 
             setup();
 
-            String pidFile = System.getProperty("cassandra-pidfile");
+            String pidFile = CASSANDRA_PID_FILE.getString();
 
             if (pidFile != null)
             {
                 new File(pidFile).deleteOnExit();
             }
 
-            if (System.getProperty("cassandra-foreground") == null)
+            if (CASSANDRA_FOREGROUND.getString() == null)
             {
                 System.out.close();
                 System.err.close();
diff --git a/src/java/org/apache/cassandra/service/StartupChecks.java b/src/java/org/apache/cassandra/service/StartupChecks.java
index 7754c6f..ecf9549 100644
--- a/src/java/org/apache/cassandra/service/StartupChecks.java
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@ -36,6 +36,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import net.jpountz.lz4.LZ4Factory;
+import org.apache.cassandra.config.CassandraRelevantProperties;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.schema.TableMetadata;
@@ -55,6 +56,10 @@ import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JavaUtils;
 import org.apache.cassandra.utils.SigarLibrary;
 
+import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_PORT;
+import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VERSION;
+import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VM_NAME;
+
 /**
  * Verifies that the system and environment is in a fit state to be started.
  * Used in CassandraDaemon#setup() to check various settings and invariants.
@@ -195,7 +200,7 @@ public class StartupChecks
     {
         public void execute()
         {
-            if (System.getProperty("com.sun.management.jmxremote.port") != null)
+            if (COM_SUN_MANAGEMENT_JMXREMOTE_PORT.isPresent())
             {
                 logger.warn("Use of com.sun.management.jmxremote.port at startup is deprecated. " +
                             "Please use cassandra.jmx.remote.port instead.");
@@ -211,7 +216,7 @@ public class StartupChecks
             if (!DatabaseDescriptor.hasLargeAddressSpace())
                 logger.warn("32bit JVM detected.  It is recommended to run Cassandra on a 64bit JVM for better performance.");
 
-            String javaVmName = System.getProperty("java.vm.name");
+            String javaVmName = JAVA_VM_NAME.getString();
             if (!(javaVmName.contains("HotSpot") || javaVmName.contains("OpenJDK")))
             {
                 logger.warn("Non-Oracle JVM detected.  Some features, such as immediate unmap of compacted SSTables, may not work as intended");
@@ -227,7 +232,7 @@ public class StartupChecks
          */
         private void checkOutOfMemoryHandling()
         {
-            if (JavaUtils.supportExitOnOutOfMemory(System.getProperty("java.version")))
+            if (JavaUtils.supportExitOnOutOfMemory(JAVA_VERSION.getString()))
             {
                 if (!jvmOptionsContainsOneOf("-XX:OnOutOfMemoryError=", "-XX:+ExitOnOutOfMemoryError", "-XX:+CrashOnOutOfMemoryError"))
                     logger.warn("The JVM is not configured to stop on OutOfMemoryError which can cause data corruption."
diff --git a/src/java/org/apache/cassandra/utils/Architecture.java b/src/java/org/apache/cassandra/utils/Architecture.java
index 3e9f579..bdea769 100644
--- a/src/java/org/apache/cassandra/utils/Architecture.java
+++ b/src/java/org/apache/cassandra/utils/Architecture.java
@@ -24,6 +24,8 @@ import java.util.Set;
 
 import com.google.common.collect.Sets;
 
+import static org.apache.cassandra.config.CassandraRelevantProperties.OS_ARCH;
+
 public final class Architecture
 {
     // Note that s390x, aarch64, & ppc64le architectures are not officially supported and adding them here is only done out
@@ -38,7 +40,7 @@ public final class Architecture
     "ppc64le"
     ));
 
-    public static final boolean IS_UNALIGNED = UNALIGNED_ARCH.contains(System.getProperty("os.arch"));
+    public static final boolean IS_UNALIGNED = UNALIGNED_ARCH.contains(OS_ARCH.getString());
 
     private Architecture()
     {
diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java
index 8e000d5..3b63d9a 100644
--- a/src/java/org/apache/cassandra/utils/FBUtilities.java
+++ b/src/java/org/apache/cassandra/utils/FBUtilities.java
@@ -26,9 +26,6 @@ import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 import java.util.*;
 import java.util.concurrent.*;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-import java.util.stream.Stream;
 import java.util.zip.CRC32;
 import java.util.zip.Checksum;
 
@@ -70,6 +67,9 @@ import org.apache.cassandra.io.util.DataOutputBufferFixed;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.locator.InetAddressAndPort;
 
+import static org.apache.cassandra.config.CassandraRelevantProperties.LINE_SEPARATOR;
+import static org.apache.cassandra.config.CassandraRelevantProperties.USER_HOME;
+
 
 public class FBUtilities
 {
@@ -912,19 +912,19 @@ public class FBUtilities
             int errCode = p.waitFor();
             if (errCode != 0)
             {
-            	try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()));
+                try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()));
                      BufferedReader err = new BufferedReader(new InputStreamReader(p.getErrorStream())))
                 {
-            		String lineSep = System.getProperty("line.separator");
-	                StringBuilder sb = new StringBuilder();
-	                String str;
-	                while ((str = in.readLine()) != null)
-	                    sb.append(str).append(lineSep);
-	                while ((str = err.readLine()) != null)
-	                    sb.append(str).append(lineSep);
-	                throw new IOException("Exception while executing the command: "+ StringUtils.join(pb.command(), " ") +
-	                                      ", command error Code: " + errCode +
-	                                      ", command output: "+ sb.toString());
+                    String lineSep = LINE_SEPARATOR.getString();
+                    StringBuilder sb = new StringBuilder();
+                    String str;
+                    while ((str = in.readLine()) != null)
+                        sb.append(str).append(lineSep);
+                    while ((str = err.readLine()) != null)
+                        sb.append(str).append(lineSep);
+                    throw new IOException("Exception while executing the command: "+ StringUtils.join(pb.command(), " ") +
+                                          ", command error Code: " + errCode +
+                                          ", command output: "+ sb.toString());
                 }
             }
         }
@@ -1036,7 +1036,7 @@ public class FBUtilities
 
     public static File getToolsOutputDirectory()
     {
-        File historyDir = new File(System.getProperty("user.home"), ".cassandra");
+        File historyDir = new File(USER_HOME.getString(), ".cassandra");
         FileUtils.createDirectory(historyDir);
         return historyDir;
     }
diff --git a/src/java/org/apache/cassandra/utils/GuidGenerator.java b/src/java/org/apache/cassandra/utils/GuidGenerator.java
index 2b3af01..aa3ee5b 100644
--- a/src/java/org/apache/cassandra/utils/GuidGenerator.java
+++ b/src/java/org/apache/cassandra/utils/GuidGenerator.java
@@ -21,6 +21,8 @@ import java.nio.ByteBuffer;
 import java.security.SecureRandom;
 import java.util.Random;
 
+import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_SECURITY_EGD;
+
 public class GuidGenerator
 {
     private static final Random myRand;
@@ -29,7 +31,7 @@ public class GuidGenerator
 
     static
     {
-        if (System.getProperty("java.security.egd") == null)
+        if (!JAVA_SECURITY_EGD.isPresent())
         {
             System.setProperty("java.security.egd", "file:/dev/urandom");
         }
diff --git a/src/java/org/apache/cassandra/utils/HeapUtils.java b/src/java/org/apache/cassandra/utils/HeapUtils.java
index ef66fde..4dd0d46 100644
--- a/src/java/org/apache/cassandra/utils/HeapUtils.java
+++ b/src/java/org/apache/cassandra/utils/HeapUtils.java
@@ -26,6 +26,8 @@ import org.apache.commons.lang3.text.StrBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.cassandra.config.CassandraRelevantEnv.JAVA_HOME;
+
 /**
  * Utility to log heap histogram.
  *
@@ -75,10 +77,9 @@ public final class HeapUtils
     {
         // Searching in the JAVA_HOME is safer than searching into System.getProperty("java.home") as the Oracle
         // JVM might use the JRE which do not contains jmap.
-        String javaHome = System.getenv("JAVA_HOME");
+        String javaHome = JAVA_HOME.getString();
         if (javaHome == null)
             return null;
-
         File javaBinDirectory = new File(javaHome, "bin");
         File[] files = javaBinDirectory.listFiles(new FilenameFilter()
         {
diff --git a/src/java/org/apache/cassandra/utils/JMXServerUtils.java b/src/java/org/apache/cassandra/utils/JMXServerUtils.java
index 1f79a33..603f106 100644
--- a/src/java/org/apache/cassandra/utils/JMXServerUtils.java
+++ b/src/java/org/apache/cassandra/utils/JMXServerUtils.java
@@ -29,7 +29,6 @@ import java.net.Inet6Address;
 import java.net.InetAddress;
 import java.rmi.AccessException;
 import java.rmi.AlreadyBoundException;
-import java.rmi.NoSuchObjectException;
 import java.rmi.NotBoundException;
 import java.rmi.Remote;
 import java.rmi.RemoteException;
@@ -55,6 +54,15 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.auth.jmx.AuthenticationProxy;
 
+import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_ACCESS_FILE;
+import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_AUTHENTICATE;
+import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_PASSWORD_FILE;
+import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_RMI_PORT;
+import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL;
+import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES;
+import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS;
+import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH;
+
 public class JMXServerUtils
 {
     private static final Logger logger = LoggerFactory.getLogger(JMXServerUtils.class);
@@ -109,7 +117,7 @@ public class JMXServerUtils
         // Set the port used to create subsequent connections to exported objects over RMI. This simplifies
         // configuration in firewalled environments, but it can't be used in conjuction with SSL sockets.
         // See: CASSANDRA-7087
-        int rmiPort = Integer.getInteger("com.sun.management.jmxremote.rmi.port", 0);
+        int rmiPort = COM_SUN_MANAGEMENT_JMXREMOTE_RMI_PORT.getInt();
 
         // We create the underlying RMIJRMPServerImpl so that we can manually bind it to the registry,
         // rather then specifying a binding address in the JMXServiceURL and letting it be done automatically
@@ -144,7 +152,7 @@ public class JMXServerUtils
     private static Map<String, Object> configureJmxAuthentication()
     {
         Map<String, Object> env = new HashMap<>();
-        if (!Boolean.getBoolean("com.sun.management.jmxremote.authenticate"))
+        if (!COM_SUN_MANAGEMENT_JMXREMOTE_AUTHENTICATE.getBoolean())
             return env;
 
         // If authentication is enabled, initialize the appropriate JMXAuthenticator
@@ -165,7 +173,7 @@ public class JMXServerUtils
         }
         else
         {
-            String passwordFile = System.getProperty("com.sun.management.jmxremote.password.file");
+            String passwordFile = COM_SUN_MANAGEMENT_JMXREMOTE_PASSWORD_FILE.getString();
             if (passwordFile != null)
             {
                 // stash the password file location where JMXPluggableAuthenticator expects it
@@ -196,7 +204,7 @@ public class JMXServerUtils
         }
         else
         {
-            String accessFile = System.getProperty("com.sun.management.jmxremote.access.file");
+            String accessFile = COM_SUN_MANAGEMENT_JMXREMOTE_ACCESS_FILE.getString();
             if (accessFile != null)
             {
                 env.put("jmx.remote.x.access.file", accessFile);
@@ -208,11 +216,11 @@ public class JMXServerUtils
     private static Map<String, Object> configureJmxSocketFactories(InetAddress serverAddress, boolean localOnly)
     {
         Map<String, Object> env = new HashMap<>();
-        if (Boolean.getBoolean("com.sun.management.jmxremote.ssl"))
+        if (COM_SUN_MANAGEMENT_JMXREMOTE_SSL.getBoolean())
         {
-            boolean requireClientAuth = Boolean.getBoolean("com.sun.management.jmxremote.ssl.need.client.auth");
+            boolean requireClientAuth = COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH.getBoolean();
             String[] protocols = null;
-            String protocolList = System.getProperty("com.sun.management.jmxremote.ssl.enabled.protocols");
+            String protocolList = COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS.getString();
             if (protocolList != null)
             {
                 System.setProperty("javax.rmi.ssl.client.enabledProtocols", protocolList);
@@ -220,7 +228,7 @@ public class JMXServerUtils
             }
 
             String[] ciphers = null;
-            String cipherList = System.getProperty("com.sun.management.jmxremote.ssl.enabled.cipher.suites");
+            String cipherList = COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES.getString();
             if (cipherList != null)
             {
                 System.setProperty("javax.rmi.ssl.client.enabledCipherSuites", cipherList);
diff --git a/src/java/org/apache/cassandra/utils/MBeanWrapper.java b/src/java/org/apache/cassandra/utils/MBeanWrapper.java
index edee6af..81d8dae 100644
--- a/src/java/org/apache/cassandra/utils/MBeanWrapper.java
+++ b/src/java/org/apache/cassandra/utils/MBeanWrapper.java
@@ -21,12 +21,15 @@ package org.apache.cassandra.utils;
 import java.lang.management.ManagementFactory;
 import java.util.function.Consumer;
 import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.CassandraRelevantProperties;
+
+import static org.apache.cassandra.config.CassandraRelevantProperties.IS_DISABLED_MBEAN_REGISTRATION;
+
 /**
  * Helper class to avoid catching and rethrowing checked exceptions on MBean and
  * allow turning of MBean registration for test purposes.
@@ -35,7 +38,7 @@ public interface MBeanWrapper
 {
     static final Logger logger = LoggerFactory.getLogger(MBeanWrapper.class);
 
-    static final MBeanWrapper instance = Boolean.getBoolean("org.apache.cassandra.disable_mbean_registration") ?
+    static final MBeanWrapper instance = IS_DISABLED_MBEAN_REGISTRATION.getBoolean() ?
                                          new NoOpMBeanWrapper() :
                                          new PlatformMBeanWrapper();
 
diff --git a/src/java/org/apache/cassandra/utils/Mx4jTool.java b/src/java/org/apache/cassandra/utils/Mx4jTool.java
index eda6354..79f163b 100644
--- a/src/java/org/apache/cassandra/utils/Mx4jTool.java
+++ b/src/java/org/apache/cassandra/utils/Mx4jTool.java
@@ -23,6 +23,11 @@ import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.CassandraRelevantProperties;
+
+import static org.apache.cassandra.config.CassandraRelevantProperties.MX4JADDRESS;
+import static org.apache.cassandra.config.CassandraRelevantProperties.MX4JPORT;
+
 /**
  * If mx4j-tools is in the classpath call maybeLoad to load the HTTP interface of mx4j.
  *
@@ -75,7 +80,7 @@ public class Mx4jTool
 
     private static String getAddress()
     {
-        String sAddress = System.getProperty("mx4jaddress");
+        String sAddress = MX4JADDRESS.getString();
         if (StringUtils.isEmpty(sAddress))
             sAddress = FBUtilities.getBroadcastAddressAndPort().address.getHostAddress();
         return sAddress;
@@ -84,7 +89,7 @@ public class Mx4jTool
     private static int getPort()
     {
         int port = 8081;
-        String sPort = System.getProperty("mx4jport");
+        String sPort = MX4JPORT.getString();
         if (StringUtils.isNotEmpty(sPort))
             port = Integer.parseInt(sPort);
         return port;
diff --git a/src/java/org/apache/cassandra/utils/NativeLibrary.java b/src/java/org/apache/cassandra/utils/NativeLibrary.java
index 8bcd6f6..eb0eaaf 100644
--- a/src/java/org/apache/cassandra/utils/NativeLibrary.java
+++ b/src/java/org/apache/cassandra/utils/NativeLibrary.java
@@ -32,6 +32,8 @@ import com.sun.jna.LastErrorException;
 
 import org.apache.cassandra.io.FSWriteError;
 
+import static org.apache.cassandra.config.CassandraRelevantProperties.OS_ARCH;
+import static org.apache.cassandra.config.CassandraRelevantProperties.OS_NAME;
 import static org.apache.cassandra.utils.NativeLibrary.OSType.LINUX;
 import static org.apache.cassandra.utils.NativeLibrary.OSType.MAC;
 import static org.apache.cassandra.utils.NativeLibrary.OSType.WINDOWS;
@@ -101,7 +103,7 @@ public final class NativeLibrary
             default: wrappedLibrary = new NativeLibraryLinux();
         }
 
-        if (System.getProperty("os.arch").toLowerCase().contains("ppc"))
+        if (OS_ARCH.getString().toLowerCase().contains("ppc"))
         {
             if (osType == LINUX)
             {
@@ -133,7 +135,7 @@ public final class NativeLibrary
      */
     private static OSType getOsType()
     {
-        String osName = System.getProperty("os.name").toLowerCase();
+        String osName = OS_NAME.getString().toLowerCase();
         if  (osName.contains("linux"))
             return LINUX;
         else if (osName.contains("mac"))
diff --git a/test/unit/org/apache/cassandra/config/CassandraRelevantPropertiesTest.java b/test/unit/org/apache/cassandra/config/CassandraRelevantPropertiesTest.java
new file mode 100644
index 0000000..1086521
--- /dev/null
+++ b/test/unit/org/apache/cassandra/config/CassandraRelevantPropertiesTest.java
@@ -0,0 +1,145 @@
+/*
+ * 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.junit.Test;
+
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.assertj.core.api.Assertions;
+
+public class CassandraRelevantPropertiesTest
+{
+    private static final CassandraRelevantProperties TEST_PROP = CassandraRelevantProperties.ORG_APACHE_CASSANDRA_CONF_CASSANDRA_RELEVANT_PROPERTIES_TEST;
+
+    @Test
+    public void testString()
+    {
+        try
+        {
+            System.setProperty(TEST_PROP.getKey(), "some-string");
+            Assertions.assertThat(TEST_PROP.getString()).isEqualTo("some-string");
+        }
+        finally
+        {
+            System.clearProperty(TEST_PROP.getKey());
+        }
+    }
+
+    @Test
+    public void testBoolean()
+    {
+        try
+        {
+            System.setProperty(TEST_PROP.getKey(), "true");
+            Assertions.assertThat(TEST_PROP.getBoolean()).isEqualTo(true);
+            System.setProperty(TEST_PROP.getKey(), "false");
+            Assertions.assertThat(TEST_PROP.getBoolean()).isEqualTo(false);
+            System.setProperty(TEST_PROP.getKey(), "junk");
+            Assertions.assertThat(TEST_PROP.getBoolean()).isEqualTo(false);
+            System.setProperty(TEST_PROP.getKey(), "");
+            Assertions.assertThat(TEST_PROP.getBoolean()).isEqualTo(false);
+        }
+        finally
+        {
+            System.clearProperty(TEST_PROP.getKey());
+        }
+    }
+
+    @Test
+    public void testBoolean_null()
+    {
+        try
+        {
+            TEST_PROP.getBoolean();
+        }
+        finally
+        {
+            System.clearProperty(TEST_PROP.getKey());
+        }
+    }
+
+    @Test
+    public void testDecimal()
+    {
+        try
+        {
+            System.setProperty(TEST_PROP.getKey(), "123456789");
+            Assertions.assertThat(TEST_PROP.getInt()).isEqualTo(123456789);
+        }
+        finally
+        {
+            System.clearProperty(TEST_PROP.getKey());
+        }
+    }
+
+    @Test
+    public void testHexadecimal()
+    {
+        try
+        {
+            System.setProperty(TEST_PROP.getKey(), "0x1234567a");
+            Assertions.assertThat(TEST_PROP.getInt()).isEqualTo(305419898);
+        }
+        finally
+        {
+            System.clearProperty(TEST_PROP.getKey());
+        }
+    }
+
+    @Test
+    public void testOctal()
+    {
+        try
+        {
+            System.setProperty(TEST_PROP.getKey(), "01234567");
+            Assertions.assertThat(TEST_PROP.getInt()).isEqualTo(342391);
+        }
+        finally
+        {
+            System.clearProperty(TEST_PROP.getKey());
+        }
+    }
+
+    @Test(expected = ConfigurationException.class)
+    public void testInteger_empty()
+    {
+        try
+        {
+            System.setProperty(TEST_PROP.getKey(), "");
+            Assertions.assertThat(TEST_PROP.getInt()).isEqualTo(342391);
+        }
+        finally
+        {
+            System.clearProperty(TEST_PROP.getKey());
+        }
+    }
+
+    @Test(expected = NullPointerException.class)
+    public void testInteger_null()
+    {
+        try
+        {
+            TEST_PROP.getInt();
+        }
+        finally
+        {
+            System.clearProperty(TEST_PROP.getKey());
+        }
+    }
+}
\ No newline at end of file


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