You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ed...@apache.org on 2022/02/06 03:44:21 UTC

[cassandra] 08/13: Transfer parameters to the newly introduced configuration framework (4) patch by Ekaterina Dimitrova; reviewed by Caleb Rackliffe, David Capwell, Michael Semb Wever and Benjamin Lerer for CASSANDRA-15234

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

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

commit ed48f3c017c5e572a523890bcd5b7c798d7eb358
Author: Ekaterina Dimitrova <ek...@datastax.com>
AuthorDate: Thu Feb 3 16:43:36 2022 -0500

    Transfer parameters to the newly introduced configuration framework (4)
    patch by Ekaterina Dimitrova; reviewed by Caleb Rackliffe, David Capwell, Michael Semb Wever and Benjamin Lerer for CASSANDRA-15234
---
 conf/cassandra.yaml                                | 18 ++---
 src/java/org/apache/cassandra/config/Config.java   | 30 ++++---
 .../cassandra/config/DatabaseDescriptor.java       | 94 ++++++++--------------
 src/java/org/apache/cassandra/db/Keyspace.java     |  4 +-
 .../apache/cassandra/streaming/StreamSession.java  |  2 +-
 .../cassandra/distributed/test/CASAddTest.java     |  4 +-
 .../apache/cassandra/distributed/test/CASTest.java | 53 ++++++------
 .../distributed/test/LargeColumnTest.java          |  4 +-
 .../distributed/test/MessageFiltersTest.java       |  3 +-
 .../test/ReadRepairEmptyRangeTombstonesTest.java   |  5 +-
 .../distributed/test/ReadRepairQueryTypesTest.java |  4 +-
 .../cassandra/distributed/test/ReadRepairTest.java |  2 +-
 .../test/ring/ReadsDuringBootstrapTest.java        |  4 +-
 .../upgrade/MixedModeAvailabilityTestBase.java     |  5 +-
 .../upgrade/MixedModeConsistencyTestBase.java      |  5 +-
 .../upgrade/MixedModeMessageForwardTest.java       |  1 -
 .../cassandra/simulator/ClusterSimulation.java     |  8 +-
 .../cassandra/config/DatabaseDescriptorTest.java   | 65 ++++++++-------
 .../LoadOldYAMLBackwardCompatibilityTest.java      |  4 +-
 19 files changed, 146 insertions(+), 169 deletions(-)

diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index e9cce68..3d8168b 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -984,28 +984,28 @@ sstable_preemptive_open_interval: 50MiB
 
 # How long the coordinator should wait for read operations to complete.
 # Lowest acceptable value is 10 ms.
-read_request_timeout_in_ms: 5000
+read_request_timeout: 5000ms
 # How long the coordinator should wait for seq or index scans to complete.
 # Lowest acceptable value is 10 ms.
-range_request_timeout_in_ms: 10000
+range_request_timeout: 10000ms
 # How long the coordinator should wait for writes to complete.
 # Lowest acceptable value is 10 ms.
-write_request_timeout_in_ms: 2000
+write_request_timeout: 2000ms
 # How long the coordinator should wait for counter writes to complete.
 # Lowest acceptable value is 10 ms.
-counter_write_request_timeout_in_ms: 5000
+counter_write_request_timeout: 5000ms
 # How long a coordinator should continue to retry a CAS operation
 # that contends with other proposals for the same row.
 # Lowest acceptable value is 10 ms.
-cas_contention_timeout_in_ms: 1000
+cas_contention_timeout: 1000ms
 # How long the coordinator should wait for truncates to complete
 # (This can be much longer, because unless auto_snapshot is disabled
 # we need to flush first so we can snapshot before removing the data.)
 # Lowest acceptable value is 10 ms.
-truncate_request_timeout_in_ms: 60000
+truncate_request_timeout: 60000ms
 # The default timeout for other, miscellaneous operations.
 # Lowest acceptable value is 10 ms.
-request_timeout_in_ms: 10000
+request_timeout: 10000ms
 
 # Defensive settings for protecting Cassandra from true network partitions.
 # See (CASSANDRA-14358) for details.
@@ -1049,7 +1049,7 @@ request_timeout_in_ms: 10000
 # How long before a node logs slow queries. Select queries that take longer than
 # this timeout to execute, will generate an aggregated log message, so that slow queries
 # can be identified. Set this value to zero to disable slow query logging.
-slow_query_log_timeout_in_ms: 500
+slow_query_log_timeout: 500ms
 
 # Enable operation timeout information exchange between nodes to accurately
 # measure request timeouts.  If disabled, replicas will assume that requests
@@ -1067,7 +1067,7 @@ slow_query_log_timeout_in_ms: 500
 # 2 keep-alive cycles the stream session times out and fail
 # Default value is 300s (5 minutes), which means stalled stream
 # times out in 10 minutes by default
-# streaming_keep_alive_period_in_secs: 300
+# streaming_keep_alive_period: 300s
 
 # Limit number of connections per host for streaming
 # Increase this when you notice that joins are CPU-bound rather that network
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index 172f16b..6c22d5f 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -102,29 +102,39 @@ public class Config
     /** Triggers automatic allocation of tokens if set, based on the provided replica count for a datacenter */
     public Integer allocate_tokens_for_local_replication_factor = null;
 
-    public long native_transport_idle_timeout_in_ms = 0L;
+    @Replaces(oldName = "native_transport_idle_timeout_in_ms", converter = Converters.MILLIS_DURATION, deprecated = true)
+    public SmallestDurationMilliseconds native_transport_idle_timeout = new SmallestDurationMilliseconds("0ms");
 
-    public volatile long request_timeout_in_ms = 10000L;
+    @Replaces(oldName = "request_timeout_in_ms", converter = Converters.MILLIS_DURATION, deprecated = true)
+    public volatile SmallestDurationMilliseconds request_timeout = new SmallestDurationMilliseconds("10000ms");
 
