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 23138252f20891c26a3692664c6affaf99e86541
Author: Ekaterina Dimitrova <ekaterina.dimitr...@datastax.com>
AuthorDate: Thu Feb 3 23:49:50 2022 -0500

    Transfer parameters to the newly introduced configuration framework (6)
    patch by Ekaterina Dimitrova; reviewed by Caleb Rackliffe, David Capwell, 
Michael Semb Wever and Benjamin Lerer for CASSANDRA-15234
---
 conf/cassandra.yaml                                |  40 ++---
 doc/native_protocol_v4.spec                        |   4 +-
 doc/native_protocol_v5.spec                        |   4 +-
 .../apache/cassandra/batchlog/BatchlogManager.java |  10 +-
 src/java/org/apache/cassandra/config/Config.java   |  50 +++++--
 .../cassandra/config/DatabaseDescriptor.java       | 166 +++++++++++----------
 .../commitlog/AbstractCommitLogSegmentManager.java |   2 +-
 .../db/commitlog/CommitLogSegmentManagerCDC.java   |   2 +-
 .../db/commitlog/GroupCommitLogService.java        |   2 +-
 .../cassandra/hints/HintsDispatchExecutor.java     |   2 +-
 .../apache/cassandra/hints/HintsWriteExecutor.java |   4 +-
 .../org/apache/cassandra/io/util/FileUtils.java    |  26 ++--
 .../apache/cassandra/service/StorageService.java   |   8 +-
 .../cassandra/service/StorageServiceMBean.java     |   6 +-
 .../nodetool/SetHintedHandoffThrottleInKB.java     |   4 +-
 test/conf/cassandra-murmur.yaml                    |   2 +-
 test/conf/cassandra-seeds.yaml                     |   2 +-
 ...dra-sslcontextfactory-invalidconfiguration.yaml |   2 +-
 test/conf/cassandra-sslcontextfactory.yaml         |   2 +-
 test/conf/cassandra.yaml                           |   6 +-
 test/conf/unit-test-conf/test-native-port.yaml     |   2 +-
 .../test/HintedHandoffNodetoolTest.java            |   4 +-
 .../distributed/test/LargeColumnTest.java          |   2 +-
 .../cassandra/config/DatabaseDescriptorTest.java   |   4 +-
 .../commitlog/CommitLogInitWithExceptionTest.java  |   1 +
 .../commitlog/CommitLogSegmentManagerCDCTest.java  |   8 +-
 26 files changed, 193 insertions(+), 172 deletions(-)

diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index 0c4acb8..9f7e657 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -68,7 +68,7 @@ max_hint_window: 3h
 # are two nodes in the cluster, each delivery thread will use the maximum
 # rate; if there are three, each will throttle to half of the maximum,
 # since we expect two nodes to be delivering hints simultaneously.)
-hinted_handoff_throttle_in_kb: 1024
+hinted_handoff_throttle: 1024KiB
 
 # Number of threads with which to deliver hints;
 # Consider increasing this number when you have multi-dc deployments, since
@@ -81,10 +81,10 @@ max_hints_delivery_threads: 2
 
 # How often hints should be flushed from the internal buffers to disk.
 # Will *not* trigger fsync.
-hints_flush_period_in_ms: 10000
+hints_flush_period: 10000ms
 
 # Maximum size for a single hints file, in megabytes.
-max_hints_file_size_in_mb: 128
+max_hints_file_size: 128MiB
 
 # Enable / disable automatic cleanup for the expired and orphaned hints file.
 # Disable the option in order to preserve those hints on the disk.
@@ -114,7 +114,7 @@ auto_hints_cleanup_enabled: false
 
 # Maximum throttle in KBs per second, total. This will be
 # reduced proportionally to the number of nodes in the cluster.
-batchlog_replay_throttle_in_kb: 1024
+batchlog_replay_throttle: 1024KiB
 
 # Authentication backend, implementing IAuthenticator; used to identify users
 # Out of the box, Cassandra provides 
