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:24 UTC

[cassandra] 11/13: Transfer parameters to the newly introduced configuration framework (7) 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 6d5203615f7a9670cb1698b74123666bc25ba471
Author: Ekaterina Dimitrova <ek...@datastax.com>
AuthorDate: Fri Feb 4 00:25:14 2022 -0500

    Transfer parameters to the newly introduced configuration framework (7)
    patch by Ekaterina Dimitrova; reviewed by Caleb Rackliffe, David Capwell, Michael Semb Wever and Benjamin Lerer for CASSANDRA-15234
---
 conf/cassandra.yaml                                |  42 ++++----
 .../org/apache/cassandra/cache/OHCProvider.java    |   2 +-
 .../cassandra/cache/SerializingCacheProvider.java  |   2 +-
 src/java/org/apache/cassandra/config/Config.java   |  61 +++++++----
 .../org/apache/cassandra/config/Converters.java    |  13 ++-
 .../cassandra/config/DatabaseDescriptor.java       | 119 ++++++++++-----------
 .../cassandra/config/SmallestDurationSeconds.java  |  29 +++++
 .../apache/cassandra/db/virtual/SettingsTable.java |  15 ++-
 .../org/apache/cassandra/service/CacheService.java |   8 +-
 test/conf/cassandra-murmur.yaml                    |   2 +-
 test/conf/cassandra-old.yaml                       |   2 +-
 test/conf/cassandra-seeds.yaml                     |   2 +-
 ...dra-sslcontextfactory-invalidconfiguration.yaml |   4 +-
 test/conf/cassandra-sslcontextfactory.yaml         |   4 +-
 test/conf/cassandra.yaml                           |   4 +-
 test/conf/unit-test-conf/test-native-port.yaml     |   4 +-
 .../cassandra/distributed/impl/InstanceConfig.java |   8 +-
 .../cassandra/distributed/test/NodeToolTest.java   |   2 +-
 .../cassandra/simulator/ClusterSimulation.java     |   4 +-
 .../LoadOldYAMLBackwardCompatibilityTest.java      |  21 ++--
 test/unit/org/apache/cassandra/cql3/CQLTester.java |   7 +-
 .../cql3/validation/entities/JsonTest.java         |   4 +-
 22 files changed, 214 insertions(+), 145 deletions(-)

diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index 9f7e657..cf4326a 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -344,8 +344,8 @@ commit_failure_policy: stop
 # fit in the cache. In most cases it is not neccessary to change this value.
 # Constantly re-preparing statements is a performance penalty.
 #
-# Default value ("auto") is 1/256th of the heap or 10MB, whichever is greater
-prepared_statements_cache_size_mb:
+# Default value ("auto") is 1/256th of the heap or 10MiB, whichever is greater
+# prepared_statements_cache_size:
 
 # Maximum size of the key cache in memory.
 #
@@ -359,7 +359,7 @@ prepared_statements_cache_size_mb:
 # NOTE: if you reduce the size, you may not get you hottest keys loaded on startup.
 #
 # Default value is empty to make it "auto" (min(5% of Heap (in MB), 100MB)). Set to 0 to disable key cache.
-key_cache_size_in_mb:
+# key_cache_size:
 
 # Duration in seconds after which Cassandra should
 # save the key cache. Caches are saved to saved_caches_directory as
@@ -370,7 +370,7 @@ key_cache_size_in_mb:
 # has limited use.
 #
 # Default is 14400 or 4 hours.
-key_cache_save_period: 14400
+key_cache_save_period: 4h
 
 # Number of keys from the key cache to save
 # Disabled by default, meaning all keys are going to be saved
@@ -394,7 +394,7 @@ key_cache_save_period: 14400
 # headroom for OS block level cache. Do never allow your system to swap.
 #
 # Default value is 0, to disable row caching.
-row_cache_size_in_mb: 0
+row_cache_size: 0MiB
 
 # Duration in seconds after which Cassandra should save the row cache.
 # Caches are saved to saved_caches_directory as specified in this configuration file.
@@ -404,7 +404,7 @@ row_cache_size_in_mb: 0
 # has limited use.
 #
 # Default is 0 to disable saving the row cache.
-row_cache_save_period: 0
+row_cache_save_period: 0s
 
 # Number of keys from the row cache to save.
 # Specify 0 (which is the default), meaning all keys are going to be saved
@@ -423,14 +423,14 @@ row_cache_save_period: 0
 #
 # Default value is empty to make it "auto" (min(2.5% of Heap (in MB), 50MB)). Set to 0 to disable counter cache.
 # NOTE: if you perform counter deletes and rely on low gcgs, you should disable the counter cache.
-counter_cache_size_in_mb:
+# counter_cache_size:
 
 # Duration in seconds after which Cassandra should
 # save the counter cache (keys only). Caches are saved to saved_caches_directory as
 # specified in this configuration file.
 #
 # Default is 7200 or 2 hours.
-counter_cache_save_period: 7200
+counter_cache_save_period: 7200s
 
 # Number of keys from the counter cache to save
 # Disabled by default, meaning all keys are going to be saved
@@ -443,7 +443,7 @@ counter_cache_save_period: 7200
 # Number of seconds the server will wait for each cache (row, key, etc ...) to load while starting
 # the Cassandra process. Setting this to a negative value is equivalent to disabling all cache loading on startup
 # while still having the cache during runtime.
-# cache_load_timeout_seconds: 30
+# cache_load_timeout: 30s
 
 # commitlog_sync may be either "periodic", "group", or "batch." 
 # 
@@ -550,7 +550,7 @@ concurrent_materialized_view_writes: 32
 # overhead which is roughly 128 bytes per chunk (i.e. 0.2% of the reserved size
 # if the default 64k chunk size is used).
 # Memory is only allocated when needed.
-# networking_cache_size_in_mb: 128
+# networking_cache_size: 128MiB
 
 # Enable the sstable chunk cache.  The chunk cache will store recently accessed
 # sections of the sstable in-memory as uncompressed buffers.
@@ -564,11 +564,11 @@ concurrent_materialized_view_writes: 32
 # overhead which is roughly 128 bytes per chunk (i.e. 0.2% of the reserved size
 # if the default 64k chunk size is used).
 # Memory is only allocated when needed.
-# file_cache_size_in_mb: 512
+# file_cache_size: 512MiB
 
 # Flag indicating whether to allocate on or off heap when the sstable buffer
 # pool is exhausted, that is when it has exceeded the maximum memory
-# file_cache_size_in_mb, beyond which it will not cache buffers but allocate on request.
+# file_cache_size, beyond which it will not cache buffers but allocate on request.
 
 # buffer_pool_use_heap_if_exhausted: true
 
@@ -683,13 +683,13 @@ memtable_allocation_type: heap_buffers
 # shrink their index summaries in order to meet this limit.  However, this
 # is a best-effort process. In extreme conditions Cassandra may need to use
 # more than this amount of memory.
-index_summary_capacity_in_mb:
+# index_summary_capacity:
 
 # How frequently index summaries should be resampled.  This is done
 # periodically to redistribute memory from the fixed-size pool to sstables
 # proportional their recent read rates.  Setting to -1 will disable this
 # process, leaving existing index summaries at their current sampling level.