-    public volatile long read_request_timeout_in_ms = 5000L;
+    @Replaces(oldName = "read_request_timeout_in_ms", converter = Converters.MILLIS_DURATION, deprecated = true)
+    public volatile SmallestDurationMilliseconds read_request_timeout = new SmallestDurationMilliseconds("5000ms");
 
-    public volatile long range_request_timeout_in_ms = 10000L;
+    @Replaces(oldName = "range_request_timeout_in_ms", converter = Converters.MILLIS_DURATION, deprecated = true)
+    public volatile SmallestDurationMilliseconds range_request_timeout = new SmallestDurationMilliseconds("10000ms");
 
-    public volatile long write_request_timeout_in_ms = 2000L;
+    @Replaces(oldName = "write_request_timeout_in_ms", converter = Converters.MILLIS_DURATION, deprecated = true)
+    public volatile SmallestDurationMilliseconds write_request_timeout = new SmallestDurationMilliseconds("2000ms");
 
-    public volatile long counter_write_request_timeout_in_ms = 5000L;
+    @Replaces(oldName = "counter_write_request_timeout_in_ms", converter = Converters.MILLIS_DURATION, deprecated = true)
+    public volatile SmallestDurationMilliseconds counter_write_request_timeout = new SmallestDurationMilliseconds("5000ms");
 
-    public volatile long cas_contention_timeout_in_ms = 1000L;
+    @Replaces(oldName = "cas_contention_timeout_in_ms", converter = Converters.MILLIS_DURATION, deprecated = true)
+    public volatile SmallestDurationMilliseconds cas_contention_timeout = new SmallestDurationMilliseconds("1000ms");
 
-    public volatile long truncate_request_timeout_in_ms = 60000L;
+    @Replaces(oldName = "truncate_request_timeout_in_ms", converter = Converters.MILLIS_DURATION, deprecated = true)
+    public volatile SmallestDurationMilliseconds truncate_request_timeout = new SmallestDurationMilliseconds("60000ms");
 
     public Integer streaming_connections_per_host = 1;
-    public Integer streaming_keep_alive_period_in_secs = 300; //5 minutes
+    @Replaces(oldName = "streaming_keep_alive_period_in_secs", converter = Converters.SECONDS_DURATION, deprecated = true)
+    public SmallestDurationSeconds streaming_keep_alive_period = new SmallestDurationSeconds("300s");
 
     @Replaces(oldName = "cross_node_timeout", converter = Converters.IDENTITY, deprecated = true)
     public boolean internode_timeout = true;
 
-    public volatile long slow_query_log_timeout_in_ms = 500L;
+    @Replaces(oldName = "slow_query_log_timeout_in_ms", converter = Converters.MILLIS_DURATION, deprecated = true)
+    public volatile SmallestDurationMilliseconds slow_query_log_timeout = new SmallestDurationMilliseconds("500ms");
 
     public volatile double phi_convict_threshold = 8.0;
 
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 978a6dd..d573def 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -104,7 +104,7 @@ public class DatabaseDescriptor
     /**
      * Request timeouts can not be less than below defined value (see CASSANDRA-9375)
      */
-    static final long LOWEST_ACCEPTED_TIMEOUT = 10L;
+    static final SmallestDurationMilliseconds LOWEST_ACCEPTED_TIMEOUT = SmallestDurationMilliseconds.inMilliseconds(10L);
 
     private static Supplier<IFailureDetector> newFailureDetector;
     private static IEndpointSnitch snitch;
@@ -1061,52 +1061,22 @@ public class DatabaseDescriptor
     @VisibleForTesting
     static void checkForLowestAcceptedTimeouts(Config conf)
     {
-        if(conf.read_request_timeout_in_ms < LOWEST_ACCEPTED_TIMEOUT)
-        {
-           logInfo("read_request_timeout_in_ms", conf.read_request_timeout_in_ms, LOWEST_ACCEPTED_TIMEOUT);
-           conf.read_request_timeout_in_ms = LOWEST_ACCEPTED_TIMEOUT;
-        }
-
-        if(conf.range_request_timeout_in_ms < LOWEST_ACCEPTED_TIMEOUT)
-        {
-           logInfo("range_request_timeout_in_ms", conf.range_request_timeout_in_ms, LOWEST_ACCEPTED_TIMEOUT);
-           conf.range_request_timeout_in_ms = LOWEST_ACCEPTED_TIMEOUT;
-        }
-
-        if(conf.request_timeout_in_ms < LOWEST_ACCEPTED_TIMEOUT)
-        {
-           logInfo("request_timeout_in_ms", conf.request_timeout_in_ms, LOWEST_ACCEPTED_TIMEOUT);
-           conf.request_timeout_in_ms = LOWEST_ACCEPTED_TIMEOUT;
-        }
-
-        if(conf.write_request_timeout_in_ms < LOWEST_ACCEPTED_TIMEOUT)
-        {
-           logInfo("write_request_timeout_in_ms", conf.write_request_timeout_in_ms, LOWEST_ACCEPTED_TIMEOUT);
-           conf.write_request_timeout_in_ms = LOWEST_ACCEPTED_TIMEOUT;
-        }
-
-        if(conf.cas_contention_timeout_in_ms < LOWEST_ACCEPTED_TIMEOUT)
-        {
-           logInfo("cas_contention_timeout_in_ms", conf.cas_contention_timeout_in_ms, LOWEST_ACCEPTED_TIMEOUT);
-           conf.cas_contention_timeout_in_ms = LOWEST_ACCEPTED_TIMEOUT;
-        }
-
-        if(conf.counter_write_request_timeout_in_ms < LOWEST_ACCEPTED_TIMEOUT)
-        {
-           logInfo("counter_write_request_timeout_in_ms", conf.counter_write_request_timeout_in_ms, LOWEST_ACCEPTED_TIMEOUT);
-           conf.counter_write_request_timeout_in_ms = LOWEST_ACCEPTED_TIMEOUT;
-        }
-
-        if(conf.truncate_request_timeout_in_ms < LOWEST_ACCEPTED_TIMEOUT)
-        {
-           logInfo("truncate_request_timeout_in_ms", conf.truncate_request_timeout_in_ms, LOWEST_ACCEPTED_TIMEOUT);
-           conf.truncate_request_timeout_in_ms = LOWEST_ACCEPTED_TIMEOUT;
-        }
+        conf.read_request_timeout = max("read_request_timeout", conf.read_request_timeout, LOWEST_ACCEPTED_TIMEOUT);
+        conf.range_request_timeout = max("range_request_timeout", conf.range_request_timeout, LOWEST_ACCEPTED_TIMEOUT);
+        conf.request_timeout = max("request_timeout", conf.request_timeout, LOWEST_ACCEPTED_TIMEOUT);
+        conf.write_request_timeout = max("write_request_timeout", conf.write_request_timeout, LOWEST_ACCEPTED_TIMEOUT);
+        conf.cas_contention_timeout = max("cas_contention_timeout", conf.cas_contention_timeout, LOWEST_ACCEPTED_TIMEOUT);
+        conf.counter_write_request_timeout = max("counter_write_request_timeout", conf.counter_write_request_timeout, LOWEST_ACCEPTED_TIMEOUT);
+        conf.truncate_request_timeout = max("truncate_request_timeout", conf.truncate_request_timeout, LOWEST_ACCEPTED_TIMEOUT);
     }
 
