You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "jacek-lewandowski (via GitHub)" <gi...@apache.org> on 2023/04/05 08:58:51 UTC

[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2046: CASSANDRA-17797 All system properties and environment variables must be accessed via CassandraRelevantProperties

jacek-lewandowski commented on code in PR #2046:
URL: https://github.com/apache/cassandra/pull/2046#discussion_r1157467432


##########
src/java/org/apache/cassandra/config/CassandraRelevantProperties.java:
##########


Review Comment:
   it feels like those entries in this file should be sorted, can be done with one click in IntelliJ



##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1835,10 +1855,14 @@ public static InetAddressAndPort getReplaceAddress()
     {
         try
         {
-            if (System.getProperty(Config.PROPERTY_PREFIX + "replace_address", null) != null)
-                return InetAddressAndPort.getByName(System.getProperty(Config.PROPERTY_PREFIX + "replace_address", null));
-            else if (System.getProperty(Config.PROPERTY_PREFIX + "replace_address_first_boot", null) != null)
-                return InetAddressAndPort.getByName(System.getProperty(Config.PROPERTY_PREFIX + "replace_address_first_boot", null));
+            String replaceAddress = REPLACE_ADDRESS.getString();

Review Comment:
   why don't you add a method `Optional<String> getStringOpt()`, such snippets could be largely simplified, like: 
   
   ```java
   return REPLACE_ADDRESS.getStringOpt()
                         .orElseGet(REPLACE_ADDRESS_FIRST_BOOT::getStringOpt)
                         .map(InetAddressAndPort::getByName)
                         .orElse(null);
   ```



##########
src/java/org/apache/cassandra/db/ExpirationDateOverflowHandling.java:
##########
@@ -49,7 +51,7 @@
     public static ExpirationDateOverflowPolicy policy;
 
     static {
-        String policyAsString = System.getProperty("cassandra.expiration_date_overflow_policy", ExpirationDateOverflowPolicy.REJECT.name());

Review Comment:
   Actually we could have some enum handling in `CassandraRelevantProperites` which could simplify this kind of snippets



##########
src/java/org/apache/cassandra/gms/FailureDetector.java:
##########
@@ -93,16 +94,14 @@ public FailureDetector()
 
     private static long getInitialValue()
     {
-        String newvalue = System.getProperty("cassandra.fd_initial_value_ms");
-        if (newvalue == null)
-        {
+        Long newValue = FD_INITIAL_VALUE_MS.getLong();
+        if (newValue == null)
             return Gossiper.intervalInMillis * 2;
-        }
-        else
-        {
-            logger.info("Overriding FD INITIAL_VALUE to {}ms", newvalue);
-            return Integer.parseInt(newvalue);
-        }
+
+        if (newValue != Gossiper.intervalInMillis * 2)

Review Comment:
   Another example where `isDefault` could be helpful



##########
src/java/org/apache/cassandra/config/CassandraRelevantProperties.java:
##########
@@ -352,14 +410,166 @@
      * This is an optimization used in unit tests becuase we never restart a node there. The only node is stopoped
      * when the JVM terminates. Therefore, we can use such optimization and not wait unnecessarily. */
     NON_GRACEFUL_SHUTDOWN("cassandra.test.messagingService.nonGracefulShutdown"),
+    CHRONICLE_ANNOUNCER_DISABLE("chronicle.announcer.disable"),
+    COMMITLOG_ALLOW_IGNORE_SYNC_CRC("cassandra.commitlog.allow_ignore_sync_crc"),
+    COMMITLOG_IGNORE_REPLAY_ERRORS("cassandra.commitlog.ignorereplayerrors"),
+    COMMITLOG_MAX_OUTSTANDING_REPLAY_BYTES("cassandra.commitlog_max_outstanding_replay_bytes", String.valueOf(1024 * 1024 * 64)),
+    COMMITLOG_MAX_OUTSTANDING_REPLAY_COUNT("cassandra.commitlog_max_outstanding_replay_count", "1024"),
+    COMMITLOG_STOP_ON_ERRORS("cassandra.commitlog.stop_on_errors"),
+    CONFIG_LOADER("cassandra.config.loader"),
+    CONSISTENT_RANGE_MOVEMENT("cassandra.consistent.rangemovement", "true"),
+    CONSISTENT_SIMULTANEOUS_MOVES_ALLOW("cassandra.consistent.simultaneousmoves.allow", "false"),
+    CUSTOM_GUARDRAILS_CONFIG_PROVIDER_CLASS("cassandra.custom_guardrails_config_provider_class"),
+    CUSTOM_QUERY_HANDLER_CLASS("cassandra.custom_query_handler_class"),
+    CUSTOM_TRACING_CLASS("cassandra.custom_tracing_class"),
+    DATA_OUTPUT_STREAM_PLUS_TEMP_BUFFER_SIZE("cassandra.data_output_stream_plus_temp_buffer_size", "8192"),
+    DECAYING_ESTIMATED_HISTOGRAM_RESERVOIR_STRIPE_COUNT("cassandra.dehr_stripe_count", "2"),
+    DIAGNOSTIC_SNAPSHOT_INTERVAL_NANOS("cassandra.diagnostic_snapshot_interval_nanos", "60000000000"),
+    DISABLE_AUTH_CACHES_REMOTE_CONFIGURATION("cassandra.disable_auth_caches_remote_configuration"),
+    DISABLE_PAXOS_AUTO_REPAIRS("cassandra.disable_paxos_auto_repairs"),
+    DISABLE_PAXOS_STATE_FLUSH("cassandra.disable_paxos_state_flush"),
+    DISABLE_STCS_IN_L0("cassandra.disable_stcs_in_l0"),
+    DISABLE_TCACTIVE_OPENSSL("cassandra.disable_tcactive_openssl"),
+    DOB_DOUBLING_THRESHOLD_MB("cassandra.DOB_DOUBLING_THRESHOLD_MB", "64"),
+    DOB_MAX_RECYCLE_BYTES("cassandra.dob_max_recycle_bytes", String.valueOf(1024 * 1024)),
+    DROP_OVERSIZED_READ_REPAIR_MUTATIONS("cassandra.drop_oversized_readrepair_mutations"),
+    DTEST_API_LOG_TOPOLOGY("cassandra.dtest.api.log.topology"),
+    ENABLE_DC_LOCAL_COMMIT("cassandra.enable_dc_local_commit", "true"),
+    EXPIRATION_DATE_OVERFLOW_POLICY("cassandra.expiration_date_overflow_policy"),
+    EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES("cassandra.expiration_overflow_warning_interval_minutes", "5"),
+    FD_INITIAL_VALUE_MS("cassandra.fd_initial_value_ms"),
+    FD_MAX_INTERVAL_MS("cassandra.fd_max_interval_ms"),
+    FILE_CACHE_ENABLED("cassandra.file_cache_enabled"),
+    FORCE_DEFAULT_INDEXING_PAGE_SIZE("cassandra.force_default_indexing_page_size"),
+    FORCE_PAXOS_STATE_REBUILD("cassandra.force_paxos_state_rebuild"),
+    GIT_SHA("cassandra.gitSHA"),
+    GOSSIP_DISABLE_THREAD_VALIDATION("cassandra.gossip.disable_thread_validation", "false"),
+    IGNORE_CORRUPTED_SCHEMA_TABLES("cassandra.ignore_corrupted_schema_tables", "false"),

Review Comment:
   Do we ever need to provide the default value "false"?



##########
src/java/org/apache/cassandra/service/CassandraDaemon.java:
##########
@@ -698,8 +703,8 @@ public void start()
 
     private void startClientTransports()
     {
-        String nativeFlag = System.getProperty("cassandra.start_native_transport");

Review Comment:
   Another example where getBooleanOpt could be useful



##########
test/distributed/org/apache/cassandra/distributed/test/BootstrapBinaryDisabledTest.java:
##########
@@ -115,9 +116,9 @@ private static void bootstrap(Cluster cluster,
         config.forEach(nodeConfig::set);
 
         //TODO can we make this more isolated?
-        System.setProperty("cassandra.ring_delay_ms", "5000");
+        RING_DELAY.setLong(5000);
         if (isWriteSurvey)
-            System.setProperty("cassandra.write_survey", "true");

Review Comment:
   check name of this property



##########
test/simulator/asm/org/apache/cassandra/simulator/asm/MethodLogger.java:
##########
@@ -37,6 +37,8 @@
 import static org.apache.cassandra.simulator.asm.MethodLogger.Level.NONE;
 import static org.apache.cassandra.simulator.asm.MethodLogger.Level.valueOf;
 
+// checkstyle: suppress below 'blockSystemPropertyUsage'

Review Comment:
   why?



##########
test/unit/org/apache/cassandra/config/CassandraRelevantPropertiesTest.java:
##########


Review Comment:
   Is there a test for cleaning the value?



##########
test/unit/org/apache/cassandra/LogbackStatusListener.java:
##########
@@ -475,9 +479,9 @@ public synchronized void onStart(LoggerContext loggerContext)
                 Logger stdoutLogger = LoggerFactory.getLogger("stdout");
                 Logger stderrLogger = LoggerFactory.getLogger("stderr");
 
-                replacementOut = wrapLogger(stdoutLogger, originalOut, "sun.stdout.encoding", false);

Review Comment:
   Perhaps all those JVM properties starting with java. or sun. should not be prefixed with TEST as they are general. Maybe prefix them with JVM_ ?



##########
test/unit/org/apache/cassandra/cql3/CQLTester.java:
##########
@@ -573,10 +579,8 @@ private static void startServer(Consumer<Server.Builder> decorator)
     private static Cluster initClientCluster(User user, ProtocolVersion version)
     {
         SocketOptions socketOptions =
-                new SocketOptions().setConnectTimeoutMillis(Integer.getInteger("cassandra.test.driver.connection_timeout_ms",
-                                                                               DEFAULT_CONNECT_TIMEOUT_MILLIS)) // default is 5000
-                                   .setReadTimeoutMillis(Integer.getInteger("cassandra.test.driver.read_timeout_ms",
-                                                                            DEFAULT_READ_TIMEOUT_MILLIS)); // default is 12000
+                new SocketOptions().setConnectTimeoutMillis(TEST_DRIVER_CONNECTION_TIMEOUT_MS.getInt(DEFAULT_CONNECT_TIMEOUT_MILLIS)) // default is 5000

Review Comment:
   Please move the defaults to CRP



##########
src/java/org/apache/cassandra/gms/FailureDetector.java:
##########
@@ -55,21 +58,19 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean
     private static final int SAMPLE_SIZE = 1000;
     protected static final long INITIAL_VALUE_NANOS = TimeUnit.NANOSECONDS.convert(getInitialValue(), TimeUnit.MILLISECONDS);
     private static final int DEBUG_PERCENTAGE = 80; // if the phi is larger than this percentage of the max, log a debug message
-    private static final long DEFAULT_MAX_PAUSE = 5000L * 1000000L; // 5 seconds
     private static final long MAX_LOCAL_PAUSE_IN_NANOS = getMaxLocalPause();
     private long lastInterpret = preciseTime.now();
     private long lastPause = 0L;
 
     private static long getMaxLocalPause()
     {
-        if (System.getProperty("cassandra.max_local_pause_in_ms") != null)
-        {
-            long pause = Long.parseLong(System.getProperty("cassandra.max_local_pause_in_ms"));
-            logger.warn("Overriding max local pause time to {}ms", pause);
-            return pause * 1000000L;
-        }
-        else
-            return DEFAULT_MAX_PAUSE;
+        long pause = MAX_LOCAL_PAUSE_IN_MS.getLong();
+
+        if (!String.valueOf(pause).equals(MAX_LOCAL_PAUSE_IN_MS.getDefaultValue()))

Review Comment:
   Maybe just add a method to CRP, something like `isDefault`?



##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -2682,7 +2706,7 @@ public static boolean startNativeTransport()
      */
     public static int getNativeTransportPort()
     {
-        return Integer.parseInt(System.getProperty(Config.PROPERTY_PREFIX + "native_transport_port", Integer.toString(conf.native_transport_port)));
+        return NATIVE_TRANSPORT_PORT.getInt(Integer.parseInt(Integer.toString(conf.native_transport_port)));

Review Comment:
   seems like a redundant conversion, intentional?



##########
test/unit/org/apache/cassandra/io/sstable/indexsummary/IndexSummaryTest.java:
##########
@@ -81,7 +82,7 @@ public static void setup()
     public void testIndexSummaryKeySizes() throws IOException
     {
         // On Circle CI we normally don't have enough off-heap memory for this test so ignore it
-        Assume.assumeTrue(System.getenv("CIRCLECI") == null);
+        Assume.assumeTrue(CIRCLECI.getString() == null);

Review Comment:
   Wondering why we just don't assume exact amount of available direct memory?



##########
src/java/org/apache/cassandra/gms/Gossiper.java:
##########
@@ -106,9 +108,12 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
 {
     public static final String MBEAN_NAME = "org.apache.cassandra.net:type=Gossiper";
 
+    /** @deprecated Use CassandraRelevantProperties.GOSSIP_DISABLE_THREAD_VALIDATION instead. */
+    @Deprecated
     public static class Props

Review Comment:
   Please remove this class entirely 



##########
src/java/org/apache/cassandra/gms/Gossiper.java:
##########
@@ -257,18 +262,19 @@ public void expireUpgradeFromVersion()
         ((ExpiringMemoizingSupplier<CassandraVersion>) upgradeFromVersionMemoized).expire();
     }
 
-    private static final boolean disableThreadValidation = Boolean.getBoolean(Props.DISABLE_THREAD_VALIDATION);
+    private static final boolean disableThreadValidation = GOSSIP_DISABLE_THREAD_VALIDATION.getBoolean();
     private static volatile boolean disableEndpointRemoval = DISABLE_GOSSIP_ENDPOINT_REMOVAL.getBoolean();
 
     private static long getVeryLongTime()
     {
-        String newVLT =  System.getProperty("cassandra.very_long_time_ms");
-        if (newVLT != null)
-        {
-            logger.info("Overriding aVeryLongTime to {}ms", newVLT);
-            return Long.parseLong(newVLT);
-        }
-        return 259200 * 1000; // 3 days
+        long time = VERY_LONG_TIME_MS.getLong();

Review Comment:
   wow, I like this property 🤣 



##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1021,9 +1041,9 @@ private static String storagedirFor(String type)
 
     private static String storagedir(String errMsgType)
     {
-        String storagedir = System.getProperty(Config.PROPERTY_PREFIX + "storagedir", null);
+        String storagedir = STORAGE_DIR.getString();
         if (storagedir == null)
-            throw new ConfigurationException(errMsgType + " is missing and -Dcassandra.storagedir is not set", false);
+            throw new ConfigurationException(errMsgType + " is missing and -D" + STORAGE_DIR.getKey() + " is not set", false);

Review Comment:
   To me, it looks weird with `-D` when we are talking about the property rather than an argument that has to be added to set it.



##########
test/unit/org/apache/cassandra/service/paxos/uncommitted/PaxosStateTrackerTest.java:
##########
@@ -206,7 +175,8 @@ public void manualRebuild() throws Throwable
         SystemKeyspace.savePaxosWritePromise(dk(0), cfm1, ballots[2]);
         SystemKeyspace.savePaxosProposal(commit(cfm1, 2, ballots[3]));
 
-        try (SystemProp forceRebuild = SystemProp.set(PaxosStateTracker.FORCE_REBUILD_PROP, true))
+        Boolean prev = FORCE_PAXOS_STATE_REBUILD.setBoolean(true);

Review Comment:
   Replace it with `WithProperties`, note that this applies to all the places which match this scenario



##########
src/java/org/apache/cassandra/schema/CompactionParams.java:
##########
@@ -77,7 +78,7 @@ public static Optional<TombstoneOption> forName(String name)
 
     public static final boolean DEFAULT_ENABLED = true;
     public static final TombstoneOption DEFAULT_PROVIDE_OVERLAPPING_TOMBSTONES =
-            TombstoneOption.valueOf(System.getProperty("default.provide.overlapping.tombstones", TombstoneOption.NONE.toString()).toUpperCase());
+            TombstoneOption.valueOf(CassandraRelevantProperties.DEFAULT_PROVIDE_OVERLAPPING_TOMBSTONES.getString(TombstoneOption.NONE.toString()).toUpperCase());

Review Comment:
   Maybe add a static import?



##########
test/distributed/org/apache/cassandra/distributed/fuzz/HarryHelper.java:
##########
@@ -26,19 +26,27 @@
 import harry.model.clock.OffsetClock;
 import harry.model.sut.PrintlnSut;
 
+import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_ALLOW_SIMPLE_STRATEGY;
+import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_MINIMUM_REPLICATION_FACTOR;
+import static org.apache.cassandra.config.CassandraRelevantProperties.DISABLE_TCACTIVE_OPENSSL;
+import static org.apache.cassandra.config.CassandraRelevantProperties.LOG4J2_DISABLE_JMX;
+import static org.apache.cassandra.config.CassandraRelevantProperties.LOG4J2_DISABLE_JMX_LEGACY;
+import static org.apache.cassandra.config.CassandraRelevantProperties.LOG4J_SHUTDOWN_HOOK_ENABLED;
+import static org.apache.cassandra.config.CassandraRelevantProperties.ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION;
+
 public class HarryHelper
 {
     public static void init()
     {
-        System.setProperty("log4j2.disableJmx", "true"); // setting both ways as changes between versions
-        System.setProperty("log4j2.disable.jmx", "true");
-        System.setProperty("log4j.shutdownHookEnabled", "false");
-        System.setProperty("cassandra.allow_simplestrategy", "true"); // makes easier to share OSS tests without RF limits
-        System.setProperty("cassandra.minimum_replication_factor", "0"); // makes easier to share OSS tests without RF limits
-
-        System.setProperty("cassandra.disable_tcactive_openssl", "true");
-        System.setProperty("relocated.shaded.io.netty.transport.noNative", "true");
-        System.setProperty("org.apache.cassandra.disable_mbean_registration", "true");
+        // setting both ways as changes between versions
+        LOG4J2_DISABLE_JMX.setBoolean(true);
+        LOG4J2_DISABLE_JMX_LEGACY.setBoolean(true);
+        LOG4J_SHUTDOWN_HOOK_ENABLED.setBoolean(false);
+        CASSANDRA_ALLOW_SIMPLE_STRATEGY.setBoolean(true); // makes easier to share OSS tests without RF limits
+        CASSANDRA_MINIMUM_REPLICATION_FACTOR.setInt(0); // makes easier to share OSS tests without RF limits
+        DISABLE_TCACTIVE_OPENSSL.setBoolean(true);
+        System.setProperty("relocated.shaded.io.netty.transport.noNative", "true"); // checkstyle: suppress nearby 'blockSystemPropertyUsage'

Review Comment:
   why not?



##########
src/java/org/apache/cassandra/service/StorageService.java:
##########
@@ -387,8 +401,7 @@ public Collection<Range<Token>> getPrimaryRangesWithinDC(String keyspace)
     private volatile boolean isBootstrapMode;
 
     /* we bootstrap but do NOT join the ring unless told to do so */
-    private boolean isSurveyMode = Boolean.parseBoolean(System.getProperty
-            ("cassandra.write_survey", "false"));

Review Comment:
   this property in CRP is defined as `write_survey`, without `cassandra.` prefix



##########
src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategyOptions.java:
##########
@@ -45,7 +46,7 @@
     public static final String EXPIRED_SSTABLE_CHECK_FREQUENCY_SECONDS_KEY = "expired_sstable_check_frequency_seconds";
     public static final String UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION_KEY = "unsafe_aggressive_sstable_expiration";
 
-    static final String UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION_PROPERTY = Config.PROPERTY_PREFIX + "allow_unsafe_aggressive_sstable_expiration";
+    static final boolean UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION_ENABLED = ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION.getBoolean();

Review Comment:
   Caching is a separate thing, maybe after this patch it can be fixed globally by storing cached values in `CassandraRelevantProperties` as with the new checkstyle we limited the number of possible access paths.



##########
test/unit/org/apache/cassandra/CassandraXMLJUnitResultFormatter.java:
##########
@@ -73,17 +76,17 @@ private static DocumentBuilder getDocumentBuilder() {
         }
     }
 
-    private static final String tag = System.getProperty("cassandra.testtag", "");
+    private static final String tag = TEST_CASSANDRA_TESTTAG.getString();
 
     /*
      * Set the property for the test suite name so that log configuration can pick it up
      * and log to a file specific to this test suite
      */
     static
     {
-        String command = System.getProperty("sun.java.command");
+        String command = TEST_SUN_JAVA_COMMAND.getString();

Review Comment:
   It is probably not a test property



##########
test/distributed/org/apache/cassandra/distributed/test/CASTest.java:
##########
@@ -76,7 +77,7 @@ public class CASTest extends CASCommonTestCases
     @BeforeClass
     public static void beforeClass() throws Throwable
     {
-        System.setProperty("cassandra.paxos.use_self_execution", "false");
+        PAXOS_EXECUTE_ON_SELF.setBoolean(false);

Review Comment:
   Why the name of this CRP does not reflect the property name?



##########
src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java:
##########
@@ -50,8 +50,10 @@
  */
 public class CommitLogSegmentReader implements Iterable<CommitLogSegmentReader.SyncSegment>
 {
-    public static final String ALLOW_IGNORE_SYNC_CRC = Config.PROPERTY_PREFIX + "commitlog.allow_ignore_sync_crc";
-    private static volatile boolean allowSkipSyncMarkerCrc = Boolean.getBoolean(ALLOW_IGNORE_SYNC_CRC);
+    /** @deprecated Use {@link org.apache.cassandra.config.CassandraRelevantProperties#COMMITLOG_ALLOW_IGNORE_SYNC_CRC}. */

Review Comment:
   I have a different opinion - we should remove it, especially that this patch is going to be in a new major release. There will be significantly more API changes anyway and this actually does not look like an API



##########
src/java/org/apache/cassandra/service/ClientState.java:
##########
@@ -101,13 +102,13 @@
     static
     {
         QueryHandler handler = QueryProcessor.instance;
-        String customHandlerClass = System.getProperty("cassandra.custom_query_handler_class");
+        String customHandlerClass = CUSTOM_QUERY_HANDLER_CLASS.getString();
         if (customHandlerClass != null)
         {
             try
             {
                 handler = FBUtilities.construct(customHandlerClass, "QueryHandler");
-                logger.info("Using {} as query handler for native protocol queries (as requested with -Dcassandra.custom_query_handler_class)", customHandlerClass);
+                logger.info("Using {} as query handler for native protocol queries (as requested with -D" + CUSTOM_QUERY_HANDLER_CLASS.getKey() + ')', customHandlerClass);

Review Comment:
   Please use interpolation {}



##########
src/java/org/apache/cassandra/gms/Gossiper.java:
##########
@@ -257,18 +262,19 @@ public void expireUpgradeFromVersion()
         ((ExpiringMemoizingSupplier<CassandraVersion>) upgradeFromVersionMemoized).expire();
     }
 
-    private static final boolean disableThreadValidation = Boolean.getBoolean(Props.DISABLE_THREAD_VALIDATION);
+    private static final boolean disableThreadValidation = GOSSIP_DISABLE_THREAD_VALIDATION.getBoolean();
     private static volatile boolean disableEndpointRemoval = DISABLE_GOSSIP_ENDPOINT_REMOVAL.getBoolean();
 
     private static long getVeryLongTime()
     {
-        String newVLT =  System.getProperty("cassandra.very_long_time_ms");
-        if (newVLT != null)
-        {
-            logger.info("Overriding aVeryLongTime to {}ms", newVLT);
-            return Long.parseLong(newVLT);
-        }
-        return 259200 * 1000; // 3 days
+        long time = VERY_LONG_TIME_MS.getLong();

Review Comment:
   and this is yet another example where some generalized dealing with defaults could be helpful



##########
src/java/org/apache/cassandra/config/CassandraRelevantProperties.java:
##########


Review Comment:
   Property `org.apache.cassandra.disable_mbean_registration` is duplicated



##########
test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java:
##########
@@ -233,11 +235,9 @@ public void testInvalidPartition() throws Exception
     @Test
     public void testInvalidPartitionPropertyOverride() throws Exception
     {
-        String key = Config.PROPERTY_PREFIX + "partitioner";
-        String previous = System.getProperty(key);
+        String previous = CassandraRelevantProperties.PARTITIONER.setString("ThisDoesNotExist");

Review Comment:
   It looks like this scenario is repeated in many places - we set some property, do a test, then we want to revert. Would you mind adding something to test utils which just store all the properties and then revert them so that we do not need to bother every time? It could be lambda or thread local variable in test utils, whatever, maybe just use `WithProperties` as it seems to serve that purpose?



##########
test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java:
##########
@@ -92,13 +97,12 @@ public void withSystemProperties()
         // It is possible that we define a common string representation for these types so they can be written to; this
         // is an issue that SettingsTable may need to worry about.
         try (WithProperties ignore = new WithProperties(CONFIG_ALLOW_SYSTEM_PROPERTIES.getKey(), "true",
-                                                        SYSTEM_PROPERTY_PREFIX + "storage_port", "123",
-                                                        SYSTEM_PROPERTY_PREFIX + "commitlog_sync", "batch",
-                                                        SYSTEM_PROPERTY_PREFIX + "seed_provider.class_name", "org.apache.cassandra.locator.SimpleSeedProvider",
-//                                                        PROPERTY_PREFIX + "client_encryption_options.cipher_suites", "[\"FakeCipher\"]",
-                                                        SYSTEM_PROPERTY_PREFIX + "client_encryption_options.optional", "false",
-                                                        SYSTEM_PROPERTY_PREFIX + "client_encryption_options.enabled", "true",
-                                                        SYSTEM_PROPERTY_PREFIX + "doesnotexist", "true"
+                                                        CASSANDRA_SETTINGS_STORAGE_PORT.getKey(), "123",

Review Comment:
   To me, this test does not look very consistent. I wouldn't expect to have those properties defined in CRP as those are not real properties. To make it consistent, I think we should define names in this test, like `COMMITLOG_SYNC="commitlog_sync"`, then  use reflection for accessing `prop` in `config` and `SYSTEM_PROPERTY_PREFIX + prop` to access system properties. This way we can be sure we consistently use that mechanism. 
   
   IMHO declaring random properties in CRP for a single test case can be confusing. In other words, I think that we should define only those properties in CRP which are directly accessed somewhere in the production code or in test infrastructure classes.
   



##########
src/java/org/apache/cassandra/config/CassandraRelevantProperties.java:
##########


Review Comment:
   As a general note to this file, I think we should:
   - remove redundant defaults, like `null`, `false` (maybe fail if such default is defined)
   - maybe fail if a default is defined for non-cassandra property - can we really assume such default?
   - perhaps have overloaded set() method rather than setString, setInt, setLong, etc.
   - have additional methods to get as optional
   - have methods to check if is unset or default
   - maybe have some methods to get values as `DataStorageSpec` and `DurationSpec` objects
   - have some static initializer which asserts there are no duplicates
   - rename enums so that they reflect the area of application
   - rename some pure test properties which lacks reasonable prefix
   - sort items



##########
src/java/org/apache/cassandra/config/CassandraRelevantProperties.java:
##########


Review Comment:
   There are some unnecessary null defaults, for example `CLOCK_MONOTONIC_APPROX`, could you clean all of them? `null`, `false`, etc.



##########
test/distributed/org/apache/cassandra/distributed/shared/WithProperties.java:
##########
@@ -80,35 +75,28 @@ public void set(CassandraRelevantProperties prop, long value)
         set(prop, Long.toString(value));
     }
 
-    public void with(String key, String value)
-    {
-        String previous = System.setProperty(key, value);
-        properties.add(new Property(key, previous));
-    }
-
-
     @Override
     public void close()
     {
         Collections.reverse(properties);
         properties.forEach(s -> {
-            if (s.value == null)
-                System.getProperties().remove(s.key);
+            if (s.prevValue == null)
+                s.prop.clearValue();
             else
-                System.setProperty(s.key, s.value);
+                s.prop.setString(s.prevValue);
         });
         properties.clear();
     }
 
     private static final class Property
     {
-        private final String key;
-        private final String value;
+        private final CassandraRelevantProperties prop;

Review Comment:
   I think this is too strict. Let keep the key being a string



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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