-index_summary_resize_interval_in_minutes: 60
+index_summary_resize_interval: 60m
 
 # Whether to, when doing sequential writing, fsync() at intervals in
 # order to force the operating system to flush the dirty
@@ -1304,10 +1304,10 @@ internode_compression: dc
 inter_dc_tcp_nodelay: false
 
 # TTL for different trace types used during logging of the repair process.
-tracetype_query_ttl: 86400
-tracetype_repair_ttl: 604800
+trace_type_query_ttl: 1d
+trace_type_repair_ttl: 7d
 
-# If unset, all GC Pauses greater than gc_log_threshold_in_ms will log at
+# If unset, all GC Pauses greater than gc_log_threshold will log at
 # INFO level
 # UDFs (user defined functions) are disabled by default.
 # As of Cassandra 3.0 there is a sandbox in place that should prevent execution of evil code.
@@ -1401,16 +1401,16 @@ compaction_tombstone_warning_threshold: 100000
 
 # GC Pauses greater than 200 ms will be logged at INFO level
 # This threshold can be adjusted to minimize logging if necessary
-# gc_log_threshold_in_ms: 200
+# gc_log_threshold: 200ms
 
-# GC Pauses greater than gc_warn_threshold_in_ms will be logged at WARN level
+# GC Pauses greater than gc_warn_threshold will be logged at WARN level
 # Adjust the threshold based on your application throughput requirement. Setting to 0
 # will deactivate the feature.
-# gc_warn_threshold_in_ms: 1000
+# gc_warn_threshold: 1000ms
 
 # Maximum size of any value in SSTables. Safety measure to detect SSTable corruption
 # early. Any value size larger than this threshold will result into marking an SSTable
-# as corrupted. This should be positive and less than 2048.
+# as corrupted. This should be positive and less than 2GiB.
 # max_value_size: 256MiB
 
 # ** Impact on keyspace creation **