-    private static void logInfo(String property, long actualValue, long lowestAcceptedValue)
+    private static void logInfo(String property, SmallestDurationMilliseconds actualValue, SmallestDurationMilliseconds lowestAcceptedValue)
     {
-        logger.info("found {}::{} less than lowest acceptable value {}, continuing with {}", property, actualValue, lowestAcceptedValue, lowestAcceptedValue);
+        logger.info("found {}::{} less than lowest acceptable value {}, continuing with {}",
+                    property,
+                    actualValue.toString(),
+                    lowestAcceptedValue.toString(),
+                    lowestAcceptedValue);
     }
 
     public static void applyTokensConfig()
@@ -1644,82 +1614,82 @@ public class DatabaseDescriptor
 
     public static long nativeTransportIdleTimeout()
     {
-        return conf.native_transport_idle_timeout_in_ms;
+        return conf.native_transport_idle_timeout.toMilliseconds();
     }
 
     public static void setNativeTransportIdleTimeout(long nativeTransportTimeout)
     {
-        conf.native_transport_idle_timeout_in_ms = nativeTransportTimeout;
+        conf.native_transport_idle_timeout= SmallestDurationMilliseconds.inMilliseconds(nativeTransportTimeout);
     }
 
     public static long getRpcTimeout(TimeUnit unit)
     {
-        return unit.convert(conf.request_timeout_in_ms, MILLISECONDS);
+        return unit.convert(conf.request_timeout.toMilliseconds(), MILLISECONDS);
     }
 
     public static void setRpcTimeout(long timeOutInMillis)
     {
-        conf.request_timeout_in_ms = timeOutInMillis;
+        conf.request_timeout = SmallestDurationMilliseconds.inMilliseconds(timeOutInMillis);
     }
 
     public static long getReadRpcTimeout(TimeUnit unit)
     {
-        return unit.convert(conf.read_request_timeout_in_ms, MILLISECONDS);
+        return unit.convert(conf.read_request_timeout.toMilliseconds(), MILLISECONDS);
     }
 
     public static void setReadRpcTimeout(long timeOutInMillis)
     {
-        conf.read_request_timeout_in_ms = timeOutInMillis;
+        conf.read_request_timeout = SmallestDurationMilliseconds.inMilliseconds(timeOutInMillis);
     }
 
     public static long getRangeRpcTimeout(TimeUnit unit)
     {
-        return unit.convert(conf.range_request_timeout_in_ms, MILLISECONDS);
+        return unit.convert(conf.range_request_timeout.toMilliseconds(), MILLISECONDS);
     }
 
     public static void setRangeRpcTimeout(long timeOutInMillis)
     {
-        conf.range_request_timeout_in_ms = timeOutInMillis;
+        conf.range_request_timeout = SmallestDurationMilliseconds.inMilliseconds(timeOutInMillis);
     }
 
     public static long getWriteRpcTimeout(TimeUnit unit)
     {
-        return unit.convert(conf.write_request_timeout_in_ms, MILLISECONDS);
+        return unit.convert(conf.write_request_timeout.toMilliseconds(), MILLISECONDS);
     }
 
     public static void setWriteRpcTimeout(long timeOutInMillis)
     {
-        conf.write_request_timeout_in_ms = timeOutInMillis;
+        conf.write_request_timeout = SmallestDurationMilliseconds.inMilliseconds(timeOutInMillis);
     }
 
     public static long getCounterWriteRpcTimeout(TimeUnit unit)
     {
-        return unit.convert(conf.counter_write_request_timeout_in_ms, MILLISECONDS);
+        return unit.convert(conf.counter_write_request_timeout.toMilliseconds(), MILLISECONDS);
     }
 
     public static void setCounterWriteRpcTimeout(long timeOutInMillis)
     {
-        conf.counter_write_request_timeout_in_ms = timeOutInMillis;
+        conf.counter_write_request_timeout = SmallestDurationMilliseconds.inMilliseconds(timeOutInMillis);
     }
 
     public static long getCasContentionTimeout(TimeUnit unit)
     {
-        return unit.convert(conf.cas_contention_timeout_in_ms, MILLISECONDS);
+        return unit.convert(conf.cas_contention_timeout.toMilliseconds(), MILLISECONDS);
     }
 
     public static void setCasContentionTimeout(long timeOutInMillis)
     {
-        conf.cas_contention_timeout_in_ms = timeOutInMillis;
+        conf.cas_contention_timeout = SmallestDurationMilliseconds.inMilliseconds(timeOutInMillis);
     }
 
     public static long getTruncateRpcTimeout(TimeUnit unit)
     {
-        return unit.convert(conf.truncate_request_timeout_in_ms, MILLISECONDS);
+        return unit.convert(conf.truncate_request_timeout.toMilliseconds(), MILLISECONDS);
     }
 
     public static void setTruncateRpcTimeout(long timeOutInMillis)
     {
-        conf.truncate_request_timeout_in_ms = timeOutInMillis;
+        conf.truncate_request_timeout = SmallestDurationMilliseconds.inMilliseconds(timeOutInMillis);
     }
 
     public static boolean hasCrossNodeTimeout()