org.apache.cassandra.auth.{AllowAllAuthenticator,
@@ -456,19 +456,19 @@ counter_cache_save_period: 7200
 #
 # group mode is similar to batch mode, where Cassandra will not ack writes
 # until the commit log has been flushed to disk. The difference is group
-# mode will wait up to commitlog_sync_group_window_in_ms between flushes.
+# mode will wait up to commitlog_sync_group_window between flushes.
 #
-# commitlog_sync_group_window_in_ms: 1000
+# commitlog_sync_group_window: 1000ms
 #
 # the default option is "periodic" where writes may be acked immediately
-# and the CommitLog is simply synced every commitlog_sync_period_in_ms
+# and the CommitLog is simply synced every commitlog_sync_period
 # milliseconds.
 commitlog_sync: periodic
-commitlog_sync_period_in_ms: 10000
+commitlog_sync_period: 10000ms
 
 # When in periodic commitlog mode, the number of milliseconds to block writes
 # while waiting for a slow disk flush to complete.
-# periodic_commitlog_sync_lag_block_in_ms: 
+# periodic_commitlog_sync_lag_block:
 
 # The size of the individual commitlog file segments.  A commitlog
 # segment may be archived, deleted, or recycled once all the data
@@ -479,14 +479,14 @@ commitlog_sync_period_in_ms: 10000
 # archiving commitlog segments (see commitlog_archiving.properties),
 # then you probably want a finer granularity of archiving; 8 or 16 MB
 # is reasonable.
-# Max mutation size is also configurable via max_mutation_size_in_kb setting in
-# cassandra.yaml. The default is half the size commitlog_segment_size_in_mb * 
1024.
+# Max mutation size is also configurable via max_mutation_size setting in
+# cassandra.yaml. The default is half the size commitlog_segment_size in bytes.
 # This should be positive and less than 2048.
 #
-# NOTE: If max_mutation_size_in_kb is set explicitly then 
commitlog_segment_size_in_mb must
-# be set to at least twice the size of max_mutation_size_in_kb / 1024
+# NOTE: If max_mutation_size is set explicitly then commitlog_segment_size must
+# be set to at least twice the size of max_mutation_size
 #
-commitlog_segment_size_in_mb: 32
+commitlog_segment_size: 32MiB
 
 # Compression to apply to the commit log. If omitted, the commit log
 # will be written uncompressed.  LZ4, Snappy, and Deflate compressors
@@ -631,7 +631,7 @@ memtable_allocation_type: heap_buffers
 # The default value is the smaller of 8192, and 1/4 of the total space
 # of the commitlog volume.
 #
-# commitlog_total_space_in_mb: 8192
+# commitlog_total_space: 8192MiB
 
 # This sets the number of memtable flush writer threads per disk
 # as well as the total number of memtables that can be flushed concurrently.
@@ -670,12 +670,12 @@ memtable_allocation_type: heap_buffers
 #
 # The default value is the min of 4096 mb and 1/8th of the total space
 # of the drive where cdc_raw_directory resides.
-# cdc_total_space_in_mb: 4096
+# cdc_total_space: 4096MiB
 
 # When we hit our cdc_raw limit and the CDCCompactor is either running behind
 # or experiencing backpressure, we check at the following interval to see if 
any
 # new space for cdc-tracked tables has been made available. Default to 250ms
-# cdc_free_space_check_interval_ms: 250
+# cdc_free_space_check_interval: 250ms
 
 # A fixed memory pool size in MB for for SSTable index summaries. If left
 # empty, this will default to 5% of the heap size. If the memory usage of
@@ -1176,10 +1176,10 @@ endpoint_snitch: SimpleSnitch
 
 # controls how often to perform the more expensive part of host score
 # calculation
-dynamic_snitch_update_interval_in_ms: 100 
+dynamic_snitch_update_interval: 100ms
 # controls how often to reset all host scores, allowing a bad host to
 # possibly recover
-dynamic_snitch_reset_interval_in_ms: 600000
+dynamic_snitch_reset_interval: 600000ms
 # if set greater than zero, this will allow
 # 'pinning' of replicas to hosts in order to increase cache capacity.
 # The badness threshold will control how much worse the pinned host has to be
@@ -1394,7 +1394,7 @@ batch_size_fail_threshold: 50KiB
 unlogged_batch_across_partitions_warn_threshold: 10
 
 # Log a warning when compacting partitions larger than this value
-compaction_large_partition_warning_threshold_mb: 100
+compaction_large_partition_warning_threshold: 100MiB
 
 # Log a warning when writing more tombstones than this value to a partition
 compaction_tombstone_warning_threshold: 100000
diff --git a/doc/native_protocol_v4.spec b/doc/native_protocol_v4.spec
index 2220000..6def737 100644
--- a/doc/native_protocol_v4.spec
+++ b/doc/native_protocol_v4.spec
@@ -1103,7 +1103,7 @@ Table of Contents
                              - "VIEW": the timeout occured when a write 
involves
                                 VIEW update and failure to acqiure local 
view(MV)
                                 lock for key within timeout
-                             - "CDC": the timeout occured when 
cdc_total_space_in_mb is
+                             - "CDC": the timeout occured when cdc_total_space 
is
                                 exceeded when doing a write to data tracked by 
cdc.
     0x1200    Read_timeout: Timeout exception during a read request. The rest
               of the ERROR message body will be
@@ -1176,7 +1176,7 @@ Table of Contents
                              - "VIEW": the failure occured when a write 
involves
                                 VIEW update and failure to acqiure local 
view(MV)
                                 lock for key within timeout
-                             - "CDC": the failure occured when 
cdc_total_space_in_mb is
+                             - "CDC": the failure occured when cdc_total_space 
is
                                 exceeded when doing a write to data tracked by 
cdc.
 
     0x2000    Syntax_error: The submitted query has a syntax error.
diff --git a/doc/native_protocol_v5.spec b/doc/native_protocol_v5.spec
index 3b13527..3f6703d 100644
--- a/doc/native_protocol_v5.spec
+++ b/doc/native_protocol_v5.spec
@@ -1298,7 +1298,7 @@ Table of Contents
                              - "VIEW": the timeout occured when a write 
involves
                                VIEW update and failure to acqiure local 
view(MV)
                                lock for key within timeout
-                             - "CDC": the timeout occured when 
cdc_total_space_in_mb is
+                             - "CDC": the timeout occured when cdc_total_space 
is
                                exceeded when doing a write to data tracked by 
cdc.
                 <contentions> is a [short] that describes the number of 
contentions occured during the CAS operation.
                               The field only presents when the <writeType> is 
"CAS".
@@ -1381,7 +1381,7 @@ Table of Contents
                              - "VIEW": the failure occured when a write 
involves
                                VIEW update and failure to acqiure local 
view(MV)
                                lock for key within timeout
-                             - "CDC": the failure occured when 
cdc_total_space_in_mb is
+                             - "CDC": the failure occured when cdc_total_space 
is
                                exceeded when doing a write to data tracked by 
cdc.
     0x1600    CDC_WRITE_FAILURE: // todo
     0x1700    CAS_WRITE_UNKNOWN: An exception occured due to contended Compare 
And Set write/update.
diff --git a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java 
b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
index 147e7e9..420d559 100644
--- a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
@@ -208,7 +208,7 @@ public class BatchlogManager implements BatchlogManagerMBean
             logger.trace("Replay cancelled as there are no peers in the 
ring.");
             return;
         }
-        setRate(DatabaseDescriptor.getBatchlogReplayThrottleInKB());
+        setRate(DatabaseDescriptor.getBatchlogReplayThrottleInKiB());
 
         UUID limitUuid = UUIDGen.maxTimeUUID(currentTimeMillis() - 
getBatchlogTimeout());
         ColumnFamilyStore store = 
Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.BATCHES);
@@ -229,18 +229,18 @@ public class BatchlogManager implements 
BatchlogManagerMBean
      * Sets the rate for the current rate limiter. When {@code throttleInKB} 