diff --git a/src/java/org/apache/cassandra/cache/OHCProvider.java b/src/java/org/apache/cassandra/cache/OHCProvider.java
index afdc872..4a705a6 100644
--- a/src/java/org/apache/cassandra/cache/OHCProvider.java
+++ b/src/java/org/apache/cassandra/cache/OHCProvider.java
@@ -37,7 +37,7 @@ public class OHCProvider implements CacheProvider<RowCacheKey, IRowCacheEntry>
     public ICache<RowCacheKey, IRowCacheEntry> create()
     {
         OHCacheBuilder<RowCacheKey, IRowCacheEntry> builder = OHCacheBuilder.newBuilder();
-        builder.capacity(DatabaseDescriptor.getRowCacheSizeInMB() * 1024 * 1024)
+        builder.capacity(DatabaseDescriptor.getRowCacheSizeInMiB() * 1024 * 1024)
                .keySerializer(KeySerializer.instance)
                .valueSerializer(ValueSerializer.instance)
                .throwOOME(true);
diff --git a/src/java/org/apache/cassandra/cache/SerializingCacheProvider.java b/src/java/org/apache/cassandra/cache/SerializingCacheProvider.java
index 813f6fe..56393c4 100644
--- a/src/java/org/apache/cassandra/cache/SerializingCacheProvider.java
+++ b/src/java/org/apache/cassandra/cache/SerializingCacheProvider.java
@@ -30,7 +30,7 @@ public class SerializingCacheProvider implements CacheProvider<RowCacheKey, IRow
 {
     public ICache<RowCacheKey, IRowCacheEntry> create()
     {
-        return SerializingCache.create(DatabaseDescriptor.getRowCacheSizeInMB() * 1024 * 1024, new RowCacheSerializer());
+        return SerializingCache.create(DatabaseDescriptor.getRowCacheSizeInMiB() * 1024 * 1024, new RowCacheSerializer());
     }
 
     // Package Public: used by external Row Cache plugins
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index 2f56fc7..a6b0413 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -376,41 +376,52 @@ public class Config
 
     public volatile boolean incremental_backups = false;
     public boolean trickle_fsync = false;
+    @Replaces(oldName = "trickle_fsync_interval_in_kb", converter = Converters.KIBIBYTES_DATASTORAGE, deprecated = true)
     public SmallestDataStorageKibibytes trickle_fsync_interval = new SmallestDataStorageKibibytes("10240KiB");
 
+    @Replaces(oldName = "sstable_preemptive_open_interval_in_mb", converter = Converters.MEBIBYTES_DATA_STORAGE, deprecated = true)
     public volatile SmallestDataStorageMebibytes sstable_preemptive_open_interval = new SmallestDataStorageMebibytes("50MiB");
 
     public volatile boolean key_cache_migrate_during_compaction = true;
-    public Long key_cache_size_in_mb = null;
-    public volatile int key_cache_save_period = 14400;
     public volatile int key_cache_keys_to_save = Integer.MAX_VALUE;
+    @Replaces(oldName = "key_cache_size_in_mb", converter = Converters.MEBIBYTES_DATA_STORAGE, deprecated = true)
+    public SmallestDataStorageMebibytes key_cache_size = null;
+    @Replaces(oldName = "key_cache_save_period", converter = Converters.SECONDS_CUSTOM_DURATION, deprecated = true)
+    public volatile SmallestDurationSeconds key_cache_save_period = new SmallestDurationSeconds("4h");
 
     public String row_cache_class_name = "org.apache.cassandra.cache.OHCProvider";
-    public long row_cache_size_in_mb = 0;
-    public volatile int row_cache_save_period = 0;
+    @Replaces(oldName = "row_cache_size_in_mb", converter = Converters.MEBIBYTES_DATA_STORAGE, deprecated = true)
+    public SmallestDataStorageMebibytes row_cache_size = new SmallestDataStorageMebibytes("0MiB");
+    @Replaces(oldName = "row_cache_save_period", converter = Converters.SECONDS_CUSTOM_DURATION, deprecated = true)
+    public volatile SmallestDurationSeconds row_cache_save_period = new SmallestDurationSeconds("0s");
     public volatile int row_cache_keys_to_save = Integer.MAX_VALUE;
 
-    public Long counter_cache_size_in_mb = null;
-    public volatile int counter_cache_save_period = 7200;
+    @Replaces(oldName = "counter_cache_size_in_mb", converter = Converters.MEBIBYTES_DATA_STORAGE, deprecated = true)
+    public SmallestDataStorageMebibytes counter_cache_size = null;
+    @Replaces(oldName = "counter_cache_save_period", converter = Converters.SECONDS_CUSTOM_DURATION, deprecated = true)
+    public volatile SmallestDurationSeconds counter_cache_save_period = new SmallestDurationSeconds("7200s");
     public volatile int counter_cache_keys_to_save = Integer.MAX_VALUE;
 
-    public int cache_load_timeout_seconds = 30;
+    @Replaces(oldName = "cache_load_timeout_seconds ", converter = Converters.SECONDS_DURATION, deprecated = true)
+    public SmallestDurationSeconds cache_load_timeout = new SmallestDurationSeconds("30s");
 
     private static boolean isClientMode = false;
     private static Supplier<Config> overrideLoadConfig = null;
 
-    public Integer networking_cache_size_in_mb;
+    @Replaces(oldName = "networking_cache_size_in_mb", converter = Converters.MEBIBYTES_DATA_STORAGE, deprecated = true)
+    public SmallestDataStorageMebibytes networking_cache_size;
 
-    public Integer file_cache_size_in_mb;
+    @Replaces(oldName = "file_cache_size_in_mb", converter = Converters.MEBIBYTES_DATA_STORAGE, deprecated = true)
+    public SmallestDataStorageMebibytes file_cache_size;
 
     public boolean file_cache_enabled = Boolean.getBoolean("cassandra.file_cache_enabled");
 
     /**
-     * Because of the current {@link org.apache.cassandra.utils.memory.BufferPool} slab sizes of 64 kb, we
-     * store in the file cache buffers that divide 64 kb, so we need to round the buffer sizes to powers of two.
+     * Because of the current {@link org.apache.cassandra.utils.memory.BufferPool} slab sizes of 64 KiB, we
+     * store in the file cache buffers that divide 64 KiB, so we need to round the buffer sizes to powers of two.
      * This boolean controls weather they are rounded up or down. Set it to true to round up to the
      * next power of two, set it to false to round down to the previous power of two. Note that buffer sizes are
-     * already rounded to 4 kb and capped between 4 kb minimum and 64 kb maximum by the {@link DiskOptimizationStrategy}.
+     * already rounded to 4 KiB and capped between 4 KiB minimum and 64 kb maximum by the {@link DiskOptimizationStrategy}.
      * By default, this boolean is set to round down when {@link #disk_optimization_strategy} is {@code ssd},
      * and to round up when it is {@code spinning}.
      */
@@ -436,15 +447,21 @@ public class Config
 
     public final ReplicaFilteringProtectionOptions replica_filtering_protection = new ReplicaFilteringProtectionOptions();
 
-    public volatile Long index_summary_capacity_in_mb;
-    public volatile int index_summary_resize_interval_in_minutes = 60;
+    @Replaces(oldName = "index_summary_capacity_in_mb", converter = Converters.MEBIBYTES_DATA_STORAGE, deprecated = true)
+    public volatile SmallestDataStorageMebibytes index_summary_capacity;
+    @Replaces(oldName = "index_summary_resize_interval_in_minutes", converter = Converters.MINUTES_DURATION, deprecated = true)
+    public volatile SmallestDurationMinutes index_summary_resize_interval = new SmallestDurationMinutes("60m");
 
-    public int gc_log_threshold_in_ms = 200;
-    public int gc_warn_threshold_in_ms = 1000;
+    @Replaces(oldName = "gc_log_threshold_in_ms", converter = Converters.MILLIS_DURATION, deprecated = true)
+    public SmallestDurationMilliseconds gc_log_threshold = new SmallestDurationMilliseconds("200ms");
+    @Replaces(oldName = "gc_warn_threshold_in_ms", converter = Converters.MILLIS_DURATION, deprecated = true)
+    public SmallestDurationMilliseconds gc_warn_threshold = new SmallestDurationMilliseconds("1s");
 
     // TTL for different types of trace events.
-    public int tracetype_query_ttl = (int) TimeUnit.DAYS.toSeconds(1);
-    public int tracetype_repair_ttl = (int) TimeUnit.DAYS.toSeconds(7);
+    @Replaces(oldName = "tracetype_query_ttl", converter = Converters.SECONDS_DURATION, deprecated=true)
+    public SmallestDurationSeconds trace_type_query_ttl = new SmallestDurationSeconds("1d");
+    @Replaces(oldName = "tracetype_repair_ttl", converter = Converters.SECONDS_DURATION, deprecated=true)
+    public SmallestDurationSeconds trace_type_repair_ttl = new SmallestDurationSeconds("7d");
 
     /**
      * Maintain statistics on whether writes achieve the ideal consistency level
@@ -453,10 +470,11 @@ public class Config
     public volatile ConsistencyLevel ideal_consistency_level = null;
 
     /**
-     * Size of the CQL prepared statements cache in MB.
-     * Defaults to 1/256th of the heap size or 10MB, whichever is greater.
+     * Size of the CQL prepared statements cache in MiB.
+     * Defaults to 1/256th of the heap size or 10MiB, whichever is greater.
      */
-    public Long prepared_statements_cache_size_mb = null;
+    @Replaces(oldName = "prepared_statements_cache_size_mb", converter = Converters.MEBIBYTES_DATA_STORAGE, deprecated = true)
+    public SmallestDataStorageMebibytes prepared_statements_cache_size = null;
 
     @Replaces(oldName = "enable_user_defined_functions", converter = Converters.IDENTITY, deprecated = true)
     public boolean user_defined_functions_enabled = false;
@@ -494,6 +512,7 @@ public class Config
      * Time in milliseconds after a warning will be emitted to the log and to the client that a UDF runs too long.
      * (Only valid, if user_defined_functions_threads_enabled==true)
      */
+    //No need of unit conversion as this parameter is not exposed in the yaml file
     public long user_defined_function_warn_timeout = 500;
     /**
      * Time in milliseconds after a fatal UDF run-time situation is detected and action according to
diff --git a/src/java/org/apache/cassandra/config/Converters.java b/src/java/org/apache/cassandra/config/Converters.java
index eb64552..f2f7120 100644
--- a/src/java/org/apache/cassandra/config/Converters.java
+++ b/src/java/org/apache/cassandra/config/Converters.java
@@ -26,7 +26,9 @@ import java.util.function.Function;
  * parameter name(suffix). (CASSANDRA-15234)
  * It is important to be noted that this converter is not intended to be used when we don't change name of a configuration
  * parameter but we want to add unit. This would always default to the old value provided without a unit at the moment.
- * In case this functionality is needed at some point, please, raise a Jira ticket.
+ * In case this functionality is needed at some point, please, raise a Jira ticket. There is only one exception handling
+ * three parameters (key_cache_save_period, row_cache_save_period, counter_cache_save_period) - the SECONDS_CUSTOM_DURATION
+ * converter.
  */
 public enum Converters
 {
@@ -51,6 +53,15 @@ public enum Converters
     SECONDS_DURATION(Long.class,
                      o -> SmallestDurationSeconds.inSeconds((Long) o),
                      o -> ((SmallestDurationSeconds)o).toSeconds()),
+    /**
+     * This converter is used to support backward compatibility for Duration parameters where we added the opportunity
+     * for the users to add a unit in the parameters' values but we didn't change the names. (key_cache_save_period,
+     * row_cache_save_period, counter_cache_save_period)
+     * Example: row_cache_save_period = 0 and row_cache_save_period = 0s (quantity of 0s) are equal.
+     */
+    SECONDS_CUSTOM_DURATION(String.class,
+                            o -> SmallestDurationSeconds.inSecondsString((String) o),
+                            o -> ((SmallestDurationSeconds)o).toSeconds()),
     MINUTES_DURATION(Long.class,
                      o -> SmallestDurationMinutes.inMinutes((Long) o),
                      o -> ((SmallestDurationMinutes)o).toMinutes()),
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index b51f2de..25e5c8b 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -128,11 +128,11 @@ public class DatabaseDescriptor
     // depend on the configured IAuthenticator, so defer creating it until that's been set.
     private static IRoleManager roleManager;
 
-    private static long preparedStatementsCacheSizeInMB;
+    private static long preparedStatementsCacheSizeInMiB;
 
-    private static long keyCacheSizeInMB;
-    private static long counterCacheSizeInMB;
-    private static long indexSummaryCapacityInMB;
+    private static long keyCacheSizeInMiB;
+    private static long counterCacheSizeInMiB;
+    private static long indexSummaryCapacityInMiB;
 
     private static String localDC;
     private static Comparator<Replica> localComparator;
@@ -431,11 +431,6 @@ public class DatabaseDescriptor
             logger.info("DiskAccessMode is {}, indexAccessMode is {}", conf.disk_access_mode, indexAccessMode);
         }
 
-        if (conf.gc_warn_threshold_in_ms < 0)
-        {
-            throw new ConfigurationException("gc_warn_threshold_in_ms must be a positive integer");
-        }
-
         /* phi convict threshold for FailureDetector */
         if (conf.phi_convict_threshold < 5 || conf.phi_convict_threshold > 16)
         {
@@ -459,11 +454,11 @@ public class DatabaseDescriptor
         if (conf.concurrent_replicates != null)
             logger.warn("concurrent_replicates has been deprecated and should be removed from cassandra.yaml");
 
-        if (conf.networking_cache_size_in_mb == null)
-            conf.networking_cache_size_in_mb = Math.min(128, (int) (Runtime.getRuntime().maxMemory() / (16 * 1048576)));
+        if (conf.networking_cache_size == null)
+            conf.networking_cache_size = SmallestDataStorageMebibytes.inMebibytes(Math.min(128, (int) (Runtime.getRuntime().maxMemory() / (16 * 1048576))));
 
-        if (conf.file_cache_size_in_mb == null)
-            conf.file_cache_size_in_mb = Math.min(512, (int) (Runtime.getRuntime().maxMemory() / (4 * 1048576)));
+        if (conf.file_cache_size == null)
+            conf.file_cache_size = SmallestDataStorageMebibytes.inMebibytes(Math.min(512, (int) (Runtime.getRuntime().maxMemory() / (4 * 1048576))));
 
         // round down for SSDs and round up for spinning disks
         if (conf.file_cache_round_up == null)
@@ -682,60 +677,60 @@ public class DatabaseDescriptor
 
         try
         {
-            // if prepared_statements_cache_size_mb option was set to "auto" then size of the cache should be "max(1/256 of Heap (in MB), 10MB)"
-            preparedStatementsCacheSizeInMB = (conf.prepared_statements_cache_size_mb == null)
+            // if prepared_statements_cache_size option was set to "auto" then size of the cache should be "max(1/256 of Heap (in MiB), 10MiB)"
+            preparedStatementsCacheSizeInMiB = (conf.prepared_statements_cache_size == null)
                                               ? Math.max(10, (int) (Runtime.getRuntime().maxMemory() / 1024 / 1024 / 256))
-                                              : conf.prepared_statements_cache_size_mb;
+                                              : conf.prepared_statements_cache_size.toMebibytes();
 
-            if (preparedStatementsCacheSizeInMB <= 0)
+            if (preparedStatementsCacheSizeInMiB <= 0)
                 throw new NumberFormatException(); // to escape duplicating error message
         }
         catch (NumberFormatException e)
         {
-            throw new ConfigurationException("prepared_statements_cache_size_mb option was set incorrectly to '"
-                                             + conf.prepared_statements_cache_size_mb + "', supported values are <integer> >= 0.", false);
+            throw new ConfigurationException("prepared_statements_cache_size option was set incorrectly to '"
+                                             + conf.prepared_statements_cache_size + "', supported values are <integer> >= 0.", false);
         }
 
         try
         {
-            // if key_cache_size_in_mb option was set to "auto" then size of the cache should be "min(5% of Heap (in MB), 100MB)
-            keyCacheSizeInMB = (conf.key_cache_size_in_mb == null)
+            // if key_cache_size option was set to "auto" then size of the cache should be "min(5% of Heap (in MiB), 100MiB)
+            keyCacheSizeInMiB = (conf.key_cache_size == null)
                                ? Math.min(Math.max(1, (int) (Runtime.getRuntime().totalMemory() * 0.05 / 1024 / 1024)), 100)
-                               : conf.key_cache_size_in_mb;
+                               : conf.key_cache_size.toMebibytes();
 
-            if (keyCacheSizeInMB < 0)
+            if (keyCacheSizeInMiB < 0)
                 throw new NumberFormatException(); // to escape duplicating error message
         }
         catch (NumberFormatException e)
         {
-            throw new ConfigurationException("key_cache_size_in_mb option was set incorrectly to '"
-                                             + conf.key_cache_size_in_mb + "', supported values are <integer> >= 0.", false);
+            throw new ConfigurationException("key_cache_size option was set incorrectly to '"
+                                             + (conf.key_cache_size != null ? conf.key_cache_size.toString() : null) + "', supported values are <integer> >= 0.", false);
         }
 
         try
         {
-            // if counter_cache_size_in_mb option was set to "auto" then size of the cache should be "min(2.5% of Heap (in MB), 50MB)
-            counterCacheSizeInMB = (conf.counter_cache_size_in_mb == null)
+            // if counter_cache_size option was set to "auto" then size of the cache should be "min(2.5% of Heap (in MiB), 50MiB)
+            counterCacheSizeInMiB = (conf.counter_cache_size == null)
                                    ? Math.min(Math.max(1, (int) (Runtime.getRuntime().totalMemory() * 0.025 / 1024 / 1024)), 50)
-                                   : conf.counter_cache_size_in_mb;
+                                   : conf.counter_cache_size.toMebibytes();
 
-            if (counterCacheSizeInMB < 0)
+            if (counterCacheSizeInMiB < 0)
                 throw new NumberFormatException(); // to escape duplicating error message
         }
         catch (NumberFormatException e)
         {
-            throw new ConfigurationException("counter_cache_size_in_mb option was set incorrectly to '"
-                                             + conf.counter_cache_size_in_mb + "', supported values are <integer> >= 0.", false);
+            throw new ConfigurationException("counter_cache_size option was set incorrectly to '"
+                                             + conf.counter_cache_size + "', supported values are <integer> >= 0.", false);
         }
 
         // if set to empty/"auto" then use 5% of Heap size
-        indexSummaryCapacityInMB = (conf.index_summary_capacity_in_mb == null)
+        indexSummaryCapacityInMiB = (conf.index_summary_capacity == null)
                                    ? Math.max(1, (int) (Runtime.getRuntime().totalMemory() * 0.05 / 1024 / 1024))
-                                   : conf.index_summary_capacity_in_mb;
+                                   : conf.index_summary_capacity.toMebibytes();
 
-        if (indexSummaryCapacityInMB < 0)
-            throw new ConfigurationException("index_summary_capacity_in_mb option was set incorrectly to '"
-                                             + conf.index_summary_capacity_in_mb + "', it should be a non-negative integer.", false);
+        if (indexSummaryCapacityInMiB < 0)
+            throw new ConfigurationException("index_summary_capacity option was set incorrectly to '"
+                                             + conf.index_summary_capacity.toString() + "', it should be a non-negative integer.", false);
 
         if (conf.user_defined_function_fail_timeout < 0)
             throw new ConfigurationException("user_defined_function_fail_timeout must not be negative", false);
@@ -2777,25 +2772,25 @@ public class DatabaseDescriptor
 
     public static int getFileCacheSizeInMB()
     {
-        if (conf.file_cache_size_in_mb == null)
+        if (conf.file_cache_size == null)
         {
             // In client mode the value is not set.
             assert DatabaseDescriptor.isClientInitialized();
             return 0;
         }
 
-        return conf.file_cache_size_in_mb;
+        return conf.file_cache_size.toMebibytesAsInt();
     }
 
     public static int getNetworkingCacheSizeInMB()
     {
-        if (conf.networking_cache_size_in_mb == null)
+        if (conf.networking_cache_size == null)
         {
             // In client mode the value is not set.
             assert DatabaseDescriptor.isClientInitialized();
             return 0;
         }
-        return conf.networking_cache_size_in_mb;
+        return conf.networking_cache_size.toMebibytesAsInt();
     }
 
     public static boolean getFileCacheRoundUp()
@@ -2856,22 +2851,22 @@ public class DatabaseDescriptor
 
     public static long getKeyCacheSizeInMB()
     {
-        return keyCacheSizeInMB;
+        return keyCacheSizeInMiB;
     }
 
     public static long getIndexSummaryCapacityInMB()
     {
-        return indexSummaryCapacityInMB;
+        return indexSummaryCapacityInMiB;
     }
 
     public static int getKeyCacheSavePeriod()
     {
-        return conf.key_cache_save_period;
+        return conf.key_cache_save_period.toSecondsAsInt();
     }
 
     public static void setKeyCacheSavePeriod(int keyCacheSavePeriod)
     {
-        conf.key_cache_save_period = keyCacheSavePeriod;
+        conf.key_cache_save_period = SmallestDurationSeconds.inSeconds(keyCacheSavePeriod);
     }
 
     public static int getKeyCacheKeysToSave()
@@ -2889,25 +2884,25 @@ public class DatabaseDescriptor
         return conf.row_cache_class_name;
     }
 
-    public static long getRowCacheSizeInMB()
+    public static long getRowCacheSizeInMiB()
     {
-        return conf.row_cache_size_in_mb;
+        return conf.row_cache_size.toMebibytes();
     }
 
     @VisibleForTesting
-    public static void setRowCacheSizeInMB(long val)
+    public static void setRowCacheSizeInMiB(long val)
     {
-        conf.row_cache_size_in_mb = val;
+        conf.row_cache_size = SmallestDataStorageMebibytes.inMebibytes(val);
     }
 
     public static int getRowCacheSavePeriod()
     {
-        return conf.row_cache_save_period;
+        return conf.row_cache_save_period.toSecondsAsInt();
     }
 
     public static void setRowCacheSavePeriod(int rowCacheSavePeriod)
     {
-        conf.row_cache_save_period = rowCacheSavePeriod;
+        conf.row_cache_save_period = SmallestDurationSeconds.inSeconds(rowCacheSavePeriod);
     }
 
     public static int getRowCacheKeysToSave()
@@ -2915,9 +2910,9 @@ public class DatabaseDescriptor
         return conf.row_cache_keys_to_save;
     }
 
-    public static long getCounterCacheSizeInMB()
+    public static long getCounterCacheSizeInMiB()
     {
-        return counterCacheSizeInMB;
+        return counterCacheSizeInMiB;
     }
 
     public static void setRowCacheKeysToSave(int rowCacheKeysToSave)
@@ -2927,23 +2922,23 @@ public class DatabaseDescriptor
 
     public static int getCounterCacheSavePeriod()
     {
-        return conf.counter_cache_save_period;
+        return conf.counter_cache_save_period.toSecondsAsInt();
     }
 
     public static void setCounterCacheSavePeriod(int counterCacheSavePeriod)
     {
-        conf.counter_cache_save_period = counterCacheSavePeriod;
+        conf.counter_cache_save_period = SmallestDurationSeconds.inSeconds(counterCacheSavePeriod);
     }
 
     public static int getCacheLoadTimeout()
     {
-        return conf.cache_load_timeout_seconds;
+        return conf.cache_load_timeout.toSecondsAsInt();
     }
 
     @VisibleForTesting
     public static void setCacheLoadTimeout(int seconds)
     {
-        conf.cache_load_timeout_seconds = seconds;
+        conf.cache_load_timeout = SmallestDurationSeconds.inSeconds(seconds);
     }
 
     public static int getCounterCacheKeysToSave()
@@ -3051,7 +3046,7 @@ public class DatabaseDescriptor
 
     public static int getIndexSummaryResizeIntervalInMinutes()
     {
-        return conf.index_summary_resize_interval_in_minutes;
+        return conf.index_summary_resize_interval.toMinutesAsInt();
     }
 
     public static boolean hasLargeAddressSpace()
@@ -3072,17 +3067,17 @@ public class DatabaseDescriptor
 
     public static int getTracetypeRepairTTL()
     {
-        return conf.tracetype_repair_ttl;
+        return conf.trace_type_repair_ttl.toSecondsAsInt();
     }
 
     public static int getTracetypeQueryTTL()
     {
-        return conf.tracetype_query_ttl;
+        return conf.trace_type_query_ttl.toSecondsAsInt();
     }
 
     public static long getPreparedStatementsCacheSizeMB()
     {
-        return preparedStatementsCacheSizeInMB;
+        return preparedStatementsCacheSizeInMiB;
     }
 
     public static boolean enableUserDefinedFunctions()
@@ -3178,7 +3173,7 @@ public class DatabaseDescriptor
 
     public static long getGCLogThreshold()
     {
-        return conf.gc_log_threshold_in_ms;
+        return conf.gc_log_threshold.toMilliseconds();
     }
 
     public static EncryptionContext getEncryptionContext()
@@ -3188,7 +3183,7 @@ public class DatabaseDescriptor
 
     public static long getGCWarnThreshold()
     {
-        return conf.gc_warn_threshold_in_ms;
+        return conf.gc_warn_threshold.toMilliseconds();
     }
 
     public static boolean isCDCEnabled()
diff --git a/src/java/org/apache/cassandra/config/SmallestDurationSeconds.java b/src/java/org/apache/cassandra/config/SmallestDurationSeconds.java
index f8ab4ad..f732315 100644
--- a/src/java/org/apache/cassandra/config/SmallestDurationSeconds.java
+++ b/src/java/org/apache/cassandra/config/SmallestDurationSeconds.java
@@ -19,6 +19,8 @@
 package org.apache.cassandra.config;
 
 import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 /**
  * Wrapper class for Cassandra duration configuration parameters which are internally represented in Seconds. In order
@@ -27,6 +29,8 @@ import java.util.concurrent.TimeUnit;
  */
 public final class SmallestDurationSeconds extends DurationSpec
 {
+    private static final Pattern VALUES_PATTERN = Pattern.compile(("\\d+"));
+
     /**
      * Creates a {@code SmallestDurationSeconds} of the specified amount of seconds and provides the smallest
      * required unit of seconds for the respective parameter of type {@code SmallestDurationSeconds}.
@@ -54,4 +58,29 @@ public final class SmallestDurationSeconds extends DurationSpec
     {
         return new SmallestDurationSeconds(seconds, TimeUnit.SECONDS);
     }
+
+    /**
+     * Creates a {@code SmallestDurationSeconds} of the specified amount of seconds. Custom method for special cases.
+     *
+     * @param value which can be in the old form only presenting the quantity or the post CASSANDRA-15234 form - a
+     * value consisting of quantity and unit. This method is necessary for three parameters which didn't change their
+     * names but only their value format. (key_cache_save_period, row_cache_save_period, counter_cache_save_period)
+     * @return a duration
+     */
+    public static SmallestDurationSeconds inSecondsString(String value)
+    {
+        //parse the string field value
+        Matcher matcher = VALUES_PATTERN.matcher(value);
+
+        long seconds;
+        //if the provided string value is just a number, then we create a Duration Spec value in seconds
+        if (matcher.matches())
+        {
+            seconds = Long.parseLong(value);
+            return new SmallestDurationSeconds(seconds, TimeUnit.SECONDS);
+        }
+
+        //otherwise we just use the standard constructors
+        return new SmallestDurationSeconds(value);
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/virtual/SettingsTable.java b/src/java/org/apache/cassandra/db/virtual/SettingsTable.java
index fa302b0..b82180d 100644
--- a/src/java/org/apache/cassandra/db/virtual/SettingsTable.java
+++ b/src/java/org/apache/cassandra/db/virtual/SettingsTable.java
@@ -19,7 +19,9 @@ package org.apache.cassandra.db.virtual;
 
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.function.BiConsumer;
@@ -56,6 +58,17 @@ final class SettingsTable extends AbstractVirtualTable
               .filter(f -> f.isAnnotationPresent(Replaces.class))
               .collect(Collectors.toMap(Field::getName, Functions.identity()));
 
+    // CASSANDRA-15234 - a few configuration parameters kept their names but added unit to their value, only the
+    // new value format is displayed for them
+    private final List<String> EXCLUDED_CONFIG = new ArrayList<String>()
+    {
+        {
+            add("key_cache_save_period");
+            add("row_cache_save_period");
+            add("counter_cache_save_period");
+        }
+    };
+
     @VisibleForTesting
     final Map<String, BiConsumer<SimpleDataSet, Field>> overrides =
         ImmutableMap.<String, BiConsumer<SimpleDataSet, Field>>builder()
@@ -116,7 +129,7 @@ final class SettingsTable extends AbstractVirtualTable
                 value = Arrays.toString((Object[]) value);
             result.row(f.getName()).column(VALUE, value.toString());
 
-            if (ANNOTATED_FIELDS.containsKey(f.getName()))
+            if (ANNOTATED_FIELDS.containsKey(f.getName()) && !EXCLUDED_CONFIG.contains(f.getName()))
             {
                 Replaces annotation = f.getAnnotation(Replaces.class);
                 result.row(annotation.oldName())
diff --git a/src/java/org/apache/cassandra/service/CacheService.java b/src/java/org/apache/cassandra/service/CacheService.java
index 1725d5f..2bacc343 100644
--- a/src/java/org/apache/cassandra/service/CacheService.java
+++ b/src/java/org/apache/cassandra/service/CacheService.java
@@ -121,10 +121,10 @@ public class CacheService implements CacheServiceMBean
      */
     private AutoSavingCache<RowCacheKey, IRowCacheEntry> initRowCache()
     {
-        logger.info("Initializing row cache with capacity of {} MBs", DatabaseDescriptor.getRowCacheSizeInMB());
+        logger.info("Initializing row cache with capacity of {} MBs", DatabaseDescriptor.getRowCacheSizeInMiB());
 
         CacheProvider<RowCacheKey, IRowCacheEntry> cacheProvider;
-        String cacheProviderClassName = DatabaseDescriptor.getRowCacheSizeInMB() > 0
+        String cacheProviderClassName = DatabaseDescriptor.getRowCacheSizeInMiB() > 0
                                         ? DatabaseDescriptor.getRowCacheClassName() : "org.apache.cassandra.cache.NopCacheProvider";
         try
         {
@@ -150,9 +150,9 @@ public class CacheService implements CacheServiceMBean
 
     private AutoSavingCache<CounterCacheKey, ClockAndCount> initCounterCache()
     {
-        logger.info("Initializing counter cache with capacity of {} MBs", DatabaseDescriptor.getCounterCacheSizeInMB());
+        logger.info("Initializing counter cache with capacity of {} MBs", DatabaseDescriptor.getCounterCacheSizeInMiB());
 
-        long capacity = DatabaseDescriptor.getCounterCacheSizeInMB() * 1024 * 1024;
+        long capacity = DatabaseDescriptor.getCounterCacheSizeInMiB() * 1024 * 1024;
 
         AutoSavingCache<CounterCacheKey, ClockAndCount> cache =
             new AutoSavingCache<>(CaffeineCache.create(capacity),
diff --git a/test/conf/cassandra-murmur.yaml b/test/conf/cassandra-murmur.yaml
index 05dac76..c0c2ae7 100644
--- a/test/conf/cassandra-murmur.yaml
+++ b/test/conf/cassandra-murmur.yaml
@@ -37,7 +37,7 @@ incremental_backups: true
 concurrent_compactors: 4
 compaction_throughput: 0MiB/s
 row_cache_class_name: org.apache.cassandra.cache.OHCProvider
-row_cache_size_in_mb: 16
+row_cache_size: 16MiB
 user_defined_functions_enabled: true
 scripted_user_defined_functions_enabled: true
 sasi_indexes_enabled: true
diff --git a/test/conf/cassandra-old.yaml b/test/conf/cassandra-old.yaml
index 11b79f2..27af143 100644
--- a/test/conf/cassandra-old.yaml
+++ b/test/conf/cassandra-old.yaml
@@ -44,7 +44,7 @@ row_cache_class_name: org.apache.cassandra.cache.OHCProvider
 row_cache_size_in_mb: 16
 enable_user_defined_functions: true
 enable_scripted_user_defined_functions: true
-prepared_statements_cache_size_mb: 1
+prepared_statements_cache_size_mb:
 corrupted_tombstone_strategy: exception
 stream_entire_sstables: true
 stream_throughput_outbound_megabits_per_sec: 200000000
diff --git a/test/conf/cassandra-seeds.yaml b/test/conf/cassandra-seeds.yaml
index d5dd83e..0cfd06b 100644
--- a/test/conf/cassandra-seeds.yaml
+++ b/test/conf/cassandra-seeds.yaml
@@ -38,6 +38,6 @@ incremental_backups: true
 concurrent_compactors: 4
 compaction_throughput: 0MiB/s
 row_cache_class_name: org.apache.cassandra.cache.OHCProvider
-row_cache_size_in_mb: 16
+row_cache_size: 16MiB
 user_defined_functions_enabled: true
 scripted_user_defined_functions_enabled: true
diff --git a/test/conf/cassandra-sslcontextfactory-invalidconfiguration.yaml b/test/conf/cassandra-sslcontextfactory-invalidconfiguration.yaml
index 8f51e85..1e473ae 100644
--- a/test/conf/cassandra-sslcontextfactory-invalidconfiguration.yaml
+++ b/test/conf/cassandra-sslcontextfactory-invalidconfiguration.yaml
@@ -70,10 +70,10 @@ incremental_backups: true
 concurrent_compactors: 4
 compaction_throughput: 0MiB/s
 row_cache_class_name: org.apache.cassandra.cache.OHCProvider
-row_cache_size_in_mb: 16
+row_cache_size: 16MiB
 user_defined_functions_enabled: true
 scripted_user_defined_functions_enabled: true
-prepared_statements_cache_size_mb: 1
+prepared_statements_cache_size: 1MiB
 corrupted_tombstone_strategy: exception
 stream_entire_sstables: true
 stream_throughput_outbound: 23841823841858MiB/s
diff --git a/test/conf/cassandra-sslcontextfactory.yaml b/test/conf/cassandra-sslcontextfactory.yaml
index 63c40a2..fde4bfd 100644
--- a/test/conf/cassandra-sslcontextfactory.yaml
+++ b/test/conf/cassandra-sslcontextfactory.yaml
@@ -73,10 +73,10 @@ incremental_backups: true
 concurrent_compactors: 4
 compaction_throughput: 0MiB/s
 row_cache_class_name: org.apache.cassandra.cache.OHCProvider
-row_cache_size_in_mb: 16
+row_cache_size: 16MiB
 user_defined_functions_enabled: true
 scripted_user_defined_functions_enabled: true
-prepared_statements_cache_size_mb: 1
+prepared_statements_cache_size: 1MiB
 corrupted_tombstone_strategy: exception
 stream_entire_sstables: true
 stream_throughput_outbound: 23841858MiB/s
diff --git a/test/conf/cassandra.yaml b/test/conf/cassandra.yaml
index e1a49f2..58afdb9 100644
--- a/test/conf/cassandra.yaml
+++ b/test/conf/cassandra.yaml
@@ -41,10 +41,10 @@ incremental_backups: true
 concurrent_compactors: 4
 compaction_throughput: 0MiB/s
 row_cache_class_name: org.apache.cassandra.cache.OHCProvider
-row_cache_size_in_mb: 16
+row_cache_size: 16MiB
 user_defined_functions_enabled: true
 scripted_user_defined_functions_enabled: true
-prepared_statements_cache_size_mb: 1
+prepared_statements_cache_size: 1MiB
 corrupted_tombstone_strategy: exception
 stream_entire_sstables: true
 stream_throughput_outbound: 23841858MiB/s
diff --git a/test/conf/unit-test-conf/test-native-port.yaml b/test/conf/unit-test-conf/test-native-port.yaml
index cef6b1d..a234ad5 100644
--- a/test/conf/unit-test-conf/test-native-port.yaml
+++ b/test/conf/unit-test-conf/test-native-port.yaml
@@ -41,10 +41,10 @@ incremental_backups: true
 concurrent_compactors: 4
 compaction_throughput_mb_per_sec: 0
 row_cache_class_name: org.apache.cassandra.cache.OHCProvider
-row_cache_size_in_mb: 16
+row_cache_size: 16MiB
 user_defined_functions_enabled: true
 scripted_user_defined_functions_enabled: true
-prepared_statements_cache_size_mb: 1
+prepared_statements_cache_size: 1MiB
 corrupted_tombstone_strategy: exception
 stream_entire_sstables: true
 stream_throughput_outbound: 23841858MiB/s
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java b/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
index d6bc7ad..7861a98 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
@@ -104,11 +104,11 @@ public class InstanceConfig implements IInstanceConfig
                 .set("diagnostic_events_enabled", true)
                 .set("auto_bootstrap", false)
                 // capacities that are based on `totalMemory` that should be fixed size
-                .set("index_summary_capacity_in_mb", 50l)
-                .set("counter_cache_size_in_mb", 50l)
-                .set("key_cache_size_in_mb", 50l)
+                .set("index_summary_capacity", "50MiB")
+                .set("counter_cache_size", "50MiB")
+                .set("key_cache_size", "50MiB")
                 // legacy parameters
-                .forceSet("commitlog_sync_batch_window_in_ms", 1.0);
+                .forceSet("commitlog_sync_batch_window_in_ms", "1");
         this.featureFlags = EnumSet.noneOf(Feature.class);
     }
 
diff --git a/test/distributed/org/apache/cassandra/distributed/test/NodeToolTest.java b/test/distributed/org/apache/cassandra/distributed/test/NodeToolTest.java
index 89e6168..ea45bd1 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/NodeToolTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/NodeToolTest.java
@@ -110,7 +110,7 @@ public class NodeToolTest extends TestBaseImpl
     @Test
     public void testSetCacheCapacityWhenDisabled() throws Throwable
     {
-        try (ICluster cluster = init(builder().withNodes(1).withConfig(c->c.set("row_cache_size_in_mb", "0")).start()))
+        try (ICluster cluster = init(builder().withNodes(1).withConfig(c->c.set("row_cache_size", "0MiB")).start()))
         {
             NodeToolResult ringResult = cluster.get(1).nodetoolResult("setcachecapacity", "1", "1", "1");
             ringResult.asserts().stderrContains("is not permitted as this cache is disabled");
diff --git a/test/simulator/main/org/apache/cassandra/simulator/ClusterSimulation.java b/test/simulator/main/org/apache/cassandra/simulator/ClusterSimulation.java
index 74c9356..6882390 100644
--- a/test/simulator/main/org/apache/cassandra/simulator/ClusterSimulation.java
+++ b/test/simulator/main/org/apache/cassandra/simulator/ClusterSimulation.java
@@ -642,9 +642,9 @@ public class ClusterSimulation<S extends Simulation> implements AutoCloseable
                              .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", "1mb")
+                             .set("memtable_heap_space", "1MiB")
                              .set("memtable_allocation_type", builder.memoryListener != null ? "unslabbed_heap_buffers_logged" : "heap_buffers")
-                             .set("file_cache_size_in_mb", 16)
+                             .set("file_cache_size", "16MiB")
                              .set("use_deterministic_table_id", true)
                              .set("disk_access_mode", "standard")
                              .set("failure_detector", SimulatedFailureDetector.Instance.class.getName())
diff --git a/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java b/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java
index ad7abb8..da1d094 100644
--- a/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java
+++ b/test/unit/org/apache/cassandra/config/LoadOldYAMLBackwardCompatibilityTest.java
@@ -71,14 +71,14 @@ public class LoadOldYAMLBackwardCompatibilityTest
         assertEquals(DataStorageSpec.inKibibytes(2), config.column_index_cache_size);
         assertEquals(DataStorageSpec.inKibibytes(5), config.batch_size_warn_threshold);
         assertEquals(DataRateSpec.inMebibytesPerSecond(64), config.compaction_throughput);
-        //assertEquals(DataStorageSpec.inMebibytes(50), config.min_free_space_per_drive);
+        assertEquals(DataStorageSpec.inMebibytes(50), config.min_free_space_per_drive);
         assertEquals(DataRateSpec.inMebibytesPerSecond(23841858).toString(), config.stream_throughput_outbound.toString());
         assertEquals(DataRateSpec.megabitsPerSecondInMebibytesPerSecond(200000000).toString(), config.stream_throughput_outbound.toString());
         assertEquals(DataRateSpec.inMebibytesPerSecond(24), config.inter_dc_stream_throughput_outbound);
-        /*assertNull(config.commitlog_total_space);
+        assertNull(config.commitlog_total_space);
         assertEquals(DurationSpec.inDoubleMilliseconds(0), config.commitlog_sync_group_window);
         assertEquals(DurationSpec.inMilliseconds(0), config.commitlog_sync_period);
-        assertEquals(DataStorageSpec.inMebibytes(32), config.commitlog_segment_size);
+        assertEquals(DataStorageSpec.inMebibytes(5), config.commitlog_segment_size);
         assertNull(config.periodic_commitlog_sync_lag_block);  //Integer
         assertNull(config.max_mutation_size);
         assertEquals(DataStorageSpec.inMebibytes(0), config.cdc_total_space);
@@ -88,11 +88,11 @@ public class LoadOldYAMLBackwardCompatibilityTest
         assertEquals(DataStorageSpec.inKibibytes(1024), config.hinted_handoff_throttle);
         assertEquals(DataStorageSpec.inKibibytes(1024), config.batchlog_replay_throttle);
         assertEquals(DurationSpec.inMilliseconds(10000), config.hints_flush_period);
-        assertEquals(DataStorageSpec.inMebibytes(128), config.max_hints_file_size);*/
+        assertEquals(DataStorageSpec.inMebibytes(128), config.max_hints_file_size);
         assertEquals(DataStorageSpec.inKibibytes(10240), config.trickle_fsync_interval);
         assertEquals(DataStorageSpec.inMebibytes(50), config.sstable_preemptive_open_interval);
-        /*assertNull( config.key_cache_size);
-        assertEquals(DataStorageSpec.inMebibytes(0), config.row_cache_size);
+        assertNull( config.key_cache_size);
+        assertEquals(DataStorageSpec.inMebibytes(16), config.row_cache_size);
         assertNull(config.counter_cache_size);
         assertNull(config.networking_cache_size);
         assertNull(config.file_cache_size);
@@ -101,7 +101,7 @@ public class LoadOldYAMLBackwardCompatibilityTest
         assertEquals(DurationSpec.inMilliseconds(1000), config.gc_warn_threshold);
         assertEquals(DurationSpec.inSeconds(86400), config.trace_type_query_ttl);
         assertEquals(DurationSpec.inSeconds(604800), config.trace_type_repair_ttl);
-        assertNull(config.prepared_statements_cache_size);*/
+        assertNull(config.prepared_statements_cache_size);
         assertTrue(config.user_defined_functions_enabled);
         assertTrue(config.scripted_user_defined_functions_enabled);
         assertTrue(config.materialized_views_enabled);
@@ -115,13 +115,14 @@ public class LoadOldYAMLBackwardCompatibilityTest
         assertEquals(DurationSpec.inMilliseconds(0), config.roles_update_interval);
         assertEquals(DurationSpec.inMilliseconds(2000), config.credentials_validity);
         assertEquals(DurationSpec.inMilliseconds(0), config.credentials_update_interval);
-        //assertEquals(DurationSpec.inMinutes(60), config.index_summary_resize_interval);
+        assertEquals(DurationSpec.inMinutes(60), config.index_summary_resize_interval);
 
         //parameters which names have not changed with CASSANDRA-15234
-        /*assertEquals(DurationSpec.inSecondsString("14400"), config.key_cache_save_period);
+        assertEquals(DurationSpec.inSecondsString("14400"), config.key_cache_save_period);
+        assertEquals(DurationSpec.inSecondsString("14400s"), config.key_cache_save_period);
         assertEquals(DurationSpec.inHours(4), config.key_cache_save_period);
         assertEquals(DurationSpec.inSecondsString("0"), config.row_cache_save_period);
         assertEquals(DurationSpec.inSeconds(0), config.row_cache_save_period);
-        assertEquals(DurationSpec.inHours(2), config.counter_cache_save_period);*/
+        assertEquals(DurationSpec.inHours(2), config.counter_cache_save_period);
     }
 }
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index e5395e1..7d5be0d 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -67,6 +67,7 @@ import org.apache.cassandra.auth.IRoleManager;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.config.EncryptionOptions;
+import org.apache.cassandra.config.SmallestDataStorageMebibytes;
 import org.apache.cassandra.db.virtual.VirtualKeyspaceRegistry;
 import org.apache.cassandra.db.virtual.VirtualSchemaKeyspace;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -127,7 +128,7 @@ public abstract class CQLTester
     public static final String KEYSPACE_PER_TEST = "cql_test_keyspace_alt";
     protected static final boolean USE_PREPARED_VALUES = Boolean.valueOf(System.getProperty("cassandra.test.use_prepared", "true"));
     protected static final boolean REUSE_PREPARED = Boolean.valueOf(System.getProperty("cassandra.test.reuse_prepared", "true"));
-    protected static final long ROW_CACHE_SIZE_IN_MB = Integer.valueOf(System.getProperty("cassandra.test.row_cache_size_in_mb", "0"));
+    protected static final long ROW_CACHE_SIZE_IN_MIB = new SmallestDataStorageMebibytes(System.getProperty("cassandra.test.row_cache_size", "0MiB")).toMebibytes();
     private static final AtomicInteger seqNumber = new AtomicInteger();
     protected static final ByteBuffer TOO_BIG = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT + 1024);
     public static final String DATA_CENTER = ServerTestUtils.DATA_CENTER;
@@ -303,8 +304,8 @@ public abstract class CQLTester
     @BeforeClass
     public static void setUpClass()
     {
-        if (ROW_CACHE_SIZE_IN_MB > 0)
-            DatabaseDescriptor.setRowCacheSizeInMB(ROW_CACHE_SIZE_IN_MB);
+        if (ROW_CACHE_SIZE_IN_MIB > 0)
+            DatabaseDescriptor.setRowCacheSizeInMiB(ROW_CACHE_SIZE_IN_MIB);
         StorageService.instance.setPartitionerUnsafe(Murmur3Partitioner.instance);
 
         // Once per-JVM is enough
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/JsonTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/JsonTest.java
index f550fdf..810ee5a 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/JsonTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/JsonTest.java
@@ -50,8 +50,8 @@ public class JsonTest extends CQLTester
     @BeforeClass
     public static void setUpClass()
     {
-        if (ROW_CACHE_SIZE_IN_MB > 0)
-            DatabaseDescriptor.setRowCacheSizeInMB(ROW_CACHE_SIZE_IN_MB);
+        if (ROW_CACHE_SIZE_IN_MIB > 0)
+            DatabaseDescriptor.setRowCacheSizeInMiB(ROW_CACHE_SIZE_IN_MIB);
 
         StorageService.instance.setPartitionerUnsafe(ByteOrderedPartitioner.instance);
 

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