@@ -1734,7 +1704,7 @@ public class DatabaseDescriptor
 
     public static long getSlowQueryTimeout(TimeUnit units)
     {
-        return units.convert(conf.slow_query_log_timeout_in_ms, MILLISECONDS);
+        return units.convert(conf.slow_query_log_timeout.toMilliseconds(), MILLISECONDS);
     }
 
     /**
@@ -2945,7 +2915,7 @@ public class DatabaseDescriptor
 
     public static int getStreamingKeepAlivePeriod()
     {
-        return conf.streaming_keep_alive_period_in_secs;
+        return conf.streaming_keep_alive_period.toSecondsAsInt();
     }
 
     public static int getStreamingConnectionsPerHost()
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index 2c27889..459f926 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -481,7 +481,7 @@ public class Keyspace
      *                       may happen concurrently, depending on the CL Executor type.
      * @param makeDurable    if true, don't return unless write has been made durable
      * @param updateIndexes  false to disable index updates (used by CollationController "defragmenting")
-     * @param isDroppable    true if this should throw WriteTimeoutException if it does not acquire lock within write_request_timeout_in_ms
+     * @param isDroppable    true if this should throw WriteTimeoutException if it does not acquire lock within write_request_timeout
      */
     public void apply(final Mutation mutation,
                       final boolean makeDurable,
@@ -498,7 +498,7 @@ public class Keyspace
      *                       may happen concurrently, depending on the CL Executor type.
      * @param makeDurable    if true, don't return unless write has been made durable
      * @param updateIndexes  false to disable index updates (used by CollationController "defragmenting")
-     * @param isDroppable    true if this should throw WriteTimeoutException if it does not acquire lock within write_request_timeout_in_ms
+     * @param isDroppable    true if this should throw WriteTimeoutException if it does not acquire lock within write_request_timeout
      * @param isDeferrable   true if caller is not waiting for future to complete, so that future may be deferred
      */
     private Future<?> applyInternal(final Mutation mutation,
diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java
index fd24539..02255dd 100644
--- a/src/java/org/apache/cassandra/streaming/StreamSession.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@ -670,7 +670,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         if (e instanceof SocketTimeoutException)
         {
             logger.error("[Stream #{}] Did not receive response from peer {}{} for {} secs. Is peer down? " +
-                         "If not, maybe try increasing streaming_keep_alive_period_in_secs.", planId(),
+                         "If not, maybe try increasing streaming_keep_alive_period.", planId(),
                          hostAddressAndPort(channel.peer()),
                          channel.peer().equals(channel.connectedTo()) ? "" : " through " + hostAddressAndPort(channel.connectedTo()),
                          2 * DatabaseDescriptor.getStreamingKeepAlivePeriod(),
diff --git a/test/distributed/org/apache/cassandra/distributed/test/CASAddTest.java b/test/distributed/org/apache/cassandra/distributed/test/CASAddTest.java
index 02f9fd8..59220cc 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/CASAddTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/CASAddTest.java
@@ -35,12 +35,12 @@ public class CASAddTest extends TestBaseImpl
     private static final Logger logger = LoggerFactory.getLogger(CASAddTest.class);
 
     /**
-     * The {@code cas_contention_timeout_in_ms} used during the tests
+     * The {@code cas_contention_timeout} used during the tests
      */
     private static final long CONTENTION_TIMEOUT = 1000L;
 
     /**
-     * The {@code write_request_timeout_in_ms} used during the tests
+     * The {@code write_request_timeout} used during the tests
      */
     private static final long REQUEST_TIMEOUT = 1000L;
 
diff --git a/test/distributed/org/apache/cassandra/distributed/test/CASTest.java b/test/distributed/org/apache/cassandra/distributed/test/CASTest.java
index 8dd5977..30a8a25 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/CASTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/CASTest.java
@@ -27,9 +27,6 @@ import org.junit.Assert;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.dht.Murmur3Partitioner;
@@ -56,17 +53,15 @@ import static org.junit.Assert.assertTrue;
 
 public class CASTest extends TestBaseImpl
 {
-    private static final Logger logger = LoggerFactory.getLogger(CASTest.class);
-
     /**
-     * The {@code cas_contention_timeout_in_ms} used during the tests
+     * The {@code cas_contention_timeout} used during the tests
      */
-    private static final long CONTENTION_TIMEOUT = 1000L;
+    private static final String CONTENTION_TIMEOUT = "1000ms";
 
     /**
-     * The {@code write_request_timeout_in_ms} used during the tests
+     * The {@code write_request_timeout} used during the tests
      */
-    private static final long REQUEST_TIMEOUT = 1000L;
+    private static final String REQUEST_TIMEOUT = "1000ms";
 
     @Test
     public void simpleUpdate() throws Throwable
@@ -90,8 +85,8 @@ public class CASTest extends TestBaseImpl
     @Test
     public void incompletePrepare() throws Throwable
     {
-        try (Cluster cluster = init(Cluster.create(3, config -> config.set("write_request_timeout_in_ms", REQUEST_TIMEOUT)
-                                                                      .set("cas_contention_timeout_in_ms", CONTENTION_TIMEOUT))))
+        try (Cluster cluster = init(Cluster.create(3, config -> config.set("write_request_timeout", REQUEST_TIMEOUT)
+                                                                      .set("cas_contention_timeout", CONTENTION_TIMEOUT))))
         {
             cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))");
 
@@ -114,8 +109,8 @@ public class CASTest extends TestBaseImpl
     @Test
     public void incompletePropose() throws Throwable
     {
-        try (Cluster cluster = init(Cluster.create(3, config -> config.set("write_request_timeout_in_ms", REQUEST_TIMEOUT)
-                                                                      .set("cas_contention_timeout_in_ms", CONTENTION_TIMEOUT))))
+        try (Cluster cluster = init(Cluster.create(3, config -> config.set("write_request_timeout", REQUEST_TIMEOUT)
+                                                                      .set("cas_contention_timeout", CONTENTION_TIMEOUT))))
         {
             cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))");
 
@@ -141,8 +136,8 @@ public class CASTest extends TestBaseImpl
     @Test
     public void incompleteCommit() throws Throwable
     {
-        try (Cluster cluster = init(Cluster.create(3, config -> config.set("write_request_timeout_in_ms", REQUEST_TIMEOUT)
-                                                                      .set("cas_contention_timeout_in_ms", CONTENTION_TIMEOUT))))
+        try (Cluster cluster = init(Cluster.create(3, config -> config.set("write_request_timeout", REQUEST_TIMEOUT)
+                                                                      .set("cas_contention_timeout", CONTENTION_TIMEOUT))))
         {
             cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))");
 
@@ -202,8 +197,8 @@ public class CASTest extends TestBaseImpl
         // does not happen on 3.0 and there is no report of such long pauses otherwise, so an hypothesis is that this
         // is due to the in-jvm dtest framework. This is is why we use a 4 seconds timeout here. Given this test is
         // not about performance, this is probably ok, even if we ideally should dug into the underlying reason.
-        try (Cluster cluster = init(Cluster.create(3, config -> config.set("write_request_timeout_in_ms", 4000L)
-                                                                      .set("cas_contention_timeout_in_ms", CONTENTION_TIMEOUT))))
+        try (Cluster cluster = init(Cluster.create(3, config -> config.set("write_request_timeout", "4000ms")
+                                                                      .set("cas_contention_timeout", CONTENTION_TIMEOUT))))
         {
             String table = KEYSPACE + ".t";
             cluster.schemaChange("CREATE TABLE " + table + " (k int PRIMARY KEY, v int)");
@@ -377,8 +372,8 @@ public class CASTest extends TestBaseImpl
     public void testSuccessfulWriteBeforeRangeMovement() throws Throwable
     {
         try (Cluster cluster = Cluster.create(4, config -> config
-                .set("write_request_timeout_in_ms", REQUEST_TIMEOUT)
-                .set("cas_contention_timeout_in_ms", CONTENTION_TIMEOUT)))
+                .set("write_request_timeout", REQUEST_TIMEOUT)
+                .set("cas_contention_timeout", CONTENTION_TIMEOUT)))
         {
             cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};");
             cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v1 int, v2 int, PRIMARY KEY (pk, ck))");
@@ -418,8 +413,8 @@ public class CASTest extends TestBaseImpl
     public void testConflictingWritesWithStaleRingInformation() throws Throwable
     {
         try (Cluster cluster = Cluster.create(4, config -> config
-                .set("write_request_timeout_in_ms", REQUEST_TIMEOUT)
-                .set("cas_contention_timeout_in_ms", CONTENTION_TIMEOUT)))
+                                                           .set("write_request_timeout", REQUEST_TIMEOUT)
+                                                           .set("cas_contention_timeout", CONTENTION_TIMEOUT)))
         {
             cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};");
             cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v1 int, v2 int, PRIMARY KEY (pk, ck))");
@@ -458,8 +453,8 @@ public class CASTest extends TestBaseImpl
     public void testSucccessfulWriteDuringRangeMovementFollowedByRead() throws Throwable
     {
         try (Cluster cluster = Cluster.create(4, config -> config
-                .set("write_request_timeout_in_ms", REQUEST_TIMEOUT)
-                .set("cas_contention_timeout_in_ms", CONTENTION_TIMEOUT)))
+                                                           .set("write_request_timeout", REQUEST_TIMEOUT)
+                                                           .set("cas_contention_timeout", CONTENTION_TIMEOUT)))
         {
             cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};");
             cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))");