is 0, this sets the rate to
      * {@link Double#MAX_VALUE} bytes per second.
      *
-     * @param throttleInKB throughput to set in KB per second
+     * @param throttleInKB throughput to set in KiB per second
      */
     public void setRate(final int throttleInKB)
     {
         int endpointsCount = 
StorageService.instance.getTokenMetadata().getSizeOfAllEndpoints();
         if (endpointsCount > 0)
         {
-            int endpointThrottleInKB = throttleInKB / endpointsCount;
-            double throughput = endpointThrottleInKB == 0 ? Double.MAX_VALUE : 
endpointThrottleInKB * 1024.0;
+            int endpointThrottleInKiB = throttleInKB / endpointsCount;
+            double throughput = endpointThrottleInKiB == 0 ? Double.MAX_VALUE 
: endpointThrottleInKiB * 1024.0;
             if (rateLimiter.getRate() != throughput)
             {
-                logger.debug("Updating batchlog replay throttle to {} KB/s, {} 
KB/s per endpoint", throttleInKB, endpointThrottleInKB);
+                logger.debug("Updating batchlog replay throttle to {} KB/s, {} 
KB/s per endpoint", throttleInKB, endpointThrottleInKiB);
                 rateLimiter.setRate(throughput);
             }
         }
diff --git a/src/java/org/apache/cassandra/config/Config.java 
b/src/java/org/apache/cassandra/config/Config.java
index 7ee8c42..2f56fc7 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -272,8 +272,10 @@ public class Config
     public volatile Integer concurrent_compactors;
     @Replaces(oldName = "compaction_throughput_mb_per_sec", converter = 
Converters.MEBIBYTES_PER_SECOND_DATA_RATE, deprecated = true)
     public volatile DataRateSpec compaction_throughput = new 
DataRateSpec("16MiB/s");
-    public volatile int compaction_large_partition_warning_threshold_mb = 100;
-    public int min_free_space_per_drive_in_mb = 50;
+    @Replaces(oldName = "compaction_large_partition_warning_threshold_mb", 
converter = Converters.MEBIBYTES_DATA_STORAGE, deprecated = true)
+    public volatile SmallestDataStorageMebibytes 
compaction_large_partition_warning_threshold = new 
SmallestDataStorageMebibytes("100MiB");
+    @Replaces(oldName = "min_free_space_per_drive_in_mb", converter = 
Converters.MEBIBYTES_DATA_STORAGE, deprecated = true)
+    public SmallestDataStorageMebibytes min_free_space_per_drive = new 
SmallestDataStorageMebibytes("50MiB");
     public volatile Integer compaction_tombstone_warning_threshold = 100000;
 
     public volatile int concurrent_materialized_view_builders = 1;
@@ -305,23 +307,29 @@ public class Config
 
     // Commit Log
     public String commitlog_directory;
-    public Integer commitlog_total_space_in_mb;
+    @Replaces(oldName = "commitlog_total_space_in_mb", converter = 
Converters.MEBIBYTES_DATA_STORAGE, deprecated = true)
+    public SmallestDataStorageMebibytes commitlog_total_space;
     public CommitLogSync commitlog_sync;
 
     /**
      * @deprecated since 4.0 This value was near useless, and we're not using 
it anymore
      */
     public double commitlog_sync_batch_window_in_ms = Double.NaN;
-    public double commitlog_sync_group_window_in_ms = Double.NaN;
-    public int commitlog_sync_period_in_ms;
-    public int commitlog_segment_size_in_mb = 32;
+    @Replaces(oldName = "commitlog_sync_group_window_in_ms", converter = 
Converters.MILLIS_DOUBLE_DURATION, deprecated = true)
+    public SmallestDurationMilliseconds commitlog_sync_group_window = new 
SmallestDurationMilliseconds("0ms");
+    @Replaces(oldName = "commitlog_sync_period_in_ms", converter = 
Converters.MILLIS_DURATION, deprecated = true)
+    public SmallestDurationMilliseconds commitlog_sync_period = new 
SmallestDurationMilliseconds("0ms");
+    @Replaces(oldName = "commitlog_segment_size_in_mb", converter = 
Converters.MEBIBYTES_DATA_STORAGE, deprecated = true)
+    public SmallestDataStorageMebibytes commitlog_segment_size = new 
SmallestDataStorageMebibytes("32MiB");
     public ParameterizedClass commitlog_compression;
     public FlushCompression flush_compression = FlushCompression.fast;
     public int commitlog_max_compression_buffers_in_pool = 3;
-    public Integer periodic_commitlog_sync_lag_block_in_ms;
+    @Replaces(oldName = "periodic_commitlog_sync_lag_block_in_ms", converter = 
Converters.MILLIS_DURATION, deprecated = true)
+    public SmallestDurationMilliseconds periodic_commitlog_sync_lag_block;
     public TransparentDataEncryptionOptions 
transparent_data_encryption_options = new TransparentDataEncryptionOptions();
 
-    public Integer max_mutation_size_in_kb;
+    @Replaces(oldName = "max_mutation_size_in_kb", converter = 
Converters.KIBIBYTES_DATASTORAGE, deprecated = true)
+    public SmallestDataStorageKibibytes max_mutation_size;
 
     // Change-data-capture logs
     public boolean cdc_enabled = false;
@@ -329,16 +337,20 @@ public class Config
     // When false, new CDC mutations can always be added. But it will remove 
the oldest CDC commit log segment on full.
     public volatile boolean cdc_block_writes = true;
     public String cdc_raw_directory;
-    public int cdc_total_space_in_mb = 0;
-    public int cdc_free_space_check_interval_ms = 250;
+    @Replaces(oldName = "cdc_total_space_in_mb", converter = 
Converters.MEBIBYTES_DATA_STORAGE, deprecated = true)
+    public SmallestDataStorageMebibytes cdc_total_space = new 
SmallestDataStorageMebibytes("0MiB");
+    @Replaces(oldName = "cdc_free_space_check_interval_ms", converter = 
Converters.MILLIS_DURATION, deprecated = true)
+    public SmallestDurationMilliseconds cdc_free_space_check_interval = new 
SmallestDurationMilliseconds("250ms");
 
     @Deprecated
     public int commitlog_periodic_queue_size = -1;
 
     public String endpoint_snitch;
     public boolean dynamic_snitch = true;
-    public int dynamic_snitch_update_interval_in_ms = 100;
-    public int dynamic_snitch_reset_interval_in_ms = 600000;
+    @Replaces(oldName = "dynamic_snitch_update_interval_in_ms", converter = 
Converters.MILLIS_DURATION, deprecated = true)
+    public SmallestDurationMilliseconds dynamic_snitch_update_interval = new 
SmallestDurationMilliseconds("100ms");
+    @Replaces(oldName = "dynamic_snitch_reset_interval_in_ms", converter = 
Converters.MILLIS_DURATION, deprecated = true)
+    public SmallestDurationMilliseconds dynamic_snitch_reset_interval = new 
SmallestDurationMilliseconds("10m");
     public double dynamic_snitch_badness_threshold = 1.0;
 
     public String failure_detector = "FailureDetector";
@@ -348,11 +360,17 @@ public class Config
 
     public InternodeCompression internode_compression = 
InternodeCompression.none;
 
-    public int hinted_handoff_throttle_in_kb = 1024;
-    public int batchlog_replay_throttle_in_kb = 1024;
+    @Replaces(oldName = "hinted_handoff_throttle_in_kb", converter = 
Converters.KIBIBYTES_DATASTORAGE, deprecated = true)
+    public SmallestDataStorageKibibytes hinted_handoff_throttle = new 
SmallestDataStorageKibibytes("1024KiB");
+    @Replaces(oldName = "batchlog_replay_throttle_in_kb", converter = 
Converters.KIBIBYTES_DATASTORAGE, deprecated = true)
+    public SmallestDataStorageKibibytes batchlog_replay_throttle = new 
SmallestDataStorageKibibytes("1024KiB");
     public int max_hints_delivery_threads = 2;
-    public int hints_flush_period_in_ms = 10000;
-    public int max_hints_file_size_in_mb = 128;
+
+    @Replaces(oldName = "hints_flush_period_in_ms", converter = 
Converters.MILLIS_DURATION, deprecated = true)
+    public SmallestDurationMilliseconds hints_flush_period = new 
SmallestDurationMilliseconds("10s");
+    @Replaces(oldName = "max_hints_file_size_in_mb", converter = 
Converters.MEBIBYTES_DATA_STORAGE, deprecated = true)
+    public SmallestDataStorageMebibytes max_hints_file_size = new 
SmallestDataStorageMebibytes("128MiB");
+
     public ParameterizedClass hints_compression;
     public volatile boolean auto_hints_cleanup_enabled = false;
 
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java 
b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 3b1ef66..b51f2de 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -78,7 +78,7 @@ import org.apache.commons.lang3.StringUtils;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.cassandra.config.CassandraRelevantProperties.OS_ARCH;
 import static 
org.apache.cassandra.config.CassandraRelevantProperties.SUN_ARCH_DATA_MODEL;
-import static org.apache.cassandra.io.util.FileUtils.ONE_GB;
+import static org.apache.cassandra.io.util.FileUtils.ONE_GIB;
 import static org.apache.cassandra.io.util.FileUtils.ONE_MIB;
 import static org.apache.cassandra.utils.Clock.Global.logInitializationOutcome;
 
@@ -381,35 +381,35 @@ public class DatabaseDescriptor
 
         if (conf.commitlog_sync == Config.CommitLogSync.batch)
         {
-            if (conf.commitlog_sync_period_in_ms != 0)
+            if (conf.commitlog_sync_period.toMilliseconds() != 0)
             {
-                throw new ConfigurationException("Batch sync specified, but 
commitlog_sync_period_in_ms found. Only specify 
commitlog_sync_batch_window_in_ms when using batch sync", false);
+                throw new ConfigurationException("Batch sync specified, but 
commitlog_sync_period found.", false);
             }
             logger.debug("Syncing log with batch mode");
         }
         else if (conf.commitlog_sync == CommitLogSync.group)
         {
-            if (Double.isNaN(conf.commitlog_sync_group_window_in_ms) || 
conf.commitlog_sync_group_window_in_ms <= 0d)
+            if (conf.commitlog_sync_group_window.toMilliseconds() == 0)
             {
-                throw new ConfigurationException("Missing value for 
commitlog_sync_group_window_in_ms: positive double value expected.", false);
+                throw new ConfigurationException("Missing value for 
commitlog_sync_group_window.", false);
             }
-            else if (conf.commitlog_sync_period_in_ms != 0)
+            else if (conf.commitlog_sync_period.toMilliseconds() != 0)
             {
-                throw new ConfigurationException("Group sync specified, but 
commitlog_sync_period_in_ms found. Only specify 
commitlog_sync_group_window_in_ms when using group sync", false);
+                throw new ConfigurationException("Group sync specified, but 
commitlog_sync_period found. Only specify commitlog_sync_group_window when 
using group sync", false);
             }
-            logger.debug("Syncing log with a group window of {}", 
conf.commitlog_sync_period_in_ms);
+            logger.debug("Syncing log with a group window of {}", 
conf.commitlog_sync_period.toString());
         }
         else
         {
-            if (conf.commitlog_sync_period_in_ms <= 0)
+            if (conf.commitlog_sync_period.toMilliseconds() <= 0)
             {
-                throw new ConfigurationException("Missing value for 
commitlog_sync_period_in_ms: positive integer expected", false);
+                throw new ConfigurationException("Missing value for 
commitlog_sync_period.", false);
             }
             else if (!Double.isNaN(conf.commitlog_sync_batch_window_in_ms))
             {
-                throw new ConfigurationException("commitlog_sync_period_in_ms 
specified, but commitlog_sync_batch_window_in_ms found.  Only specify 
commitlog_sync_period_in_ms when using periodic sync.", false);
+                throw new ConfigurationException("commitlog_sync_period 
specified, but commitlog_sync_batch_window found.  Only specify 
commitlog_sync_period when using periodic sync.", false);
             }
-            logger.debug("Syncing log with a period of {}", 
conf.commitlog_sync_period_in_ms);
+            logger.debug("Syncing log with a period of {}", 
conf.commitlog_sync_period.toString());
         }
 
         /* evaluate the DiskAccessMode Config directive, which also affects 
indexAccessMode selection */
@@ -551,16 +551,17 @@ public class DatabaseDescriptor
         else
             logger.info("Native transport rate-limiting disabled.");
 
-        if (conf.commitlog_total_space_in_mb == null)
+        if (conf.commitlog_total_space == null)
         {
-            final int preferredSizeInMB = 8192;
+            final int preferredSizeInMiB = 8192;
             // use 1/4 of available space.  See discussion on #10013 and #10199
             final long totalSpaceInBytes = 
tryGetSpace(conf.commitlog_directory, FileStore::getTotalSpace);
-            conf.commitlog_total_space_in_mb = 
calculateDefaultSpaceInMB("commitlog",
-                                                                         
conf.commitlog_directory,
-                                                                         
"commitlog_total_space_in_mb",
-                                                                         
preferredSizeInMB,
-                                                                         
totalSpaceInBytes, 1, 4);
+            int defaultSpaceInMiB = calculateDefaultSpaceInMiB("commitlog",
+                                                               
conf.commitlog_directory,
+                                                               
"commitlog_total_space",
+                                                               
preferredSizeInMiB,
+                                                               
totalSpaceInBytes, 1, 4);
+            conf.commitlog_total_space = 
SmallestDataStorageMebibytes.inMebibytes(defaultSpaceInMiB);
         }
 
         if (conf.cdc_enabled)
@@ -570,16 +571,17 @@ public class DatabaseDescriptor
                 conf.cdc_raw_directory = storagedirFor("cdc_raw");
             }
 
-            if (conf.cdc_total_space_in_mb == 0)
+            if (conf.cdc_total_space.toMebibytes() == 0)
             {
-                final int preferredSizeInMB = 4096;
+                final int preferredSizeInMiB = 4096;
                 // use 1/8th of available space.  See discussion on #10013 and 
#10199 on the CL, taking half that for CDC
                 final long totalSpaceInBytes = 
tryGetSpace(conf.cdc_raw_directory, FileStore::getTotalSpace);
-                conf.cdc_total_space_in_mb = calculateDefaultSpaceInMB("cdc",
-                                                                       
conf.cdc_raw_directory,
-                                                                       
"cdc_total_space_in_mb",
-                                                                       
preferredSizeInMB,
-                                                                       
totalSpaceInBytes, 1, 8);
+                int defaultSpaceInMiB = calculateDefaultSpaceInMiB("cdc",
+                                                                   
conf.cdc_raw_directory,
+                                                                   
"cdc_total_space",
+                                                                   
preferredSizeInMiB,
+                                                                   
totalSpaceInBytes, 1, 8);
+                conf.cdc_total_space = 
SmallestDataStorageMebibytes.inMebibytes(defaultSpaceInMiB);
             }
 
             logger.info("cdc_enabled is true. Starting casssandra node with 
Change-Data-Capture enabled.");
@@ -611,7 +613,7 @@ public class DatabaseDescriptor
 
             dataFreeBytes = saturatedSum(dataFreeBytes, tryGetSpace(datadir, 
FileStore::getUnallocatedSpace));
         }
-        if (dataFreeBytes < 64 * ONE_GB) // 64 GB
+        if (dataFreeBytes < 64 * ONE_GIB) // 64 GB
             logger.warn("Only {} free across all data volumes. Consider adding 
more capacity to your cluster or removing obsolete snapshots",
                         FBUtilities.prettyPrintMemory(dataFreeBytes));
 
@@ -626,7 +628,7 @@ public class DatabaseDescriptor
 
             long freeBytes = 
tryGetSpace(conf.local_system_data_file_directory, 
FileStore::getUnallocatedSpace);
 
-            if (freeBytes < ONE_GB)
+            if (freeBytes < ONE_GIB)
                 logger.warn("Only {} free in the system data volume. Consider 
adding more capacity or removing obsolete snapshots",
                             FBUtilities.prettyPrintMemory(freeBytes));
         }
@@ -743,17 +745,17 @@ public class DatabaseDescriptor
         if (conf.user_defined_function_fail_timeout < 
conf.user_defined_function_warn_timeout)
             throw new 
ConfigurationException("user_defined_function_warn_timeout must less than 
user_defined_function_fail_timeout", false);
 
-        if (conf.commitlog_segment_size_in_mb <= 0)
-            throw new ConfigurationException("commitlog_segment_size_in_mb 
must be positive, but was "
-                    + conf.commitlog_segment_size_in_mb, false);
-        else if (conf.commitlog_segment_size_in_mb >= 2048)
-            throw new ConfigurationException("commitlog_segment_size_in_mb 
must be smaller than 2048, but was "
-                    + conf.commitlog_segment_size_in_mb, false);
+        if (conf.commitlog_segment_size.toMebibytes() == 0)
+            throw new ConfigurationException("commitlog_segment_size must be 
positive, but was "
+                    + conf.commitlog_segment_size.toString(), false);
+        else if (conf.commitlog_segment_size.toMebibytes() >= 2048)
+            throw new ConfigurationException("commitlog_segment_size must be 
smaller than 2048, but was "
+                    + conf.commitlog_segment_size.toString(), false);
 
-        if (conf.max_mutation_size_in_kb == null)
-            conf.max_mutation_size_in_kb = conf.commitlog_segment_size_in_mb * 
1024 / 2;
-        else if (conf.commitlog_segment_size_in_mb * 1024 < 2 * 
conf.max_mutation_size_in_kb)
-            throw new ConfigurationException("commitlog_segment_size_in_mb 
must be at least twice the size of max_mutation_size_in_kb / 1024", false);
+        if (conf.max_mutation_size == null)
+            conf.max_mutation_size = 
SmallestDataStorageKibibytes.inKibibytes(conf.commitlog_segment_size.toKibibytes()
 / 2);
+        else if (conf.commitlog_segment_size.toKibibytes() < 2 * 
conf.max_mutation_size.toKibibytes())
+            throw new ConfigurationException("commitlog_segment_size must be 
at least twice the size of max_mutation_size / 1024", false);
 
         // native transport encryption options
         if (conf.client_encryption_options != null)
@@ -904,20 +906,20 @@ public class DatabaseDescriptor
         return storagedir;
     }
 
-    static int calculateDefaultSpaceInMB(String type, String path, String 
setting, int preferredSizeInMB, long totalSpaceInBytes, long 
totalSpaceNumerator, long totalSpaceDenominator)
+    static int calculateDefaultSpaceInMiB(String type, String path, String 
setting, int preferredSizeInMiB, long totalSpaceInBytes, long 
totalSpaceNumerator, long totalSpaceDenominator)
     {
-        final long totalSizeInMB = totalSpaceInBytes / ONE_MIB;
-        final int minSizeInMB = Ints.saturatedCast(totalSpaceNumerator * 
totalSizeInMB / totalSpaceDenominator);
+        final long totalSizeInMiB = totalSpaceInBytes / ONE_MIB;
+        final int minSizeInMiB = Ints.saturatedCast(totalSpaceNumerator * 
totalSizeInMiB / totalSpaceDenominator);
 
-        if (minSizeInMB < preferredSizeInMB)
+        if (minSizeInMiB < preferredSizeInMiB)
         {
             logger.warn("Small {} volume detected at '{}'; setting {} to {}.  
You can override this in cassandra.yaml",
-                        type, path, setting, minSizeInMB);
-            return minSizeInMB;
+                        type, path, setting, minSizeInMiB);
+            return minSizeInMiB;
         }
         else
         {
-            return preferredSizeInMB;
+            return preferredSizeInMiB;
         }
     }
 
@@ -1867,7 +1869,7 @@ public class DatabaseDescriptor
         conf.compaction_throughput = DataRateSpec.inMebibytesPerSecond(value);
     }
 
-    public static long getCompactionLargePartitionWarningThreshold() { return 
ByteUnit.MEBI_BYTES.toBytes(conf.compaction_large_partition_warning_threshold_mb);
 }
+    public static long getCompactionLargePartitionWarningThreshold() { return 
conf.compaction_large_partition_warning_threshold.toBytes(); }
 
     public static int getCompactionTombstoneWarningThreshold()
     {
@@ -1902,7 +1904,7 @@ public class DatabaseDescriptor
 
     public static long getMinFreeSpacePerDriveInBytes()
     {
-        return 
ByteUnit.MEBI_BYTES.toBytes(conf.min_free_space_per_drive_in_mb);
+        return conf.min_free_space_per_drive.toBytes();
     }
 
     public static boolean getDisableSTCSInL0()
@@ -2074,7 +2076,7 @@ public class DatabaseDescriptor
 
     public static int getMaxMutationSize()
     {
-        return (int) ByteUnit.KIBI_BYTES.toBytes(conf.max_mutation_size_in_kb);
+        return conf.max_mutation_size.toBytesAsInt();
     }
 
     public static int getTombstoneWarnThreshold()
@@ -2122,19 +2124,19 @@ public class DatabaseDescriptor
      */
     public static int getCommitLogSegmentSize()
     {
-        return (int) 
ByteUnit.MEBI_BYTES.toBytes(conf.commitlog_segment_size_in_mb);
+        return conf.commitlog_segment_size.toBytesAsInt();
     }
 
     /**
-     * Update commitlog_segment_size_in_mb in the tests.
+     * Update commitlog_segment_size in the tests.
      * {@link CommitLogSegmentManagerCDC} uses the CommitLogSegmentSize to 
estimate the file size on allocation.
      * It is important to keep the value unchanged for the estimation to be 
correct.
-     * @param sizeMegabytes
+     * @param sizeMebibytes
      */
     @VisibleForTesting /* Only for testing */
-    public static void setCommitLogSegmentSize(int sizeMegabytes)
+    public static void setCommitLogSegmentSize(int sizeMebibytes)
     {
-        conf.commitlog_segment_size_in_mb = sizeMegabytes;
+        conf.commitlog_segment_size = 
SmallestDataStorageMebibytes.inMebibytes(sizeMebibytes);
     }
 
     public static String getSavedCachesLocation()
@@ -2404,14 +2406,14 @@ public class DatabaseDescriptor
         conf.native_transport_allow_older_protocols = isEnabled;
     }
 
-    public static double getCommitLogSyncGroupWindow()
+    public static long getCommitLogSyncGroupWindow()
     {
-        return conf.commitlog_sync_group_window_in_ms;
+        return conf.commitlog_sync_group_window.toMilliseconds();
     }
 
-    public static void setCommitLogSyncGroupWindow(double windowMillis)
+    public static void setCommitLogSyncGroupWindow(long windowMillis)
     {
-        conf.commitlog_sync_group_window_in_ms = windowMillis;
+        conf.commitlog_sync_group_window = 
SmallestDurationMilliseconds.inMilliseconds(windowMillis);
     }
 
     public static int getNativeTransportReceiveQueueCapacityInBytes()
@@ -2478,20 +2480,20 @@ public class DatabaseDescriptor
 
     public static int getCommitLogSyncPeriod()
     {
-        return conf.commitlog_sync_period_in_ms;
+        return conf.commitlog_sync_period.toMillisecondsAsInt();
     }
 
     public static long getPeriodicCommitLogSyncBlock()
     {
-        Integer blockMillis = conf.periodic_commitlog_sync_lag_block_in_ms;
+        SmallestDurationMilliseconds blockMillis = 
conf.periodic_commitlog_sync_lag_block;
         return blockMillis == null
                ? (long)(getCommitLogSyncPeriod() * 1.5)
-               : blockMillis;
+               : blockMillis.toMilliseconds();
     }
 
     public static void setCommitLogSyncPeriod(int periodMillis)
     {
-        conf.commitlog_sync_period_in_ms = periodMillis;
+        conf.commitlog_sync_period = 
SmallestDurationMilliseconds.inMilliseconds(periodMillis);
     }
 
     public static Config.CommitLogSync getCommitLogSync()
@@ -2656,20 +2658,20 @@ public class DatabaseDescriptor
 
     public static int getDynamicUpdateInterval()
     {
-        return conf.dynamic_snitch_update_interval_in_ms;
+        return conf.dynamic_snitch_update_interval.toMillisecondsAsInt();
     }
     public static void setDynamicUpdateInterval(int dynamicUpdateInterval)
     {
-        conf.dynamic_snitch_update_interval_in_ms = dynamicUpdateInterval;
+        conf.dynamic_snitch_update_interval = 
SmallestDurationMilliseconds.inMilliseconds(dynamicUpdateInterval);
     }
 
     public static int getDynamicResetInterval()
     {
-        return conf.dynamic_snitch_reset_interval_in_ms;
+        return conf.dynamic_snitch_reset_interval.toMillisecondsAsInt();
     }
     public static void setDynamicResetInterval(int dynamicResetInterval)
     {
-        conf.dynamic_snitch_reset_interval_in_ms = dynamicResetInterval;
+        conf.dynamic_snitch_reset_interval = 
SmallestDurationMilliseconds.inMilliseconds(dynamicResetInterval);
     }
 
     public static double getDynamicBadnessThreshold()
@@ -2703,24 +2705,24 @@ public class DatabaseDescriptor
         conf.client_encryption_options = 
update.apply(conf.client_encryption_options);
     }
 
-    public static int getHintedHandoffThrottleInKB()
+    public static int getHintedHandoffThrottleInKiB()
     {
-        return conf.hinted_handoff_throttle_in_kb;
+        return conf.hinted_handoff_throttle.toKibibytesAsInt();
     }
 
-    public static void setHintedHandoffThrottleInKB(int throttleInKB)
+    public static void setHintedHandoffThrottleInKiB(int throttleInKiB)
     {
-        conf.hinted_handoff_throttle_in_kb = throttleInKB;
+        conf.hinted_handoff_throttle = 
SmallestDataStorageKibibytes.inKibibytes(throttleInKiB);
     }
 
-    public static int getBatchlogReplayThrottleInKB()
+    public static int getBatchlogReplayThrottleInKiB()
     {
-        return conf.batchlog_replay_throttle_in_kb;
+        return conf.batchlog_replay_throttle.toKibibytesAsInt();
     }
 
-    public static void setBatchlogReplayThrottleInKB(int throttleInKB)
+    public static void setBatchlogReplayThrottleInKiB(int throttleInKiB)
     {
-        conf.batchlog_replay_throttle_in_kb = throttleInKB;
+        conf.batchlog_replay_throttle = 
SmallestDataStorageKibibytes.inKibibytes(throttleInKiB);
     }
 
     public static int getMaxHintsDeliveryThreads()
@@ -2730,12 +2732,12 @@ public class DatabaseDescriptor
 
     public static int getHintsFlushPeriodInMS()
     {
-        return conf.hints_flush_period_in_ms;
+        return conf.hints_flush_period.toMillisecondsAsInt();
     }
 
     public static long getMaxHintsFileSize()
     {
-        return  ByteUnit.MEBI_BYTES.toBytes(conf.max_hints_file_size_in_mb);
+        return  conf.max_hints_file_size.toBytes();
     }
 
     public static ParameterizedClass getHintsCompression()
@@ -2818,9 +2820,9 @@ public class DatabaseDescriptor
         return conf.disk_optimization_estimate_percentile;
     }
 
-    public static long getTotalCommitlogSpaceInMB()
+    public static long getTotalCommitlogSpaceInMiB()
     {
-        return conf.commitlog_total_space_in_mb;
+        return conf.commitlog_total_space.toMebibytes();
     }
 
     public static boolean shouldMigrateKeycacheOnCompaction()
@@ -3215,20 +3217,20 @@ public class DatabaseDescriptor
         return conf.cdc_raw_directory;
     }
 
-    public static int getCDCSpaceInMB()
+    public static int getCDCSpaceInMiB()
     {
-        return conf.cdc_total_space_in_mb;
+        return conf.cdc_total_space.toMebibytesAsInt();
     }
 
     @VisibleForTesting
-    public static void setCDCSpaceInMB(int input)
+    public static void setCDCSpaceInMiB(int input)
     {
-        conf.cdc_total_space_in_mb = input;
+        conf.cdc_total_space = SmallestDataStorageMebibytes.inMebibytes(input);
     }
 
     public static int getCDCDiskCheckInterval()
     {
-        return conf.cdc_free_space_check_interval_ms;
+        return conf.cdc_free_space_check_interval.toMillisecondsAsInt();
     }
 
     @VisibleForTesting
diff --git 
a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
 
b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
index 485e3fd..de29224 100644
--- 
a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
+++ 
b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
@@ -383,7 +383,7 @@ public abstract class AbstractCommitLogSegmentManager
 
     private long unusedCapacity()
     {
-        long total = DatabaseDescriptor.getTotalCommitlogSpaceInMB() * 1024 * 
1024;
+        long total = DatabaseDescriptor.getTotalCommitlogSpaceInMiB() * 1024 * 
1024;
         long currentSize = size.get();
         logger.trace("Total active commitlog segment space used is {} out of 
{}", currentSize, total);
         return total - currentSize;
diff --git 
a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java 
b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
index cbd88ca..71c6efa 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
@@ -315,7 +315,7 @@ public class CommitLogSegmentManagerCDC extends 
AbstractCommitLogSegmentManager
 
         private long allowableCDCBytes()
         {
-            return (long)DatabaseDescriptor.getCDCSpaceInMB() * 1024 * 1024;
+            return (long)DatabaseDescriptor.getCDCSpaceInMiB() * 1024 * 1024;
         }
 
         public void submitOverflowSizeRecalculation()
diff --git 
a/src/java/org/apache/cassandra/db/commitlog/GroupCommitLogService.java 
b/src/java/org/apache/cassandra/db/commitlog/GroupCommitLogService.java
index a76923e..ad4448a 100644
--- a/src/java/org/apache/cassandra/db/commitlog/GroupCommitLogService.java
+++ b/src/java/org/apache/cassandra/db/commitlog/GroupCommitLogService.java
@@ -35,7 +35,7 @@ public class GroupCommitLogService extends 
AbstractCommitLogService
     {
         // wait until record has been safely persisted to disk
         pending.incrementAndGet();
-        // wait for commitlog_sync_group_window_in_ms
+        // wait for commitlog_sync_group_window
         alloc.awaitDiskSync(commitLog.metrics.waitingOnCommit);
         pending.decrementAndGet();
     }
diff --git a/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java 
b/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
index 7f70b21..991e2a2 100644
--- a/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
+++ b/src/java/org/apache/cassandra/hints/HintsDispatchExecutor.java
@@ -216,7 +216,7 @@ final class HintsDispatchExecutor
             // not total outgoing hints traffic from this node - this is why 
the rate limiter is not shared between
             // all the dispatch tasks (as there will be at most one dispatch 
task for a particular host id at a time).
             int nodesCount = Math.max(1, 
StorageService.instance.getTokenMetadata().getAllEndpoints().size() - 1);
-            double throttleInBytes = 
DatabaseDescriptor.getHintedHandoffThrottleInKB() * 1024.0 / nodesCount;
+            double throttleInBytes = 
DatabaseDescriptor.getHintedHandoffThrottleInKiB() * 1024.0 / nodesCount;
             this.rateLimiter = RateLimiter.create(throttleInBytes == 0 ? 
Double.MAX_VALUE : throttleInBytes);
         }
 
diff --git a/src/java/org/apache/cassandra/hints/HintsWriteExecutor.java 
b/src/java/org/apache/cassandra/hints/HintsWriteExecutor.java
index 8089d61..9d64e4a 100644
--- a/src/java/org/apache/cassandra/hints/HintsWriteExecutor.java
+++ b/src/java/org/apache/cassandra/hints/HintsWriteExecutor.java
@@ -62,7 +62,7 @@ final class HintsWriteExecutor
     }
 
     /*
-     * Should be very fast (worst case scenario - write a few 10s of megabytes 
to disk).
+     * Should be very fast (worst case scenario - write a few 10s of mebibytes 
to disk).
      */
     void shutdownBlocking()
     {
@@ -242,7 +242,7 @@ final class HintsWriteExecutor
                 we are removing it specifically after we closed the store 
above in try block
                 so hints are persisted on disk before.
 
-                There is a periodic flushing of a buffer driven by 
hints_flush_period_in_ms and clearing
+                There is a periodic flushing of a buffer driven by 
hints_flush_period and clearing
                 this entry upon every flush would remove the information what 
is the earliest hint in the buffer
                 for a respective node prematurely.
 
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java 
b/src/java/org/apache/cassandra/io/util/FileUtils.java
index 063e3fb..08ab4f2 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -75,10 +75,10 @@ public final class FileUtils
 
     private static final Logger logger = 
LoggerFactory.getLogger(FileUtils.class);
 
-    public static final long ONE_KB = 1024;
-    public static final long ONE_MIB = 1024 * ONE_KB;
-    public static final long ONE_GB = 1024 * ONE_MIB;
-    public static final long ONE_TB = 1024 * ONE_GB;
+    public static final long ONE_KIB = 1024;
+    public static final long ONE_MIB = 1024 * ONE_KIB;
+    public static final long ONE_GIB = 1024 * ONE_MIB;
+    public static final long ONE_TIB = 1024 * ONE_GIB;
 
     private static final DecimalFormat df = new DecimalFormat("#.##");
     private static final AtomicReference<Optional<FSErrorHandler>> 
fsErrorHandler = new AtomicReference<>(Optional.empty());
@@ -390,17 +390,17 @@ public final class FileUtils
         }
         if (value.endsWith(" TiB"))
         {
-            result = Math.round(Double.valueOf(value.replace(" TiB", "")) * 
ONE_TB);
+            result = Math.round(Double.valueOf(value.replace(" TiB", "")) * 
ONE_TIB);
             return result;
         }
         else if (value.endsWith(" GiB"))
         {
-            result = Math.round(Double.valueOf(value.replace(" GiB", "")) * 
ONE_GB);
+            result = Math.round(Double.valueOf(value.replace(" GiB", "")) * 
ONE_GIB);
             return result;
         }
         else if (value.endsWith(" KiB"))
         {
-            result = Math.round(Double.valueOf(value.replace(" KiB", "")) * 
ONE_KB);
+            result = Math.round(Double.valueOf(value.replace(" KiB", "")) * 
ONE_KIB);
             return result;
         }
         else if (value.endsWith(" MiB"))
@@ -422,15 +422,15 @@ public final class FileUtils
     public static String stringifyFileSize(double value)
     {
         double d;
-        if ( value >= ONE_TB )
+        if (value >= ONE_TIB)
         {
-            d = value / ONE_TB;
+            d = value / ONE_TIB;
             String val = df.format(d);
             return val + " TiB";
         }
-        else if ( value >= ONE_GB )
+        else if (value >= ONE_GIB)
         {
-            d = value / ONE_GB;
+            d = value / ONE_GIB;
             String val = df.format(d);
             return val + " GiB";
         }
@@ -440,9 +440,9 @@ public final class FileUtils
             String val = df.format(d);
             return val + " MiB";
         }
-        else if ( value >= ONE_KB )
+        else if (value >= ONE_KIB)
         {
-            d = value / ONE_KB;
+            d = value / ONE_KIB;
             String val = df.format(d);
             return val + " KiB";
         }
diff --git a/src/java/org/apache/cassandra/service/StorageService.java 
b/src/java/org/apache/cassandra/service/StorageService.java
index 4133119..1236279 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -1591,12 +1591,12 @@ public class StorageService extends 
NotificationBroadcasterSupport implements IE
 
     public int getBatchlogReplayThrottleInKB()
     {
-        return DatabaseDescriptor.getBatchlogReplayThrottleInKB();
+        return DatabaseDescriptor.getBatchlogReplayThrottleInKiB();
     }
 
     public void setBatchlogReplayThrottleInKB(int throttleInKB)
     {
-        DatabaseDescriptor.setBatchlogReplayThrottleInKB(throttleInKB);
+        DatabaseDescriptor.setBatchlogReplayThrottleInKiB(throttleInKB);
         BatchlogManager.instance.setRate(throttleInKB);
     }
 
@@ -5971,8 +5971,8 @@ public class StorageService extends 
NotificationBroadcasterSupport implements IE
 
     public void setHintedHandoffThrottleInKB(int throttleInKB)
     {
-        DatabaseDescriptor.setHintedHandoffThrottleInKB(throttleInKB);
-        logger.info("updated hinted_handoff_throttle_in_kb to {}", 
throttleInKB);
+        DatabaseDescriptor.setHintedHandoffThrottleInKiB(throttleInKB);
+        logger.info("updated hinted_handoff_throttle to {} KiB", throttleInKB);
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java 
b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 8e386ec..103b4dd 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -536,12 +536,12 @@ public interface StorageServiceMBean extends 
NotificationEmitter
     public void updateSnitch(String epSnitchClassName, Boolean dynamic, 
Integer dynamicUpdateInterval, Integer dynamicResetInterval, Double 
dynamicBadnessThreshold) throws ClassNotFoundException;
 
     /*
-      Update dynamic_snitch_update_interval_in_ms
+      Update dynamic_snitch_update_interval in ms
      */
     public void setDynamicUpdateInterval(int dynamicUpdateInterval);
 
     /*
-      Get dynamic_snitch_update_interval_in_ms
+      Get dynamic_snitch_update_interval in ms
      */
     public int getDynamicUpdateInterval();
 
@@ -796,7 +796,7 @@ public interface StorageServiceMBean extends 
NotificationEmitter
     /** Sets the threshold for warning queries due to a large batch size */
     public void setBatchSizeWarnThreshold(int batchSizeDebugThreshold);
 
-    /** Sets the hinted handoff throttle in kb per second, per delivery 
thread. */
+    /** Sets the hinted handoff throttle in KiB per second, per delivery 
thread. */
     public void setHintedHandoffThrottleInKB(int throttleInKB);
 
     /**
diff --git 
a/src/java/org/apache/cassandra/tools/nodetool/SetHintedHandoffThrottleInKB.java
 
b/src/java/org/apache/cassandra/tools/nodetool/SetHintedHandoffThrottleInKB.java
index 96f1bdf..b7b7b61 100644
--- 
a/src/java/org/apache/cassandra/tools/nodetool/SetHintedHandoffThrottleInKB.java
+++ 
b/src/java/org/apache/cassandra/tools/nodetool/SetHintedHandoffThrottleInKB.java
@@ -23,10 +23,10 @@ import io.airlift.airline.Command;
 import org.apache.cassandra.tools.NodeProbe;
 import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
 
-@Command(name = "sethintedhandoffthrottlekb", description =  "Set hinted 
handoff throttle in kb per second, per delivery thread.")
+@Command(name = "sethintedhandoffthrottlekb", description =  "Set hinted 
handoff throttle in KiB per second, per delivery thread.")
 public class SetHintedHandoffThrottleInKB extends NodeToolCmd
 {
-    @Arguments(title = "throttle_in_kb", usage = "<value_in_kb_per_sec>", 
description = "Value in KB per second", required = true)
+    @Arguments(title = "throttle_in_kb", usage = "<value_in_kb_per_sec>", 
description = "Value in KiB per second", required = true)
     private Integer throttleInKB = null;
 
     @Override
diff --git a/test/conf/cassandra-murmur.yaml b/test/conf/cassandra-murmur.yaml
index 9da6917..05dac76 100644
--- a/test/conf/cassandra-murmur.yaml
+++ b/test/conf/cassandra-murmur.yaml
@@ -6,7 +6,7 @@ cluster_name: Test Cluster
 memtable_allocation_type: heap_buffers
 commitlog_sync: batch
 commitlog_sync_batch_window_in_ms: 1.0
-commitlog_segment_size_in_mb: 5
+commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 cdc_raw_directory: build/test/cassandra/cdc_raw
 cdc_enabled: false
diff --git a/test/conf/cassandra-seeds.yaml b/test/conf/cassandra-seeds.yaml
index 5e016cb..d5dd83e 100644
--- a/test/conf/cassandra-seeds.yaml
+++ b/test/conf/cassandra-seeds.yaml
@@ -7,7 +7,7 @@ cluster_name: Test Cluster
 memtable_allocation_type: offheap_objects
 commitlog_sync: batch
 commitlog_sync_batch_window_in_ms: 1.0
-commitlog_segment_size_in_mb: 5
+commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 cdc_raw_directory: build/test/cassandra/cdc_raw
 cdc_enabled: false
diff --git a/test/conf/cassandra-sslcontextfactory-invalidconfiguration.yaml 
b/test/conf/cassandra-sslcontextfactory-invalidconfiguration.yaml
index b11f248..8f51e85 100644
--- a/test/conf/cassandra-sslcontextfactory-invalidconfiguration.yaml
+++ b/test/conf/cassandra-sslcontextfactory-invalidconfiguration.yaml
@@ -25,7 +25,7 @@ cluster_name: Test Cluster
 memtable_allocation_type: offheap_objects
 commitlog_sync: batch
 commitlog_sync_batch_window_in_ms: 1.0
-commitlog_segment_size_in_mb: 5
+commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 # commitlog_compression:
 # - class_name: LZ4Compressor
diff --git a/test/conf/cassandra-sslcontextfactory.yaml 
b/test/conf/cassandra-sslcontextfactory.yaml
index 6e55b49..63c40a2 100644
--- a/test/conf/cassandra-sslcontextfactory.yaml
+++ b/test/conf/cassandra-sslcontextfactory.yaml
@@ -25,7 +25,7 @@ cluster_name: Test Cluster
 memtable_allocation_type: offheap_objects
 commitlog_sync: batch
 commitlog_sync_batch_window_in_ms: 1.0
-commitlog_segment_size_in_mb: 5
+commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 # commitlog_compression:
 # - class_name: LZ4Compressor
diff --git a/test/conf/cassandra.yaml b/test/conf/cassandra.yaml
index bd5a1b8..e1a49f2 100644
--- a/test/conf/cassandra.yaml
+++ b/test/conf/cassandra.yaml
@@ -7,7 +7,7 @@ cluster_name: Test Cluster
 memtable_allocation_type: offheap_objects
 commitlog_sync: batch
 commitlog_sync_batch_window_in_ms: 1.0
-commitlog_segment_size_in_mb: 5
+commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 # commitlog_compression:
 # - class_name: LZ4Compressor
@@ -20,7 +20,7 @@ storage_port: 7012
 ssl_storage_port: 17012
 start_native_transport: true
 native_transport_port: 9042
-column_index_size_in_kb: 4
+column_index_size: 4KiB
 saved_caches_directory: build/test/cassandra/saved_caches
 data_file_directories:
     - build/test/cassandra/data
@@ -50,7 +50,7 @@ stream_entire_sstables: true
 stream_throughput_outbound: 23841858MiB/s
 sasi_indexes_enabled: true
 materialized_views_enabled: true
-enable_drop_compact_storage: true
+drop_compact_storage_enabled: true
 file_cache_enabled: true
 auto_hints_cleanup_enabled: true
 track_warnings:
diff --git a/test/conf/unit-test-conf/test-native-port.yaml 
b/test/conf/unit-test-conf/test-native-port.yaml
index d3c5516..cef6b1d 100644
--- a/test/conf/unit-test-conf/test-native-port.yaml
+++ b/test/conf/unit-test-conf/test-native-port.yaml
@@ -7,7 +7,7 @@ cluster_name: Test Cluster
 memtable_allocation_type: offheap_objects
 commitlog_sync: batch
 commitlog_sync_batch_window_in_ms: 1.0
-commitlog_segment_size_in_mb: 5
+commitlog_segment_size: 5MiB
 commitlog_directory: build/test/cassandra/commitlog
 # commitlog_compression:
 # - class_name: LZ4Compressor
diff --git 
a/test/distributed/org/apache/cassandra/distributed/test/HintedHandoffNodetoolTest.java
 
b/test/distributed/org/apache/cassandra/distributed/test/HintedHandoffNodetoolTest.java
index 5045301..5ab83ba 100644
--- 
a/test/distributed/org/apache/cassandra/distributed/test/HintedHandoffNodetoolTest.java
+++ 
b/test/distributed/org/apache/cassandra/distributed/test/HintedHandoffNodetoolTest.java
@@ -122,9 +122,9 @@ public class HintedHandoffNodetoolTest extends TestBaseImpl
     @Test
     public void testThrottle()
     {
-        Integer throttleInKB = 
cluster.get(node).callOnInstance(DatabaseDescriptor::getHintedHandoffThrottleInKB);
+        Integer throttleInKB = 
cluster.get(node).callOnInstance(DatabaseDescriptor::getHintedHandoffThrottleInKiB);
         cluster.get(node).nodetoolResult("sethintedhandoffthrottlekb", 
String.valueOf(throttleInKB * 2)).asserts().success();
-        Integer newThrottleInKB = 
cluster.get(node).callOnInstance(DatabaseDescriptor::getHintedHandoffThrottleInKB);
+        Integer newThrottleInKB = 
cluster.get(node).callOnInstance(DatabaseDescriptor::getHintedHandoffThrottleInKiB);
         assertEquals(throttleInKB * 2, newThrottleInKB.intValue());
     }
 
diff --git 
a/test/distributed/org/apache/cassandra/distributed/test/LargeColumnTest.java 
b/test/distributed/org/apache/cassandra/distributed/test/LargeColumnTest.java
index 34432dc..b03171b 100644
--- 
a/test/distributed/org/apache/cassandra/distributed/test/LargeColumnTest.java
+++ 
b/test/distributed/org/apache/cassandra/distributed/test/LargeColumnTest.java
@@ -65,7 +65,7 @@ public class LargeColumnTest extends TestBaseImpl
         try (ICluster cluster = init(builder()
                                      .withNodes(nodes)
                                      .withConfig(config ->
-                                                 
config.set("commitlog_segment_size_in_mb", (columnSize * 3) >> 20)
+                                                 
config.set("commitlog_segment_size", String.format("%dMiB",(columnSize * 3) >> 
20))
                                                        
.set("internode_application_send_queue_reserve_endpoint_capacity", 
String.format("%dB", (columnSize * 2)))
                                                        
.set("internode_application_send_queue_reserve_global_capacity", 
String.format("%dB", (columnSize * 3)))
                                                        
.set("write_request_timeout", "30s")
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java 
b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index 0d13654..eea55f7 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -451,7 +451,7 @@ public class DatabaseDescriptorTest
         int spaceInBytes = 999 * 1024 * 1024;
 
         assertEquals(666, // total size is less than preferred, so return 
lower limit
-                     DatabaseDescriptor.calculateDefaultSpaceInMB("type", 
"/path", "setting_name", preferredInMB, spaceInBytes, numerator, denominator));
+                     DatabaseDescriptor.calculateDefaultSpaceInMiB("type", 
"/path", "setting_name", preferredInMB, spaceInBytes, numerator, denominator));
 
         // check preferred size is used for a small storage volume
         preferredInMB = 100;
@@ -460,7 +460,7 @@ public class DatabaseDescriptorTest
         spaceInBytes = 999 * 1024 * 1024;
 
         assertEquals(100, // total size is more than preferred so keep the 
configured limit
-                     DatabaseDescriptor.calculateDefaultSpaceInMB("type", 
"/path", "setting_name", preferredInMB, spaceInBytes, numerator, denominator));
+                     DatabaseDescriptor.calculateDefaultSpaceInMiB("type", 
"/path", "setting_name", preferredInMB, spaceInBytes, numerator, denominator));
     }
 
     @Test
diff --git 
a/test/unit/org/apache/cassandra/db/commitlog/CommitLogInitWithExceptionTest.java
 
b/test/unit/org/apache/cassandra/db/commitlog/CommitLogInitWithExceptionTest.java
index fbad0b4..7cd1445 100644
--- 
a/test/unit/org/apache/cassandra/db/commitlog/CommitLogInitWithExceptionTest.java
+++ 
b/test/unit/org/apache/cassandra/db/commitlog/CommitLogInitWithExceptionTest.java
@@ -90,6 +90,7 @@ public class CommitLogInitWithExceptionTest
             Assert.fail();
         }
 
+
         
Assert.assertTrue(CommitLog.instance.segmentManager.executor.isTerminated()); 
// exit successfully
     }
 
diff --git 
a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
 
b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
index 6c1d375..b5902e5 100644
--- 
a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
+++ 
b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
@@ -49,7 +49,7 @@ public class CommitLogSegmentManagerCDCTest extends CQLTester
     public static void setUpClass()
     {
         DatabaseDescriptor.setCDCEnabled(true);
-        DatabaseDescriptor.setCDCSpaceInMB(1024);
+        DatabaseDescriptor.setCDCSpaceInMiB(1024);
         CQLTester.setUpClass();
     }
 
@@ -407,15 +407,15 @@ public class CommitLogSegmentManagerCDCTest extends 
CQLTester
 
     private void testWithCDCSpaceInMb(int size, Testable test) throws Throwable
     {
-        int origSize = DatabaseDescriptor.getCDCSpaceInMB();
-        DatabaseDescriptor.setCDCSpaceInMB(size);
+        int origSize = DatabaseDescriptor.getCDCSpaceInMiB();
+        DatabaseDescriptor.setCDCSpaceInMiB(size);
         try
         {
             test.run();
         }
         finally
         {
-            DatabaseDescriptor.setCDCSpaceInMB(origSize);
+            DatabaseDescriptor.setCDCSpaceInMiB(origSize);
         }
     }
 

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

Reply via email to