@@ -502,8 +497,8 @@ public class CASTest extends TestBaseImpl
     public void testSuccessfulWriteDuringRangeMovementFollowedByConflicting() throws Throwable
     {
         try (Cluster cluster = Cluster.create(4, config -> config
-                .set("write_request_timeout_in_ms", REQUEST_TIMEOUT)
-                .set("cas_contention_timeout_in_ms", CONTENTION_TIMEOUT)))
+                                                           .set("write_request_timeout", REQUEST_TIMEOUT)
+                                                           .set("cas_contention_timeout", CONTENTION_TIMEOUT)))
         {
             cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};");
             cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v1 int, v2 int, PRIMARY KEY (pk, ck))");
@@ -554,8 +549,8 @@ public class CASTest extends TestBaseImpl
     public void testIncompleteWriteFollowedBySuccessfulWriteWithStaleRingDuringRangeMovementFollowedByRead() throws Throwable
     {
         try (Cluster cluster = Cluster.create(4, config -> config
-                .set("write_request_timeout_in_ms", REQUEST_TIMEOUT)
-                .set("cas_contention_timeout_in_ms", CONTENTION_TIMEOUT)))
+                                                           .set("write_request_timeout", REQUEST_TIMEOUT)
+                                                           .set("cas_contention_timeout", CONTENTION_TIMEOUT)))
         {
             cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};");
             cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v1 int, v2 int, PRIMARY KEY (pk, ck))");
@@ -618,8 +613,8 @@ public class CASTest extends TestBaseImpl
     public void testIncompleteWriteFollowedBySuccessfulWriteWithStaleRingDuringRangeMovementFollowedByWrite() throws Throwable
     {
         try (Cluster cluster = Cluster.create(4, config -> config
-                .set("write_request_timeout_in_ms", REQUEST_TIMEOUT)
-                .set("cas_contention_timeout_in_ms", CONTENTION_TIMEOUT)))
+                                                           .set("write_request_timeout", REQUEST_TIMEOUT)
+                                                           .set("cas_contention_timeout", CONTENTION_TIMEOUT)))
         {
             cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};");
             cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v1 int, v2 int, PRIMARY KEY (pk, ck))");
diff --git a/test/distributed/org/apache/cassandra/distributed/test/LargeColumnTest.java b/test/distributed/org/apache/cassandra/distributed/test/LargeColumnTest.java
index e1733ce..7a59876 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/LargeColumnTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/LargeColumnTest.java
@@ -70,8 +70,8 @@ public class LargeColumnTest extends TestBaseImpl
                                                  config.set("commitlog_segment_size_in_mb", (columnSize * 3) >> 20)
                                                        .set("internode_application_send_queue_reserve_endpoint_capacity_in_bytes", columnSize * 2)
                                                        .set("internode_application_send_queue_reserve_global_capacity_in_bytes", columnSize * 3)
-                                                       .set("write_request_timeout_in_ms", SECONDS.toMillis(30L))
-                                                       .set("read_request_timeout_in_ms", SECONDS.toMillis(30L))
+                                                       .set("write_request_timeout", "30s")
+                                                       .set("read_request_timeout", "30s")
                                                        .set("memtable_heap_space_in_mb", 1024)
                                      )
                                      .start()))
diff --git a/test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java b/test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java
index 9cae1bd..9ecea99 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java
@@ -30,6 +30,7 @@ import com.google.common.collect.Sets;
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.cassandra.config.SmallestDurationMilliseconds;
 import org.apache.cassandra.distributed.Cluster;
 import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.ICluster;
@@ -187,7 +188,7 @@ public class MessageFiltersTest extends TestBaseImpl
         String read = "SELECT * FROM " + KEYSPACE + ".tbl";
         String write = "INSERT INTO " + KEYSPACE + ".tbl (pk, ck, v) VALUES (1, 1, 1)";
 
-        try (ICluster<IInvokableInstance> cluster = builder().withNodes(2).withConfig(c -> c.set("range_request_timeout_in_ms", 20000)).start())
+        try (ICluster<IInvokableInstance> cluster = builder().withNodes(2).withConfig(c -> c.set("range_request_timeout", "2000ms")).start())
         {
             cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': " + cluster.size() + "};");
             cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))");
diff --git a/test/distributed/org/apache/cassandra/distributed/test/ReadRepairEmptyRangeTombstonesTest.java b/test/distributed/org/apache/cassandra/distributed/test/ReadRepairEmptyRangeTombstonesTest.java
index 44ee07e..6cdea9b 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/ReadRepairEmptyRangeTombstonesTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/ReadRepairEmptyRangeTombstonesTest.java
@@ -33,7 +33,6 @@ import org.apache.cassandra.distributed.Cluster;
 import org.apache.cassandra.distributed.shared.AssertUtils;
 import org.apache.cassandra.service.reads.repair.ReadRepairStrategy;
 
-import static java.util.concurrent.TimeUnit.MINUTES;
 import static org.apache.cassandra.distributed.shared.AssertUtils.row;
 
 
@@ -96,8 +95,8 @@ public class ReadRepairEmptyRangeTombstonesTest extends TestBaseImpl
     public static void setupCluster() throws IOException
     {
         cluster = init(Cluster.build(NUM_NODES)
-                              .withConfig(config -> config.set("read_request_timeout_in_ms", MINUTES.toMillis(1))
-                                                          .set("write_request_timeout_in_ms", MINUTES.toMillis(1)))
+                              .withConfig(config -> config.set("read_request_timeout", "1m")
+                                                          .set("write_request_timeout", "1m"))
                               .start());
         cluster.schemaChange(withKeyspace("CREATE TYPE %s.udt (x int, y int)"));
     }
diff --git a/test/distributed/org/apache/cassandra/distributed/test/ReadRepairQueryTypesTest.java b/test/distributed/org/apache/cassandra/distributed/test/ReadRepairQueryTypesTest.java
index b79e0c4..221ac86 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/ReadRepairQueryTypesTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/ReadRepairQueryTypesTest.java
@@ -118,8 +118,8 @@ public class ReadRepairQueryTypesTest extends TestBaseImpl
     public static void setupCluster() throws IOException
     {
         cluster = init(Cluster.build(NUM_NODES)
-                              .withConfig(config -> config.set("read_request_timeout_in_ms", MINUTES.toMillis(1))
-                                                          .set("write_request_timeout_in_ms", MINUTES.toMillis(1)))
+                              .withConfig(config -> config.set("read_request_timeout", "1m")
+                                                          .set("write_request_timeout", "1m"))
                               .start());
         cluster.schemaChange(withKeyspace("CREATE TYPE %s.udt (x int, y int)"));
     }
diff --git a/test/distributed/org/apache/cassandra/distributed/test/ReadRepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/ReadRepairTest.java
index c5e3a71..7ce55bc 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/ReadRepairTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/ReadRepairTest.java
@@ -355,7 +355,7 @@ public class ReadRepairTest extends TestBaseImpl
         String key = "test1";
         try (Cluster cluster = init(Cluster.build()
                                            .withConfig(config -> config.with(Feature.GOSSIP, Feature.NETWORK)
-                                                                       .set("read_request_timeout_in_ms", Integer.MAX_VALUE))
+                                                                       .set("read_request_timeout", String.format("%dms", Integer.MAX_VALUE)))
                                            .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(4))
                                            .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(4, "dc0", "rack0"))
                                            .withNodes(3)
diff --git a/test/distributed/org/apache/cassandra/distributed/test/ring/ReadsDuringBootstrapTest.java b/test/distributed/org/apache/cassandra/distributed/test/ring/ReadsDuringBootstrapTest.java
index 932dbdc..ba2cdc5 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/ring/ReadsDuringBootstrapTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/ring/ReadsDuringBootstrapTest.java
@@ -61,8 +61,8 @@ public class ReadsDuringBootstrapTest extends TestBaseImpl
                                         .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(expandedNodeCount))
                                         .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(expandedNodeCount, "dc0", "rack0"))
                                         .withConfig(config -> config.with(NETWORK, GOSSIP)
-                                                                    .set("read_request_timeout_in_ms", Integer.MAX_VALUE)
-                                                                    .set("request_timeout_in_ms", Integer.MAX_VALUE))
+                                                                    .set("read_request_timeout", String.format("%dms", Integer.MAX_VALUE))
+                                                                    .set("request_timeout", String.format("%dms", Integer.MAX_VALUE)))
                                         .withInstanceInitializer(BB::install)
                                         .start())
         {
diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTestBase.java b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTestBase.java
index c1ae153..e801f76 100644
--- a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTestBase.java
+++ b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTestBase.java
@@ -30,7 +30,6 @@ import org.apache.cassandra.exceptions.ReadTimeoutException;
 import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.net.Verb;
 
-import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.ONE;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
@@ -70,8 +69,8 @@ public class MixedModeAvailabilityTestBase extends UpgradeTestBase
         .nodes(NUM_NODES)
         .nodesToUpgrade(upgradedCoordinator ? 1 : 2)
         .upgrades(initial, upgrade)
-        .withConfig(config -> config.set("read_request_timeout_in_ms", SECONDS.toMillis(2))
-                                    .set("write_request_timeout_in_ms", SECONDS.toMillis(2)))
+        .withConfig(config -> config.set("read_request_timeout", "2s")
+                                    .set("write_request_timeout", "2s"))
         .setup(c -> c.schemaChange(withKeyspace("CREATE TABLE %s.t (k uuid, c int, v int, PRIMARY KEY (k, c))")))
         .runAfterNodeUpgrade((cluster, n) -> {
 
diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeConsistencyTestBase.java b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeConsistencyTestBase.java
index f98fc8a..70b83e2 100644
--- a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeConsistencyTestBase.java
+++ b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeConsistencyTestBase.java
@@ -31,7 +31,6 @@ import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.IUpgradeableInstance;
 
 import static java.lang.String.format;
-import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.ONE;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
@@ -56,8 +55,8 @@ public class MixedModeConsistencyTestBase extends UpgradeTestBase
         .nodes(3)
         .nodesToUpgrade(1)
         .upgrades(initial, upgrade)
-        .withConfig(config -> config.set("read_request_timeout_in_ms", SECONDS.toMillis(30))
-                                    .set("write_request_timeout_in_ms", SECONDS.toMillis(30)))
+        .withConfig(config -> config.set("read_request_timeout", "30s")
+                                    .set("write_request_timeout", "30s"))
         .setup(cluster -> {
             Tester.createTable(cluster);
             for (Tester tester : testers)
diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeMessageForwardTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeMessageForwardTest.java
index 8010853..935cc8e 100644
--- a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeMessageForwardTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeMessageForwardTest.java
@@ -28,7 +28,6 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.distributed.UpgradeableCluster;
 import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.Feature;
-import org.apache.cassandra.utils.Shared;
 
 import static org.apache.cassandra.distributed.shared.AssertUtils.*;
 
diff --git a/test/simulator/main/org/apache/cassandra/simulator/ClusterSimulation.java b/test/simulator/main/org/apache/cassandra/simulator/ClusterSimulation.java
index a4938f1..c56a5a7 100644
--- a/test/simulator/main/org/apache/cassandra/simulator/ClusterSimulation.java
+++ b/test/simulator/main/org/apache/cassandra/simulator/ClusterSimulation.java
@@ -638,10 +638,10 @@ public class ClusterSimulation<S extends Simulation> implements AutoCloseable
                          .withSharedClasses(getSharedClassPredicate(ISOLATE, SHARE, ANY, SIMULATION))
                          .withConfig(config -> configUpdater.accept(threadAllocator.update(config
                              .with(Feature.BLANK_GOSSIP)
-                             .set("read_request_timeout_in_ms", NANOSECONDS.toMillis(builder.readTimeoutNanos))
-                             .set("write_request_timeout_in_ms", NANOSECONDS.toMillis(builder.writeTimeoutNanos))
-                             .set("cas_contention_timeout_in_ms", NANOSECONDS.toMillis(builder.contentionTimeoutNanos))
-                             .set("request_timeout_in_ms", NANOSECONDS.toMillis(builder.requestTimeoutNanos))
+                             .set("read_request_timeout", String.format("%dms", NANOSECONDS.toMillis(builder.readTimeoutNanos)))
+                             .set("write_request_timeout", String.format("%dms", NANOSECONDS.toMillis(builder.writeTimeoutNanos)))
+                             .set("cas_contention_timeout", String.format("%dms", NANOSECONDS.toMillis(builder.contentionTimeoutNanos)))
+                             .set("request_timeout", String.format("%dms", NANOSECONDS.toMillis(builder.requestTimeoutNanos)))
                              .set("memtable_heap_space_in_mb", 1)
                              .set("memtable_allocation_type", builder.memoryListener != null ? "unslabbed_heap_buffers_logged" : "heap_buffers")
                              .set("file_cache_size_in_mb", 16)
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index 4af1f8a..5aabc80 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -113,7 +113,7 @@ public class DatabaseDescriptorTest
     }
 
     @Test
-    public void testRpcInterface() throws Exception
+    public void testRpcInterface()
     {
         Config testConfig = DatabaseDescriptor.loadConfig();
         testConfig.rpc_interface = suitableInterface.getName();
@@ -332,40 +332,45 @@ public class DatabaseDescriptorTest
     public void testLowestAcceptableTimeouts() throws ConfigurationException
     {
         Config testConfig = new Config();
-        testConfig.read_request_timeout_in_ms = DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT + 1;
-        testConfig.range_request_timeout_in_ms = DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT + 1;
-        testConfig.write_request_timeout_in_ms = DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT + 1;
-        testConfig.truncate_request_timeout_in_ms = DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT + 1;
-        testConfig.cas_contention_timeout_in_ms = DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT + 1;
-        testConfig.counter_write_request_timeout_in_ms = DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT + 1;
-        testConfig.request_timeout_in_ms = DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT + 1;
-
-        assertTrue(testConfig.read_request_timeout_in_ms > DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
-        assertTrue(testConfig.range_request_timeout_in_ms > DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
-        assertTrue(testConfig.write_request_timeout_in_ms > DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
-        assertTrue(testConfig.truncate_request_timeout_in_ms > DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
-        assertTrue(testConfig.cas_contention_timeout_in_ms > DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
-        assertTrue(testConfig.counter_write_request_timeout_in_ms > DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
-        assertTrue(testConfig.request_timeout_in_ms > DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
+
+        SmallestDurationMilliseconds greaterThanLowestTimeout = SmallestDurationMilliseconds.inMilliseconds(DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT.toMilliseconds() + 1);
+
+        testConfig.read_request_timeout = greaterThanLowestTimeout;
+        testConfig.range_request_timeout = greaterThanLowestTimeout;
+        testConfig.write_request_timeout = greaterThanLowestTimeout;
+        testConfig.truncate_request_timeout = greaterThanLowestTimeout;
+        testConfig.cas_contention_timeout = greaterThanLowestTimeout;
+        testConfig.counter_write_request_timeout = greaterThanLowestTimeout;
+        testConfig.request_timeout = greaterThanLowestTimeout;
+
+        assertEquals(testConfig.read_request_timeout, greaterThanLowestTimeout);
+        assertEquals(testConfig.range_request_timeout, greaterThanLowestTimeout);
+        assertEquals(testConfig.write_request_timeout, greaterThanLowestTimeout);
+        assertEquals(testConfig.truncate_request_timeout, greaterThanLowestTimeout);
+        assertEquals(testConfig.cas_contention_timeout, greaterThanLowestTimeout);
+        assertEquals(testConfig.counter_write_request_timeout, greaterThanLowestTimeout);
+        assertEquals(testConfig.request_timeout, greaterThanLowestTimeout);
 
         //set less than Lowest acceptable value
-        testConfig.read_request_timeout_in_ms = DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT - 1;
-        testConfig.range_request_timeout_in_ms = DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT - 1;
-        testConfig.write_request_timeout_in_ms = DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT - 1;
-        testConfig.truncate_request_timeout_in_ms = DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT - 1;
-        testConfig.cas_contention_timeout_in_ms = DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT - 1;
-        testConfig.counter_write_request_timeout_in_ms = DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT - 1;
-        testConfig.request_timeout_in_ms = DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT - 1;
+        SmallestDurationMilliseconds lowerThanLowestTimeout = SmallestDurationMilliseconds.inMilliseconds(DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT.toMilliseconds() - 1);
+
+        testConfig.read_request_timeout = lowerThanLowestTimeout;
+        testConfig.range_request_timeout = lowerThanLowestTimeout;
+        testConfig.write_request_timeout = lowerThanLowestTimeout;
+        testConfig.truncate_request_timeout = lowerThanLowestTimeout;
+        testConfig.cas_contention_timeout = lowerThanLowestTimeout;
+        testConfig.counter_write_request_timeout = lowerThanLowestTimeout;
+        testConfig.request_timeout = lowerThanLowestTimeout;
 
         DatabaseDescriptor.checkForLowestAcceptedTimeouts(testConfig);
 
-        assertTrue(testConfig.read_request_timeout_in_ms == DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
-        assertTrue(testConfig.range_request_timeout_in_ms == DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
-        assertTrue(testConfig.write_request_timeout_in_ms == DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
-        assertTrue(testConfig.truncate_request_timeout_in_ms == DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
-        assertTrue(testConfig.cas_contention_timeout_in_ms == DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
-        assertTrue(testConfig.counter_write_request_timeout_in_ms == DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
-        assertTrue(testConfig.request_timeout_in_ms == DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
+        assertEquals(testConfig.read_request_timeout, DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
+        assertEquals(testConfig.range_request_timeout, DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
+        assertEquals(testConfig.write_request_timeout, DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
+        assertEquals(testConfig.truncate_request_timeout, DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
+        assertEquals(testConfig.cas_contention_timeout, DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
+        assertEquals(testConfig.counter_write_request_timeout, DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
+        assertEquals(testConfig.request_timeout, DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
     }
 
     @Test
diff --git a/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java b/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java
index 528da31..885f28f 100644
--- a/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java
+++ b/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java
@@ -43,7 +43,7 @@ public class LoadOldYAMLBackwardCompatibilityTest
         /*assertEquals(5, config.internode_socket_send_buffer_size_in_bytes);
         assertEquals(5, config.internode_socket_receive_buffer_size_in_bytes);
         assertEquals(DurationSpec.inMilliseconds(10800000), config.max_hint_window);
-        assertEquals(DurationSpec.inHours(0), config.max_hint_window);
+        assertEquals(DurationSpec.inHours(0), config.max_hint_window);*/
         assertEquals(DurationSpec.inMilliseconds(0), config.native_transport_idle_timeout);
         assertEquals(DurationSpec.inMilliseconds(10000), config.request_timeout);
         assertEquals(DurationSpec.inMilliseconds(5000), config.read_request_timeout);
@@ -54,7 +54,7 @@ public class LoadOldYAMLBackwardCompatibilityTest
         assertEquals(DurationSpec.inMilliseconds(60000), config.truncate_request_timeout);
         assertEquals(DurationSpec.inSeconds(300), config.streaming_keep_alive_period);
         assertEquals(DurationSpec.inMilliseconds(500), config.slow_query_log_timeout);
-        assertNull(config.memtable_heap_space);
+        /*assertNull(config.memtable_heap_space);
         assertNull(config.memtable_offheap_space);
         assertNull( config.repair_session_space);
         assertEquals(DataStorageSpec.inBytes(4194304), config.internode_application_send_queue_capacity);

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